HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / HRegion.java
blobcf65859356c0bd84f53a7a12287e8d09b4c2593d
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 com.google.errorprone.annotations.RestrictedApi;
27 import edu.umd.cs.findbugs.annotations.Nullable;
28 import io.opentelemetry.api.trace.Span;
29 import java.io.EOFException;
30 import java.io.FileNotFoundException;
31 import java.io.IOException;
32 import java.io.InterruptedIOException;
33 import java.lang.reflect.Constructor;
34 import java.nio.ByteBuffer;
35 import java.nio.charset.StandardCharsets;
36 import java.text.ParseException;
37 import java.util.ArrayList;
38 import java.util.Arrays;
39 import java.util.Collection;
40 import java.util.Collections;
41 import java.util.HashMap;
42 import java.util.HashSet;
43 import java.util.Iterator;
44 import java.util.List;
45 import java.util.Map;
46 import java.util.Map.Entry;
47 import java.util.NavigableMap;
48 import java.util.NavigableSet;
49 import java.util.Objects;
50 import java.util.Optional;
51 import java.util.RandomAccess;
52 import java.util.Set;
53 import java.util.TreeMap;
54 import java.util.UUID;
55 import java.util.concurrent.Callable;
56 import java.util.concurrent.CompletionService;
57 import java.util.concurrent.ConcurrentHashMap;
58 import java.util.concurrent.ConcurrentMap;
59 import java.util.concurrent.ConcurrentSkipListMap;
60 import java.util.concurrent.ExecutionException;
61 import java.util.concurrent.ExecutorCompletionService;
62 import java.util.concurrent.Future;
63 import java.util.concurrent.ThreadFactory;
64 import java.util.concurrent.ThreadPoolExecutor;
65 import java.util.concurrent.TimeUnit;
66 import java.util.concurrent.atomic.AtomicBoolean;
67 import java.util.concurrent.atomic.AtomicInteger;
68 import java.util.concurrent.atomic.LongAdder;
69 import java.util.concurrent.locks.Lock;
70 import java.util.concurrent.locks.ReadWriteLock;
71 import java.util.concurrent.locks.ReentrantLock;
72 import java.util.concurrent.locks.ReentrantReadWriteLock;
73 import java.util.function.Function;
74 import java.util.stream.Collectors;
75 import java.util.stream.Stream;
77 import org.apache.hadoop.conf.Configuration;
78 import org.apache.hadoop.fs.FileStatus;
79 import org.apache.hadoop.fs.FileSystem;
80 import org.apache.hadoop.fs.LocatedFileStatus;
81 import org.apache.hadoop.fs.Path;
82 import org.apache.hadoop.hbase.Cell;
83 import org.apache.hadoop.hbase.CellBuilderType;
84 import org.apache.hadoop.hbase.CellComparator;
85 import org.apache.hadoop.hbase.CellComparatorImpl;
86 import org.apache.hadoop.hbase.CellScanner;
87 import org.apache.hadoop.hbase.CellUtil;
88 import org.apache.hadoop.hbase.CompareOperator;
89 import org.apache.hadoop.hbase.CompoundConfiguration;
90 import org.apache.hadoop.hbase.DoNotRetryIOException;
91 import org.apache.hadoop.hbase.DroppedSnapshotException;
92 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
93 import org.apache.hadoop.hbase.HConstants;
94 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
95 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
96 import org.apache.hadoop.hbase.KeyValue;
97 import org.apache.hadoop.hbase.MetaCellComparator;
98 import org.apache.hadoop.hbase.NamespaceDescriptor;
99 import org.apache.hadoop.hbase.NotServingRegionException;
100 import org.apache.hadoop.hbase.PrivateCellUtil;
101 import org.apache.hadoop.hbase.RegionTooBusyException;
102 import org.apache.hadoop.hbase.Tag;
103 import org.apache.hadoop.hbase.TagUtil;
104 import org.apache.hadoop.hbase.client.Append;
105 import org.apache.hadoop.hbase.client.CheckAndMutate;
106 import org.apache.hadoop.hbase.client.CheckAndMutateResult;
107 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
108 import org.apache.hadoop.hbase.client.CompactionState;
109 import org.apache.hadoop.hbase.client.Delete;
110 import org.apache.hadoop.hbase.client.Durability;
111 import org.apache.hadoop.hbase.client.Get;
112 import org.apache.hadoop.hbase.client.Increment;
113 import org.apache.hadoop.hbase.client.IsolationLevel;
114 import org.apache.hadoop.hbase.client.Mutation;
115 import org.apache.hadoop.hbase.client.Put;
116 import org.apache.hadoop.hbase.client.RegionInfo;
117 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
118 import org.apache.hadoop.hbase.client.Result;
119 import org.apache.hadoop.hbase.client.Row;
120 import org.apache.hadoop.hbase.client.RowMutations;
121 import org.apache.hadoop.hbase.client.Scan;
122 import org.apache.hadoop.hbase.client.TableDescriptor;
123 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
124 import org.apache.hadoop.hbase.conf.ConfigurationManager;
125 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
126 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
127 import org.apache.hadoop.hbase.coprocessor.ReadOnlyConfiguration;
128 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
129 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
130 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
131 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
132 import org.apache.hadoop.hbase.filter.BinaryComparator;
133 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
134 import org.apache.hadoop.hbase.filter.Filter;
135 import org.apache.hadoop.hbase.io.HFileLink;
136 import org.apache.hadoop.hbase.io.HeapSize;
137 import org.apache.hadoop.hbase.io.TimeRange;
138 import org.apache.hadoop.hbase.io.hfile.BlockCache;
139 import org.apache.hadoop.hbase.io.hfile.HFile;
140 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
141 import org.apache.hadoop.hbase.ipc.RpcCall;
142 import org.apache.hadoop.hbase.ipc.RpcServer;
143 import org.apache.hadoop.hbase.ipc.ServerCall;
144 import org.apache.hadoop.hbase.mob.MobFileCache;
145 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
146 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
147 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
148 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
149 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
150 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
151 import org.apache.hadoop.hbase.regionserver.compactions.ForbidMajorCompactionChecker;
152 import org.apache.hadoop.hbase.regionserver.regionreplication.RegionReplicationSink;
153 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
154 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
155 import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector;
156 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
157 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
158 import org.apache.hadoop.hbase.replication.ReplicationUtils;
159 import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver;
160 import org.apache.hadoop.hbase.security.User;
161 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
162 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
163 import org.apache.hadoop.hbase.trace.TraceUtil;
164 import org.apache.hadoop.hbase.util.Bytes;
165 import org.apache.hadoop.hbase.util.CancelableProgressable;
166 import org.apache.hadoop.hbase.util.ClassSize;
167 import org.apache.hadoop.hbase.util.CommonFSUtils;
168 import org.apache.hadoop.hbase.util.CoprocessorConfigurationUtil;
169 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
170 import org.apache.hadoop.hbase.util.FSUtils;
171 import org.apache.hadoop.hbase.util.HashedBytes;
172 import org.apache.hadoop.hbase.util.NonceKey;
173 import org.apache.hadoop.hbase.util.Pair;
174 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
175 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
176 import org.apache.hadoop.hbase.util.Threads;
177 import org.apache.hadoop.hbase.wal.WAL;
178 import org.apache.hadoop.hbase.wal.WALEdit;
179 import org.apache.hadoop.hbase.wal.WALFactory;
180 import org.apache.hadoop.hbase.wal.WALKey;
181 import org.apache.hadoop.hbase.wal.WALKeyImpl;
182 import org.apache.hadoop.hbase.wal.WALSplitUtil;
183 import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay;
184 import org.apache.hadoop.util.StringUtils;
185 import org.apache.yetus.audience.InterfaceAudience;
186 import org.slf4j.Logger;
187 import org.slf4j.LoggerFactory;
189 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
190 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
191 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
192 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
193 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
194 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
195 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
196 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
197 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
198 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
199 import org.apache.hbase.thirdparty.com.google.protobuf.Service;
200 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
201 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
202 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
204 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
205 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
206 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
207 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
208 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
209 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
210 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
211 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
212 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
213 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
214 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
215 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
216 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
217 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
218 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
221 * Regions store data for a certain region of a table. It stores all columns
222 * for each row. A given table consists of one or more Regions.
224 * <p>An Region is defined by its table and its key extent.
226 * <p>Locking at the Region level serves only one purpose: preventing the
227 * region from being closed (and consequently split) while other operations
228 * are ongoing. Each row level operation obtains both a row lock and a region
229 * read lock for the duration of the operation. While a scanner is being
230 * constructed, getScanner holds a read lock. If the scanner is successfully
231 * constructed, it holds a read lock until it is closed. A close takes out a
232 * write lock and consequently will block for ongoing operations and will block
233 * new operations from starting while the close is in progress.
235 @SuppressWarnings("deprecation")
236 @InterfaceAudience.Private
237 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
238 private static final Logger LOG = LoggerFactory.getLogger(HRegion.class);
240 public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
241 "hbase.hregion.scan.loadColumnFamiliesOnDemand";
243 public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
244 public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
246 public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE =
247 "hbase.regionserver.minibatch.size";
248 public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000;
250 public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync";
251 public static final boolean DEFAULT_WAL_HSYNC = false;
253 /** Parameter name for compaction after bulkload */
254 public static final String COMPACTION_AFTER_BULKLOAD_ENABLE =
255 "hbase.compaction.after.bulkload.enable";
257 /** Config for allow split when file count greater than the configured blocking file count*/
258 public static final String SPLIT_IGNORE_BLOCKING_ENABLED_KEY =
259 "hbase.hregion.split.ignore.blocking.enabled";
262 * This is for for using HRegion as a local storage, where we may put the recovered edits in a
263 * special place. Once this is set, we will only replay the recovered edits under this directory
264 * and ignore the original replay directory configs.
266 public static final String SPECIAL_RECOVERED_EDITS_DIR =
267 "hbase.hregion.special.recovered.edits.dir";
270 * Whether to use {@link MetaCellComparator} even if we are not meta region. Used when creating
271 * master local region.
273 public static final String USE_META_CELL_COMPARATOR = "hbase.region.use.meta.cell.comparator";
275 public static final boolean DEFAULT_USE_META_CELL_COMPARATOR = false;
277 final AtomicBoolean closed = new AtomicBoolean(false);
279 /* Closing can take some time; use the closing flag if there is stuff we don't
280 * want to do while in closing state; e.g. like offer this region up to the
281 * master as a region to close if the carrying regionserver is overloaded.
282 * Once set, it is never cleared.
284 final AtomicBoolean closing = new AtomicBoolean(false);
287 * The max sequence id of flushed data on this region. There is no edit in memory that is
288 * less that this sequence id.
290 private volatile long maxFlushedSeqId = HConstants.NO_SEQNUM;
293 * Record the sequence id of last flush operation. Can be in advance of
294 * {@link #maxFlushedSeqId} when flushing a single column family. In this case,
295 * {@link #maxFlushedSeqId} will be older than the oldest edit in memory.
297 private volatile long lastFlushOpSeqId = HConstants.NO_SEQNUM;
300 * The sequence id of the last replayed open region event from the primary region. This is used
301 * to skip entries before this due to the possibility of replay edits coming out of order from
302 * replication.
304 protected volatile long lastReplayedOpenRegionSeqId = -1L;
305 protected volatile long lastReplayedCompactionSeqId = -1L;
307 //////////////////////////////////////////////////////////////////////////////
308 // Members
309 //////////////////////////////////////////////////////////////////////////////
311 // map from a locked row to the context for that lock including:
312 // - CountDownLatch for threads waiting on that row
313 // - the thread that owns the lock (allow reentrancy)
314 // - reference count of (reentrant) locks held by the thread
315 // - the row itself
316 private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows =
317 new ConcurrentHashMap<>();
319 protected final Map<byte[], HStore> stores =
320 new ConcurrentSkipListMap<>(Bytes.BYTES_RAWCOMPARATOR);
322 // TODO: account for each registered handler in HeapSize computation
323 private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
325 // Track data size in all memstores
326 private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();
327 RegionServicesForStores regionServicesForStores;
329 // Debug possible data loss due to WAL off
330 final LongAdder numMutationsWithoutWAL = new LongAdder();
331 final LongAdder dataInMemoryWithoutWAL = new LongAdder();
333 // Debug why CAS operations are taking a while.
334 final LongAdder checkAndMutateChecksPassed = new LongAdder();
335 final LongAdder checkAndMutateChecksFailed = new LongAdder();
337 // Number of requests
338 // Count rows for scan
339 final LongAdder readRequestsCount = new LongAdder();
340 final LongAdder cpRequestsCount = new LongAdder();
341 final LongAdder filteredReadRequestsCount = new LongAdder();
342 // Count rows for multi row mutations
343 final LongAdder writeRequestsCount = new LongAdder();
345 // Number of requests blocked by memstore size.
346 private final LongAdder blockedRequestsCount = new LongAdder();
348 // Compaction LongAdders
349 final LongAdder compactionsFinished = new LongAdder();
350 final LongAdder compactionsFailed = new LongAdder();
351 final LongAdder compactionNumFilesCompacted = new LongAdder();
352 final LongAdder compactionNumBytesCompacted = new LongAdder();
353 final LongAdder compactionsQueued = new LongAdder();
354 final LongAdder flushesQueued = new LongAdder();
356 private BlockCache blockCache;
357 private MobFileCache mobFileCache;
358 private final WAL wal;
359 private final HRegionFileSystem fs;
360 protected final Configuration conf;
361 private final Configuration baseConf;
362 private final int rowLockWaitDuration;
363 static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
365 private Path regionDir;
366 private FileSystem walFS;
368 // set to true if the region is restored from snapshot for reading by ClientSideRegionScanner
369 private boolean isRestoredRegion = false;
371 public void setRestoredRegion(boolean restoredRegion) {
372 isRestoredRegion = restoredRegion;
375 // The internal wait duration to acquire a lock before read/update
376 // from the region. It is not per row. The purpose of this wait time
377 // is to avoid waiting a long time while the region is busy, so that
378 // we can release the IPC handler soon enough to improve the
379 // availability of the region server. It can be adjusted by
380 // tuning configuration "hbase.busy.wait.duration".
381 final long busyWaitDuration;
382 static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
384 // If updating multiple rows in one call, wait longer,
385 // i.e. waiting for busyWaitDuration * # of rows. However,
386 // we can limit the max multiplier.
387 final int maxBusyWaitMultiplier;
389 // Max busy wait duration. There is no point to wait longer than the RPC
390 // purge timeout, when a RPC call will be terminated by the RPC engine.
391 final long maxBusyWaitDuration;
393 // Max cell size. If nonzero, the maximum allowed size for any given cell
394 // in bytes
395 final long maxCellSize;
397 // Number of mutations for minibatch processing.
398 private final int miniBatchSize;
400 final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
403 * The sequence ID that was enLongAddered when this region was opened.
405 private long openSeqNum = HConstants.NO_SEQNUM;
408 * The default setting for whether to enable on-demand CF loading for
409 * scan requests to this region. Requests can override it.
411 private boolean isLoadingCfsOnDemandDefault = false;
413 private final AtomicInteger majorInProgress = new AtomicInteger(0);
414 private final AtomicInteger minorInProgress = new AtomicInteger(0);
417 // Context: During replay we want to ensure that we do not lose any data. So, we
418 // have to be conservative in how we replay wals. For each store, we calculate
419 // the maxSeqId up to which the store was flushed. And, skip the edits which
420 // are equal to or lower than maxSeqId for each store.
421 // The following map is populated when opening the region
422 Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
424 // lock used to protect the replay operation for secondary replicas, so the below two fields does
425 // not need to be volatile.
426 private Lock replayLock;
428 /** Saved state from replaying prepare flush cache */
429 private PrepareFlushResult prepareFlushResult = null;
431 private long lastReplayedSequenceId = HConstants.NO_SEQNUM;
433 private volatile ConfigurationManager configurationManager;
435 // Used for testing.
436 private volatile Long timeoutForWriteLock = null;
438 private final CellComparator cellComparator;
441 * @return The smallest mvcc readPoint across all the scanners in this
442 * region. Writes older than this readPoint, are included in every
443 * read operation.
445 public long getSmallestReadPoint() {
446 long minimumReadPoint;
447 // We need to ensure that while we are calculating the smallestReadPoint
448 // no new RegionScanners can grab a readPoint that we are unaware of.
449 // We achieve this by synchronizing on the scannerReadPoints object.
450 synchronized (scannerReadPoints) {
451 minimumReadPoint = mvcc.getReadPoint();
452 for (Long readPoint : this.scannerReadPoints.values()) {
453 if (readPoint < minimumReadPoint) {
454 minimumReadPoint = readPoint;
458 return minimumReadPoint;
462 * Data structure of write state flags used coordinating flushes,
463 * compactions and closes.
465 static class WriteState {
466 // Set while a memstore flush is happening.
467 volatile boolean flushing = false;
468 // Set when a flush has been requested.
469 volatile boolean flushRequested = false;
470 // Number of compactions running.
471 AtomicInteger compacting = new AtomicInteger(0);
472 // Gets set in close. If set, cannot compact or flush again.
473 volatile boolean writesEnabled = true;
474 // Set if region is read-only
475 volatile boolean readOnly = false;
476 // whether the reads are enabled. This is different than readOnly, because readOnly is
477 // static in the lifetime of the region, while readsEnabled is dynamic
478 volatile boolean readsEnabled = true;
481 * Set flags that make this region read-only.
483 * @param onOff flip value for region r/o setting
485 synchronized void setReadOnly(final boolean onOff) {
486 this.writesEnabled = !onOff;
487 this.readOnly = onOff;
490 boolean isReadOnly() {
491 return this.readOnly;
494 boolean isFlushRequested() {
495 return this.flushRequested;
498 void setReadsEnabled(boolean readsEnabled) {
499 this.readsEnabled = readsEnabled;
502 static final long HEAP_SIZE = ClassSize.align(
503 ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
507 * Objects from this class are created when flushing to describe all the different states that
508 * that method ends up in. The Result enum describes those states. The sequence id should only
509 * be specified if the flush was successful, and the failure message should only be specified
510 * if it didn't flush.
512 public static class FlushResultImpl implements FlushResult {
513 final Result result;
514 final String failureReason;
515 final long flushSequenceId;
516 final boolean wroteFlushWalMarker;
519 * Convenience constructor to use when the flush is successful, the failure message is set to
520 * null.
521 * @param result Expecting FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_COMPACTION_NEEDED.
522 * @param flushSequenceId Generated sequence id that comes right after the edits in the
523 * memstores.
525 FlushResultImpl(Result result, long flushSequenceId) {
526 this(result, flushSequenceId, null, false);
527 assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
528 .FLUSHED_COMPACTION_NEEDED;
532 * Convenience constructor to use when we cannot flush.
533 * @param result Expecting CANNOT_FLUSH_MEMSTORE_EMPTY or CANNOT_FLUSH.
534 * @param failureReason Reason why we couldn't flush.
536 FlushResultImpl(Result result, String failureReason, boolean wroteFlushMarker) {
537 this(result, -1, failureReason, wroteFlushMarker);
538 assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH;
542 * Constructor with all the parameters.
543 * @param result Any of the Result.
544 * @param flushSequenceId Generated sequence id if the memstores were flushed else -1.
545 * @param failureReason Reason why we couldn't flush, or null.
547 FlushResultImpl(Result result, long flushSequenceId, String failureReason,
548 boolean wroteFlushMarker) {
549 this.result = result;
550 this.flushSequenceId = flushSequenceId;
551 this.failureReason = failureReason;
552 this.wroteFlushWalMarker = wroteFlushMarker;
556 * Convenience method, the equivalent of checking if result is
557 * FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_NO_COMPACTION_NEEDED.
558 * @return true if the memstores were flushed, else false.
560 @Override
561 public boolean isFlushSucceeded() {
562 return result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
563 .FLUSHED_COMPACTION_NEEDED;
567 * Convenience method, the equivalent of checking if result is FLUSHED_COMPACTION_NEEDED.
568 * @return True if the flush requested a compaction, else false (doesn't even mean it flushed).
570 @Override
571 public boolean isCompactionNeeded() {
572 return result == Result.FLUSHED_COMPACTION_NEEDED;
575 @Override
576 public String toString() {
577 return new StringBuilder()
578 .append("flush result:").append(result).append(", ")
579 .append("failureReason:").append(failureReason).append(",")
580 .append("flush seq id").append(flushSequenceId).toString();
583 @Override
584 public Result getResult() {
585 return result;
589 /** A result object from prepare flush cache stage */
590 protected static class PrepareFlushResult {
591 final FlushResultImpl result; // indicating a failure result from prepare
592 final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
593 final TreeMap<byte[], List<Path>> committedFiles;
594 final TreeMap<byte[], MemStoreSize> storeFlushableSize;
595 final long startTime;
596 final long flushOpSeqId;
597 final long flushedSeqId;
598 final MemStoreSizing totalFlushableSize;
600 /** Constructs an early exit case */
601 PrepareFlushResult(FlushResultImpl result, long flushSeqId) {
602 this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, MemStoreSizing.DUD);
605 /** Constructs a successful prepare flush result */
606 PrepareFlushResult(
607 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
608 TreeMap<byte[], List<Path>> committedFiles,
609 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId,
610 long flushedSeqId, MemStoreSizing totalFlushableSize) {
611 this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
612 flushSeqId, flushedSeqId, totalFlushableSize);
615 private PrepareFlushResult(
616 FlushResultImpl result,
617 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
618 TreeMap<byte[], List<Path>> committedFiles,
619 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId,
620 long flushedSeqId, MemStoreSizing totalFlushableSize) {
621 this.result = result;
622 this.storeFlushCtxs = storeFlushCtxs;
623 this.committedFiles = committedFiles;
624 this.storeFlushableSize = storeFlushableSize;
625 this.startTime = startTime;
626 this.flushOpSeqId = flushSeqId;
627 this.flushedSeqId = flushedSeqId;
628 this.totalFlushableSize = totalFlushableSize;
631 public FlushResult getResult() {
632 return this.result;
637 * A class that tracks exceptions that have been observed in one batch. Not thread safe.
639 static class ObservedExceptionsInBatch {
640 private boolean wrongRegion = false;
641 private boolean failedSanityCheck = false;
642 private boolean wrongFamily = false;
645 * @return If a {@link WrongRegionException} has been observed.
647 boolean hasSeenWrongRegion() {
648 return wrongRegion;
652 * Records that a {@link WrongRegionException} has been observed.
654 void sawWrongRegion() {
655 wrongRegion = true;
659 * @return If a {@link FailedSanityCheckException} has been observed.
661 boolean hasSeenFailedSanityCheck() {
662 return failedSanityCheck;
666 * Records that a {@link FailedSanityCheckException} has been observed.
668 void sawFailedSanityCheck() {
669 failedSanityCheck = true;
673 * @return If a {@link NoSuchColumnFamilyException} has been observed.
675 boolean hasSeenNoSuchFamily() {
676 return wrongFamily;
680 * Records that a {@link NoSuchColumnFamilyException} has been observed.
682 void sawNoSuchFamily() {
683 wrongFamily = true;
687 final WriteState writestate = new WriteState();
689 long memstoreFlushSize;
690 final long timestampSlop;
692 // Last flush time for each Store. Useful when we are flushing for each column
693 private final ConcurrentMap<HStore, Long> lastStoreFlushTimeMap = new ConcurrentHashMap<>();
695 protected RegionServerServices rsServices;
696 private RegionServerAccounting rsAccounting;
697 private long flushCheckInterval;
698 // flushPerChanges is to prevent too many changes in memstore
699 private long flushPerChanges;
700 private long blockingMemStoreSize;
701 // Used to guard closes
702 final ReentrantReadWriteLock lock;
703 // Used to track interruptible holders of the region lock. Currently that is only RPC handler
704 // threads. Boolean value in map determines if lock holder can be interrupted, normally true,
705 // but may be false when thread is transiting a critical section.
706 final ConcurrentHashMap<Thread, Boolean> regionLockHolders;
708 // Stop updates lock
709 private final ReentrantReadWriteLock updatesLock = new ReentrantReadWriteLock();
711 private final MultiVersionConcurrencyControl mvcc;
713 // Coprocessor host
714 private volatile RegionCoprocessorHost coprocessorHost;
716 private TableDescriptor htableDescriptor = null;
717 private RegionSplitPolicy splitPolicy;
718 private RegionSplitRestriction splitRestriction;
719 private FlushPolicy flushPolicy;
721 private final MetricsRegion metricsRegion;
722 private final MetricsRegionWrapperImpl metricsRegionWrapper;
723 private final Durability regionDurability;
724 private final boolean regionStatsEnabled;
725 // Stores the replication scope of the various column families of the table
726 // that has non-default scope
727 private final NavigableMap<byte[], Integer> replicationScope = new TreeMap<>(
728 Bytes.BYTES_COMPARATOR);
730 private final StoreHotnessProtector storeHotnessProtector;
732 protected Optional<RegionReplicationSink> regionReplicationSink = Optional.empty();
735 * HRegion constructor. This constructor should only be used for testing and
736 * extensions. Instances of HRegion should be instantiated with the
737 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
739 * @param tableDir qualified path of directory where region should be located,
740 * usually the table directory.
741 * @param wal The WAL is the outbound log for any updates to the HRegion
742 * The wal file is a logfile from the previous execution that's
743 * custom-computed for this HRegion. The HRegionServer computes and sorts the
744 * appropriate wal info for this HRegion. If there is a previous wal file
745 * (implying that the HRegion has been written-to before), then read it from
746 * the supplied path.
747 * @param fs is the filesystem.
748 * @param confParam is global configuration settings.
749 * @param regionInfo - RegionInfo that describes the region
750 * is new), then read them from the supplied path.
751 * @param htd the table descriptor
752 * @param rsServices reference to {@link RegionServerServices} or null
753 * @deprecated Use other constructors.
755 @Deprecated
756 public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
757 final Configuration confParam, final RegionInfo regionInfo,
758 final TableDescriptor htd, final RegionServerServices rsServices) {
759 this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
760 wal, confParam, htd, rsServices);
764 * HRegion constructor. This constructor should only be used for testing and
765 * extensions. Instances of HRegion should be instantiated with the
766 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
768 * @param fs is the filesystem.
769 * @param wal The WAL is the outbound log for any updates to the HRegion
770 * The wal file is a logfile from the previous execution that's
771 * custom-computed for this HRegion. The HRegionServer computes and sorts the
772 * appropriate wal info for this HRegion. If there is a previous wal file
773 * (implying that the HRegion has been written-to before), then read it from
774 * the supplied path.
775 * @param confParam is global configuration settings.
776 * @param htd the table descriptor
777 * @param rsServices reference to {@link RegionServerServices} or null
779 public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam,
780 final TableDescriptor htd, final RegionServerServices rsServices) {
781 if (htd == null) {
782 throw new IllegalArgumentException("Need table descriptor");
785 if (confParam instanceof CompoundConfiguration) {
786 throw new IllegalArgumentException("Need original base configuration");
789 this.wal = wal;
790 this.fs = fs;
791 this.mvcc = new MultiVersionConcurrencyControl(getRegionInfo().getShortNameToLog());
793 // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
794 this.baseConf = confParam;
795 this.conf = new CompoundConfiguration().add(confParam).addBytesMap(htd.getValues());
796 this.cellComparator = htd.isMetaTable() ||
797 conf.getBoolean(USE_META_CELL_COMPARATOR, DEFAULT_USE_META_CELL_COMPARATOR) ?
798 MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR;
799 this.lock = new ReentrantReadWriteLock(conf.getBoolean(FAIR_REENTRANT_CLOSE_LOCK,
800 DEFAULT_FAIR_REENTRANT_CLOSE_LOCK));
801 this.regionLockHolders = new ConcurrentHashMap<>();
802 this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL,
803 DEFAULT_CACHE_FLUSH_INTERVAL);
804 this.flushPerChanges = conf.getLong(MEMSTORE_FLUSH_PER_CHANGES, DEFAULT_FLUSH_PER_CHANGES);
805 if (this.flushPerChanges > MAX_FLUSH_PER_CHANGES) {
806 throw new IllegalArgumentException(MEMSTORE_FLUSH_PER_CHANGES + " can not exceed "
807 + MAX_FLUSH_PER_CHANGES);
809 int tmpRowLockDuration = conf.getInt("hbase.rowlock.wait.duration",
810 DEFAULT_ROWLOCK_WAIT_DURATION);
811 if (tmpRowLockDuration <= 0) {
812 LOG.info("Found hbase.rowlock.wait.duration set to {}. values <= 0 will cause all row " +
813 "locking to fail. Treating it as 1ms to avoid region failure.", tmpRowLockDuration);
814 tmpRowLockDuration = 1;
816 this.rowLockWaitDuration = tmpRowLockDuration;
818 this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
819 this.htableDescriptor = htd;
820 Set<byte[]> families = this.htableDescriptor.getColumnFamilyNames();
821 for (byte[] family : families) {
822 if (!replicationScope.containsKey(family)) {
823 int scope = htd.getColumnFamily(family).getScope();
824 // Only store those families that has NON-DEFAULT scope
825 if (scope != REPLICATION_SCOPE_LOCAL) {
826 // Do a copy before storing it here.
827 replicationScope.put(Bytes.copy(family), scope);
832 this.rsServices = rsServices;
833 if (this.rsServices != null) {
834 this.blockCache = rsServices.getBlockCache().orElse(null);
835 this.mobFileCache = rsServices.getMobFileCache().orElse(null);
837 this.regionServicesForStores = new RegionServicesForStores(this, rsServices);
839 setHTableSpecificConf();
840 this.scannerReadPoints = new ConcurrentHashMap<>();
842 this.busyWaitDuration = conf.getLong(
843 "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
844 this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
845 if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
846 throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
847 + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
848 + maxBusyWaitMultiplier + "). Their product should be positive");
850 this.maxBusyWaitDuration = conf.getLong("hbase.ipc.client.call.purge.timeout",
851 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
854 * timestamp.slop provides a server-side constraint on the timestamp. This
855 * assumes that you base your TS around EnvironmentEdgeManager.currentTime(). In this case,
856 * throw an error to the user if the user-specified TS is newer than now +
857 * slop. LATEST_TIMESTAMP == don't use this functionality
859 this.timestampSlop = conf.getLong(
860 "hbase.hregion.keyvalue.timestamp.slop.millisecs",
861 HConstants.LATEST_TIMESTAMP);
863 this.storeHotnessProtector = new StoreHotnessProtector(this, conf);
865 boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC);
867 * This is the global default value for durability. All tables/mutations not defining a
868 * durability or using USE_DEFAULT will default to this value.
870 Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL;
871 this.regionDurability =
872 this.htableDescriptor.getDurability() == Durability.USE_DEFAULT ? defaultDurability :
873 this.htableDescriptor.getDurability();
875 decorateRegionConfiguration(conf);
876 if (rsServices != null) {
877 this.rsAccounting = this.rsServices.getRegionServerAccounting();
878 // don't initialize coprocessors if not running within a regionserver
879 // TODO: revisit if coprocessors should load in other cases
880 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
881 this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
882 this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper, conf);
883 } else {
884 this.metricsRegionWrapper = null;
885 this.metricsRegion = null;
887 if (LOG.isDebugEnabled()) {
888 // Write out region name, its encoded name and storeHotnessProtector as string.
889 LOG.debug("Instantiated " + this +"; "+ storeHotnessProtector.toString());
892 configurationManager = null;
894 // disable stats tracking system tables, but check the config for everything else
895 this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals(
896 NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) ?
897 false :
898 conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
899 HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
901 this.maxCellSize = conf.getLong(HBASE_MAX_CELL_SIZE_KEY, DEFAULT_MAX_CELL_SIZE);
902 this.miniBatchSize = conf.getInt(HBASE_REGIONSERVER_MINIBATCH_SIZE,
903 DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE);
905 // recover the metrics of read and write requests count if they were retained
906 if (rsServices != null && rsServices.getRegionServerAccounting() != null) {
907 Pair<Long, Long> retainedRWRequestsCnt = rsServices.getRegionServerAccounting()
908 .getRetainedRegionRWRequestsCnt().get(getRegionInfo().getEncodedName());
909 if (retainedRWRequestsCnt != null) {
910 this.addReadRequestsCount(retainedRWRequestsCnt.getFirst());
911 this.addWriteRequestsCount(retainedRWRequestsCnt.getSecond());
912 // remove them since won't use again
913 rsServices.getRegionServerAccounting().getRetainedRegionRWRequestsCnt()
914 .remove(getRegionInfo().getEncodedName());
919 private void setHTableSpecificConf() {
920 if (this.htableDescriptor == null) {
921 return;
923 long flushSize = this.htableDescriptor.getMemStoreFlushSize();
925 if (flushSize <= 0) {
926 flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
927 TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE);
929 this.memstoreFlushSize = flushSize;
930 long mult = conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,
931 HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER);
932 this.blockingMemStoreSize = this.memstoreFlushSize * mult;
936 * Initialize this region.
937 * Used only by tests and SplitTransaction to reopen the region.
938 * You should use createHRegion() or openHRegion()
939 * @return What the next sequence (edit) id should be.
940 * @throws IOException e
941 * @deprecated use HRegion.createHRegion() or HRegion.openHRegion()
943 @Deprecated
944 public long initialize() throws IOException {
945 return initialize(null);
949 * Initialize this region.
951 * @param reporter Tickle every so often if initialize is taking a while.
952 * @return What the next sequence (edit) id should be.
953 * @throws IOException e
955 long initialize(final CancelableProgressable reporter) throws IOException {
957 //Refuse to open the region if there is no column family in the table
958 if (htableDescriptor.getColumnFamilyCount() == 0) {
959 throw new DoNotRetryIOException("Table " + htableDescriptor.getTableName().getNameAsString()+
960 " should have at least one column family.");
963 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
964 status.enableStatusJournal(true);
965 long nextSeqId = -1;
966 try {
967 nextSeqId = initializeRegionInternals(reporter, status);
968 return nextSeqId;
969 } catch (IOException e) {
970 LOG.warn("Failed initialize of region= {}, starting to roll back memstore",
971 getRegionInfo().getRegionNameAsString(), e);
972 // global memstore size will be decreased when dropping memstore
973 try {
974 //drop the memory used by memstore if open region fails
975 dropMemStoreContents();
976 } catch (IOException ioE) {
977 if (conf.getBoolean(MemStoreLAB.USEMSLAB_KEY, MemStoreLAB.USEMSLAB_DEFAULT)) {
978 LOG.warn("Failed drop memstore of region= {}, "
979 + "some chunks may not released forever since MSLAB is enabled",
980 getRegionInfo().getRegionNameAsString());
984 throw e;
985 } finally {
986 // nextSeqid will be -1 if the initialization fails.
987 // At least it will be 0 otherwise.
988 if (nextSeqId == -1) {
989 status.abort("Exception during region " + getRegionInfo().getRegionNameAsString() +
990 " initialization.");
992 if (LOG.isDebugEnabled()) {
993 LOG.debug("Region open journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
994 status.prettyPrintJournal());
996 status.cleanup();
1000 private long initializeRegionInternals(final CancelableProgressable reporter,
1001 final MonitoredTask status) throws IOException {
1002 if (coprocessorHost != null) {
1003 status.setStatus("Running coprocessor pre-open hook");
1004 coprocessorHost.preOpen();
1007 // Write HRI to a file in case we need to recover hbase:meta
1008 // Only the primary replica should write .regioninfo
1009 if (this.getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
1010 status.setStatus("Writing region info on filesystem");
1011 fs.checkRegionInfoOnFilesystem();
1014 // Initialize all the HStores
1015 status.setStatus("Initializing all the Stores");
1016 long maxSeqId = initializeStores(reporter, status);
1017 this.mvcc.advanceTo(maxSeqId);
1018 if (!isRestoredRegion && ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
1019 Collection<HStore> stores = this.stores.values();
1020 try {
1021 // update the stores that we are replaying
1022 LOG.debug("replaying wal for " + this.getRegionInfo().getEncodedName());
1023 stores.forEach(HStore::startReplayingFromWAL);
1024 // Recover any edits if available.
1025 maxSeqId = Math.max(maxSeqId,
1026 replayRecoveredEditsIfAny(maxSeqIdInStores, reporter, status));
1027 // Recover any hfiles if available
1028 maxSeqId = Math.max(maxSeqId, loadRecoveredHFilesIfAny(stores));
1029 // Make sure mvcc is up to max.
1030 this.mvcc.advanceTo(maxSeqId);
1031 } finally {
1032 LOG.debug("stopping wal replay for " + this.getRegionInfo().getEncodedName());
1033 // update the stores that we are done replaying
1034 stores.forEach(HStore::stopReplayingFromWAL);
1037 this.lastReplayedOpenRegionSeqId = maxSeqId;
1039 this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this));
1040 this.writestate.flushRequested = false;
1041 this.writestate.compacting.set(0);
1043 if (this.writestate.writesEnabled) {
1044 LOG.debug("Cleaning up temporary data for " + this.getRegionInfo().getEncodedName());
1045 // Remove temporary data left over from old regions
1046 status.setStatus("Cleaning up temporary data from old regions");
1047 fs.cleanupTempDir();
1050 // Initialize split policy
1051 this.splitPolicy = RegionSplitPolicy.create(this, conf);
1053 // Initialize split restriction
1054 splitRestriction = RegionSplitRestriction.create(getTableDescriptor(), conf);
1056 // Initialize flush policy
1057 this.flushPolicy = FlushPolicyFactory.create(this, conf);
1059 long lastFlushTime = EnvironmentEdgeManager.currentTime();
1060 for (HStore store: stores.values()) {
1061 this.lastStoreFlushTimeMap.put(store, lastFlushTime);
1064 // Use maximum of log sequenceid or that which was found in stores
1065 // (particularly if no recovered edits, seqid will be -1).
1066 long nextSeqId = maxSeqId + 1;
1067 if (!isRestoredRegion) {
1068 // always get openSeqNum from the default replica, even if we are secondary replicas
1069 long maxSeqIdFromFile = WALSplitUtil.getMaxRegionSequenceId(conf,
1070 RegionReplicaUtil.getRegionInfoForDefaultReplica(getRegionInfo()), this::getFilesystem,
1071 this::getWalFileSystem);
1072 nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
1073 // The openSeqNum will always be increase even for read only region, as we rely on it to
1074 // determine whether a region has been successfully reopened, so here we always need to update
1075 // the max sequence id file.
1076 if (RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
1077 LOG.debug("writing seq id for {}", this.getRegionInfo().getEncodedName());
1078 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
1079 nextSeqId - 1);
1080 // This means we have replayed all the recovered edits and also written out the max sequence
1081 // id file, let's delete the wrong directories introduced in HBASE-20734, see HBASE-22617
1082 // for more details.
1083 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf,
1084 getRegionInfo().getTable(), getRegionInfo().getEncodedName());
1085 FileSystem walFs = getWalFileSystem();
1086 if (walFs.exists(wrongRegionWALDir)) {
1087 if (!walFs.delete(wrongRegionWALDir, true)) {
1088 LOG.debug("Failed to clean up wrong region WAL directory {}", wrongRegionWALDir);
1091 } else {
1092 lastReplayedSequenceId = nextSeqId - 1;
1093 replayLock = new ReentrantLock();
1095 initializeRegionReplicationSink(reporter, status);
1098 LOG.info("Opened {}; next sequenceid={}; {}, {}", this.getRegionInfo().getShortNameToLog(),
1099 nextSeqId, this.splitPolicy, this.flushPolicy);
1101 // A region can be reopened if failed a split; reset flags
1102 this.closing.set(false);
1103 this.closed.set(false);
1105 if (coprocessorHost != null) {
1106 LOG.debug("Running coprocessor post-open hooks for " + this.getRegionInfo().getEncodedName());
1107 status.setStatus("Running coprocessor post-open hooks");
1108 coprocessorHost.postOpen();
1110 status.markComplete("Region opened successfully");
1111 return nextSeqId;
1114 private void initializeRegionReplicationSink(CancelableProgressable reporter,
1115 MonitoredTask status) {
1116 RegionServerServices rss = getRegionServerServices();
1117 TableDescriptor td = getTableDescriptor();
1118 int regionReplication = td.getRegionReplication();
1119 RegionInfo regionInfo = getRegionInfo();
1120 if (regionReplication <= 1 || !RegionReplicaUtil.isDefaultReplica(regionInfo) ||
1121 !ServerRegionReplicaUtil.isRegionReplicaReplicationEnabled(conf, regionInfo.getTable()) ||
1122 rss == null) {
1123 regionReplicationSink = Optional.empty();
1124 return;
1126 status.setStatus("Initializaing region replication sink");
1127 regionReplicationSink = Optional.of(new RegionReplicationSink(conf, regionInfo, td,
1128 rss.getRegionReplicationBufferManager(), () -> rss.getFlushRequester().requestFlush(this,
1129 new ArrayList<>(td.getColumnFamilyNames()), FlushLifeCycleTracker.DUMMY),
1130 rss.getAsyncClusterConnection()));
1134 * Open all Stores.
1135 * @param reporter
1136 * @param status
1137 * @return Highest sequenceId found out in a Store.
1138 * @throws IOException
1140 private long initializeStores(CancelableProgressable reporter, MonitoredTask status)
1141 throws IOException {
1142 return initializeStores(reporter, status, false);
1145 private long initializeStores(CancelableProgressable reporter, MonitoredTask status,
1146 boolean warmup) throws IOException {
1147 // Load in all the HStores.
1148 long maxSeqId = -1;
1149 // initialized to -1 so that we pick up MemstoreTS from column families
1150 long maxMemstoreTS = -1;
1152 if (htableDescriptor.getColumnFamilyCount() != 0) {
1153 // initialize the thread pool for opening stores in parallel.
1154 ThreadPoolExecutor storeOpenerThreadPool =
1155 getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog());
1156 CompletionService<HStore> completionService = new ExecutorCompletionService<>(storeOpenerThreadPool);
1158 // initialize each store in parallel
1159 for (final ColumnFamilyDescriptor family : htableDescriptor.getColumnFamilies()) {
1160 status.setStatus("Instantiating store for column family " + family);
1161 completionService.submit(new Callable<HStore>() {
1162 @Override
1163 public HStore call() throws IOException {
1164 return instantiateHStore(family, warmup);
1168 boolean allStoresOpened = false;
1169 boolean hasSloppyStores = false;
1170 try {
1171 for (int i = 0; i < htableDescriptor.getColumnFamilyCount(); i++) {
1172 Future<HStore> future = completionService.take();
1173 HStore store = future.get();
1174 this.stores.put(store.getColumnFamilyDescriptor().getName(), store);
1175 if (store.isSloppyMemStore()) {
1176 hasSloppyStores = true;
1179 long storeMaxSequenceId = store.getMaxSequenceId().orElse(0L);
1180 maxSeqIdInStores.put(Bytes.toBytes(store.getColumnFamilyName()),
1181 storeMaxSequenceId);
1182 if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) {
1183 maxSeqId = storeMaxSequenceId;
1185 long maxStoreMemstoreTS = store.getMaxMemStoreTS().orElse(0L);
1186 if (maxStoreMemstoreTS > maxMemstoreTS) {
1187 maxMemstoreTS = maxStoreMemstoreTS;
1190 allStoresOpened = true;
1191 if(hasSloppyStores) {
1192 htableDescriptor = TableDescriptorBuilder.newBuilder(htableDescriptor)
1193 .setFlushPolicyClassName(FlushNonSloppyStoresFirstPolicy.class.getName())
1194 .build();
1195 LOG.info("Setting FlushNonSloppyStoresFirstPolicy for the region=" + this);
1197 } catch (InterruptedException e) {
1198 throw throwOnInterrupt(e);
1199 } catch (ExecutionException e) {
1200 throw new IOException(e.getCause());
1201 } finally {
1202 storeOpenerThreadPool.shutdownNow();
1203 if (!allStoresOpened) {
1204 // something went wrong, close all opened stores
1205 LOG.error("Could not initialize all stores for the region=" + this);
1206 for (HStore store : this.stores.values()) {
1207 try {
1208 store.close();
1209 } catch (IOException e) {
1210 LOG.warn("close store {} failed in region {}", store.toString(), this, e);
1216 return Math.max(maxSeqId, maxMemstoreTS + 1);
1219 private void initializeWarmup(final CancelableProgressable reporter) throws IOException {
1220 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
1221 // Initialize all the HStores
1222 status.setStatus("Warmup all stores of " + this.getRegionInfo().getRegionNameAsString());
1223 try {
1224 initializeStores(reporter, status, true);
1225 } finally {
1226 status.markComplete("Warmed up " + this.getRegionInfo().getRegionNameAsString());
1231 * @return Map of StoreFiles by column family
1233 private NavigableMap<byte[], List<Path>> getStoreFiles() {
1234 NavigableMap<byte[], List<Path>> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1235 for (HStore store : stores.values()) {
1236 Collection<HStoreFile> storeFiles = store.getStorefiles();
1237 if (storeFiles == null) {
1238 continue;
1240 List<Path> storeFileNames = new ArrayList<>();
1241 for (HStoreFile storeFile : storeFiles) {
1242 storeFileNames.add(storeFile.getPath());
1244 allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames);
1246 return allStoreFiles;
1249 protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
1250 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1251 RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
1252 RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId,
1253 getRegionServerServices().getServerName(), storeFiles);
1254 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionOpenDesc,
1255 mvcc, regionReplicationSink.orElse(null));
1258 private void writeRegionCloseMarker(WAL wal) throws IOException {
1259 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1260 RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor(
1261 RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(),
1262 getRegionServerServices().getServerName(), storeFiles);
1263 // we do not care region close event at secondary replica side so just pass a null
1264 // RegionReplicationSink
1265 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionEventDesc,
1266 mvcc, null);
1268 // Store SeqId in WAL FileSystem when a region closes
1269 // checking region folder exists is due to many tests which delete the table folder while a
1270 // table is still online
1271 if (getWalFileSystem().exists(getWALRegionDir())) {
1272 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
1273 mvcc.getReadPoint());
1278 * @return True if this region has references.
1280 public boolean hasReferences() {
1281 return stores.values().stream().anyMatch(HStore::hasReferences);
1284 public void blockUpdates() {
1285 this.updatesLock.writeLock().lock();
1288 public void unblockUpdates() {
1289 this.updatesLock.writeLock().unlock();
1292 public HDFSBlocksDistribution getHDFSBlocksDistribution() {
1293 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1294 stores.values().stream().filter(s -> s.getStorefiles() != null)
1295 .flatMap(s -> s.getStorefiles().stream()).map(HStoreFile::getHDFSBlockDistribution)
1296 .forEachOrdered(hdfsBlocksDistribution::add);
1297 return hdfsBlocksDistribution;
1301 * This is a helper function to compute HDFS block distribution on demand
1302 * @param conf configuration
1303 * @param tableDescriptor TableDescriptor of the table
1304 * @param regionInfo encoded name of the region
1305 * @return The HDFS blocks distribution for the given region.
1307 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
1308 TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
1309 Path tablePath =
1310 CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), tableDescriptor.getTableName());
1311 return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
1315 * This is a helper function to compute HDFS block distribution on demand
1316 * @param conf configuration
1317 * @param tableDescriptor TableDescriptor of the table
1318 * @param regionInfo encoded name of the region
1319 * @param tablePath the table directory
1320 * @return The HDFS blocks distribution for the given region.
1321 * @throws IOException
1323 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
1324 TableDescriptor tableDescriptor, RegionInfo regionInfo, Path tablePath) throws IOException {
1325 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1326 FileSystem fs = tablePath.getFileSystem(conf);
1328 HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
1329 for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()) {
1330 List<LocatedFileStatus> locatedFileStatusList = HRegionFileSystem
1331 .getStoreFilesLocatedStatus(regionFs, family.getNameAsString(), true);
1332 if (locatedFileStatusList == null) {
1333 continue;
1336 for (LocatedFileStatus status : locatedFileStatusList) {
1337 Path p = status.getPath();
1338 if (StoreFileInfo.isReference(p) || HFileLink.isHFileLink(p)) {
1339 // Only construct StoreFileInfo object if its not a hfile, save obj
1340 // creation
1341 StoreFileInfo storeFileInfo = new StoreFileInfo(conf, fs, status);
1342 hdfsBlocksDistribution.add(storeFileInfo
1343 .computeHDFSBlocksDistribution(fs));
1344 } else if (StoreFileInfo.isHFile(p)) {
1345 // If its a HFile, then lets just add to the block distribution
1346 // lets not create more objects here, not even another HDFSBlocksDistribution
1347 FSUtils.addToHDFSBlocksDistribution(hdfsBlocksDistribution,
1348 status.getBlockLocations());
1349 } else {
1350 throw new IOException("path=" + p
1351 + " doesn't look like a valid StoreFile");
1355 return hdfsBlocksDistribution;
1359 * Increase the size of mem store in this region and the size of global mem
1360 * store
1362 private void incMemStoreSize(MemStoreSize mss) {
1363 incMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(),
1364 mss.getCellsCount());
1367 void incMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta,
1368 int cellsCountDelta) {
1369 if (this.rsAccounting != null) {
1370 rsAccounting.incGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta);
1372 long dataSize = this.memStoreSizing.incMemStoreSize(dataSizeDelta, heapSizeDelta,
1373 offHeapSizeDelta, cellsCountDelta);
1374 checkNegativeMemStoreDataSize(dataSize, dataSizeDelta);
1377 void decrMemStoreSize(MemStoreSize mss) {
1378 decrMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(),
1379 mss.getCellsCount());
1382 private void decrMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta,
1383 int cellsCountDelta) {
1384 if (this.rsAccounting != null) {
1385 rsAccounting.decGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta);
1387 long dataSize = this.memStoreSizing.decMemStoreSize(dataSizeDelta, heapSizeDelta,
1388 offHeapSizeDelta, cellsCountDelta);
1389 checkNegativeMemStoreDataSize(dataSize, -dataSizeDelta);
1392 private void checkNegativeMemStoreDataSize(long memStoreDataSize, long delta) {
1393 // This is extremely bad if we make memStoreSizing negative. Log as much info on the offending
1394 // caller as possible. (memStoreSizing might be a negative value already -- freeing memory)
1395 if (memStoreDataSize < 0) {
1396 LOG.error("Asked to modify this region's (" + this.toString()
1397 + ") memStoreSizing to a negative value which is incorrect. Current memStoreSizing="
1398 + (memStoreDataSize - delta) + ", delta=" + delta, new Exception());
1402 @Override
1403 public RegionInfo getRegionInfo() {
1404 return this.fs.getRegionInfo();
1408 * @return Instance of {@link RegionServerServices} used by this HRegion.
1409 * Can be null.
1411 RegionServerServices getRegionServerServices() {
1412 return this.rsServices;
1415 @Override
1416 public long getReadRequestsCount() {
1417 return readRequestsCount.sum();
1420 @Override
1421 public long getCpRequestsCount() {
1422 return cpRequestsCount.sum();
1425 @Override
1426 public long getFilteredReadRequestsCount() {
1427 return filteredReadRequestsCount.sum();
1430 @Override
1431 public long getWriteRequestsCount() {
1432 return writeRequestsCount.sum();
1435 @Override
1436 public long getMemStoreDataSize() {
1437 return memStoreSizing.getDataSize();
1440 @Override
1441 public long getMemStoreHeapSize() {
1442 return memStoreSizing.getHeapSize();
1445 @Override
1446 public long getMemStoreOffHeapSize() {
1447 return memStoreSizing.getOffHeapSize();
1450 /** @return store services for this region, to access services required by store level needs */
1451 public RegionServicesForStores getRegionServicesForStores() {
1452 return regionServicesForStores;
1455 @Override
1456 public long getNumMutationsWithoutWAL() {
1457 return numMutationsWithoutWAL.sum();
1460 @Override
1461 public long getDataInMemoryWithoutWAL() {
1462 return dataInMemoryWithoutWAL.sum();
1465 @Override
1466 public long getBlockedRequestsCount() {
1467 return blockedRequestsCount.sum();
1470 @Override
1471 public long getCheckAndMutateChecksPassed() {
1472 return checkAndMutateChecksPassed.sum();
1475 @Override
1476 public long getCheckAndMutateChecksFailed() {
1477 return checkAndMutateChecksFailed.sum();
1480 // TODO Needs to check whether we should expose our metrics system to CPs. If CPs themselves doing
1481 // the op and bypassing the core, this might be needed? Should be stop supporting the bypass
1482 // feature?
1483 public MetricsRegion getMetrics() {
1484 return metricsRegion;
1487 @Override
1488 public boolean isClosed() {
1489 return this.closed.get();
1492 @Override
1493 public boolean isClosing() {
1494 return this.closing.get();
1497 @Override
1498 public boolean isReadOnly() {
1499 return this.writestate.isReadOnly();
1502 @Override
1503 public boolean isAvailable() {
1504 return !isClosed() && !isClosing();
1507 @Override
1508 public boolean isSplittable() {
1509 return splitPolicy.canSplit();
1512 @Override
1513 public boolean isMergeable() {
1514 if (!isAvailable()) {
1515 LOG.debug("Region " + this
1516 + " is not mergeable because it is closing or closed");
1517 return false;
1519 if (hasReferences()) {
1520 LOG.debug("Region " + this
1521 + " is not mergeable because it has references");
1522 return false;
1525 return true;
1528 public boolean areWritesEnabled() {
1529 synchronized(this.writestate) {
1530 return this.writestate.writesEnabled;
1534 public MultiVersionConcurrencyControl getMVCC() {
1535 return mvcc;
1538 @Override
1539 public long getMaxFlushedSeqId() {
1540 return maxFlushedSeqId;
1544 * @return readpoint considering given IsolationLevel. Pass {@code null} for default
1546 public long getReadPoint(IsolationLevel isolationLevel) {
1547 if (isolationLevel != null && isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
1548 // This scan can read even uncommitted transactions
1549 return Long.MAX_VALUE;
1551 return mvcc.getReadPoint();
1554 public boolean isLoadingCfsOnDemandDefault() {
1555 return this.isLoadingCfsOnDemandDefault;
1559 * Close down this HRegion. Flush the cache, shut down each HStore, don't
1560 * service any more calls.
1562 * <p>This method could take some time to execute, so don't call it from a
1563 * time-sensitive thread.
1565 * @return Vector of all the storage files that the HRegion's component
1566 * HStores make use of. It's a list of all StoreFile objects. Returns empty
1567 * vector if already closed and null if judged that it should not close.
1569 * @throws IOException e
1570 * @throws DroppedSnapshotException Thrown when replay of wal is required
1571 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1572 * caller MUST abort after this.
1574 public Map<byte[], List<HStoreFile>> close() throws IOException {
1575 return close(false);
1578 private final Object closeLock = new Object();
1580 /** Conf key for fair locking policy */
1581 public static final String FAIR_REENTRANT_CLOSE_LOCK =
1582 "hbase.regionserver.fair.region.close.lock";
1583 public static final boolean DEFAULT_FAIR_REENTRANT_CLOSE_LOCK = true;
1584 /** Conf key for the periodic flush interval */
1585 public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL =
1586 "hbase.regionserver.optionalcacheflushinterval";
1587 /** Default interval for the memstore flush */
1588 public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
1589 /** Default interval for System tables memstore flush */
1590 public static final int SYSTEM_CACHE_FLUSH_INTERVAL = 300000; // 5 minutes
1592 /** Conf key to force a flush if there are already enough changes for one region in memstore */
1593 public static final String MEMSTORE_FLUSH_PER_CHANGES =
1594 "hbase.regionserver.flush.per.changes";
1595 public static final long DEFAULT_FLUSH_PER_CHANGES = 30000000; // 30 millions
1597 * The following MAX_FLUSH_PER_CHANGES is large enough because each KeyValue has 20+ bytes
1598 * overhead. Therefore, even 1G empty KVs occupy at least 20GB memstore size for a single region
1600 public static final long MAX_FLUSH_PER_CHANGES = 1000000000; // 1G
1602 public static final String CLOSE_WAIT_ABORT = "hbase.regionserver.close.wait.abort";
1603 public static final boolean DEFAULT_CLOSE_WAIT_ABORT = true;
1604 public static final String CLOSE_WAIT_TIME = "hbase.regionserver.close.wait.time.ms";
1605 public static final long DEFAULT_CLOSE_WAIT_TIME = 60000; // 1 minute
1606 public static final String CLOSE_WAIT_INTERVAL = "hbase.regionserver.close.wait.interval.ms";
1607 public static final long DEFAULT_CLOSE_WAIT_INTERVAL = 10000; // 10 seconds
1609 public Map<byte[], List<HStoreFile>> close(boolean abort) throws IOException {
1610 return close(abort, false);
1614 * Close down this HRegion. Flush the cache unless abort parameter is true,
1615 * Shut down each HStore, don't service any more calls.
1617 * This method could take some time to execute, so don't call it from a
1618 * time-sensitive thread.
1620 * @param abort true if server is aborting (only during testing)
1621 * @param ignoreStatus true if ignore the status (wont be showed on task list)
1622 * @return Vector of all the storage files that the HRegion's component
1623 * HStores make use of. It's a list of StoreFile objects. Can be null if
1624 * we are not to close at this time or we are already closed.
1626 * @throws IOException e
1627 * @throws DroppedSnapshotException Thrown when replay of wal is required
1628 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1629 * caller MUST abort after this.
1631 public Map<byte[], List<HStoreFile>> close(boolean abort, boolean ignoreStatus)
1632 throws IOException {
1633 // Only allow one thread to close at a time. Serialize them so dual
1634 // threads attempting to close will run up against each other.
1635 MonitoredTask status = TaskMonitor.get().createStatus(
1636 "Closing region " + this.getRegionInfo().getEncodedName() +
1637 (abort ? " due to abort" : ""), ignoreStatus);
1638 status.enableStatusJournal(true);
1639 status.setStatus("Waiting for close lock");
1640 try {
1641 synchronized (closeLock) {
1642 return doClose(abort, status);
1644 } finally {
1645 if (LOG.isDebugEnabled()) {
1646 LOG.debug("Region close journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
1647 status.prettyPrintJournal());
1649 status.cleanup();
1654 * Exposed for some very specific unit tests.
1656 public void setClosing(boolean closing) {
1657 this.closing.set(closing);
1661 * The {@link HRegion#doClose} will block forever if someone tries proving the dead lock via the unit test.
1662 * Instead of blocking, the {@link HRegion#doClose} will throw exception if you set the timeout.
1663 * @param timeoutForWriteLock the second time to wait for the write lock in {@link HRegion#doClose}
1665 public void setTimeoutForWriteLock(long timeoutForWriteLock) {
1666 assert timeoutForWriteLock >= 0;
1667 this.timeoutForWriteLock = timeoutForWriteLock;
1670 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
1671 justification="I think FindBugs is confused")
1672 private Map<byte[], List<HStoreFile>> doClose(boolean abort, MonitoredTask status)
1673 throws IOException {
1674 if (isClosed()) {
1675 LOG.warn("Region " + this + " already closed");
1676 return null;
1679 if (coprocessorHost != null) {
1680 status.setStatus("Running coprocessor pre-close hooks");
1681 this.coprocessorHost.preClose(abort);
1683 status.setStatus("Disabling compacts and flushes for region");
1684 boolean canFlush = true;
1685 synchronized (writestate) {
1686 // Disable compacting and flushing by background threads for this
1687 // region.
1688 canFlush = !writestate.readOnly;
1689 writestate.writesEnabled = false;
1690 LOG.debug("Closing {}, disabling compactions & flushes",
1691 this.getRegionInfo().getEncodedName());
1692 waitForFlushesAndCompactions();
1694 // If we were not just flushing, is it worth doing a preflush...one
1695 // that will clear out of the bulk of the memstore before we put up
1696 // the close flag?
1697 if (!abort && worthPreFlushing() && canFlush) {
1698 status.setStatus("Pre-flushing region before close");
1699 LOG.info("Running close preflush of {}", this.getRegionInfo().getEncodedName());
1700 try {
1701 internalFlushcache(status);
1702 } catch (IOException ioe) {
1703 // Failed to flush the region. Keep going.
1704 status.setStatus("Failed pre-flush " + this + "; " + ioe.getMessage());
1707 if (regionReplicationSink.isPresent()) {
1708 // stop replicating to secondary replicas
1709 // the open event marker can make secondary replicas refresh store files and catch up
1710 // everything, so here we just give up replicating later edits, to speed up the reopen process
1711 RegionReplicationSink sink = regionReplicationSink.get();
1712 sink.stop();
1713 try {
1714 regionReplicationSink.get().waitUntilStopped();
1715 } catch (InterruptedException e) {
1716 throw throwOnInterrupt(e);
1719 // Set the closing flag
1720 // From this point new arrivals at the region lock will get NSRE.
1722 this.closing.set(true);
1723 LOG.info("Closing region {}", this);
1725 // Acquire the close lock
1727 // The configuration parameter CLOSE_WAIT_ABORT is overloaded to enable both
1728 // the new regionserver abort condition and interrupts for running requests.
1729 // If CLOSE_WAIT_ABORT is not enabled there is no change from earlier behavior,
1730 // we will not attempt to interrupt threads servicing requests nor crash out
1731 // the regionserver if something remains stubborn.
1733 final boolean canAbort = conf.getBoolean(CLOSE_WAIT_ABORT, DEFAULT_CLOSE_WAIT_ABORT);
1734 boolean useTimedWait = false;
1735 if (timeoutForWriteLock != null && timeoutForWriteLock != Long.MAX_VALUE) {
1736 // convert legacy use of timeoutForWriteLock in seconds to new use in millis
1737 timeoutForWriteLock = TimeUnit.SECONDS.toMillis(timeoutForWriteLock);
1738 useTimedWait = true;
1739 } else if (canAbort) {
1740 timeoutForWriteLock = conf.getLong(CLOSE_WAIT_TIME, DEFAULT_CLOSE_WAIT_TIME);
1741 useTimedWait = true;
1743 if (LOG.isDebugEnabled()) {
1744 LOG.debug((useTimedWait ? "Time limited wait" : "Waiting without time limit") +
1745 " for close lock on " + this);
1747 final long closeWaitInterval = conf.getLong(CLOSE_WAIT_INTERVAL, DEFAULT_CLOSE_WAIT_INTERVAL);
1748 long elapsedWaitTime = 0;
1749 if (useTimedWait) {
1750 // Sanity check configuration
1751 long remainingWaitTime = timeoutForWriteLock;
1752 if (remainingWaitTime < closeWaitInterval) {
1753 LOG.warn("Time limit for close wait of " + timeoutForWriteLock +
1754 " ms is less than the configured lock acquisition wait interval " +
1755 closeWaitInterval + " ms, using wait interval as time limit");
1756 remainingWaitTime = closeWaitInterval;
1758 boolean acquired = false;
1759 do {
1760 long start = EnvironmentEdgeManager.currentTime();
1761 try {
1762 acquired = lock.writeLock().tryLock(Math.min(remainingWaitTime, closeWaitInterval),
1763 TimeUnit.MILLISECONDS);
1764 } catch (InterruptedException e) {
1765 // Interrupted waiting for close lock. More likely the server is shutting down, not
1766 // normal operation, so aborting upon interrupt while waiting on this lock would not
1767 // provide much value. Throw an IOE (as IIOE) like we would in the case where we
1768 // fail to acquire the lock.
1769 String msg = "Interrupted while waiting for close lock on " + this;
1770 LOG.warn(msg, e);
1771 throw (InterruptedIOException) new InterruptedIOException(msg).initCause(e);
1773 long elapsed = EnvironmentEdgeManager.currentTime() - start;
1774 elapsedWaitTime += elapsed;
1775 remainingWaitTime -= elapsed;
1776 if (canAbort && !acquired && remainingWaitTime > 0) {
1777 // Before we loop to wait again, interrupt all region operations that might
1778 // still be in progress, to encourage them to break out of waiting states or
1779 // inner loops, throw an exception to clients, and release the read lock via
1780 // endRegionOperation.
1781 if (LOG.isDebugEnabled()) {
1782 LOG.debug("Interrupting region operations after waiting for close lock for " +
1783 elapsedWaitTime + " ms on " + this + ", " + remainingWaitTime +
1784 " ms remaining");
1786 interruptRegionOperations();
1788 } while (!acquired && remainingWaitTime > 0);
1790 // If we fail to acquire the lock, trigger an abort if we can; otherwise throw an IOE
1791 // to let the caller know we could not proceed with the close.
1792 if (!acquired) {
1793 String msg = "Failed to acquire close lock on " + this + " after waiting " +
1794 elapsedWaitTime + " ms";
1795 LOG.error(msg);
1796 if (canAbort) {
1797 // If we failed to acquire the write lock, abort the server
1798 rsServices.abort(msg, null);
1800 throw new IOException(msg);
1803 } else {
1805 long start = EnvironmentEdgeManager.currentTime();
1806 lock.writeLock().lock();
1807 elapsedWaitTime = EnvironmentEdgeManager.currentTime() - start;
1811 if (LOG.isDebugEnabled()) {
1812 LOG.debug("Acquired close lock on " + this + " after waiting " +
1813 elapsedWaitTime + " ms");
1816 status.setStatus("Disabling writes for close");
1817 try {
1818 if (this.isClosed()) {
1819 status.abort("Already got closed by another process");
1820 // SplitTransaction handles the null
1821 return null;
1823 LOG.debug("Updates disabled for region " + this);
1824 // Don't flush the cache if we are aborting
1825 if (!abort && canFlush) {
1826 int failedfFlushCount = 0;
1827 int flushCount = 0;
1828 long tmp = 0;
1829 long remainingSize = this.memStoreSizing.getDataSize();
1830 while (remainingSize > 0) {
1831 try {
1832 internalFlushcache(status);
1833 if(flushCount >0) {
1834 LOG.info("Running extra flush, " + flushCount +
1835 " (carrying snapshot?) " + this);
1837 flushCount++;
1838 tmp = this.memStoreSizing.getDataSize();
1839 if (tmp >= remainingSize) {
1840 failedfFlushCount++;
1842 remainingSize = tmp;
1843 if (failedfFlushCount > 5) {
1844 // If we failed 5 times and are unable to clear memory, abort
1845 // so we do not lose data
1846 throw new DroppedSnapshotException("Failed clearing memory after " +
1847 flushCount + " attempts on region: " +
1848 Bytes.toStringBinary(getRegionInfo().getRegionName()));
1850 } catch (IOException ioe) {
1851 status.setStatus("Failed flush " + this + ", putting online again");
1852 synchronized (writestate) {
1853 writestate.writesEnabled = true;
1855 // Have to throw to upper layers. I can't abort server from here.
1856 throw ioe;
1861 Map<byte[], List<HStoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1862 if (!stores.isEmpty()) {
1863 // initialize the thread pool for closing stores in parallel.
1864 ThreadPoolExecutor storeCloserThreadPool =
1865 getStoreOpenAndCloseThreadPool("StoreCloser-" +
1866 getRegionInfo().getRegionNameAsString());
1867 CompletionService<Pair<byte[], Collection<HStoreFile>>> completionService =
1868 new ExecutorCompletionService<>(storeCloserThreadPool);
1870 // close each store in parallel
1871 for (HStore store : stores.values()) {
1872 MemStoreSize mss = store.getFlushableSize();
1873 if (!(abort || mss.getDataSize() == 0 || writestate.readOnly)) {
1874 if (getRegionServerServices() != null) {
1875 getRegionServerServices().abort("Assertion failed while closing store "
1876 + getRegionInfo().getRegionNameAsString() + " " + store
1877 + ". flushableSize expected=0, actual={" + mss
1878 + "}. Current memStoreSize=" + this.memStoreSizing.getMemStoreSize() +
1879 ". Maybe a coprocessor "
1880 + "operation failed and left the memstore in a partially updated state.", null);
1883 completionService
1884 .submit(new Callable<Pair<byte[], Collection<HStoreFile>>>() {
1885 @Override
1886 public Pair<byte[], Collection<HStoreFile>> call() throws IOException {
1887 return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close());
1891 try {
1892 for (int i = 0; i < stores.size(); i++) {
1893 Future<Pair<byte[], Collection<HStoreFile>>> future = completionService.take();
1894 Pair<byte[], Collection<HStoreFile>> storeFiles = future.get();
1895 List<HStoreFile> familyFiles = result.get(storeFiles.getFirst());
1896 if (familyFiles == null) {
1897 familyFiles = new ArrayList<>();
1898 result.put(storeFiles.getFirst(), familyFiles);
1900 familyFiles.addAll(storeFiles.getSecond());
1902 } catch (InterruptedException e) {
1903 throw throwOnInterrupt(e);
1904 } catch (ExecutionException e) {
1905 Throwable cause = e.getCause();
1906 if (cause instanceof IOException) {
1907 throw (IOException) cause;
1909 throw new IOException(cause);
1910 } finally {
1911 storeCloserThreadPool.shutdownNow();
1915 status.setStatus("Writing region close event to WAL");
1916 // Always write close marker to wal even for read only table. This is not a big problem as we
1917 // do not write any data into the region; it is just a meta edit in the WAL file.
1918 if (!abort && wal != null && getRegionServerServices() != null &&
1919 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
1920 writeRegionCloseMarker(wal);
1922 this.closed.set(true);
1923 if (!canFlush) {
1924 decrMemStoreSize(this.memStoreSizing.getMemStoreSize());
1925 } else if (this.memStoreSizing.getDataSize() != 0) {
1926 LOG.error("Memstore data size is {} in region {}", this.memStoreSizing.getDataSize(), this);
1928 if (coprocessorHost != null) {
1929 status.setStatus("Running coprocessor post-close hooks");
1930 this.coprocessorHost.postClose(abort);
1932 if (this.metricsRegion != null) {
1933 this.metricsRegion.close();
1935 if (this.metricsRegionWrapper != null) {
1936 Closeables.close(this.metricsRegionWrapper, true);
1938 status.markComplete("Closed");
1939 LOG.info("Closed {}", this);
1940 return result;
1941 } finally {
1942 lock.writeLock().unlock();
1946 /** Wait for all current flushes and compactions of the region to complete */
1947 // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for
1948 // Phoenix needs.
1949 public void waitForFlushesAndCompactions() {
1950 synchronized (writestate) {
1951 if (this.writestate.readOnly) {
1952 // we should not wait for replayed flushed if we are read only (for example in case the
1953 // region is a secondary replica).
1954 return;
1956 boolean interrupted = false;
1957 try {
1958 while (writestate.compacting.get() > 0 || writestate.flushing) {
1959 LOG.debug("waiting for " + writestate.compacting + " compactions"
1960 + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
1961 try {
1962 writestate.wait();
1963 } catch (InterruptedException iex) {
1964 // essentially ignore and propagate the interrupt back up
1965 LOG.warn("Interrupted while waiting in region {}", this);
1966 interrupted = true;
1967 break;
1970 } finally {
1971 if (interrupted) {
1972 Thread.currentThread().interrupt();
1979 * Wait for all current flushes of the region to complete
1981 public void waitForFlushes() {
1982 waitForFlushes(0);// Unbound wait
1985 @Override
1986 public boolean waitForFlushes(long timeout) {
1987 synchronized (writestate) {
1988 if (this.writestate.readOnly) {
1989 // we should not wait for replayed flushed if we are read only (for example in case the
1990 // region is a secondary replica).
1991 return true;
1993 if (!writestate.flushing) return true;
1994 long start = EnvironmentEdgeManager.currentTime();
1995 long duration = 0;
1996 boolean interrupted = false;
1997 LOG.debug("waiting for cache flush to complete for region " + this);
1998 try {
1999 while (writestate.flushing) {
2000 if (timeout > 0 && duration >= timeout) break;
2001 try {
2002 long toWait = timeout == 0 ? 0 : (timeout - duration);
2003 writestate.wait(toWait);
2004 } catch (InterruptedException iex) {
2005 // essentially ignore and propagate the interrupt back up
2006 LOG.warn("Interrupted while waiting in region {}", this);
2007 interrupted = true;
2008 break;
2009 } finally {
2010 duration = EnvironmentEdgeManager.currentTime() - start;
2013 } finally {
2014 if (interrupted) {
2015 Thread.currentThread().interrupt();
2018 LOG.debug("Waited {} ms for region {} flush to complete", duration, this);
2019 return !(writestate.flushing);
2023 @Override
2024 public Configuration getReadOnlyConfiguration() {
2025 return new ReadOnlyConfiguration(this.conf);
2028 private ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
2029 final String threadNamePrefix) {
2030 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
2031 int maxThreads = Math.min(numStores,
2032 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
2033 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
2034 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
2037 ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
2038 final String threadNamePrefix) {
2039 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
2040 int maxThreads = Math.max(1,
2041 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
2042 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
2043 / numStores);
2044 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
2047 private static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
2048 final String threadNamePrefix) {
2049 return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
2050 new ThreadFactory() {
2051 private int count = 1;
2053 @Override
2054 public Thread newThread(Runnable r) {
2055 return new Thread(r, threadNamePrefix + "-" + count++);
2061 * @return True if its worth doing a flush before we put up the close flag.
2063 private boolean worthPreFlushing() {
2064 return this.memStoreSizing.getDataSize() >
2065 this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
2068 //////////////////////////////////////////////////////////////////////////////
2069 // HRegion accessors
2070 //////////////////////////////////////////////////////////////////////////////
2072 @Override
2073 public TableDescriptor getTableDescriptor() {
2074 return this.htableDescriptor;
2077 public void setTableDescriptor(TableDescriptor desc) {
2078 htableDescriptor = desc;
2081 /** @return WAL in use for this region */
2082 public WAL getWAL() {
2083 return this.wal;
2086 public BlockCache getBlockCache() {
2087 return this.blockCache;
2091 * Only used for unit test which doesn't start region server.
2093 public void setBlockCache(BlockCache blockCache) {
2094 this.blockCache = blockCache;
2097 public MobFileCache getMobFileCache() {
2098 return this.mobFileCache;
2102 * Only used for unit test which doesn't start region server.
2104 public void setMobFileCache(MobFileCache mobFileCache) {
2105 this.mobFileCache = mobFileCache;
2109 * @return split policy for this region.
2111 RegionSplitPolicy getSplitPolicy() {
2112 return this.splitPolicy;
2116 * A split takes the config from the parent region & passes it to the daughter
2117 * region's constructor. If 'conf' was passed, you would end up using the HTD
2118 * of the parent region in addition to the new daughter HTD. Pass 'baseConf'
2119 * to the daughter regions to avoid this tricky dedupe problem.
2120 * @return Configuration object
2122 Configuration getBaseConf() {
2123 return this.baseConf;
2126 /** @return {@link FileSystem} being used by this region */
2127 public FileSystem getFilesystem() {
2128 return fs.getFileSystem();
2131 /** @return the {@link HRegionFileSystem} used by this region */
2132 public HRegionFileSystem getRegionFileSystem() {
2133 return this.fs;
2136 /** @return the WAL {@link HRegionFileSystem} used by this region */
2137 HRegionWALFileSystem getRegionWALFileSystem() throws IOException {
2138 return new HRegionWALFileSystem(conf, getWalFileSystem(),
2139 CommonFSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
2142 /** @return the WAL {@link FileSystem} being used by this region */
2143 FileSystem getWalFileSystem() throws IOException {
2144 if (walFS == null) {
2145 walFS = CommonFSUtils.getWALFileSystem(conf);
2147 return walFS;
2151 * @return the Region directory under WALRootDirectory
2152 * @throws IOException if there is an error getting WALRootDir
2154 public Path getWALRegionDir() throws IOException {
2155 if (regionDir == null) {
2156 regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
2157 getRegionInfo().getEncodedName());
2159 return regionDir;
2162 @Override
2163 public long getEarliestFlushTimeForAllStores() {
2164 return Collections.min(lastStoreFlushTimeMap.values());
2167 @Override
2168 public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
2169 long result = Long.MAX_VALUE;
2170 for (HStore store : stores.values()) {
2171 Collection<HStoreFile> storeFiles = store.getStorefiles();
2172 if (storeFiles == null) {
2173 continue;
2175 for (HStoreFile file : storeFiles) {
2176 StoreFileReader sfReader = file.getReader();
2177 if (sfReader == null) {
2178 continue;
2180 HFile.Reader reader = sfReader.getHFileReader();
2181 if (reader == null) {
2182 continue;
2184 if (majorCompactionOnly) {
2185 byte[] val = reader.getHFileInfo().get(MAJOR_COMPACTION_KEY);
2186 if (val == null || !Bytes.toBoolean(val)) {
2187 continue;
2190 result = Math.min(result, reader.getFileContext().getFileCreateTime());
2193 return result == Long.MAX_VALUE ? 0 : result;
2196 RegionLoad.Builder setCompleteSequenceId(RegionLoad.Builder regionLoadBldr) {
2197 long lastFlushOpSeqIdLocal = this.lastFlushOpSeqId;
2198 byte[] encodedRegionName = this.getRegionInfo().getEncodedNameAsBytes();
2199 regionLoadBldr.clearStoreCompleteSequenceId();
2200 for (byte[] familyName : this.stores.keySet()) {
2201 long earliest = this.wal.getEarliestMemStoreSeqNum(encodedRegionName, familyName);
2202 // Subtract - 1 to go earlier than the current oldest, unflushed edit in memstore; this will
2203 // give us a sequence id that is for sure flushed. We want edit replay to start after this
2204 // sequence id in this region. If NO_SEQNUM, use the regions maximum flush id.
2205 long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1;
2206 regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId.newBuilder()
2207 .setFamilyName(UnsafeByteOperations.unsafeWrap(familyName)).setSequenceId(csid).build());
2209 return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId());
2212 //////////////////////////////////////////////////////////////////////////////
2213 // HRegion maintenance.
2215 // These methods are meant to be called periodically by the HRegionServer for
2216 // upkeep.
2217 //////////////////////////////////////////////////////////////////////////////
2219 * Do preparation for pending compaction.
2220 * @throws IOException
2222 protected void doRegionCompactionPrep() throws IOException {
2226 * Synchronously compact all stores in the region.
2227 * <p>This operation could block for a long time, so don't call it from a
2228 * time-sensitive thread.
2229 * <p>Note that no locks are taken to prevent possible conflicts between
2230 * compaction and splitting activities. The regionserver does not normally compact
2231 * and split in parallel. However by calling this method you may introduce
2232 * unexpected and unhandled concurrency. Don't do this unless you know what
2233 * you are doing.
2235 * @param majorCompaction True to force a major compaction regardless of thresholds
2236 * @throws IOException
2238 public void compact(boolean majorCompaction) throws IOException {
2239 if (majorCompaction) {
2240 stores.values().forEach(HStore::triggerMajorCompaction);
2242 for (HStore s : stores.values()) {
2243 Optional<CompactionContext> compaction = s.requestCompaction();
2244 if (compaction.isPresent()) {
2245 ThroughputController controller = null;
2246 if (rsServices != null) {
2247 controller = CompactionThroughputControllerFactory.create(rsServices, conf);
2249 if (controller == null) {
2250 controller = NoLimitThroughputController.INSTANCE;
2252 compact(compaction.get(), s, controller, null);
2258 * This is a helper function that compact all the stores synchronously.
2259 * <p>
2260 * It is used by utilities and testing
2262 public void compactStores() throws IOException {
2263 for (HStore s : stores.values()) {
2264 Optional<CompactionContext> compaction = s.requestCompaction();
2265 if (compaction.isPresent()) {
2266 compact(compaction.get(), s, NoLimitThroughputController.INSTANCE, null);
2272 * This is a helper function that compact the given store.
2273 * <p>
2274 * It is used by utilities and testing
2276 void compactStore(byte[] family, ThroughputController throughputController) throws IOException {
2277 HStore s = getStore(family);
2278 Optional<CompactionContext> compaction = s.requestCompaction();
2279 if (compaction.isPresent()) {
2280 compact(compaction.get(), s, throughputController, null);
2285 * Called by compaction thread and after region is opened to compact the
2286 * HStores if necessary.
2288 * <p>This operation could block for a long time, so don't call it from a
2289 * time-sensitive thread.
2291 * Note that no locking is necessary at this level because compaction only
2292 * conflicts with a region split, and that cannot happen because the region
2293 * server does them sequentially and not in parallel.
2295 * @param compaction Compaction details, obtained by requestCompaction()
2296 * @param throughputController
2297 * @return whether the compaction completed
2299 public boolean compact(CompactionContext compaction, HStore store,
2300 ThroughputController throughputController) throws IOException {
2301 return compact(compaction, store, throughputController, null);
2304 private boolean shouldForbidMajorCompaction() {
2305 if (rsServices != null && rsServices.getReplicationSourceService() != null) {
2306 return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
2307 .checkState(getRegionInfo().getTable(), ForbidMajorCompactionChecker.get());
2309 return false;
2313 * We are trying to remove / relax the region read lock for compaction.
2314 * Let's see what are the potential race conditions among the operations (user scan,
2315 * region split, region close and region bulk load).
2317 * user scan ---> region read lock
2318 * region split --> region close first --> region write lock
2319 * region close --> region write lock
2320 * region bulk load --> region write lock
2322 * read lock is compatible with read lock. ---> no problem with user scan/read
2323 * region bulk load does not cause problem for compaction (no consistency problem, store lock
2324 * will help the store file accounting).
2325 * They can run almost concurrently at the region level.
2327 * The only remaining race condition is between the region close and compaction.
2328 * So we will evaluate, below, how region close intervenes with compaction if compaction does
2329 * not acquire region read lock.
2331 * Here are the steps for compaction:
2332 * 1. obtain list of StoreFile's
2333 * 2. create StoreFileScanner's based on list from #1
2334 * 3. perform compaction and save resulting files under tmp dir
2335 * 4. swap in compacted files
2337 * #1 is guarded by store lock. This patch does not change this --> no worse or better
2338 * For #2, we obtain smallest read point (for region) across all the Scanners (for both default
2339 * compactor and stripe compactor).
2340 * The read points are for user scans. Region keeps the read points for all currently open
2341 * user scanners.
2342 * Compaction needs to know the smallest read point so that during re-write of the hfiles,
2343 * it can remove the mvcc points for the cells if their mvccs are older than the smallest
2344 * since they are not needed anymore.
2345 * This will not conflict with compaction.
2346 * For #3, it can be performed in parallel to other operations.
2347 * For #4 bulk load and compaction don't conflict with each other on the region level
2348 * (for multi-family atomicy).
2349 * Region close and compaction are guarded pretty well by the 'writestate'.
2350 * In HRegion#doClose(), we have :
2351 * synchronized (writestate) {
2352 * // Disable compacting and flushing by background threads for this
2353 * // region.
2354 * canFlush = !writestate.readOnly;
2355 * writestate.writesEnabled = false;
2356 * LOG.debug("Closing " + this + ": disabling compactions & flushes");
2357 * waitForFlushesAndCompactions();
2359 * waitForFlushesAndCompactions() would wait for writestate.compacting to come down to 0.
2360 * and in HRegion.compact()
2361 * try {
2362 * synchronized (writestate) {
2363 * if (writestate.writesEnabled) {
2364 * wasStateSet = true;
2365 * ++writestate.compacting;
2366 * } else {
2367 * String msg = "NOT compacting region " + this + ". Writes disabled.";
2368 * LOG.info(msg);
2369 * status.abort(msg);
2370 * return false;
2373 * Also in compactor.performCompaction():
2374 * check periodically to see if a system stop is requested
2375 * if (closeChecker != null && closeChecker.isTimeLimit(store, now)) {
2376 * progress.cancel();
2377 * return false;
2379 * if (closeChecker != null && closeChecker.isSizeLimit(store, len)) {
2380 * progress.cancel();
2381 * return false;
2384 public boolean compact(CompactionContext compaction, HStore store,
2385 ThroughputController throughputController, User user) throws IOException {
2386 assert compaction != null && compaction.hasSelection();
2387 assert !compaction.getRequest().getFiles().isEmpty();
2388 if (this.closing.get() || this.closed.get()) {
2389 LOG.debug("Skipping compaction on " + this + " because closing/closed");
2390 store.cancelRequestedCompaction(compaction);
2391 return false;
2394 if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) {
2395 LOG.warn("Skipping major compaction on " + this
2396 + " because this cluster is transiting sync replication state"
2397 + " from STANDBY to DOWNGRADE_ACTIVE");
2398 store.cancelRequestedCompaction(compaction);
2399 return false;
2402 MonitoredTask status = null;
2403 boolean requestNeedsCancellation = true;
2404 try {
2405 byte[] cf = Bytes.toBytes(store.getColumnFamilyName());
2406 if (stores.get(cf) != store) {
2407 LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this
2408 + " has been re-instantiated, cancel this compaction request. "
2409 + " It may be caused by the roll back of split transaction");
2410 return false;
2413 status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this);
2414 status.enableStatusJournal(false);
2415 if (this.closed.get()) {
2416 String msg = "Skipping compaction on " + this + " because closed";
2417 LOG.debug(msg);
2418 status.abort(msg);
2419 return false;
2421 boolean wasStateSet = false;
2422 try {
2423 synchronized (writestate) {
2424 if (writestate.writesEnabled) {
2425 wasStateSet = true;
2426 writestate.compacting.incrementAndGet();
2427 } else {
2428 String msg = "NOT compacting region " + this + ". Writes disabled.";
2429 LOG.info(msg);
2430 status.abort(msg);
2431 return false;
2434 LOG.info("Starting compaction of {} in {}{}", store, this,
2435 (compaction.getRequest().isOffPeak()?" as an off-peak compaction":""));
2436 doRegionCompactionPrep();
2437 try {
2438 status.setStatus("Compacting store " + store);
2439 // We no longer need to cancel the request on the way out of this
2440 // method because Store#compact will clean up unconditionally
2441 requestNeedsCancellation = false;
2442 store.compact(compaction, throughputController, user);
2443 } catch (InterruptedIOException iioe) {
2444 String msg = "region " + this + " compaction interrupted";
2445 LOG.info(msg, iioe);
2446 status.abort(msg);
2447 return false;
2449 } finally {
2450 if (wasStateSet) {
2451 synchronized (writestate) {
2452 writestate.compacting.decrementAndGet();
2453 if (writestate.compacting.get() <= 0) {
2454 writestate.notifyAll();
2459 status.markComplete("Compaction complete");
2460 return true;
2461 } finally {
2462 if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction);
2463 if (status != null) {
2464 LOG.debug("Compaction status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2465 status.prettyPrintJournal());
2466 status.cleanup();
2472 * Flush the cache.
2474 * <p>When this method is called the cache will be flushed unless:
2475 * <ol>
2476 * <li>the cache is empty</li>
2477 * <li>the region is closed.</li>
2478 * <li>a flush is already in progress</li>
2479 * <li>writes are disabled</li>
2480 * </ol>
2482 * <p>This method may block for some time, so it should not be called from a
2483 * time-sensitive thread.
2484 * @param flushAllStores whether we want to force a flush of all stores
2485 * @return FlushResult indicating whether the flush was successful or not and if
2486 * the region needs compacting
2488 * @throws IOException general io exceptions
2489 * because a snapshot was not properly persisted.
2491 // TODO HBASE-18905. We might have to expose a requestFlush API for CPs
2492 public FlushResult flush(boolean flushAllStores) throws IOException {
2493 return flushcache(flushAllStores, false, FlushLifeCycleTracker.DUMMY);
2496 public interface FlushResult {
2497 enum Result {
2498 FLUSHED_NO_COMPACTION_NEEDED,
2499 FLUSHED_COMPACTION_NEEDED,
2500 // Special case where a flush didn't run because there's nothing in the memstores. Used when
2501 // bulk loading to know when we can still load even if a flush didn't happen.
2502 CANNOT_FLUSH_MEMSTORE_EMPTY,
2503 CANNOT_FLUSH
2506 /** @return the detailed result code */
2507 Result getResult();
2509 /** @return true if the memstores were flushed, else false */
2510 boolean isFlushSucceeded();
2512 /** @return True if the flush requested a compaction, else false */
2513 boolean isCompactionNeeded();
2516 public FlushResultImpl flushcache(boolean flushAllStores, boolean writeFlushRequestWalMarker,
2517 FlushLifeCycleTracker tracker) throws IOException {
2518 List<byte[]> families = null;
2519 if (flushAllStores) {
2520 families = new ArrayList<>();
2521 families.addAll(this.getTableDescriptor().getColumnFamilyNames());
2523 return this.flushcache(families, writeFlushRequestWalMarker, tracker);
2527 * Flush the cache.
2529 * When this method is called the cache will be flushed unless:
2530 * <ol>
2531 * <li>the cache is empty</li>
2532 * <li>the region is closed.</li>
2533 * <li>a flush is already in progress</li>
2534 * <li>writes are disabled</li>
2535 * </ol>
2537 * <p>This method may block for some time, so it should not be called from a
2538 * time-sensitive thread.
2539 * @param families stores of region to flush.
2540 * @param writeFlushRequestWalMarker whether to write the flush request marker to WAL
2541 * @param tracker used to track the life cycle of this flush
2542 * @return whether the flush is success and whether the region needs compacting
2544 * @throws IOException general io exceptions
2545 * @throws DroppedSnapshotException Thrown when replay of wal is required
2546 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
2547 * caller MUST abort after this.
2549 public FlushResultImpl flushcache(List<byte[]> families,
2550 boolean writeFlushRequestWalMarker, FlushLifeCycleTracker tracker) throws IOException {
2551 // fail-fast instead of waiting on the lock
2552 if (this.closing.get()) {
2553 String msg = "Skipping flush on " + this + " because closing";
2554 LOG.debug(msg);
2555 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2557 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
2558 status.enableStatusJournal(false);
2559 status.setStatus("Acquiring readlock on region");
2560 // block waiting for the lock for flushing cache
2561 lock.readLock().lock();
2562 boolean flushed = true;
2563 try {
2564 if (this.closed.get()) {
2565 String msg = "Skipping flush on " + this + " because closed";
2566 LOG.debug(msg);
2567 status.abort(msg);
2568 flushed = false;
2569 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2571 if (coprocessorHost != null) {
2572 status.setStatus("Running coprocessor pre-flush hooks");
2573 coprocessorHost.preFlush(tracker);
2575 // TODO: this should be managed within memstore with the snapshot, updated only after flush
2576 // successful
2577 if (numMutationsWithoutWAL.sum() > 0) {
2578 numMutationsWithoutWAL.reset();
2579 dataInMemoryWithoutWAL.reset();
2581 synchronized (writestate) {
2582 if (!writestate.flushing && writestate.writesEnabled) {
2583 this.writestate.flushing = true;
2584 } else {
2585 String msg = "NOT flushing " + this + " as " + (writestate.flushing ? "already flushing"
2586 : "writes are not enabled");
2587 LOG.debug(msg);
2588 status.abort(msg);
2589 flushed = false;
2590 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2594 try {
2595 // The reason that we do not always use flushPolicy is, when the flush is
2596 // caused by logRoller, we should select stores which must be flushed
2597 // rather than could be flushed.
2598 Collection<HStore> specificStoresToFlush = null;
2599 if (families != null) {
2600 specificStoresToFlush = getSpecificStores(families);
2601 } else {
2602 specificStoresToFlush = flushPolicy.selectStoresToFlush();
2604 FlushResultImpl fs =
2605 internalFlushcache(specificStoresToFlush, status, writeFlushRequestWalMarker, tracker);
2607 if (coprocessorHost != null) {
2608 status.setStatus("Running post-flush coprocessor hooks");
2609 coprocessorHost.postFlush(tracker);
2612 if(fs.isFlushSucceeded()) {
2613 flushesQueued.reset();
2616 status.markComplete("Flush successful " + fs.toString());
2617 return fs;
2618 } finally {
2619 synchronized (writestate) {
2620 writestate.flushing = false;
2621 this.writestate.flushRequested = false;
2622 writestate.notifyAll();
2625 } finally {
2626 lock.readLock().unlock();
2627 if (flushed) {
2628 // Don't log this journal stuff if no flush -- confusing.
2629 LOG.debug("Flush status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2630 status.prettyPrintJournal());
2632 status.cleanup();
2637 * get stores which matches the specified families
2639 * @return the stores need to be flushed.
2641 private Collection<HStore> getSpecificStores(List<byte[]> families) {
2642 Collection<HStore> specificStoresToFlush = new ArrayList<>();
2643 for (byte[] family : families) {
2644 specificStoresToFlush.add(stores.get(family));
2646 return specificStoresToFlush;
2650 * Should the store be flushed because it is old enough.
2651 * <p>
2652 * Every FlushPolicy should call this to determine whether a store is old enough to flush (except
2653 * that you always flush all stores). Otherwise the method will always
2654 * returns true which will make a lot of flush requests.
2656 boolean shouldFlushStore(HStore store) {
2657 long earliest = this.wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
2658 store.getColumnFamilyDescriptor().getName()) - 1;
2659 if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
2660 if (LOG.isDebugEnabled()) {
2661 LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " +
2662 getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest +
2663 " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint());
2665 return true;
2667 if (this.flushCheckInterval <= 0) {
2668 return false;
2670 long now = EnvironmentEdgeManager.currentTime();
2671 if (store.timeOfOldestEdit() < now - this.flushCheckInterval) {
2672 if (LOG.isDebugEnabled()) {
2673 LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of " +
2674 getRegionInfo().getEncodedName() + " because time of oldest edit=" +
2675 store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now);
2677 return true;
2679 return false;
2683 * Should the memstore be flushed now
2685 boolean shouldFlush(final StringBuilder whyFlush) {
2686 whyFlush.setLength(0);
2687 // This is a rough measure.
2688 if (this.maxFlushedSeqId > 0
2689 && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())) {
2690 whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush");
2691 return true;
2693 long modifiedFlushCheckInterval = flushCheckInterval;
2694 if (getRegionInfo().getTable().isSystemTable() &&
2695 getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
2696 modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
2698 if (modifiedFlushCheckInterval <= 0) { //disabled
2699 return false;
2701 long now = EnvironmentEdgeManager.currentTime();
2702 //if we flushed in the recent past, we don't need to do again now
2703 if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) {
2704 return false;
2706 //since we didn't flush in the recent past, flush now if certain conditions
2707 //are met. Return true on first such memstore hit.
2708 for (HStore s : stores.values()) {
2709 if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
2710 // we have an old enough edit in the memstore, flush
2711 whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
2712 return true;
2715 return false;
2719 * Flushing all stores.
2720 * @see #internalFlushcache(Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2722 private FlushResult internalFlushcache(MonitoredTask status) throws IOException {
2723 return internalFlushcache(stores.values(), status, false, FlushLifeCycleTracker.DUMMY);
2727 * Flushing given stores.
2728 * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2730 private FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status,
2731 boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException {
2732 return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush, status,
2733 writeFlushWalMarker, tracker);
2737 * Flush the memstore. Flushing the memstore is a little tricky. We have a lot of updates in the
2738 * memstore, all of which have also been written to the wal. We need to write those updates in the
2739 * memstore out to disk, while being able to process reads/writes as much as possible during the
2740 * flush operation.
2741 * <p>
2742 * This method may block for some time. Every time you call it, we up the regions sequence id even
2743 * if we don't flush; i.e. the returned region id will be at least one larger than the last edit
2744 * applied to this region. The returned id does not refer to an actual edit. The returned id can
2745 * be used for say installing a bulk loaded file just ahead of the last hfile that was the result
2746 * of this flush, etc.
2747 * @param wal Null if we're NOT to go via wal.
2748 * @param myseqid The seqid to use if <code>wal</code> is null writing out flush file.
2749 * @param storesToFlush The list of stores to flush.
2750 * @return object describing the flush's state
2751 * @throws IOException general io exceptions
2752 * @throws DroppedSnapshotException Thrown when replay of WAL is required.
2754 protected FlushResultImpl internalFlushcache(WAL wal, long myseqid,
2755 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2756 FlushLifeCycleTracker tracker) throws IOException {
2757 PrepareFlushResult result =
2758 internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker, tracker);
2759 if (result.result == null) {
2760 return internalFlushCacheAndCommit(wal, status, result, storesToFlush);
2761 } else {
2762 return result.result; // early exit due to failure from prepare stage
2766 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
2767 justification="FindBugs seems confused about trxId")
2768 protected PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid,
2769 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2770 FlushLifeCycleTracker tracker) throws IOException {
2771 if (this.rsServices != null && this.rsServices.isAborted()) {
2772 // Don't flush when server aborting, it's unsafe
2773 throw new IOException("Aborting flush because server is aborted...");
2775 final long startTime = EnvironmentEdgeManager.currentTime();
2776 // If nothing to flush, return, but return with a valid unused sequenceId.
2777 // Its needed by bulk upload IIRC. It flushes until no edits in memory so it can insert a
2778 // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs
2779 // to no other that it can use to associate with the bulk load. Hence this little dance below
2780 // to go get one.
2781 if (this.memStoreSizing.getDataSize() <= 0) {
2782 // Take an update lock so no edits can come into memory just yet.
2783 this.updatesLock.writeLock().lock();
2784 WriteEntry writeEntry = null;
2785 try {
2786 if (this.memStoreSizing.getDataSize() <= 0) {
2787 // Presume that if there are still no edits in the memstore, then there are no edits for
2788 // this region out in the WAL subsystem so no need to do any trickery clearing out
2789 // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
2790 // sure just beyond the last appended region edit and not associated with any edit
2791 // (useful as marker when bulk loading, etc.).
2792 if (wal != null) {
2793 writeEntry = mvcc.begin();
2794 long flushOpSeqId = writeEntry.getWriteNumber();
2795 FlushResultImpl flushResult =
2796 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId,
2797 "Nothing to flush", writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
2798 mvcc.completeAndWait(writeEntry);
2799 // Set to null so we don't complete it again down in finally block.
2800 writeEntry = null;
2801 return new PrepareFlushResult(flushResult, myseqid);
2802 } else {
2803 return new PrepareFlushResult(new FlushResultImpl(
2804 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", false), myseqid);
2807 } finally {
2808 if (writeEntry != null) {
2809 // If writeEntry is non-null, this operation failed; the mvcc transaction failed...
2810 // but complete it anyways so it doesn't block the mvcc queue.
2811 mvcc.complete(writeEntry);
2813 this.updatesLock.writeLock().unlock();
2816 logFatLineOnFlush(storesToFlush, myseqid);
2817 // Stop updates while we snapshot the memstore of all of these regions' stores. We only have
2818 // to do this for a moment. It is quick. We also set the memstore size to zero here before we
2819 // allow updates again so its value will represent the size of the updates received
2820 // during flush
2822 // We have to take an update lock during snapshot, or else a write could end up in both snapshot
2823 // and memstore (makes it difficult to do atomic rows then)
2824 status.setStatus("Obtaining lock to block concurrent updates");
2825 // block waiting for the lock for internal flush
2826 this.updatesLock.writeLock().lock();
2827 status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName());
2828 MemStoreSizing totalSizeOfFlushableStores = new NonThreadSafeMemStoreSizing();
2830 Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>();
2831 for (HStore store : storesToFlush) {
2832 flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(),
2833 store.preFlushSeqIDEstimation());
2836 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2837 TreeMap<byte[], List<Path>> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2838 TreeMap<byte[], MemStoreSize> storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2839 // The sequence id of this flush operation which is used to log FlushMarker and pass to
2840 // createFlushContext to use as the store file's sequence id. It can be in advance of edits
2841 // still in the memstore, edits that are in other column families yet to be flushed.
2842 long flushOpSeqId = HConstants.NO_SEQNUM;
2843 // The max flushed sequence id after this flush operation completes. All edits in memstore
2844 // will be in advance of this sequence id.
2845 long flushedSeqId = HConstants.NO_SEQNUM;
2846 byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes();
2847 try {
2848 if (wal != null) {
2849 Long earliestUnflushedSequenceIdForTheRegion =
2850 wal.startCacheFlush(encodedRegionName, flushedFamilyNamesToSeq);
2851 if (earliestUnflushedSequenceIdForTheRegion == null) {
2852 // This should never happen. This is how startCacheFlush signals flush cannot proceed.
2853 String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
2854 status.setStatus(msg);
2855 return new PrepareFlushResult(
2856 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false),
2857 myseqid);
2859 flushOpSeqId = getNextSequenceId(wal);
2860 // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
2861 flushedSeqId =
2862 earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM?
2863 flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
2864 } else {
2865 // use the provided sequence Id as WAL is not being used for this flush.
2866 flushedSeqId = flushOpSeqId = myseqid;
2869 for (HStore s : storesToFlush) {
2870 storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(),
2871 s.createFlushContext(flushOpSeqId, tracker));
2872 // for writing stores to WAL
2873 committedFiles.put(s.getColumnFamilyDescriptor().getName(), null);
2876 // write the snapshot start to WAL
2877 if (wal != null && !writestate.readOnly) {
2878 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
2879 getRegionInfo(), flushOpSeqId, committedFiles);
2880 // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH
2881 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false,
2882 mvcc, regionReplicationSink.orElse(null));
2885 // Prepare flush (take a snapshot)
2886 storeFlushCtxs.forEach((name, flush) -> {
2887 MemStoreSize snapshotSize = flush.prepare();
2888 totalSizeOfFlushableStores.incMemStoreSize(snapshotSize);
2889 storeFlushableSize.put(name, snapshotSize);
2891 } catch (IOException ex) {
2892 doAbortFlushToWAL(wal, flushOpSeqId, committedFiles);
2893 throw ex;
2894 } finally {
2895 this.updatesLock.writeLock().unlock();
2897 String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " +
2898 "flushsize=" + totalSizeOfFlushableStores;
2899 status.setStatus(s);
2900 doSyncOfUnflushedWALChanges(wal, getRegionInfo());
2901 return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
2902 flushOpSeqId, flushedSeqId, totalSizeOfFlushableStores);
2906 * Utility method broken out of internalPrepareFlushCache so that method is smaller.
2908 private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) {
2909 if (!LOG.isInfoEnabled()) {
2910 return;
2912 // Log a fat line detailing what is being flushed.
2913 StringBuilder perCfExtras = null;
2914 if (!isAllFamilies(storesToFlush)) {
2915 perCfExtras = new StringBuilder();
2916 for (HStore store: storesToFlush) {
2917 MemStoreSize mss = store.getFlushableSize();
2918 perCfExtras.append("; ").append(store.getColumnFamilyName());
2919 perCfExtras.append("={dataSize=")
2920 .append(StringUtils.byteDesc(mss.getDataSize()));
2921 perCfExtras.append(", heapSize=")
2922 .append(StringUtils.byteDesc(mss.getHeapSize()));
2923 perCfExtras.append(", offHeapSize=")
2924 .append(StringUtils.byteDesc(mss.getOffHeapSize()));
2925 perCfExtras.append("}");
2928 MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
2929 LOG.info("Flushing " + this.getRegionInfo().getEncodedName() + " " +
2930 storesToFlush.size() + "/" + stores.size() + " column families," +
2931 " dataSize=" + StringUtils.byteDesc(mss.getDataSize()) +
2932 " heapSize=" + StringUtils.byteDesc(mss.getHeapSize()) +
2933 ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
2934 ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId));
2937 private void doAbortFlushToWAL(final WAL wal, final long flushOpSeqId,
2938 final Map<byte[], List<Path>> committedFiles) {
2939 if (wal == null) return;
2940 try {
2941 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2942 getRegionInfo(), flushOpSeqId, committedFiles);
2943 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false, mvcc,
2944 null);
2945 } catch (Throwable t) {
2946 LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL: {} in "
2947 + " region {}", StringUtils.stringifyException(t), this);
2948 // ignore this since we will be aborting the RS with DSE.
2950 // we have called wal.startCacheFlush(), now we have to abort it
2951 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2955 * Sync unflushed WAL changes. See HBASE-8208 for details
2957 private static void doSyncOfUnflushedWALChanges(final WAL wal, final RegionInfo hri)
2958 throws IOException {
2959 if (wal == null) {
2960 return;
2962 try {
2963 wal.sync(); // ensure that flush marker is sync'ed
2964 } catch (IOException ioe) {
2965 wal.abortCacheFlush(hri.getEncodedNameAsBytes());
2966 throw ioe;
2971 * @return True if passed Set is all families in the region.
2973 private boolean isAllFamilies(Collection<HStore> families) {
2974 return families == null || this.stores.size() == families.size();
2978 * Writes a marker to WAL indicating a flush is requested but cannot be complete due to various
2979 * reasons. Ignores exceptions from WAL. Returns whether the write succeeded.
2980 * @return whether WAL write was successful
2982 private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker) {
2983 if (writeFlushWalMarker && wal != null && !writestate.readOnly) {
2984 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH,
2985 getRegionInfo(), -1, new TreeMap<>(Bytes.BYTES_COMPARATOR));
2986 try {
2987 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc,
2988 regionReplicationSink.orElse(null));
2989 return true;
2990 } catch (IOException e) {
2991 LOG.warn(getRegionInfo().getEncodedName() + " : " +
2992 "Received exception while trying to write the flush request to wal", e);
2995 return false;
2998 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
2999 justification="Intentional; notify is about completed flush")
3000 FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status,
3001 PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException {
3002 // prepare flush context is carried via PrepareFlushResult
3003 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs;
3004 TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles;
3005 long startTime = prepareResult.startTime;
3006 long flushOpSeqId = prepareResult.flushOpSeqId;
3007 long flushedSeqId = prepareResult.flushedSeqId;
3009 String s = "Flushing stores of " + this;
3010 status.setStatus(s);
3011 if (LOG.isTraceEnabled()) LOG.trace(s);
3013 // Any failure from here on out will be catastrophic requiring server
3014 // restart so wal content can be replayed and put back into the memstore.
3015 // Otherwise, the snapshot content while backed up in the wal, it will not
3016 // be part of the current running servers state.
3017 boolean compactionRequested = false;
3018 long flushedOutputFileSize = 0;
3019 try {
3020 // A. Flush memstore to all the HStores.
3021 // Keep running vector of all store files that includes both old and the
3022 // just-made new flush store file. The new flushed file is still in the
3023 // tmp directory.
3025 for (StoreFlushContext flush : storeFlushCtxs.values()) {
3026 flush.flushCache(status);
3029 // Switch snapshot (in memstore) -> new hfile (thus causing
3030 // all the store scanners to reset/reseek).
3031 for (Map.Entry<byte[], StoreFlushContext> flushEntry : storeFlushCtxs.entrySet()) {
3032 StoreFlushContext sfc = flushEntry.getValue();
3033 boolean needsCompaction = sfc.commit(status);
3034 if (needsCompaction) {
3035 compactionRequested = true;
3037 byte[] storeName = flushEntry.getKey();
3038 List<Path> storeCommittedFiles = sfc.getCommittedFiles();
3039 committedFiles.put(storeName, storeCommittedFiles);
3040 // Flush committed no files, indicating flush is empty or flush was canceled
3041 if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
3042 MemStoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName);
3043 prepareResult.totalFlushableSize.decMemStoreSize(storeFlushableSize);
3045 flushedOutputFileSize += sfc.getOutputFileSize();
3047 storeFlushCtxs.clear();
3049 // Set down the memstore size by amount of flush.
3050 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3051 this.decrMemStoreSize(mss);
3053 // Increase the size of this Region for the purposes of quota. Noop if quotas are disabled.
3054 // During startup, quota manager may not be initialized yet.
3055 if (rsServices != null) {
3056 RegionServerSpaceQuotaManager quotaManager = rsServices.getRegionServerSpaceQuotaManager();
3057 if (quotaManager != null) {
3058 quotaManager.getRegionSizeStore().incrementRegionSize(
3059 this.getRegionInfo(), flushedOutputFileSize);
3063 if (wal != null) {
3064 // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
3065 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
3066 getRegionInfo(), flushOpSeqId, committedFiles);
3067 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc,
3068 regionReplicationSink.orElse(null));
3070 } catch (Throwable t) {
3071 // An exception here means that the snapshot was not persisted.
3072 // The wal needs to be replayed so its content is restored to memstore.
3073 // Currently, only a server restart will do this.
3074 // We used to only catch IOEs but its possible that we'd get other
3075 // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
3076 // all and sundry.
3077 if (wal != null) {
3078 try {
3079 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
3080 getRegionInfo(), flushOpSeqId, committedFiles);
3081 WALUtil.writeFlushMarker(wal, this.replicationScope, getRegionInfo(), desc, false, mvcc,
3082 null);
3083 } catch (Throwable ex) {
3084 LOG.warn(getRegionInfo().getEncodedName() + " : "
3085 + "failed writing ABORT_FLUSH marker to WAL", ex);
3086 // ignore this since we will be aborting the RS with DSE.
3088 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
3090 DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
3091 Bytes.toStringBinary(getRegionInfo().getRegionName()), t);
3092 status.abort("Flush failed: " + StringUtils.stringifyException(t));
3094 // Callers for flushcache() should catch DroppedSnapshotException and abort the region server.
3095 // However, since we may have the region read lock, we cannot call close(true) here since
3096 // we cannot promote to a write lock. Instead we are setting closing so that all other region
3097 // operations except for close will be rejected.
3098 this.closing.set(true);
3100 if (rsServices != null) {
3101 // This is a safeguard against the case where the caller fails to explicitly handle aborting
3102 rsServices.abort("Replay of WAL required. Forcing server shutdown", dse);
3105 throw dse;
3108 // If we get to here, the HStores have been written.
3109 if (wal != null) {
3110 wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes(), flushedSeqId);
3113 // Record latest flush time
3114 for (HStore store: storesToFlush) {
3115 this.lastStoreFlushTimeMap.put(store, startTime);
3118 this.maxFlushedSeqId = flushedSeqId;
3119 this.lastFlushOpSeqId = flushOpSeqId;
3121 // C. Finally notify anyone waiting on memstore to clear:
3122 // e.g. checkResources().
3123 synchronized (this) {
3124 notifyAll(); // FindBugs NN_NAKED_NOTIFY
3127 long time = EnvironmentEdgeManager.currentTime() - startTime;
3128 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3129 long memstoresize = this.memStoreSizing.getMemStoreSize().getDataSize();
3130 String msg = "Finished flush of"
3131 + " dataSize ~" + StringUtils.byteDesc(mss.getDataSize()) + "/" + mss.getDataSize()
3132 + ", heapSize ~" + StringUtils.byteDesc(mss.getHeapSize()) + "/" + mss.getHeapSize()
3133 + ", currentSize=" + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
3134 + " for " + this.getRegionInfo().getEncodedName() + " in " + time + "ms, sequenceid="
3135 + flushOpSeqId + ", compaction requested=" + compactionRequested
3136 + ((wal == null) ? "; wal=null" : "");
3137 LOG.info(msg);
3138 status.setStatus(msg);
3140 if (rsServices != null && rsServices.getMetrics() != null) {
3141 rsServices.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(),
3142 time,
3143 mss.getDataSize(), flushedOutputFileSize);
3146 return new FlushResultImpl(compactionRequested ?
3147 FlushResult.Result.FLUSHED_COMPACTION_NEEDED :
3148 FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId);
3152 * Method to safely get the next sequence number.
3153 * @return Next sequence number unassociated with any actual edit.
3154 * @throws IOException
3156 protected long getNextSequenceId(final WAL wal) throws IOException {
3157 WriteEntry we = mvcc.begin();
3158 mvcc.completeAndWait(we);
3159 return we.getWriteNumber();
3162 //////////////////////////////////////////////////////////////////////////////
3163 // get() methods for client use.
3164 //////////////////////////////////////////////////////////////////////////////
3166 @Override
3167 public RegionScannerImpl getScanner(Scan scan) throws IOException {
3168 return getScanner(scan, null);
3171 @Override
3172 public RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners)
3173 throws IOException {
3174 return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE);
3177 private RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
3178 long nonceGroup, long nonce) throws IOException {
3179 return TraceUtil.trace(() -> {
3180 startRegionOperation(Operation.SCAN);
3181 try {
3182 // Verify families are all valid
3183 if (!scan.hasFamilies()) {
3184 // Adding all families to scanner
3185 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
3186 scan.addFamily(family);
3188 } else {
3189 for (byte[] family : scan.getFamilyMap().keySet()) {
3190 checkFamily(family);
3193 return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
3194 } finally {
3195 closeRegionOperation(Operation.SCAN);
3197 }, () -> createRegionSpan("Region.getScanner"));
3200 protected RegionScannerImpl instantiateRegionScanner(Scan scan,
3201 List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException {
3202 if (scan.isReversed()) {
3203 if (scan.getFilter() != null) {
3204 scan.getFilter().setReversed(true);
3206 return new ReversedRegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3208 return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3212 * Prepare a delete for a row mutation processor
3213 * @param delete The passed delete is modified by this method. WARNING!
3215 private void prepareDelete(Delete delete) throws IOException {
3216 // Check to see if this is a deleteRow insert
3217 if(delete.getFamilyCellMap().isEmpty()){
3218 for(byte [] family : this.htableDescriptor.getColumnFamilyNames()){
3219 // Don't eat the timestamp
3220 delete.addFamily(family, delete.getTimestamp());
3222 } else {
3223 for(byte [] family : delete.getFamilyCellMap().keySet()) {
3224 if(family == null) {
3225 throw new NoSuchColumnFamilyException("Empty family is invalid");
3227 checkFamily(family, delete.getDurability());
3232 @Override
3233 public void delete(Delete delete) throws IOException {
3234 TraceUtil.trace(() -> {
3235 checkReadOnly();
3236 checkResources();
3237 startRegionOperation(Operation.DELETE);
3238 try {
3239 // All edits for the given row (across all column families) must happen atomically.
3240 return mutate(delete);
3241 } finally {
3242 closeRegionOperation(Operation.DELETE);
3244 }, () -> createRegionSpan("Region.delete"));
3248 * Set up correct timestamps in the KVs in Delete object.
3249 * <p/>
3250 * Caller should have the row and region locks.
3252 private void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap,
3253 byte[] byteNow) throws IOException {
3254 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3256 byte[] family = e.getKey();
3257 List<Cell> cells = e.getValue();
3258 assert cells instanceof RandomAccess;
3260 Map<byte[], Integer> kvCount = new TreeMap<>(Bytes.BYTES_COMPARATOR);
3261 int listSize = cells.size();
3262 for (int i=0; i < listSize; i++) {
3263 Cell cell = cells.get(i);
3264 // Check if time is LATEST, change to time of most recent addition if so
3265 // This is expensive.
3266 if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP
3267 && PrivateCellUtil.isDeleteType(cell)) {
3268 byte[] qual = CellUtil.cloneQualifier(cell);
3270 Integer count = kvCount.get(qual);
3271 if (count == null) {
3272 kvCount.put(qual, 1);
3273 } else {
3274 kvCount.put(qual, count + 1);
3276 count = kvCount.get(qual);
3278 Get get = new Get(CellUtil.cloneRow(cell));
3279 get.readVersions(count);
3280 get.addColumn(family, qual);
3281 if (coprocessorHost != null) {
3282 if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
3283 byteNow, get)) {
3284 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3286 } else {
3287 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3289 } else {
3290 PrivateCellUtil.updateLatestStamp(cell, byteNow);
3296 private void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
3297 throws IOException {
3298 try (RegionScanner scanner = getScanner(new Scan(get))) {
3299 // NOTE: Please don't use HRegion.get() instead,
3300 // because it will copy cells to heap. See HBASE-26036
3301 List<Cell> result = new ArrayList<>();
3302 scanner.next(result);
3304 if (result.size() < count) {
3305 // Nothing to delete
3306 PrivateCellUtil.updateLatestStamp(cell, byteNow);
3307 return;
3309 if (result.size() > count) {
3310 throw new RuntimeException("Unexpected size: " + result.size());
3312 Cell getCell = result.get(count - 1);
3313 PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
3317 @Override
3318 public void put(Put put) throws IOException {
3319 TraceUtil.trace(() -> {
3320 checkReadOnly();
3322 // Do a rough check that we have resources to accept a write. The check is
3323 // 'rough' in that between the resource check and the call to obtain a
3324 // read lock, resources may run out. For now, the thought is that this
3325 // will be extremely rare; we'll deal with it when it happens.
3326 checkResources();
3327 startRegionOperation(Operation.PUT);
3328 try {
3329 // All edits for the given row (across all column families) must happen atomically.
3330 return mutate(put);
3331 } finally {
3332 closeRegionOperation(Operation.PUT);
3334 }, () -> createRegionSpan("Region.put"));
3338 * Class that tracks the progress of a batch operations, accumulating status codes and tracking
3339 * the index at which processing is proceeding. These batch operations may get split into
3340 * mini-batches for processing.
3342 private abstract static class BatchOperation<T> {
3343 protected final T[] operations;
3344 protected final OperationStatus[] retCodeDetails;
3345 protected final WALEdit[] walEditsFromCoprocessors;
3346 // reference family cell maps directly so coprocessors can mutate them if desired
3347 protected final Map<byte[], List<Cell>>[] familyCellMaps;
3348 // For Increment/Append operations
3349 protected final Result[] results;
3351 protected final HRegion region;
3352 protected int nextIndexToProcess = 0;
3353 protected final ObservedExceptionsInBatch observedExceptions;
3354 //Durability of the batch (highest durability of all operations)
3355 protected Durability durability;
3356 protected boolean atomic = false;
3358 public BatchOperation(final HRegion region, T[] operations) {
3359 this.operations = operations;
3360 this.retCodeDetails = new OperationStatus[operations.length];
3361 Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
3362 this.walEditsFromCoprocessors = new WALEdit[operations.length];
3363 familyCellMaps = new Map[operations.length];
3364 this.results = new Result[operations.length];
3366 this.region = region;
3367 observedExceptions = new ObservedExceptionsInBatch();
3368 durability = Durability.USE_DEFAULT;
3372 * Visitor interface for batch operations
3374 @FunctionalInterface
3375 interface Visitor {
3377 * @param index operation index
3378 * @return If true continue visiting remaining entries, break otherwise
3380 boolean visit(int index) throws IOException;
3384 * Helper method for visiting pending/ all batch operations
3386 public void visitBatchOperations(boolean pendingOnly, int lastIndexExclusive, Visitor visitor)
3387 throws IOException {
3388 assert lastIndexExclusive <= this.size();
3389 for (int i = nextIndexToProcess; i < lastIndexExclusive; i++) {
3390 if (!pendingOnly || isOperationPending(i)) {
3391 if (!visitor.visit(i)) {
3392 break;
3398 public abstract Mutation getMutation(int index);
3400 public abstract long getNonceGroup(int index);
3402 public abstract long getNonce(int index);
3405 * This method is potentially expensive and useful mostly for non-replay CP path.
3407 public abstract Mutation[] getMutationsForCoprocs();
3409 public abstract boolean isInReplay();
3411 public abstract long getOrigLogSeqNum();
3413 public abstract void startRegionOperation() throws IOException;
3415 public abstract void closeRegionOperation() throws IOException;
3418 * Validates each mutation and prepares a batch for write. If necessary (non-replay case), runs
3419 * CP prePut()/preDelete()/preIncrement()/preAppend() hooks for all mutations in a batch. This
3420 * is intended to operate on entire batch and will be called from outside of class to check
3421 * and prepare batch. This can be implemented by calling helper method
3422 * {@link #checkAndPrepareMutation(int, long)} in a 'for' loop over mutations.
3424 public abstract void checkAndPrepare() throws IOException;
3427 * Implement any Put request specific check and prepare logic here. Please refer to
3428 * {@link #checkAndPrepareMutation(Mutation, long)} for how its used.
3430 protected abstract void checkAndPreparePut(final Put p) throws IOException;
3433 * If necessary, calls preBatchMutate() CP hook for a mini-batch and updates metrics, cell
3434 * count, tags and timestamp for all cells of all operations in a mini-batch.
3436 public abstract void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation>
3437 miniBatchOp, long timestamp, final List<RowLock> acquiredRowLocks) throws IOException;
3440 * Write mini-batch operations to MemStore
3442 public abstract WriteEntry writeMiniBatchOperationsToMemStore(
3443 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
3444 throws IOException;
3446 protected void writeMiniBatchOperationsToMemStore(
3447 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final long writeNumber)
3448 throws IOException {
3449 MemStoreSizing memStoreAccounting = new NonThreadSafeMemStoreSizing();
3450 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
3451 // We need to update the sequence id for following reasons.
3452 // 1) If the op is in replay mode, FSWALEntry#stampRegionSequenceId won't stamp sequence id.
3453 // 2) If no WAL, FSWALEntry won't be used
3454 // we use durability of the original mutation for the mutation passed by CP.
3455 if (isInReplay() || getMutation(index).getDurability() == Durability.SKIP_WAL) {
3456 region.updateSequenceId(familyCellMaps[index].values(), writeNumber);
3458 applyFamilyMapToMemStore(familyCellMaps[index], memStoreAccounting);
3459 return true;
3461 // update memStore size
3462 region.incMemStoreSize(memStoreAccounting.getDataSize(), memStoreAccounting.getHeapSize(),
3463 memStoreAccounting.getOffHeapSize(), memStoreAccounting.getCellsCount());
3466 public boolean isDone() {
3467 return nextIndexToProcess == operations.length;
3470 public int size() {
3471 return operations.length;
3474 public boolean isOperationPending(int index) {
3475 return retCodeDetails[index].getOperationStatusCode() == OperationStatusCode.NOT_RUN;
3478 public List<UUID> getClusterIds() {
3479 assert size() != 0;
3480 return getMutation(0).getClusterIds();
3483 boolean isAtomic() {
3484 return atomic;
3488 * Helper method that checks and prepares only one mutation. This can be used to implement
3489 * {@link #checkAndPrepare()} for entire Batch.
3490 * NOTE: As CP prePut()/preDelete()/preIncrement()/preAppend() hooks may modify mutations,
3491 * this method should be called after prePut()/preDelete()/preIncrement()/preAppend() CP hooks
3492 * are run for the mutation
3494 protected void checkAndPrepareMutation(Mutation mutation, final long timestamp)
3495 throws IOException {
3496 region.checkRow(mutation.getRow(), "batchMutate");
3497 if (mutation instanceof Put) {
3498 // Check the families in the put. If bad, skip this one.
3499 checkAndPreparePut((Put) mutation);
3500 region.checkTimestamps(mutation.getFamilyCellMap(), timestamp);
3501 } else if (mutation instanceof Delete) {
3502 region.prepareDelete((Delete) mutation);
3503 } else if (mutation instanceof Increment || mutation instanceof Append) {
3504 region.checkFamilies(mutation.getFamilyCellMap().keySet(), mutation.getDurability());
3508 protected void checkAndPrepareMutation(int index, long timestamp) throws IOException {
3509 Mutation mutation = getMutation(index);
3510 try {
3511 this.checkAndPrepareMutation(mutation, timestamp);
3513 if (mutation instanceof Put || mutation instanceof Delete) {
3514 // store the family map reference to allow for mutations
3515 familyCellMaps[index] = mutation.getFamilyCellMap();
3518 // store durability for the batch (highest durability of all operations in the batch)
3519 Durability tmpDur = region.getEffectiveDurability(mutation.getDurability());
3520 if (tmpDur.ordinal() > durability.ordinal()) {
3521 durability = tmpDur;
3523 } catch (NoSuchColumnFamilyException nscfe) {
3524 final String msg = "No such column family in batch mutation in region " + this;
3525 if (observedExceptions.hasSeenNoSuchFamily()) {
3526 LOG.warn(msg + nscfe.getMessage());
3527 } else {
3528 LOG.warn(msg, nscfe);
3529 observedExceptions.sawNoSuchFamily();
3531 retCodeDetails[index] = new OperationStatus(
3532 OperationStatusCode.BAD_FAMILY, nscfe.getMessage());
3533 if (isAtomic()) { // fail, atomic means all or none
3534 throw nscfe;
3536 } catch (FailedSanityCheckException fsce) {
3537 final String msg = "Batch Mutation did not pass sanity check in region " + this;
3538 if (observedExceptions.hasSeenFailedSanityCheck()) {
3539 LOG.warn(msg + fsce.getMessage());
3540 } else {
3541 LOG.warn(msg, fsce);
3542 observedExceptions.sawFailedSanityCheck();
3544 retCodeDetails[index] = new OperationStatus(
3545 OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
3546 if (isAtomic()) {
3547 throw fsce;
3549 } catch (WrongRegionException we) {
3550 final String msg = "Batch mutation had a row that does not belong to this region " + this;
3551 if (observedExceptions.hasSeenWrongRegion()) {
3552 LOG.warn(msg + we.getMessage());
3553 } else {
3554 LOG.warn(msg, we);
3555 observedExceptions.sawWrongRegion();
3557 retCodeDetails[index] = new OperationStatus(
3558 OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
3559 if (isAtomic()) {
3560 throw we;
3566 * Creates Mini-batch of all operations [nextIndexToProcess, lastIndexExclusive) for which
3567 * a row lock can be acquired. All mutations with locked rows are considered to be
3568 * In-progress operations and hence the name {@link MiniBatchOperationInProgress}. Mini batch
3569 * is window over {@link BatchOperation} and contains contiguous pending operations.
3571 * @param acquiredRowLocks keeps track of rowLocks acquired.
3573 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch(
3574 List<RowLock> acquiredRowLocks) throws IOException {
3575 int readyToWriteCount = 0;
3576 int lastIndexExclusive = 0;
3577 RowLock prevRowLock = null;
3578 for (; lastIndexExclusive < size(); lastIndexExclusive++) {
3579 // It reaches the miniBatchSize, stop here and process the miniBatch
3580 // This only applies to non-atomic batch operations.
3581 if (!isAtomic() && (readyToWriteCount == region.miniBatchSize)) {
3582 break;
3585 if (!isOperationPending(lastIndexExclusive)) {
3586 continue;
3589 // HBASE-19389 Limit concurrency of put with dense (hundreds) columns to avoid exhausting
3590 // RS handlers, covering both MutationBatchOperation and ReplayBatchOperation
3591 // The BAD_FAMILY/SANITY_CHECK_FAILURE cases are handled in checkAndPrepare phase and won't
3592 // pass the isOperationPending check
3593 Map<byte[], List<Cell>> curFamilyCellMap =
3594 getMutation(lastIndexExclusive).getFamilyCellMap();
3595 try {
3596 // start the protector before acquiring row lock considering performance, and will finish
3597 // it when encountering exception
3598 region.storeHotnessProtector.start(curFamilyCellMap);
3599 } catch (RegionTooBusyException rtbe) {
3600 region.storeHotnessProtector.finish(curFamilyCellMap);
3601 if (isAtomic()) {
3602 throw rtbe;
3604 retCodeDetails[lastIndexExclusive] =
3605 new OperationStatus(OperationStatusCode.STORE_TOO_BUSY, rtbe.getMessage());
3606 continue;
3609 Mutation mutation = getMutation(lastIndexExclusive);
3610 // If we haven't got any rows in our batch, we should block to get the next one.
3611 RowLock rowLock = null;
3612 boolean throwException = false;
3613 try {
3614 // if atomic then get exclusive lock, else shared lock
3615 rowLock = region.getRowLock(mutation.getRow(), !isAtomic(), prevRowLock);
3616 } catch (TimeoutIOException | InterruptedIOException e) {
3617 // NOTE: We will retry when other exceptions, but we should stop if we receive
3618 // TimeoutIOException or InterruptedIOException as operation has timed out or
3619 // interrupted respectively.
3620 throwException = true;
3621 throw e;
3622 } catch (IOException ioe) {
3623 LOG.warn("Failed getting lock, row={}, in region {}",
3624 Bytes.toStringBinary(mutation.getRow()), this, ioe);
3625 if (isAtomic()) { // fail, atomic means all or none
3626 throwException = true;
3627 throw ioe;
3629 } catch (Throwable throwable) {
3630 throwException = true;
3631 throw throwable;
3632 } finally {
3633 if (throwException) {
3634 region.storeHotnessProtector.finish(curFamilyCellMap);
3637 if (rowLock == null) {
3638 // We failed to grab another lock
3639 if (isAtomic()) {
3640 region.storeHotnessProtector.finish(curFamilyCellMap);
3641 throw new IOException("Can't apply all operations atomically!");
3643 break; // Stop acquiring more rows for this batch
3644 } else {
3645 if (rowLock != prevRowLock) {
3646 // It is a different row now, add this to the acquiredRowLocks and
3647 // set prevRowLock to the new returned rowLock
3648 acquiredRowLocks.add(rowLock);
3649 prevRowLock = rowLock;
3653 readyToWriteCount++;
3655 return createMiniBatch(lastIndexExclusive, readyToWriteCount);
3658 protected MiniBatchOperationInProgress<Mutation> createMiniBatch(final int lastIndexExclusive,
3659 final int readyToWriteCount) {
3660 return new MiniBatchOperationInProgress<>(getMutationsForCoprocs(), retCodeDetails,
3661 walEditsFromCoprocessors, nextIndexToProcess, lastIndexExclusive, readyToWriteCount);
3665 * Builds separate WALEdit per nonce by applying input mutations. If WALEdits from CP are
3666 * present, they are merged to result WALEdit.
3668 public List<Pair<NonceKey, WALEdit>> buildWALEdits(
3669 final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
3670 List<Pair<NonceKey, WALEdit>> walEdits = new ArrayList<>();
3672 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), new Visitor() {
3673 private Pair<NonceKey, WALEdit> curWALEditForNonce;
3675 @Override
3676 public boolean visit(int index) throws IOException {
3677 Mutation m = getMutation(index);
3678 // we use durability of the original mutation for the mutation passed by CP.
3679 if (region.getEffectiveDurability(m.getDurability()) == Durability.SKIP_WAL) {
3680 region.recordMutationWithoutWal(m.getFamilyCellMap());
3681 return true;
3684 // the batch may contain multiple nonce keys (replay case). If so, write WALEdit for each.
3685 // Given how nonce keys are originally written, these should be contiguous.
3686 // They don't have to be, it will still work, just write more WALEdits than needed.
3687 long nonceGroup = getNonceGroup(index);
3688 long nonce = getNonce(index);
3689 if (curWALEditForNonce == null ||
3690 curWALEditForNonce.getFirst().getNonceGroup() != nonceGroup ||
3691 curWALEditForNonce.getFirst().getNonce() != nonce) {
3692 curWALEditForNonce = new Pair<>(new NonceKey(nonceGroup, nonce),
3693 new WALEdit(miniBatchOp.getCellCount(), isInReplay()));
3694 walEdits.add(curWALEditForNonce);
3696 WALEdit walEdit = curWALEditForNonce.getSecond();
3698 // Add WAL edits from CPs.
3699 WALEdit fromCP = walEditsFromCoprocessors[index];
3700 if (fromCP != null) {
3701 for (Cell cell : fromCP.getCells()) {
3702 walEdit.add(cell);
3705 walEdit.add(familyCellMaps[index]);
3707 return true;
3710 return walEdits;
3714 * This method completes mini-batch operations by calling postBatchMutate() CP hook (if
3715 * required) and completing mvcc.
3717 public void completeMiniBatchOperations(
3718 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
3719 throws IOException {
3720 if (writeEntry != null) {
3721 region.mvcc.completeAndWait(writeEntry);
3725 public void doPostOpCleanupForMiniBatch(
3726 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WALEdit walEdit,
3727 boolean success) throws IOException {
3728 doFinishHotnessProtector(miniBatchOp);
3731 private void doFinishHotnessProtector(
3732 final MiniBatchOperationInProgress<Mutation> miniBatchOp) {
3733 // check and return if the protector is not enabled
3734 if (!region.storeHotnessProtector.isEnable()) {
3735 return;
3737 // miniBatchOp is null, if and only if lockRowsAndBuildMiniBatch throwing exception.
3738 // This case was handled.
3739 if (miniBatchOp == null) {
3740 return;
3743 final int finalLastIndexExclusive = miniBatchOp.getLastIndexExclusive();
3745 for (int i = nextIndexToProcess; i < finalLastIndexExclusive; i++) {
3746 switch (retCodeDetails[i].getOperationStatusCode()) {
3747 case SUCCESS:
3748 case FAILURE:
3749 region.storeHotnessProtector.finish(getMutation(i).getFamilyCellMap());
3750 break;
3751 default:
3752 // do nothing
3753 // We won't start the protector for NOT_RUN/BAD_FAMILY/SANITY_CHECK_FAILURE and the
3754 // STORE_TOO_BUSY case is handled in StoreHotnessProtector#start
3755 break;
3761 * Atomically apply the given map of family->edits to the memstore.
3762 * This handles the consistency control on its own, but the caller
3763 * should already have locked updatesLock.readLock(). This also does
3764 * <b>not</b> check the families for validity.
3766 * @param familyMap Map of Cells by family
3768 protected void applyFamilyMapToMemStore(Map<byte[], List<Cell>> familyMap,
3769 MemStoreSizing memstoreAccounting) throws IOException {
3770 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3771 byte[] family = e.getKey();
3772 List<Cell> cells = e.getValue();
3773 assert cells instanceof RandomAccess;
3774 region.applyToMemStore(region.getStore(family), cells, false, memstoreAccounting);
3781 * Batch of mutation operations. Base class is shared with {@link ReplayBatchOperation} as most of
3782 * the logic is same.
3784 private static class MutationBatchOperation extends BatchOperation<Mutation> {
3786 // For nonce operations
3787 private long nonceGroup;
3788 private long nonce;
3789 protected boolean canProceed;
3791 public MutationBatchOperation(final HRegion region, Mutation[] operations, boolean atomic,
3792 long nonceGroup, long nonce) {
3793 super(region, operations);
3794 this.atomic = atomic;
3795 this.nonceGroup = nonceGroup;
3796 this.nonce = nonce;
3799 @Override
3800 public Mutation getMutation(int index) {
3801 return this.operations[index];
3804 @Override
3805 public long getNonceGroup(int index) {
3806 return nonceGroup;
3809 @Override
3810 public long getNonce(int index) {
3811 return nonce;
3814 @Override
3815 public Mutation[] getMutationsForCoprocs() {
3816 return this.operations;
3819 @Override
3820 public boolean isInReplay() {
3821 return false;
3824 @Override
3825 public long getOrigLogSeqNum() {
3826 return SequenceId.NO_SEQUENCE_ID;
3829 @Override
3830 public void startRegionOperation() throws IOException {
3831 region.startRegionOperation(Operation.BATCH_MUTATE);
3834 @Override
3835 public void closeRegionOperation() throws IOException {
3836 region.closeRegionOperation(Operation.BATCH_MUTATE);
3839 @Override
3840 public void checkAndPreparePut(Put p) throws IOException {
3841 region.checkFamilies(p.getFamilyCellMap().keySet(), p.getDurability());
3844 @Override
3845 public void checkAndPrepare() throws IOException {
3846 // index 0: puts, index 1: deletes, index 2: increments, index 3: append
3847 final int[] metrics = {0, 0, 0, 0};
3849 visitBatchOperations(true, this.size(), new Visitor() {
3850 private long now = EnvironmentEdgeManager.currentTime();
3851 private WALEdit walEdit;
3852 @Override
3853 public boolean visit(int index) throws IOException {
3854 // Run coprocessor pre hook outside of locks to avoid deadlock
3855 if (region.coprocessorHost != null) {
3856 if (walEdit == null) {
3857 walEdit = new WALEdit();
3859 callPreMutateCPHook(index, walEdit, metrics);
3860 if (!walEdit.isEmpty()) {
3861 walEditsFromCoprocessors[index] = walEdit;
3862 walEdit = null;
3865 if (isOperationPending(index)) {
3866 // TODO: Currently validation is done with current time before acquiring locks and
3867 // updates are done with different timestamps after acquiring locks. This behavior is
3868 // inherited from the code prior to this change. Can this be changed?
3869 checkAndPrepareMutation(index, now);
3871 return true;
3875 // FIXME: we may update metrics twice! here for all operations bypassed by CP and later in
3876 // normal processing.
3877 // Update metrics in same way as it is done when we go the normal processing route (we now
3878 // update general metrics though a Coprocessor did the work).
3879 if (region.metricsRegion != null) {
3880 if (metrics[0] > 0) {
3881 // There were some Puts in the batch.
3882 region.metricsRegion.updatePut();
3884 if (metrics[1] > 0) {
3885 // There were some Deletes in the batch.
3886 region.metricsRegion.updateDelete();
3888 if (metrics[2] > 0) {
3889 // There were some Increment in the batch.
3890 region.metricsRegion.updateIncrement();
3892 if (metrics[3] > 0) {
3893 // There were some Append in the batch.
3894 region.metricsRegion.updateAppend();
3899 @Override
3900 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
3901 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
3902 // For nonce operations
3903 canProceed = startNonceOperation();
3905 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
3906 Mutation mutation = getMutation(index);
3907 if (mutation instanceof Put) {
3908 HRegion.updateCellTimestamps(familyCellMaps[index].values(), Bytes.toBytes(timestamp));
3909 miniBatchOp.incrementNumOfPuts();
3910 } else if (mutation instanceof Delete) {
3911 region.prepareDeleteTimestamps(mutation, familyCellMaps[index],
3912 Bytes.toBytes(timestamp));
3913 miniBatchOp.incrementNumOfDeletes();
3914 } else if (mutation instanceof Increment || mutation instanceof Append) {
3915 boolean returnResults;
3916 if (mutation instanceof Increment) {
3917 returnResults = ((Increment) mutation).isReturnResults();
3918 } else {
3919 returnResults = ((Append) mutation).isReturnResults();
3922 // For nonce operations
3923 if (!canProceed) {
3924 Result result;
3925 if (returnResults) {
3926 // convert duplicate increment/append to get
3927 List<Cell> results = region.get(toGet(mutation), false, nonceGroup, nonce);
3928 result = Result.create(results);
3929 } else {
3930 result = Result.EMPTY_RESULT;
3932 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
3933 return true;
3936 Result result = null;
3937 if (region.coprocessorHost != null) {
3938 if (mutation instanceof Increment) {
3939 result = region.coprocessorHost.preIncrementAfterRowLock((Increment) mutation);
3940 } else {
3941 result = region.coprocessorHost.preAppendAfterRowLock((Append) mutation);
3944 if (result != null) {
3945 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS,
3946 returnResults ? result : Result.EMPTY_RESULT);
3947 return true;
3950 List<Cell> results = returnResults ? new ArrayList<>(mutation.size()) : null;
3951 familyCellMaps[index] = reckonDeltas(mutation, results, timestamp);
3952 this.results[index] = results != null ? Result.create(results) : Result.EMPTY_RESULT;
3954 if (mutation instanceof Increment) {
3955 miniBatchOp.incrementNumOfIncrements();
3956 } else {
3957 miniBatchOp.incrementNumOfAppends();
3960 region.rewriteCellTags(familyCellMaps[index], mutation);
3962 // update cell count
3963 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
3964 for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
3965 miniBatchOp.addCellCount(cells.size());
3969 WALEdit fromCP = walEditsFromCoprocessors[index];
3970 if (fromCP != null) {
3971 miniBatchOp.addCellCount(fromCP.size());
3973 return true;
3976 if (region.coprocessorHost != null) {
3977 // calling the pre CP hook for batch mutation
3978 region.coprocessorHost.preBatchMutate(miniBatchOp);
3979 checkAndMergeCPMutations(miniBatchOp, acquiredRowLocks, timestamp);
3984 * Starts the nonce operation for a mutation, if needed.
3985 * @return whether to proceed this mutation.
3987 private boolean startNonceOperation() throws IOException {
3988 if (region.rsServices == null || region.rsServices.getNonceManager() == null
3989 || nonce == HConstants.NO_NONCE) {
3990 return true;
3992 boolean canProceed;
3993 try {
3994 canProceed = region.rsServices.getNonceManager()
3995 .startOperation(nonceGroup, nonce, region.rsServices);
3996 } catch (InterruptedException ex) {
3997 throw new InterruptedIOException("Nonce start operation interrupted");
3999 return canProceed;
4003 * Ends nonce operation for a mutation, if needed.
4004 * @param success Whether the operation for this nonce has succeeded.
4006 private void endNonceOperation(boolean success) {
4007 if (region.rsServices != null && region.rsServices.getNonceManager() != null
4008 && nonce != HConstants.NO_NONCE) {
4009 region.rsServices.getNonceManager().endOperation(nonceGroup, nonce, success);
4013 private static Get toGet(final Mutation mutation) throws IOException {
4014 assert mutation instanceof Increment || mutation instanceof Append;
4015 Get get = new Get(mutation.getRow());
4016 CellScanner cellScanner = mutation.cellScanner();
4017 while (!cellScanner.advance()) {
4018 Cell cell = cellScanner.current();
4019 get.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell));
4021 if (mutation instanceof Increment) {
4022 // Increment
4023 Increment increment = (Increment) mutation;
4024 get.setTimeRange(increment.getTimeRange().getMin(), increment.getTimeRange().getMax());
4025 } else {
4026 // Append
4027 Append append = (Append) mutation;
4028 get.setTimeRange(append.getTimeRange().getMin(), append.getTimeRange().getMax());
4030 for (Entry<String, byte[]> entry : mutation.getAttributesMap().entrySet()) {
4031 get.setAttribute(entry.getKey(), entry.getValue());
4033 return get;
4036 private Map<byte[], List<Cell>> reckonDeltas(Mutation mutation, List<Cell> results,
4037 long now) throws IOException {
4038 assert mutation instanceof Increment || mutation instanceof Append;
4039 Map<byte[], List<Cell>> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR);
4040 // Process a Store/family at a time.
4041 for (Map.Entry<byte [], List<Cell>> entry: mutation.getFamilyCellMap().entrySet()) {
4042 final byte[] columnFamilyName = entry.getKey();
4043 List<Cell> deltas = entry.getValue();
4044 // Reckon for the Store what to apply to WAL and MemStore.
4045 List<Cell> toApply = reckonDeltasByStore(region.stores.get(columnFamilyName), mutation,
4046 now, deltas, results);
4047 if (!toApply.isEmpty()) {
4048 for (Cell cell : toApply) {
4049 HStore store = region.getStore(cell);
4050 if (store == null) {
4051 region.checkFamily(CellUtil.cloneFamily(cell));
4052 } else {
4053 ret.computeIfAbsent(store.getColumnFamilyDescriptor().getName(),
4054 key -> new ArrayList<>()).add(cell);
4059 return ret;
4063 * Reckon the Cells to apply to WAL, memstore, and to return to the Client in passed
4064 * column family/Store.
4066 * Does Get of current value and then adds passed in deltas for this Store returning the
4067 * result.
4069 * @param mutation The encompassing Mutation object
4070 * @param deltas Changes to apply to this Store; either increment amount or data to append
4071 * @param results In here we accumulate all the Cells we are to return to the client. If null,
4072 * client doesn't want results returned.
4073 * @return Resulting Cells after <code>deltas</code> have been applied to current
4074 * values. Side effect is our filling out of the <code>results</code> List.
4076 private List<Cell> reckonDeltasByStore(HStore store, Mutation mutation, long now,
4077 List<Cell> deltas, List<Cell> results) throws IOException {
4078 assert mutation instanceof Increment || mutation instanceof Append;
4079 byte[] columnFamily = store.getColumnFamilyDescriptor().getName();
4080 List<Pair<Cell, Cell>> cellPairs = new ArrayList<>(deltas.size());
4082 // Sort the cells so that they match the order that they appear in the Get results.
4083 // Otherwise, we won't be able to find the existing values if the cells are not specified
4084 // in order by the client since cells are in an array list.
4085 deltas.sort(store.getComparator());
4087 // Get previous values for all columns in this family.
4088 Get get = new Get(mutation.getRow());
4089 for (Cell cell: deltas) {
4090 get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
4092 TimeRange tr;
4093 if (mutation instanceof Increment) {
4094 tr = ((Increment) mutation).getTimeRange();
4095 } else {
4096 tr = ((Append) mutation).getTimeRange();
4099 if (tr != null) {
4100 get.setTimeRange(tr.getMin(), tr.getMax());
4103 try (RegionScanner scanner = region.getScanner(new Scan(get))) {
4104 // NOTE: Please don't use HRegion.get() instead,
4105 // because it will copy cells to heap. See HBASE-26036
4106 List<Cell> currentValues = new ArrayList<>();
4107 scanner.next(currentValues);
4108 // Iterate the input columns and update existing values if they were found, otherwise
4109 // add new column initialized to the delta amount
4110 int currentValuesIndex = 0;
4111 for (int i = 0; i < deltas.size(); i++) {
4112 Cell delta = deltas.get(i);
4113 Cell currentValue = null;
4114 if (currentValuesIndex < currentValues.size() && CellUtil
4115 .matchingQualifier(currentValues.get(currentValuesIndex), delta)) {
4116 currentValue = currentValues.get(currentValuesIndex);
4117 if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) {
4118 currentValuesIndex++;
4121 // Switch on whether this an increment or an append building the new Cell to apply.
4122 Cell newCell;
4123 if (mutation instanceof Increment) {
4124 long deltaAmount = getLongValue(delta);
4125 final long newValue = currentValue == null ? deltaAmount :
4126 getLongValue(currentValue) + deltaAmount;
4127 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4128 (oldCell) -> Bytes.toBytes(newValue));
4129 } else {
4130 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4131 (oldCell) -> ByteBuffer.wrap(new byte[delta.getValueLength() +
4132 oldCell.getValueLength()])
4133 .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength())
4134 .put(delta.getValueArray(), delta.getValueOffset(), delta.getValueLength())
4135 .array());
4137 if (region.maxCellSize > 0) {
4138 int newCellSize = PrivateCellUtil.estimatedSerializedSizeOf(newCell);
4139 if (newCellSize > region.maxCellSize) {
4140 String msg =
4141 "Cell with size " + newCellSize + " exceeds limit of " + region.maxCellSize +
4142 " bytes in region " + this;
4143 LOG.debug(msg);
4144 throw new DoNotRetryIOException(msg);
4147 cellPairs.add(new Pair<>(currentValue, newCell));
4148 // Add to results to get returned to the Client. If null, cilent does not want results.
4149 if (results != null) {
4150 results.add(newCell);
4153 // Give coprocessors a chance to update the new cells before apply to WAL or memstore
4154 if (region.coprocessorHost != null) {
4155 // Here the operation must be increment or append.
4156 cellPairs = mutation instanceof Increment ?
4157 region.coprocessorHost.postIncrementBeforeWAL(mutation, cellPairs) :
4158 region.coprocessorHost.postAppendBeforeWAL(mutation, cellPairs);
4161 return cellPairs.stream().map(Pair::getSecond).collect(Collectors.toList());
4164 private static Cell reckonDelta(final Cell delta, final Cell currentCell,
4165 final byte[] columnFamily, final long now, Mutation mutation,
4166 Function<Cell, byte[]> supplier) throws IOException {
4167 // Forward any tags found on the delta.
4168 List<Tag> tags = TagUtil.carryForwardTags(delta);
4169 if (currentCell != null) {
4170 tags = TagUtil.carryForwardTags(tags, currentCell);
4171 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4172 byte[] newValue = supplier.apply(currentCell);
4173 return ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
4174 .setRow(mutation.getRow(), 0, mutation.getRow().length)
4175 .setFamily(columnFamily, 0, columnFamily.length)
4176 // copy the qualifier if the cell is located in shared memory.
4177 .setQualifier(CellUtil.cloneQualifier(delta))
4178 .setTimestamp(Math.max(currentCell.getTimestamp() + 1, now))
4179 .setType(KeyValue.Type.Put.getCode())
4180 .setValue(newValue, 0, newValue.length)
4181 .setTags(TagUtil.fromList(tags))
4182 .build();
4183 } else {
4184 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4185 PrivateCellUtil.updateLatestStamp(delta, now);
4186 return CollectionUtils.isEmpty(tags) ? delta : PrivateCellUtil.createCell(delta, tags);
4191 * @return Get the long out of the passed in Cell
4193 private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
4194 int len = cell.getValueLength();
4195 if (len != Bytes.SIZEOF_LONG) {
4196 // throw DoNotRetryIOException instead of IllegalArgumentException
4197 throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
4199 return PrivateCellUtil.getValueAsLong(cell);
4202 @Override
4203 public List<Pair<NonceKey, WALEdit>> buildWALEdits(final MiniBatchOperationInProgress<Mutation>
4204 miniBatchOp) throws IOException {
4205 List<Pair<NonceKey, WALEdit>> walEdits = super.buildWALEdits(miniBatchOp);
4206 // for MutationBatchOperation, more than one nonce is not allowed
4207 if (walEdits.size() > 1) {
4208 throw new IOException("Found multiple nonce keys per batch!");
4210 return walEdits;
4213 @Override
4214 public WriteEntry writeMiniBatchOperationsToMemStore(
4215 final MiniBatchOperationInProgress<Mutation> miniBatchOp, @Nullable WriteEntry writeEntry)
4216 throws IOException {
4217 if (writeEntry == null) {
4218 writeEntry = region.mvcc.begin();
4220 super.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry.getWriteNumber());
4221 return writeEntry;
4224 @Override
4225 public void completeMiniBatchOperations(
4226 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4227 throws IOException {
4228 // TODO: can it be done after completing mvcc?
4229 // calling the post CP hook for batch mutation
4230 if (region.coprocessorHost != null) {
4231 region.coprocessorHost.postBatchMutate(miniBatchOp);
4233 super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4235 if (nonce != HConstants.NO_NONCE) {
4236 if (region.rsServices != null && region.rsServices.getNonceManager() != null) {
4237 region.rsServices.getNonceManager()
4238 .addMvccToOperationContext(nonceGroup, nonce, writeEntry.getWriteNumber());
4243 @Override
4244 public void doPostOpCleanupForMiniBatch(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4245 final WALEdit walEdit, boolean success) throws IOException {
4247 super.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, success);
4248 if (miniBatchOp != null) {
4249 // synced so that the coprocessor contract is adhered to.
4250 if (region.coprocessorHost != null) {
4251 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4252 // only for successful puts/deletes/increments/appends
4253 if (retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.SUCCESS) {
4254 Mutation m = getMutation(i);
4255 if (m instanceof Put) {
4256 region.coprocessorHost.postPut((Put) m, walEdit);
4257 } else if (m instanceof Delete) {
4258 region.coprocessorHost.postDelete((Delete) m, walEdit);
4259 } else if (m instanceof Increment) {
4260 Result result = region.getCoprocessorHost().postIncrement((Increment) m,
4261 results[i], walEdit);
4262 if (result != results[i]) {
4263 retCodeDetails[i] =
4264 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4266 } else if (m instanceof Append) {
4267 Result result = region.getCoprocessorHost().postAppend((Append) m, results[i],
4268 walEdit);
4269 if (result != results[i]) {
4270 retCodeDetails[i] =
4271 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4275 return true;
4279 // For nonce operations
4280 if (canProceed && nonce != HConstants.NO_NONCE) {
4281 boolean[] areAllIncrementsAndAppendsSuccessful = new boolean[]{true};
4282 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4283 Mutation mutation = getMutation(i);
4284 if (mutation instanceof Increment || mutation instanceof Append) {
4285 if (retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
4286 areAllIncrementsAndAppendsSuccessful[0] = false;
4287 return false;
4290 return true;
4292 endNonceOperation(areAllIncrementsAndAppendsSuccessful[0]);
4295 // See if the column families were consistent through the whole thing.
4296 // if they were then keep them. If they were not then pass a null.
4297 // null will be treated as unknown.
4298 // Total time taken might be involving Puts, Deletes, Increments and Appends.
4299 // Split the time for puts and deletes based on the total number of Puts, Deletes,
4300 // Increments and Appends.
4301 if (region.metricsRegion != null) {
4302 if (miniBatchOp.getNumOfPuts() > 0) {
4303 // There were some Puts in the batch.
4304 region.metricsRegion.updatePut();
4306 if (miniBatchOp.getNumOfDeletes() > 0) {
4307 // There were some Deletes in the batch.
4308 region.metricsRegion.updateDelete();
4310 if (miniBatchOp.getNumOfIncrements() > 0) {
4311 // There were some Increments in the batch.
4312 region.metricsRegion.updateIncrement();
4314 if (miniBatchOp.getNumOfAppends() > 0) {
4315 // There were some Appends in the batch.
4316 region.metricsRegion.updateAppend();
4321 if (region.coprocessorHost != null) {
4322 // call the coprocessor hook to do any finalization steps after the put is done
4323 region.coprocessorHost.postBatchMutateIndispensably(
4324 miniBatchOp != null ? miniBatchOp : createMiniBatch(size(), 0), success);
4329 * Runs prePut/preDelete/preIncrement/preAppend coprocessor hook for input mutation in a batch
4330 * @param metrics Array of 2 ints. index 0: count of puts, index 1: count of deletes, index 2:
4331 * count of increments and 3: count of appends
4333 private void callPreMutateCPHook(int index, final WALEdit walEdit, final int[] metrics)
4334 throws IOException {
4335 Mutation m = getMutation(index);
4336 if (m instanceof Put) {
4337 if (region.coprocessorHost.prePut((Put) m, walEdit)) {
4338 // pre hook says skip this Put
4339 // mark as success and skip in doMiniBatchMutation
4340 metrics[0]++;
4341 retCodeDetails[index] = OperationStatus.SUCCESS;
4343 } else if (m instanceof Delete) {
4344 Delete curDel = (Delete) m;
4345 if (curDel.getFamilyCellMap().isEmpty()) {
4346 // handle deleting a row case
4347 // TODO: prepareDelete() has been called twice, before and after preDelete() CP hook.
4348 // Can this be avoided?
4349 region.prepareDelete(curDel);
4351 if (region.coprocessorHost.preDelete(curDel, walEdit)) {
4352 // pre hook says skip this Delete
4353 // mark as success and skip in doMiniBatchMutation
4354 metrics[1]++;
4355 retCodeDetails[index] = OperationStatus.SUCCESS;
4357 } else if (m instanceof Increment) {
4358 Increment increment = (Increment) m;
4359 Result result = region.coprocessorHost.preIncrement(increment, walEdit);
4360 if (result != null) {
4361 // pre hook says skip this Increment
4362 // mark as success and skip in doMiniBatchMutation
4363 metrics[2]++;
4364 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4366 } else if (m instanceof Append) {
4367 Append append = (Append) m;
4368 Result result = region.coprocessorHost.preAppend(append, walEdit);
4369 if (result != null) {
4370 // pre hook says skip this Append
4371 // mark as success and skip in doMiniBatchMutation
4372 metrics[3]++;
4373 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4375 } else {
4376 String msg = "Put/Delete/Increment/Append mutations only supported in a batch";
4377 retCodeDetails[index] = new OperationStatus(OperationStatusCode.FAILURE, msg);
4378 if (isAtomic()) { // fail, atomic means all or none
4379 throw new IOException(msg);
4384 // TODO Support Increment/Append operations
4385 private void checkAndMergeCPMutations(final MiniBatchOperationInProgress<Mutation> miniBatchOp,
4386 final List<RowLock> acquiredRowLocks, final long timestamp) throws IOException {
4387 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), (int i) -> {
4388 // we pass (i - firstIndex) below since the call expects a relative index
4389 Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - nextIndexToProcess);
4390 if (cpMutations == null) {
4391 return true;
4393 // Else Coprocessor added more Mutations corresponding to the Mutation at this index.
4394 Mutation mutation = getMutation(i);
4395 for (Mutation cpMutation : cpMutations) {
4396 this.checkAndPrepareMutation(cpMutation, timestamp);
4398 // Acquire row locks. If not, the whole batch will fail.
4399 acquiredRowLocks.add(region.getRowLock(cpMutation.getRow(), true, null));
4401 // Returned mutations from coprocessor correspond to the Mutation at index i. We can
4402 // directly add the cells from those mutations to the familyMaps of this mutation.
4403 Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap();
4404 region.rewriteCellTags(cpFamilyMap, mutation);
4405 // will get added to the memStore later
4406 mergeFamilyMaps(familyCellMaps[i], cpFamilyMap);
4408 // The durability of returned mutation is replaced by the corresponding mutation.
4409 // If the corresponding mutation contains the SKIP_WAL, we shouldn't count the
4410 // cells of returned mutation.
4411 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
4412 for (List<Cell> cells : cpFamilyMap.values()) {
4413 miniBatchOp.addCellCount(cells.size());
4417 return true;
4421 private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
4422 Map<byte[], List<Cell>> toBeMerged) {
4423 for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
4424 List<Cell> cells = familyMap.get(entry.getKey());
4425 if (cells == null) {
4426 familyMap.put(entry.getKey(), entry.getValue());
4427 } else {
4428 cells.addAll(entry.getValue());
4435 * Batch of mutations for replay. Base class is shared with {@link MutationBatchOperation} as most
4436 * of the logic is same.
4437 * @deprecated Since 3.0.0, will be removed in 4.0.0. Now we will not use this operation to apply
4438 * edits at secondary replica side.
4440 @Deprecated
4441 private static final class ReplayBatchOperation extends BatchOperation<MutationReplay> {
4443 private long origLogSeqNum = 0;
4445 public ReplayBatchOperation(final HRegion region, MutationReplay[] operations,
4446 long origLogSeqNum) {
4447 super(region, operations);
4448 this.origLogSeqNum = origLogSeqNum;
4451 @Override
4452 public Mutation getMutation(int index) {
4453 return this.operations[index].mutation;
4456 @Override
4457 public long getNonceGroup(int index) {
4458 return this.operations[index].nonceGroup;
4461 @Override
4462 public long getNonce(int index) {
4463 return this.operations[index].nonce;
4466 @Override
4467 public Mutation[] getMutationsForCoprocs() {
4468 return null;
4471 @Override
4472 public boolean isInReplay() {
4473 return true;
4476 @Override
4477 public long getOrigLogSeqNum() {
4478 return this.origLogSeqNum;
4481 @Override
4482 public void startRegionOperation() throws IOException {
4483 region.startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4486 @Override
4487 public void closeRegionOperation() throws IOException {
4488 region.closeRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4492 * During replay, there could exist column families which are removed between region server
4493 * failure and replay
4495 @Override
4496 protected void checkAndPreparePut(Put p) throws IOException {
4497 Map<byte[], List<Cell>> familyCellMap = p.getFamilyCellMap();
4498 List<byte[]> nonExistentList = null;
4499 for (byte[] family : familyCellMap.keySet()) {
4500 if (!region.htableDescriptor.hasColumnFamily(family)) {
4501 if (nonExistentList == null) {
4502 nonExistentList = new ArrayList<>();
4504 nonExistentList.add(family);
4507 if (nonExistentList != null) {
4508 for (byte[] family : nonExistentList) {
4509 // Perhaps schema was changed between crash and replay
4510 LOG.info("No family for {} omit from reply in region {}.", Bytes.toString(family), this);
4511 familyCellMap.remove(family);
4516 @Override
4517 public void checkAndPrepare() throws IOException {
4518 long now = EnvironmentEdgeManager.currentTime();
4519 visitBatchOperations(true, this.size(), (int index) -> {
4520 checkAndPrepareMutation(index, now);
4521 return true;
4525 @Override
4526 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4527 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
4528 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
4529 // update cell count
4530 for (List<Cell> cells : getMutation(index).getFamilyCellMap().values()) {
4531 miniBatchOp.addCellCount(cells.size());
4533 return true;
4537 @Override
4538 public WriteEntry writeMiniBatchOperationsToMemStore(
4539 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4540 throws IOException {
4541 super.writeMiniBatchOperationsToMemStore(miniBatchOp, getOrigLogSeqNum());
4542 return writeEntry;
4545 @Override
4546 public void completeMiniBatchOperations(
4547 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4548 throws IOException {
4549 super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4550 region.mvcc.advanceTo(getOrigLogSeqNum());
4554 public OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic, long nonceGroup,
4555 long nonce) throws IOException {
4556 // As it stands, this is used for 3 things
4557 // * batchMutate with single mutation - put/delete/increment/append, separate or from
4558 // checkAndMutate.
4559 // * coprocessor calls (see ex. BulkDeleteEndpoint).
4560 // So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd...
4561 return batchMutate(new MutationBatchOperation(this, mutations, atomic, nonceGroup, nonce));
4564 @Override
4565 public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
4566 // If the mutations has any Increment/Append operations, we need to do batchMutate atomically
4567 boolean atomic =
4568 Arrays.stream(mutations).anyMatch(m -> m instanceof Increment || m instanceof Append);
4569 return batchMutate(mutations, atomic);
4572 OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic) throws IOException {
4573 return TraceUtil.trace(
4574 () -> batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE),
4575 () -> createRegionSpan("Region.batchMutate"));
4579 * @deprecated Since 3.0.0, will be removed in 4.0.0. Now we use
4580 * {@link #replayWALEntry(WALEntry, CellScanner)} for replaying edits at secondary
4581 * replica side.
4583 @Deprecated
4584 OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException {
4585 if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo())
4586 && replaySeqId < lastReplayedOpenRegionSeqId) {
4587 // if it is a secondary replica we should ignore these entries silently
4588 // since they are coming out of order
4589 if (LOG.isTraceEnabled()) {
4590 LOG.trace(getRegionInfo().getEncodedName() + " : "
4591 + "Skipping " + mutations.length + " mutations with replaySeqId=" + replaySeqId
4592 + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId);
4593 for (MutationReplay mut : mutations) {
4594 LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation);
4598 OperationStatus[] statuses = new OperationStatus[mutations.length];
4599 for (int i = 0; i < statuses.length; i++) {
4600 statuses[i] = OperationStatus.SUCCESS;
4602 return statuses;
4604 return batchMutate(new ReplayBatchOperation(this, mutations, replaySeqId));
4608 * Perform a batch of mutations.
4609 * <p/>
4610 * Operations in a batch are stored with highest durability specified of for all operations in a
4611 * batch, except for {@link Durability#SKIP_WAL}.
4612 * <p/>
4613 * This function is called from {@link #batchReplay(WALSplitUtil.MutationReplay[], long)} with
4614 * {@link ReplayBatchOperation} instance and {@link #batchMutate(Mutation[])} with
4615 * {@link MutationBatchOperation} instance as an argument. As the processing of replay batch and
4616 * mutation batch is very similar, lot of code is shared by providing generic methods in base
4617 * class {@link BatchOperation}. The logic for this method and
4618 * {@link #doMiniBatchMutate(BatchOperation)} is implemented using methods in base class which are
4619 * overridden by derived classes to implement special behavior.
4620 * @param batchOp contains the list of mutations
4621 * @return an array of OperationStatus which internally contains the OperationStatusCode and the
4622 * exceptionMessage if any.
4623 * @throws IOException if an IO problem is encountered
4625 private OperationStatus[] batchMutate(BatchOperation<?> batchOp) throws IOException {
4626 boolean initialized = false;
4627 batchOp.startRegionOperation();
4628 try {
4629 while (!batchOp.isDone()) {
4630 if (!batchOp.isInReplay()) {
4631 checkReadOnly();
4633 checkResources();
4635 if (!initialized) {
4636 this.writeRequestsCount.add(batchOp.size());
4637 // validate and prepare batch for write, for MutationBatchOperation it also calls CP
4638 // prePut()/preDelete()/preIncrement()/preAppend() hooks
4639 batchOp.checkAndPrepare();
4640 initialized = true;
4642 doMiniBatchMutate(batchOp);
4643 requestFlushIfNeeded();
4645 } finally {
4646 if (rsServices != null && rsServices.getMetrics() != null) {
4647 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
4648 getTableName(), batchOp.size());
4650 batchOp.closeRegionOperation();
4652 return batchOp.retCodeDetails;
4656 * Called to do a piece of the batch that came in to {@link #batchMutate(Mutation[])}
4657 * In here we also handle replay of edits on region recover. Also gets change in size brought
4658 * about by applying {@code batchOp}.
4660 private void doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException {
4661 boolean success = false;
4662 WALEdit walEdit = null;
4663 WriteEntry writeEntry = null;
4664 boolean locked = false;
4665 // We try to set up a batch in the range [batchOp.nextIndexToProcess,lastIndexExclusive)
4666 MiniBatchOperationInProgress<Mutation> miniBatchOp = null;
4667 /** Keep track of the locks we hold so we can release them in finally clause */
4668 List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size());
4670 // Check for thread interrupt status in case we have been signaled from
4671 // #interruptRegionOperation.
4672 checkInterrupt();
4674 try {
4675 // STEP 1. Try to acquire as many locks as we can and build mini-batch of operations with
4676 // locked rows
4677 miniBatchOp = batchOp.lockRowsAndBuildMiniBatch(acquiredRowLocks);
4679 // We've now grabbed as many mutations off the list as we can
4680 // Ensure we acquire at least one.
4681 if (miniBatchOp.getReadyToWriteCount() <= 0) {
4682 // Nothing to put/delete/increment/append -- an exception in the above such as
4683 // NoSuchColumnFamily?
4684 return;
4687 // Check for thread interrupt status in case we have been signaled from
4688 // #interruptRegionOperation. Do it before we take the lock and disable interrupts for
4689 // the WAL append.
4690 checkInterrupt();
4692 lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount());
4693 locked = true;
4695 // From this point until memstore update this operation should not be interrupted.
4696 disableInterrupts();
4698 // STEP 2. Update mini batch of all operations in progress with LATEST_TIMESTAMP timestamp
4699 // We should record the timestamp only after we have acquired the rowLock,
4700 // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer
4701 // timestamp
4703 long now = EnvironmentEdgeManager.currentTime();
4704 batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks);
4706 // STEP 3. Build WAL edit
4708 List<Pair<NonceKey, WALEdit>> walEdits = batchOp.buildWALEdits(miniBatchOp);
4710 // STEP 4. Append the WALEdits to WAL and sync.
4712 for(Iterator<Pair<NonceKey, WALEdit>> it = walEdits.iterator(); it.hasNext();) {
4713 Pair<NonceKey, WALEdit> nonceKeyWALEditPair = it.next();
4714 walEdit = nonceKeyWALEditPair.getSecond();
4715 NonceKey nonceKey = nonceKeyWALEditPair.getFirst();
4717 if (walEdit != null && !walEdit.isEmpty()) {
4718 writeEntry = doWALAppend(walEdit, batchOp.durability, batchOp.getClusterIds(), now,
4719 nonceKey.getNonceGroup(), nonceKey.getNonce(), batchOp.getOrigLogSeqNum());
4722 // Complete mvcc for all but last writeEntry (for replay case)
4723 if (it.hasNext() && writeEntry != null) {
4724 mvcc.complete(writeEntry);
4725 writeEntry = null;
4729 // STEP 5. Write back to memStore
4730 // NOTE: writeEntry can be null here
4731 writeEntry = batchOp.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry);
4733 // STEP 6. Complete MiniBatchOperations: If required calls postBatchMutate() CP hook and
4734 // complete mvcc for last writeEntry
4735 batchOp.completeMiniBatchOperations(miniBatchOp, writeEntry);
4736 writeEntry = null;
4737 success = true;
4738 } finally {
4739 // Call complete rather than completeAndWait because we probably had error if walKey != null
4740 if (writeEntry != null) mvcc.complete(writeEntry);
4742 if (locked) {
4743 this.updatesLock.readLock().unlock();
4745 releaseRowLocks(acquiredRowLocks);
4747 enableInterrupts();
4749 final int finalLastIndexExclusive =
4750 miniBatchOp != null ? miniBatchOp.getLastIndexExclusive() : batchOp.size();
4751 final boolean finalSuccess = success;
4752 batchOp.visitBatchOperations(true, finalLastIndexExclusive,
4753 (int i) -> {
4754 Mutation mutation = batchOp.getMutation(i);
4755 if (mutation instanceof Increment || mutation instanceof Append) {
4756 if (finalSuccess) {
4757 batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.SUCCESS,
4758 batchOp.results[i]);
4759 } else {
4760 batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
4762 } else {
4763 batchOp.retCodeDetails[i] =
4764 finalSuccess ? OperationStatus.SUCCESS : OperationStatus.FAILURE;
4766 return true;
4769 batchOp.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, finalSuccess);
4771 batchOp.nextIndexToProcess = finalLastIndexExclusive;
4776 * Returns effective durability from the passed durability and
4777 * the table descriptor.
4779 private Durability getEffectiveDurability(Durability d) {
4780 return d == Durability.USE_DEFAULT ? this.regionDurability : d;
4783 @Override
4784 @Deprecated
4785 public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
4786 ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException {
4787 CheckAndMutate checkAndMutate;
4788 try {
4789 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row)
4790 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange);
4791 if (mutation instanceof Put) {
4792 checkAndMutate = builder.build((Put) mutation);
4793 } else if (mutation instanceof Delete) {
4794 checkAndMutate = builder.build((Delete) mutation);
4795 } else {
4796 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass()
4797 .getSimpleName().toUpperCase());
4799 } catch (IllegalArgumentException e) {
4800 throw new DoNotRetryIOException(e.getMessage());
4802 return checkAndMutate(checkAndMutate).isSuccess();
4805 @Override
4806 @Deprecated
4807 public boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation)
4808 throws IOException {
4809 CheckAndMutate checkAndMutate;
4810 try {
4811 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row).ifMatches(filter)
4812 .timeRange(timeRange);
4813 if (mutation instanceof Put) {
4814 checkAndMutate = builder.build((Put) mutation);
4815 } else if (mutation instanceof Delete) {
4816 checkAndMutate = builder.build((Delete) mutation);
4817 } else {
4818 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass()
4819 .getSimpleName().toUpperCase());
4821 } catch (IllegalArgumentException e) {
4822 throw new DoNotRetryIOException(e.getMessage());
4824 return checkAndMutate(checkAndMutate).isSuccess();
4827 @Override
4828 @Deprecated
4829 public boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
4830 ByteArrayComparable comparator, TimeRange timeRange, RowMutations rm) throws IOException {
4831 CheckAndMutate checkAndMutate;
4832 try {
4833 checkAndMutate = CheckAndMutate.newBuilder(row)
4834 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange).build(rm);
4835 } catch (IllegalArgumentException e) {
4836 throw new DoNotRetryIOException(e.getMessage());
4838 return checkAndMutate(checkAndMutate).isSuccess();
4841 @Override
4842 @Deprecated
4843 public boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations rm)
4844 throws IOException {
4845 CheckAndMutate checkAndMutate;
4846 try {
4847 checkAndMutate = CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange)
4848 .build(rm);
4849 } catch (IllegalArgumentException e) {
4850 throw new DoNotRetryIOException(e.getMessage());
4852 return checkAndMutate(checkAndMutate).isSuccess();
4855 @Override
4856 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException {
4857 return checkAndMutate(checkAndMutate, HConstants.NO_NONCE, HConstants.NO_NONCE);
4860 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long nonceGroup,
4861 long nonce) throws IOException {
4862 return TraceUtil.trace(() -> checkAndMutateInternal(checkAndMutate, nonceGroup, nonce),
4863 () -> createRegionSpan("Region.checkAndMutate"));
4866 private CheckAndMutateResult checkAndMutateInternal(CheckAndMutate checkAndMutate,
4867 long nonceGroup, long nonce) throws IOException {
4868 byte[] row = checkAndMutate.getRow();
4869 Filter filter = null;
4870 byte[] family = null;
4871 byte[] qualifier = null;
4872 CompareOperator op = null;
4873 ByteArrayComparable comparator = null;
4874 if (checkAndMutate.hasFilter()) {
4875 filter = checkAndMutate.getFilter();
4876 } else {
4877 family = checkAndMutate.getFamily();
4878 qualifier = checkAndMutate.getQualifier();
4879 op = checkAndMutate.getCompareOp();
4880 comparator = new BinaryComparator(checkAndMutate.getValue());
4882 TimeRange timeRange = checkAndMutate.getTimeRange();
4884 Mutation mutation = null;
4885 RowMutations rowMutations = null;
4886 if (checkAndMutate.getAction() instanceof Mutation) {
4887 mutation = (Mutation) checkAndMutate.getAction();
4888 } else {
4889 rowMutations = (RowMutations) checkAndMutate.getAction();
4892 if (mutation != null) {
4893 checkMutationType(mutation);
4894 checkRow(mutation, row);
4895 } else {
4896 checkRow(rowMutations, row);
4898 checkReadOnly();
4899 // TODO, add check for value length also move this check to the client
4900 checkResources();
4901 startRegionOperation();
4902 try {
4903 Get get = new Get(row);
4904 if (family != null) {
4905 checkFamily(family);
4906 get.addColumn(family, qualifier);
4908 if (filter != null) {
4909 get.setFilter(filter);
4911 if (timeRange != null) {
4912 get.setTimeRange(timeRange.getMin(), timeRange.getMax());
4914 // Lock row - note that doBatchMutate will relock this row if called
4915 checkRow(row, "doCheckAndRowMutate");
4916 RowLock rowLock = getRowLock(get.getRow(), false, null);
4917 try {
4918 if (this.getCoprocessorHost() != null) {
4919 CheckAndMutateResult result =
4920 getCoprocessorHost().preCheckAndMutateAfterRowLock(checkAndMutate);
4921 if (result != null) {
4922 return result;
4926 // NOTE: We used to wait here until mvcc caught up: mvcc.await();
4927 // Supposition is that now all changes are done under row locks, then when we go to read,
4928 // we'll get the latest on this row.
4929 boolean matches = false;
4930 long cellTs = 0;
4931 try (RegionScanner scanner = getScanner(new Scan(get))) {
4932 // NOTE: Please don't use HRegion.get() instead,
4933 // because it will copy cells to heap. See HBASE-26036
4934 List<Cell> result = new ArrayList<>(1);
4935 scanner.next(result);
4936 if (filter != null) {
4937 if (!result.isEmpty()) {
4938 matches = true;
4939 cellTs = result.get(0).getTimestamp();
4941 } else {
4942 boolean valueIsNull =
4943 comparator.getValue() == null || comparator.getValue().length == 0;
4944 if (result.isEmpty() && valueIsNull) {
4945 matches = op != CompareOperator.NOT_EQUAL;
4946 } else if (result.size() > 0 && valueIsNull) {
4947 matches = (result.get(0).getValueLength() == 0) == (op != CompareOperator.NOT_EQUAL);
4948 cellTs = result.get(0).getTimestamp();
4949 } else if (result.size() == 1) {
4950 Cell kv = result.get(0);
4951 cellTs = kv.getTimestamp();
4952 int compareResult = PrivateCellUtil.compareValue(kv, comparator);
4953 matches = matches(op, compareResult);
4958 // If matches, perform the mutation or the rowMutations
4959 if (matches) {
4960 // We have acquired the row lock already. If the system clock is NOT monotonically
4961 // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
4962 // larger timestamp than what was observed via Get. doBatchMutate already does this, but
4963 // there is no way to pass the cellTs. See HBASE-14054.
4964 long now = EnvironmentEdgeManager.currentTime();
4965 long ts = Math.max(now, cellTs); // ensure write is not eclipsed
4966 byte[] byteTs = Bytes.toBytes(ts);
4967 if (mutation != null) {
4968 if (mutation instanceof Put) {
4969 updateCellTimestamps(mutation.getFamilyCellMap().values(), byteTs);
4971 // And else 'delete' is not needed since it already does a second get, and sets the
4972 // timestamp from get (see prepareDeleteTimestamps).
4973 } else {
4974 for (Mutation m : rowMutations.getMutations()) {
4975 if (m instanceof Put) {
4976 updateCellTimestamps(m.getFamilyCellMap().values(), byteTs);
4979 // And else 'delete' is not needed since it already does a second get, and sets the
4980 // timestamp from get (see prepareDeleteTimestamps).
4982 // All edits for the given row (across all column families) must happen atomically.
4983 Result r;
4984 if (mutation != null) {
4985 r = mutate(mutation, true, nonceGroup, nonce).getResult();
4986 } else {
4987 r = mutateRow(rowMutations, nonceGroup, nonce);
4989 this.checkAndMutateChecksPassed.increment();
4990 return new CheckAndMutateResult(true, r);
4992 this.checkAndMutateChecksFailed.increment();
4993 return new CheckAndMutateResult(false, null);
4994 } finally {
4995 rowLock.release();
4997 } finally {
4998 closeRegionOperation();
5002 private void checkMutationType(final Mutation mutation) throws DoNotRetryIOException {
5003 if (!(mutation instanceof Put) && !(mutation instanceof Delete) &&
5004 !(mutation instanceof Increment) && !(mutation instanceof Append)) {
5005 throw new org.apache.hadoop.hbase.DoNotRetryIOException(
5006 "Action must be Put or Delete or Increment or Delete");
5010 private void checkRow(final Row action, final byte[] row)
5011 throws DoNotRetryIOException {
5012 if (!Bytes.equals(row, action.getRow())) {
5013 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's getRow must match");
5017 private boolean matches(final CompareOperator op, final int compareResult) {
5018 boolean matches = false;
5019 switch (op) {
5020 case LESS:
5021 matches = compareResult < 0;
5022 break;
5023 case LESS_OR_EQUAL:
5024 matches = compareResult <= 0;
5025 break;
5026 case EQUAL:
5027 matches = compareResult == 0;
5028 break;
5029 case NOT_EQUAL:
5030 matches = compareResult != 0;
5031 break;
5032 case GREATER_OR_EQUAL:
5033 matches = compareResult >= 0;
5034 break;
5035 case GREATER:
5036 matches = compareResult > 0;
5037 break;
5038 default:
5039 throw new RuntimeException("Unknown Compare op " + op.name());
5041 return matches;
5044 private OperationStatus mutate(Mutation mutation) throws IOException {
5045 return mutate(mutation, false);
5048 private OperationStatus mutate(Mutation mutation, boolean atomic) throws IOException {
5049 return mutate(mutation, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE);
5052 private OperationStatus mutate(Mutation mutation, boolean atomic, long nonceGroup, long nonce)
5053 throws IOException {
5054 OperationStatus[] status =
5055 this.batchMutate(new Mutation[] { mutation }, atomic, nonceGroup, nonce);
5056 if (status[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
5057 throw new FailedSanityCheckException(status[0].getExceptionMsg());
5058 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
5059 throw new NoSuchColumnFamilyException(status[0].getExceptionMsg());
5060 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.STORE_TOO_BUSY)) {
5061 throw new RegionTooBusyException(status[0].getExceptionMsg());
5063 return status[0];
5067 * Complete taking the snapshot on the region. Writes the region info and adds references to the
5068 * working snapshot directory.
5070 * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
5071 * arg. (In the future other cancellable HRegion methods could eventually add a
5072 * {@link ForeignExceptionSnare}, or we could do something fancier).
5074 * @param desc snapshot description object
5075 * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to
5076 * bail out. This is allowed to be null and will just be ignored in that case.
5077 * @throws IOException if there is an external or internal error causing the snapshot to fail
5079 public void addRegionToSnapshot(SnapshotDescription desc,
5080 ForeignExceptionSnare exnSnare) throws IOException {
5081 Path rootDir = CommonFSUtils.getRootDir(conf);
5082 Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
5084 SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
5085 snapshotDir, desc, exnSnare);
5086 manifest.addRegion(this);
5089 private void updateSequenceId(final Iterable<List<Cell>> cellItr, final long sequenceId)
5090 throws IOException {
5091 for (List<Cell> cells: cellItr) {
5092 if (cells == null) return;
5093 for (Cell cell : cells) {
5094 PrivateCellUtil.setSequenceId(cell, sequenceId);
5100 * Replace any cell timestamps set to {@link org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP}
5101 * provided current timestamp.
5102 * @param cellItr
5103 * @param now
5105 private static void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
5106 throws IOException {
5107 for (List<Cell> cells: cellItr) {
5108 if (cells == null) continue;
5109 // Optimization: 'foreach' loop is not used. See:
5110 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5111 assert cells instanceof RandomAccess;
5112 int listSize = cells.size();
5113 for (int i = 0; i < listSize; i++) {
5114 PrivateCellUtil.updateLatestStamp(cells.get(i), now);
5120 * Possibly rewrite incoming cell tags.
5122 private void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
5123 // Check if we have any work to do and early out otherwise
5124 // Update these checks as more logic is added here
5125 if (m.getTTL() == Long.MAX_VALUE) {
5126 return;
5129 // From this point we know we have some work to do
5130 for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
5131 List<Cell> cells = e.getValue();
5132 assert cells instanceof RandomAccess;
5133 int listSize = cells.size();
5134 for (int i = 0; i < listSize; i++) {
5135 Cell cell = cells.get(i);
5136 List<Tag> newTags = TagUtil.carryForwardTags(null, cell);
5137 newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL());
5138 // Rewrite the cell with the updated set of tags
5139 cells.set(i, PrivateCellUtil.createCell(cell, newTags));
5145 * Check if resources to support an update.
5146 * <p/>
5147 * We throw RegionTooBusyException if above memstore limit and expect client to retry using some
5148 * kind of backoff
5150 private void checkResources() throws RegionTooBusyException {
5151 // If catalog region, do not impose resource constraints or block updates.
5152 if (this.getRegionInfo().isMetaRegion()) {
5153 return;
5156 MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
5157 if (mss.getHeapSize() + mss.getOffHeapSize() > this.blockingMemStoreSize) {
5158 blockedRequestsCount.increment();
5159 requestFlush();
5160 // Don't print current limit because it will vary too much. The message is used as a key
5161 // over in RetriesExhaustedWithDetailsException processing.
5162 final String regionName =
5163 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getEncodedName();
5164 final String serverName = this.getRegionServerServices() == null ?
5165 "unknown" : (this.getRegionServerServices().getServerName() == null ? "unknown" :
5166 this.getRegionServerServices().getServerName().toString());
5167 RegionTooBusyException rtbe = new RegionTooBusyException(
5168 "Over memstore limit=" + org.apache.hadoop.hbase.procedure2.util.StringUtils
5169 .humanSize(this.blockingMemStoreSize) + ", regionName=" + regionName + ", server="
5170 + serverName);
5171 LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe);
5172 throw rtbe;
5177 * @throws IOException Throws exception if region is in read-only mode.
5179 private void checkReadOnly() throws IOException {
5180 if (isReadOnly()) {
5181 throw new DoNotRetryIOException("region is read only");
5185 private void checkReadsEnabled() throws IOException {
5186 if (!this.writestate.readsEnabled) {
5187 throw new IOException(getRegionInfo().getEncodedName()
5188 + ": The region's reads are disabled. Cannot serve the request");
5192 public void setReadsEnabled(boolean readsEnabled) {
5193 if (readsEnabled && !this.writestate.readsEnabled) {
5194 LOG.info("Enabling reads for {}", getRegionInfo().getEncodedName());
5196 this.writestate.setReadsEnabled(readsEnabled);
5200 * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be
5201 * set; when set we will run operations that make sense in the increment/append scenario
5202 * but that do not make sense otherwise.
5204 private void applyToMemStore(HStore store, List<Cell> cells, boolean delta,
5205 MemStoreSizing memstoreAccounting) throws IOException {
5206 // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!!
5207 boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
5208 if (upsert) {
5209 store.upsert(cells, getSmallestReadPoint(), memstoreAccounting);
5210 } else {
5211 store.add(cells, memstoreAccounting);
5215 private void checkFamilies(Collection<byte[]> families, Durability durability)
5216 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5217 for (byte[] family : families) {
5218 checkFamily(family, durability);
5222 private void checkFamily(final byte[] family, Durability durability)
5223 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5224 checkFamily(family);
5225 if (durability.equals(Durability.SKIP_WAL)
5226 && htableDescriptor.getColumnFamily(family).getScope()
5227 != HConstants.REPLICATION_SCOPE_LOCAL) {
5228 throw new InvalidMutationDurabilityException(
5229 "Mutation's durability is SKIP_WAL but table's column family " + Bytes.toString(family)
5230 + " need replication");
5234 private void checkFamily(final byte[] family) throws NoSuchColumnFamilyException {
5235 if (!this.htableDescriptor.hasColumnFamily(family)) {
5236 throw new NoSuchColumnFamilyException(
5237 "Column family " + Bytes.toString(family) + " does not exist in region " + this
5238 + " in table " + this.htableDescriptor);
5243 * Check the collection of families for valid timestamps
5244 * @param familyMap
5245 * @param now current timestamp
5246 * @throws FailedSanityCheckException
5248 public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
5249 throws FailedSanityCheckException {
5250 if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
5251 return;
5253 long maxTs = now + timestampSlop;
5254 for (List<Cell> kvs : familyMap.values()) {
5255 // Optimization: 'foreach' loop is not used. See:
5256 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5257 assert kvs instanceof RandomAccess;
5258 int listSize = kvs.size();
5259 for (int i=0; i < listSize; i++) {
5260 Cell cell = kvs.get(i);
5261 // see if the user-side TS is out of range. latest = server-side
5262 long ts = cell.getTimestamp();
5263 if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
5264 throw new FailedSanityCheckException("Timestamp for KV out of range "
5265 + cell + " (too.new=" + timestampSlop + ")");
5272 * @param size
5273 * @return True if size is over the flush threshold
5275 private boolean isFlushSize(MemStoreSize size) {
5276 return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize();
5279 private void deleteRecoveredEdits(FileSystem fs, Iterable<Path> files) throws IOException {
5280 for (Path file : files) {
5281 if (!fs.delete(file, false)) {
5282 LOG.error("Failed delete of {}", file);
5283 } else {
5284 LOG.debug("Deleted recovered.edits file={}", file);
5290 * Read the edits put under this region by wal splitting process. Put
5291 * the recovered edits back up into this region.
5293 * <p>We can ignore any wal message that has a sequence ID that's equal to or
5294 * lower than minSeqId. (Because we know such messages are already
5295 * reflected in the HFiles.)
5297 * <p>While this is running we are putting pressure on memory yet we are
5298 * outside of our usual accounting because we are not yet an onlined region
5299 * (this stuff is being run as part of Region initialization). This means
5300 * that if we're up against global memory limits, we'll not be flagged to flush
5301 * because we are not online. We can't be flushed by usual mechanisms anyways;
5302 * we're not yet online so our relative sequenceids are not yet aligned with
5303 * WAL sequenceids -- not till we come up online, post processing of split
5304 * edits.
5306 * <p>But to help relieve memory pressure, at least manage our own heap size
5307 * flushing if are in excess of per-region limits. Flushing, though, we have
5308 * to be careful and avoid using the regionserver/wal sequenceid. Its running
5309 * on a different line to whats going on in here in this region context so if we
5310 * crashed replaying these edits, but in the midst had a flush that used the
5311 * regionserver wal with a sequenceid in excess of whats going on in here
5312 * in this region and with its split editlogs, then we could miss edits the
5313 * next time we go to recover. So, we have to flush inline, using seqids that
5314 * make sense in a this single region context only -- until we online.
5316 * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of
5317 * the maxSeqId for the store to be applied, else its skipped.
5318 * @return the sequence id of the last edit added to this region out of the
5319 * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
5321 long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
5322 final CancelableProgressable reporter, final MonitoredTask status) throws IOException {
5323 long minSeqIdForTheRegion = -1;
5324 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
5325 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
5326 minSeqIdForTheRegion = maxSeqIdInStore;
5329 long seqId = minSeqIdForTheRegion;
5330 String specialRecoveredEditsDirStr = conf.get(SPECIAL_RECOVERED_EDITS_DIR);
5331 if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) {
5332 FileSystem walFS = getWalFileSystem();
5333 FileSystem rootFS = getFilesystem();
5334 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
5335 getRegionInfo().getEncodedName());
5336 Path regionWALDir = getWALRegionDir();
5337 Path regionDir =
5338 FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo());
5340 // We made a mistake in HBASE-20734 so we need to do this dirty hack...
5341 NavigableSet<Path> filesUnderWrongRegionWALDir =
5342 WALSplitUtil.getSplitEditFilesSorted(walFS, wrongRegionWALDir);
5343 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
5344 filesUnderWrongRegionWALDir, reporter, regionDir));
5345 // This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear
5346 // under the root dir even if walDir is set.
5347 NavigableSet<Path> filesUnderRootDir = Collections.emptyNavigableSet();
5348 if (!regionWALDir.equals(regionDir)) {
5349 filesUnderRootDir = WALSplitUtil.getSplitEditFilesSorted(rootFS, regionDir);
5350 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS,
5351 filesUnderRootDir, reporter, regionDir));
5354 NavigableSet<Path> files = WALSplitUtil.getSplitEditFilesSorted(walFS, regionWALDir);
5355 seqId = Math.max(seqId,
5356 replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, files, reporter, regionWALDir));
5357 if (seqId > minSeqIdForTheRegion) {
5358 // Then we added some edits to memory. Flush and cleanup split edit files.
5359 internalFlushcache(null, seqId, stores.values(), status, false,
5360 FlushLifeCycleTracker.DUMMY);
5362 // Now delete the content of recovered edits. We're done w/ them.
5363 if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
5364 // For debugging data loss issues!
5365 // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
5366 // column family. Have to fake out file type too by casting our recovered.edits as
5367 // storefiles
5368 String fakeFamilyName = WALSplitUtil.getRegionDirRecoveredEditsDir(regionWALDir).getName();
5369 Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size());
5370 for (Path file : files) {
5371 fakeStoreFiles.add(new HStoreFile(walFS, file, this.conf, null, null, true));
5373 getRegionWALFileSystem().archiveRecoveredEdits(fakeFamilyName, fakeStoreFiles);
5374 } else {
5375 deleteRecoveredEdits(walFS, Iterables.concat(files, filesUnderWrongRegionWALDir));
5376 deleteRecoveredEdits(rootFS, filesUnderRootDir);
5378 } else {
5379 Path recoveredEditsDir = new Path(specialRecoveredEditsDirStr);
5380 FileSystem fs = recoveredEditsDir.getFileSystem(conf);
5381 FileStatus[] files = fs.listStatus(recoveredEditsDir);
5382 LOG.debug("Found {} recovered edits file(s) under {}", files == null ? 0 : files.length,
5383 recoveredEditsDir);
5384 if (files != null) {
5385 for (FileStatus file : files) {
5386 // it is safe to trust the zero-length in this case because we've been through rename and
5387 // lease recovery in the above.
5388 if (isZeroLengthThenDelete(fs, file, file.getPath())) {
5389 continue;
5391 seqId =
5392 Math.max(seqId, replayRecoveredEdits(file.getPath(), maxSeqIdInStores, reporter, fs));
5395 if (seqId > minSeqIdForTheRegion) {
5396 // Then we added some edits to memory. Flush and cleanup split edit files.
5397 internalFlushcache(null, seqId, stores.values(), status, false,
5398 FlushLifeCycleTracker.DUMMY);
5400 deleteRecoveredEdits(fs,
5401 Stream.of(files).map(FileStatus::getPath).collect(Collectors.toList()));
5404 return seqId;
5407 private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
5408 final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir)
5409 throws IOException {
5410 long seqid = minSeqIdForTheRegion;
5411 if (LOG.isDebugEnabled()) {
5412 LOG.debug("Found " + (files == null ? 0 : files.size())
5413 + " recovered edits file(s) under " + regionDir);
5416 if (files == null || files.isEmpty()) {
5417 return minSeqIdForTheRegion;
5420 for (Path edits: files) {
5421 if (edits == null || !fs.exists(edits)) {
5422 LOG.warn("Null or non-existent edits file: " + edits);
5423 continue;
5425 if (isZeroLengthThenDelete(fs, fs.getFileStatus(edits), edits)) {
5426 continue;
5429 long maxSeqId;
5430 String fileName = edits.getName();
5431 maxSeqId = Math.abs(Long.parseLong(fileName));
5432 if (maxSeqId <= minSeqIdForTheRegion) {
5433 if (LOG.isDebugEnabled()) {
5434 String msg = "Maximum sequenceid for this wal is " + maxSeqId
5435 + " and minimum sequenceid for the region " + this + " is " + minSeqIdForTheRegion
5436 + ", skipped the whole file, path=" + edits;
5437 LOG.debug(msg);
5439 continue;
5442 try {
5443 // replay the edits. Replay can return -1 if everything is skipped, only update
5444 // if seqId is greater
5445 seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs));
5446 } catch (IOException e) {
5447 handleException(fs, edits, e);
5450 return seqid;
5453 private void handleException(FileSystem fs, Path edits, IOException e) throws IOException {
5454 boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
5455 conf.getBoolean("hbase.skip.errors", HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
5456 if (conf.get("hbase.skip.errors") != null) {
5457 LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use "
5458 + HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
5460 if (skipErrors) {
5461 Path p = WALSplitUtil.moveAsideBadEditsFile(fs, edits);
5462 LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + "=true so continuing. Renamed "
5463 + edits + " as " + p,
5465 } else {
5466 throw e;
5471 * @param edits File of recovered edits.
5472 * @param maxSeqIdInStores Maximum sequenceid found in each store. Edits in wal must be larger
5473 * than this to be replayed for each store.
5474 * @return the sequence id of the last edit added to this region out of the recovered edits log or
5475 * <code>minSeqId</code> if nothing added from editlogs.
5477 private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores,
5478 final CancelableProgressable reporter, FileSystem fs) throws IOException {
5479 String msg = "Replaying edits from " + edits;
5480 LOG.info(msg);
5481 MonitoredTask status = TaskMonitor.get().createStatus(msg);
5483 status.setStatus("Opening recovered edits");
5484 WAL.Reader reader = null;
5485 try {
5486 reader = WALFactory.createReader(fs, edits, conf);
5487 long currentEditSeqId = -1;
5488 long currentReplaySeqId = -1;
5489 long firstSeqIdInLog = -1;
5490 long skippedEdits = 0;
5491 long editsCount = 0;
5492 long intervalEdits = 0;
5493 WAL.Entry entry;
5494 HStore store = null;
5495 boolean reported_once = false;
5496 ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
5498 try {
5499 // How many edits seen before we check elapsed time
5500 int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
5501 // How often to send a progress report (default 1/2 master timeout)
5502 int period = this.conf.getInt("hbase.hstore.report.period", 300000);
5503 long lastReport = EnvironmentEdgeManager.currentTime();
5505 if (coprocessorHost != null) {
5506 coprocessorHost.preReplayWALs(this.getRegionInfo(), edits);
5509 while ((entry = reader.next()) != null) {
5510 WALKey key = entry.getKey();
5511 WALEdit val = entry.getEdit();
5513 if (ng != null) { // some test, or nonces disabled
5514 ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
5517 if (reporter != null) {
5518 intervalEdits += val.size();
5519 if (intervalEdits >= interval) {
5520 // Number of edits interval reached
5521 intervalEdits = 0;
5522 long cur = EnvironmentEdgeManager.currentTime();
5523 if (lastReport + period <= cur) {
5524 status.setStatus("Replaying edits..." +
5525 " skipped=" + skippedEdits +
5526 " edits=" + editsCount);
5527 // Timeout reached
5528 if(!reporter.progress()) {
5529 msg = "Progressable reporter failed, stopping replay for region " + this;
5530 LOG.warn(msg);
5531 status.abort(msg);
5532 throw new IOException(msg);
5534 reported_once = true;
5535 lastReport = cur;
5540 if (firstSeqIdInLog == -1) {
5541 firstSeqIdInLog = key.getSequenceId();
5543 if (currentEditSeqId > key.getSequenceId()) {
5544 // when this condition is true, it means we have a serious defect because we need to
5545 // maintain increasing SeqId for WAL edits per region
5546 LOG.error(getRegionInfo().getEncodedName() + " : "
5547 + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
5548 + "; edit=" + val);
5549 } else {
5550 currentEditSeqId = key.getSequenceId();
5552 currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
5553 key.getOrigLogSeqNum() : currentEditSeqId;
5555 // Start coprocessor replay here. The coprocessor is for each WALEdit
5556 // instead of a KeyValue.
5557 if (coprocessorHost != null) {
5558 status.setStatus("Running pre-WAL-restore hook in coprocessors");
5559 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
5560 // if bypass this wal entry, ignore it ...
5561 continue;
5564 boolean checkRowWithinBoundary = false;
5565 // Check this edit is for this region.
5566 if (!Bytes.equals(key.getEncodedRegionName(),
5567 this.getRegionInfo().getEncodedNameAsBytes())) {
5568 checkRowWithinBoundary = true;
5571 boolean flush = false;
5572 MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
5573 for (Cell cell: val.getCells()) {
5574 // Check this edit is for me. Also, guard against writing the special
5575 // METACOLUMN info such as HBASE::CACHEFLUSH entries
5576 if (WALEdit.isMetaEditFamily(cell)) {
5577 // if region names don't match, skipp replaying compaction marker
5578 if (!checkRowWithinBoundary) {
5579 //this is a special edit, we should handle it
5580 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
5581 if (compaction != null) {
5582 //replay the compaction
5583 replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
5586 skippedEdits++;
5587 continue;
5589 // Figure which store the edit is meant for.
5590 if (store == null || !CellUtil.matchingFamily(cell,
5591 store.getColumnFamilyDescriptor().getName())) {
5592 store = getStore(cell);
5594 if (store == null) {
5595 // This should never happen. Perhaps schema was changed between
5596 // crash and redeploy?
5597 LOG.warn("No family for cell {} in region {}", cell, this);
5598 skippedEdits++;
5599 continue;
5601 if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
5602 cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
5603 LOG.warn("Row of {} is not within region boundary for region {}", cell, this);
5604 skippedEdits++;
5605 continue;
5607 // Now, figure if we should skip this edit.
5608 if (key.getSequenceId() <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor()
5609 .getName())) {
5610 skippedEdits++;
5611 continue;
5613 PrivateCellUtil.setSequenceId(cell, currentReplaySeqId);
5615 restoreEdit(store, cell, memStoreSizing);
5616 editsCount++;
5618 MemStoreSize mss = memStoreSizing.getMemStoreSize();
5619 incMemStoreSize(mss);
5620 flush = isFlushSize(this.memStoreSizing.getMemStoreSize());
5621 if (flush) {
5622 internalFlushcache(null, currentEditSeqId, stores.values(), status, false,
5623 FlushLifeCycleTracker.DUMMY);
5626 if (coprocessorHost != null) {
5627 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
5631 if (coprocessorHost != null) {
5632 coprocessorHost.postReplayWALs(this.getRegionInfo(), edits);
5634 } catch (EOFException eof) {
5635 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5636 msg = "EnLongAddered EOF. Most likely due to Master failure during "
5637 + "wal splitting, so we have this data in another edit. Continuing, but renaming "
5638 + edits + " as " + p + " for region " + this;
5639 LOG.warn(msg, eof);
5640 status.abort(msg);
5641 } catch (IOException ioe) {
5642 // If the IOE resulted from bad file format,
5643 // then this problem is idempotent and retrying won't help
5644 if (ioe.getCause() instanceof ParseException) {
5645 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5646 msg = "File corruption enLongAddered! " +
5647 "Continuing, but renaming " + edits + " as " + p;
5648 LOG.warn(msg, ioe);
5649 status.setStatus(msg);
5650 } else {
5651 status.abort(StringUtils.stringifyException(ioe));
5652 // other IO errors may be transient (bad network connection,
5653 // checksum exception on one datanode, etc). throw & retry
5654 throw ioe;
5657 if (reporter != null && !reported_once) {
5658 reporter.progress();
5660 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
5661 ", firstSequenceIdInLog=" + firstSeqIdInLog +
5662 ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
5663 status.markComplete(msg);
5664 LOG.debug(msg);
5665 return currentEditSeqId;
5666 } finally {
5667 status.cleanup();
5668 if (reader != null) {
5669 reader.close();
5675 * Call to complete a compaction. Its for the case where we find in the WAL a compaction
5676 * that was not finished. We could find one recovering a WAL after a regionserver crash.
5677 * See HBASE-2331.
5679 void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
5680 boolean removeFiles, long replaySeqId)
5681 throws IOException {
5682 try {
5683 checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
5684 "Compaction marker from WAL ", compaction);
5685 } catch (WrongRegionException wre) {
5686 if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5687 // skip the compaction marker since it is not for this region
5688 return;
5690 throw wre;
5693 synchronized (writestate) {
5694 if (replaySeqId < lastReplayedOpenRegionSeqId) {
5695 LOG.warn(getRegionInfo().getEncodedName() + " : "
5696 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
5697 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5698 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
5699 return;
5701 if (replaySeqId < lastReplayedCompactionSeqId) {
5702 LOG.warn(getRegionInfo().getEncodedName() + " : "
5703 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
5704 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5705 + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId);
5706 return;
5707 } else {
5708 lastReplayedCompactionSeqId = replaySeqId;
5711 if (LOG.isDebugEnabled()) {
5712 LOG.debug(getRegionInfo().getEncodedName() + " : "
5713 + "Replaying compaction marker " + TextFormat.shortDebugString(compaction)
5714 + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId="
5715 + lastReplayedOpenRegionSeqId);
5718 startRegionOperation(Operation.REPLAY_EVENT);
5719 try {
5720 HStore store = this.getStore(compaction.getFamilyName().toByteArray());
5721 if (store == null) {
5722 LOG.warn(getRegionInfo().getEncodedName() + " : "
5723 + "Found Compaction WAL edit for deleted family:"
5724 + Bytes.toString(compaction.getFamilyName().toByteArray()));
5725 return;
5727 store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
5728 logRegionFiles();
5729 } catch (FileNotFoundException ex) {
5730 LOG.warn(getRegionInfo().getEncodedName() + " : "
5731 + "At least one of the store files in compaction: "
5732 + TextFormat.shortDebugString(compaction)
5733 + " doesn't exist any more. Skip loading the file(s)", ex);
5734 } finally {
5735 closeRegionOperation(Operation.REPLAY_EVENT);
5741 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
5742 * replica implementation.
5744 @Deprecated
5745 void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
5746 checkTargetRegion(flush.getEncodedRegionName().toByteArray(), "Flush marker from WAL ", flush);
5748 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5749 return; // if primary nothing to do
5752 if (LOG.isDebugEnabled()) {
5753 LOG.debug(getRegionInfo().getEncodedName() + " : "
5754 + "Replaying flush marker " + TextFormat.shortDebugString(flush));
5757 startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close
5758 try {
5759 FlushAction action = flush.getAction();
5760 switch (action) {
5761 case START_FLUSH:
5762 replayWALFlushStartMarker(flush);
5763 break;
5764 case COMMIT_FLUSH:
5765 replayWALFlushCommitMarker(flush);
5766 break;
5767 case ABORT_FLUSH:
5768 replayWALFlushAbortMarker(flush);
5769 break;
5770 case CANNOT_FLUSH:
5771 replayWALFlushCannotFlushMarker(flush, replaySeqId);
5772 break;
5773 default:
5774 LOG.warn(getRegionInfo().getEncodedName() + " : " +
5775 "Received a flush event with unknown action, ignoring. " +
5776 TextFormat.shortDebugString(flush));
5777 break;
5780 logRegionFiles();
5781 } finally {
5782 closeRegionOperation(Operation.REPLAY_EVENT);
5786 private Collection<HStore> getStoresToFlush(FlushDescriptor flushDesc) {
5787 List<HStore> storesToFlush = new ArrayList<>();
5788 for (StoreFlushDescriptor storeFlush : flushDesc.getStoreFlushesList()) {
5789 byte[] family = storeFlush.getFamilyName().toByteArray();
5790 HStore store = getStore(family);
5791 if (store == null) {
5792 LOG.warn(getRegionInfo().getEncodedName() + " : " +
5793 "Received a flush start marker from primary, but the family is not found. Ignoring" +
5794 " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
5795 continue;
5797 storesToFlush.add(store);
5799 return storesToFlush;
5803 * Replay the flush marker from primary region by creating a corresponding snapshot of the store
5804 * memstores, only if the memstores do not have a higher seqId from an earlier wal edit (because
5805 * the events may be coming out of order).
5806 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
5807 * replica implementation.
5809 @Deprecated
5810 PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
5811 long flushSeqId = flush.getFlushSequenceNumber();
5813 Collection<HStore> storesToFlush = getStoresToFlush(flush);
5815 MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
5817 // we will use writestate as a coarse-grain lock for all the replay events
5818 // (flush, compaction, region open etc)
5819 synchronized (writestate) {
5820 try {
5821 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
5822 LOG.warn(getRegionInfo().getEncodedName() + " : "
5823 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5824 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5825 + " of " + lastReplayedOpenRegionSeqId);
5826 return null;
5828 if (numMutationsWithoutWAL.sum() > 0) {
5829 numMutationsWithoutWAL.reset();
5830 dataInMemoryWithoutWAL.reset();
5833 if (!writestate.flushing) {
5834 // we do not have an active snapshot and corresponding this.prepareResult. This means
5835 // we can just snapshot our memstores and continue as normal.
5837 // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal
5838 PrepareFlushResult prepareResult = internalPrepareFlushCache(null, flushSeqId,
5839 storesToFlush, status, false, FlushLifeCycleTracker.DUMMY);
5840 if (prepareResult.result == null) {
5841 // save the PrepareFlushResult so that we can use it later from commit flush
5842 this.writestate.flushing = true;
5843 this.prepareFlushResult = prepareResult;
5844 status.markComplete("Flush prepare successful");
5845 if (LOG.isDebugEnabled()) {
5846 LOG.debug(getRegionInfo().getEncodedName() + " : "
5847 + " Prepared flush with seqId:" + flush.getFlushSequenceNumber());
5849 } else {
5850 // special case empty memstore. We will still save the flush result in this case, since
5851 // our memstore ie empty, but the primary is still flushing
5852 if (prepareResult.getResult().getResult() ==
5853 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
5854 this.writestate.flushing = true;
5855 this.prepareFlushResult = prepareResult;
5856 if (LOG.isDebugEnabled()) {
5857 LOG.debug(getRegionInfo().getEncodedName() + " : "
5858 + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
5861 status.abort("Flush prepare failed with " + prepareResult.result);
5862 // nothing much to do. prepare flush failed because of some reason.
5864 return prepareResult;
5865 } else {
5866 // we already have an active snapshot.
5867 if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
5868 // They define the same flush. Log and continue.
5869 LOG.warn(getRegionInfo().getEncodedName() + " : "
5870 + "Received a flush prepare marker with the same seqId: " +
5871 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5872 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5873 // ignore
5874 } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
5875 // We received a flush with a smaller seqNum than what we have prepared. We can only
5876 // ignore this prepare flush request.
5877 LOG.warn(getRegionInfo().getEncodedName() + " : "
5878 + "Received a flush prepare marker with a smaller seqId: " +
5879 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5880 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5881 // ignore
5882 } else {
5883 // We received a flush with a larger seqNum than what we have prepared
5884 LOG.warn(getRegionInfo().getEncodedName() + " : "
5885 + "Received a flush prepare marker with a larger seqId: " +
5886 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5887 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5888 // We do not have multiple active snapshots in the memstore or a way to merge current
5889 // memstore snapshot with the contents and resnapshot for now. We cannot take
5890 // another snapshot and drop the previous one because that will cause temporary
5891 // data loss in the secondary. So we ignore this for now, deferring the resolution
5892 // to happen when we see the corresponding flush commit marker. If we have a memstore
5893 // snapshot with x, and later received another prepare snapshot with y (where x < y),
5894 // when we see flush commit for y, we will drop snapshot for x, and can also drop all
5895 // the memstore edits if everything in memstore is < y. This is the usual case for
5896 // RS crash + recovery where we might see consequtive prepare flush wal markers.
5897 // Otherwise, this will cause more memory to be used in secondary replica until a
5898 // further prapare + commit flush is seen and replayed.
5901 } finally {
5902 status.cleanup();
5903 writestate.notifyAll();
5906 return null;
5910 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
5911 * replica implementation.
5913 @Deprecated
5914 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
5915 justification="Intentional; post memstore flush")
5916 void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
5917 MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
5919 // check whether we have the memstore snapshot with the corresponding seqId. Replay to
5920 // secondary region replicas are in order, except for when the region moves or then the
5921 // region server crashes. In those cases, we may receive replay requests out of order from
5922 // the original seqIds.
5923 synchronized (writestate) {
5924 try {
5925 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
5926 LOG.warn(getRegionInfo().getEncodedName() + " : "
5927 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5928 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5929 + " of " + lastReplayedOpenRegionSeqId);
5930 return;
5933 if (writestate.flushing) {
5934 PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
5935 if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
5936 if (LOG.isDebugEnabled()) {
5937 LOG.debug(getRegionInfo().getEncodedName() + " : "
5938 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5939 + " and a previous prepared snapshot was found");
5941 // This is the regular case where we received commit flush after prepare flush
5942 // corresponding to the same seqId.
5943 replayFlushInStores(flush, prepareFlushResult, true);
5945 // Set down the memstore size by amount of flush.
5946 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
5947 this.prepareFlushResult = null;
5948 writestate.flushing = false;
5949 } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
5950 // This should not happen normally. However, lets be safe and guard against these cases
5951 // we received a flush commit with a smaller seqId than what we have prepared
5952 // we will pick the flush file up from this commit (if we have not seen it), but we
5953 // will not drop the memstore
5954 LOG.warn(getRegionInfo().getEncodedName() + " : "
5955 + "Received a flush commit marker with smaller seqId: "
5956 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
5957 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
5958 +" prepared memstore snapshot");
5959 replayFlushInStores(flush, prepareFlushResult, false);
5961 // snapshot is not dropped, so memstore sizes should not be decremented
5962 // we still have the prepared snapshot, flushing should still be true
5963 } else {
5964 // This should not happen normally. However, lets be safe and guard against these cases
5965 // we received a flush commit with a larger seqId than what we have prepared
5966 // we will pick the flush file for this. We will also obtain the updates lock and
5967 // look for contents of the memstore to see whether we have edits after this seqId.
5968 // If not, we will drop all the memstore edits and the snapshot as well.
5969 LOG.warn(getRegionInfo().getEncodedName() + " : "
5970 + "Received a flush commit marker with larger seqId: "
5971 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " +
5972 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
5973 +" memstore snapshot");
5975 replayFlushInStores(flush, prepareFlushResult, true);
5977 // Set down the memstore size by amount of flush.
5978 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
5980 // Inspect the memstore contents to see whether the memstore contains only edits
5981 // with seqId smaller than the flush seqId. If so, we can discard those edits.
5982 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5984 this.prepareFlushResult = null;
5985 writestate.flushing = false;
5987 // If we were waiting for observing a flush or region opening event for not showing
5988 // partial data after a secondary region crash, we can allow reads now. We can only make
5989 // sure that we are not showing partial data (for example skipping some previous edits)
5990 // until we observe a full flush start and flush commit. So if we were not able to find
5991 // a previous flush we will not enable reads now.
5992 this.setReadsEnabled(true);
5993 } else {
5994 LOG.warn(getRegionInfo().getEncodedName() + " : "
5995 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5996 + ", but no previous prepared snapshot was found");
5997 // There is no corresponding prepare snapshot from before.
5998 // We will pick up the new flushed file
5999 replayFlushInStores(flush, null, false);
6001 // Inspect the memstore contents to see whether the memstore contains only edits
6002 // with seqId smaller than the flush seqId. If so, we can discard those edits.
6003 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
6006 status.markComplete("Flush commit successful");
6008 // Update the last flushed sequence id for region.
6009 this.maxFlushedSeqId = flush.getFlushSequenceNumber();
6011 // advance the mvcc read point so that the new flushed file is visible.
6012 mvcc.advanceTo(flush.getFlushSequenceNumber());
6014 } catch (FileNotFoundException ex) {
6015 LOG.warn(getRegionInfo().getEncodedName() + " : "
6016 + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
6017 + " doesn't exist any more. Skip loading the file(s)", ex);
6019 finally {
6020 status.cleanup();
6021 writestate.notifyAll();
6025 // C. Finally notify anyone waiting on memstore to clear:
6026 // e.g. checkResources().
6027 synchronized (this) {
6028 notifyAll(); // FindBugs NN_NAKED_NOTIFY
6033 * Replays the given flush descriptor by opening the flush files in stores and dropping the
6034 * memstore snapshots if requested.
6035 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6036 * replica implementation.
6038 @Deprecated
6039 private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
6040 boolean dropMemstoreSnapshot)
6041 throws IOException {
6042 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
6043 byte[] family = storeFlush.getFamilyName().toByteArray();
6044 HStore store = getStore(family);
6045 if (store == null) {
6046 LOG.warn(getRegionInfo().getEncodedName() + " : "
6047 + "Received a flush commit marker from primary, but the family is not found."
6048 + "Ignoring StoreFlushDescriptor:" + storeFlush);
6049 continue;
6051 List<String> flushFiles = storeFlush.getFlushOutputList();
6052 StoreFlushContext ctx = null;
6053 long startTime = EnvironmentEdgeManager.currentTime();
6054 if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
6055 ctx = store.createFlushContext(flush.getFlushSequenceNumber(), FlushLifeCycleTracker.DUMMY);
6056 } else {
6057 ctx = prepareFlushResult.storeFlushCtxs.get(family);
6058 startTime = prepareFlushResult.startTime;
6061 if (ctx == null) {
6062 LOG.warn(getRegionInfo().getEncodedName() + " : "
6063 + "Unexpected: flush commit marker received from store "
6064 + Bytes.toString(family) + " but no associated flush context. Ignoring");
6065 continue;
6068 ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush
6070 // Record latest flush time
6071 this.lastStoreFlushTimeMap.put(store, startTime);
6075 private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException {
6076 Path regionDir = fs.getRegionDir();
6077 long maxSeqId = -1;
6078 for (HStore store : stores) {
6079 String familyName = store.getColumnFamilyName();
6080 FileStatus[] files =
6081 WALSplitUtil.getRecoveredHFiles(fs.getFileSystem(), regionDir, familyName);
6082 if (files != null && files.length != 0) {
6083 for (FileStatus file : files) {
6084 Path filePath = file.getPath();
6085 // If file length is zero then delete it
6086 if (isZeroLengthThenDelete(fs.getFileSystem(), file, filePath)) {
6087 continue;
6089 try {
6090 HStoreFile storefile = store.tryCommitRecoveredHFile(file.getPath());
6091 maxSeqId = Math.max(maxSeqId, storefile.getReader().getSequenceID());
6092 } catch (IOException e) {
6093 handleException(fs.getFileSystem(), filePath, e);
6094 continue;
6097 if (this.rsServices != null && store.needsCompaction()) {
6098 this.rsServices.getCompactionRequestor()
6099 .requestCompaction(this, store, "load recovered hfiles request compaction",
6100 Store.PRIORITY_USER + 1, CompactionLifeCycleTracker.DUMMY, null);
6104 return maxSeqId;
6108 * Be careful, this method will drop all data in the memstore of this region.
6109 * Currently, this method is used to drop memstore to prevent memory leak
6110 * when replaying recovered.edits while opening region.
6112 private MemStoreSize dropMemStoreContents() throws IOException {
6113 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6114 this.updatesLock.writeLock().lock();
6115 try {
6116 for (HStore s : stores.values()) {
6117 MemStoreSize memStoreSize = doDropStoreMemStoreContentsForSeqId(s, HConstants.NO_SEQNUM);
6118 LOG.info("Drop memstore for Store " + s.getColumnFamilyName() + " in region "
6119 + this.getRegionInfo().getRegionNameAsString()
6120 + " , dropped memstoresize: [" + memStoreSize + " }");
6121 totalFreedSize.incMemStoreSize(memStoreSize);
6123 return totalFreedSize.getMemStoreSize();
6124 } finally {
6125 this.updatesLock.writeLock().unlock();
6130 * Drops the memstore contents after replaying a flush descriptor or region open event replay
6131 * if the memstore edits have seqNums smaller than the given seq id
6133 private MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException {
6134 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6135 this.updatesLock.writeLock().lock();
6136 try {
6138 long currentSeqId = mvcc.getReadPoint();
6139 if (seqId >= currentSeqId) {
6140 // then we can drop the memstore contents since everything is below this seqId
6141 LOG.info(getRegionInfo().getEncodedName() + " : "
6142 + "Dropping memstore contents as well since replayed flush seqId: "
6143 + seqId + " is greater than current seqId:" + currentSeqId);
6145 // Prepare flush (take a snapshot) and then abort (drop the snapshot)
6146 if (store == null) {
6147 for (HStore s : stores.values()) {
6148 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(s, currentSeqId));
6150 } else {
6151 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store, currentSeqId));
6153 } else {
6154 LOG.info(getRegionInfo().getEncodedName() + " : "
6155 + "Not dropping memstore contents since replayed flush seqId: "
6156 + seqId + " is smaller than current seqId:" + currentSeqId);
6158 } finally {
6159 this.updatesLock.writeLock().unlock();
6161 return totalFreedSize.getMemStoreSize();
6164 private MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId)
6165 throws IOException {
6166 MemStoreSize flushableSize = s.getFlushableSize();
6167 this.decrMemStoreSize(flushableSize);
6168 StoreFlushContext ctx = s.createFlushContext(currentSeqId, FlushLifeCycleTracker.DUMMY);
6169 ctx.prepare();
6170 ctx.abort();
6171 return flushableSize;
6174 private void replayWALFlushAbortMarker(FlushDescriptor flush) {
6175 // nothing to do for now. A flush abort will cause a RS abort which means that the region
6176 // will be opened somewhere else later. We will see the region open event soon, and replaying
6177 // that will drop the snapshot
6180 private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
6181 synchronized (writestate) {
6182 if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
6183 LOG.warn(getRegionInfo().getEncodedName() + " : "
6184 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
6185 + " because its sequence id " + replaySeqId + " is smaller than this regions "
6186 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
6187 return;
6190 // If we were waiting for observing a flush or region opening event for not showing partial
6191 // data after a secondary region crash, we can allow reads now. This event means that the
6192 // primary was not able to flush because memstore is empty when we requested flush. By the
6193 // time we observe this, we are guaranteed to have up to date seqId with our previous
6194 // assignment.
6195 this.setReadsEnabled(true);
6199 PrepareFlushResult getPrepareFlushResult() {
6200 return prepareFlushResult;
6204 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6205 * replica implementation.
6207 @Deprecated
6208 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
6209 justification = "Intentional; cleared the memstore")
6210 void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
6211 checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
6212 "RegionEvent marker from WAL ", regionEvent);
6214 startRegionOperation(Operation.REPLAY_EVENT);
6215 try {
6216 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6217 return; // if primary nothing to do
6220 if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
6221 // nothing to do on REGION_CLOSE for now.
6222 return;
6224 if (regionEvent.getEventType() != EventType.REGION_OPEN) {
6225 LOG.warn(getRegionInfo().getEncodedName() + " : "
6226 + "Unknown region event received, ignoring :"
6227 + TextFormat.shortDebugString(regionEvent));
6228 return;
6231 if (LOG.isDebugEnabled()) {
6232 LOG.debug(getRegionInfo().getEncodedName() + " : "
6233 + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent));
6236 // we will use writestate as a coarse-grain lock for all the replay events
6237 synchronized (writestate) {
6238 // Replication can deliver events out of order when primary region moves or the region
6239 // server crashes, since there is no coordination between replication of different wal files
6240 // belonging to different region servers. We have to safe guard against this case by using
6241 // region open event's seqid. Since this is the first event that the region puts (after
6242 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6243 // smaller than this seqId
6244 if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
6245 this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
6246 } else {
6247 LOG.warn(getRegionInfo().getEncodedName() + " : "
6248 + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent)
6249 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
6250 + " of " + lastReplayedOpenRegionSeqId);
6251 return;
6254 // region open lists all the files that the region has at the time of the opening. Just pick
6255 // all the files and drop prepared flushes and empty memstores
6256 for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
6257 // stores of primary may be different now
6258 byte[] family = storeDescriptor.getFamilyName().toByteArray();
6259 HStore store = getStore(family);
6260 if (store == null) {
6261 LOG.warn(getRegionInfo().getEncodedName() + " : "
6262 + "Received a region open marker from primary, but the family is not found. "
6263 + "Ignoring. StoreDescriptor:" + storeDescriptor);
6264 continue;
6267 long storeSeqId = store.getMaxSequenceId().orElse(0L);
6268 List<String> storeFiles = storeDescriptor.getStoreFileList();
6269 try {
6270 store.refreshStoreFiles(storeFiles); // replace the files with the new ones
6271 } catch (FileNotFoundException ex) {
6272 LOG.warn(getRegionInfo().getEncodedName() + " : "
6273 + "At least one of the store files: " + storeFiles
6274 + " doesn't exist any more. Skip loading the file(s)", ex);
6275 continue;
6277 if (store.getMaxSequenceId().orElse(0L) != storeSeqId) {
6278 // Record latest flush time if we picked up new files
6279 lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
6282 if (writestate.flushing) {
6283 // only drop memstore snapshots if they are smaller than last flush for the store
6284 if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
6285 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
6286 null : this.prepareFlushResult.storeFlushCtxs.get(family);
6287 if (ctx != null) {
6288 MemStoreSize mss = store.getFlushableSize();
6289 ctx.abort();
6290 this.decrMemStoreSize(mss);
6291 this.prepareFlushResult.storeFlushCtxs.remove(family);
6296 // Drop the memstore contents if they are now smaller than the latest seen flushed file
6297 dropMemStoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
6298 if (storeSeqId > this.maxFlushedSeqId) {
6299 this.maxFlushedSeqId = storeSeqId;
6303 // if all stores ended up dropping their snapshots, we can safely drop the
6304 // prepareFlushResult
6305 dropPrepareFlushIfPossible();
6307 // advance the mvcc read point so that the new flushed file is visible.
6308 mvcc.await();
6310 // If we were waiting for observing a flush or region opening event for not showing partial
6311 // data after a secondary region crash, we can allow reads now.
6312 this.setReadsEnabled(true);
6314 // C. Finally notify anyone waiting on memstore to clear:
6315 // e.g. checkResources().
6316 synchronized (this) {
6317 notifyAll(); // FindBugs NN_NAKED_NOTIFY
6320 logRegionFiles();
6321 } finally {
6322 closeRegionOperation(Operation.REPLAY_EVENT);
6327 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6328 * replica implementation.
6330 @Deprecated
6331 void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
6332 checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
6333 "BulkLoad marker from WAL ", bulkLoadEvent);
6335 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6336 return; // if primary nothing to do
6339 if (LOG.isDebugEnabled()) {
6340 LOG.debug(getRegionInfo().getEncodedName() + " : "
6341 + "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent));
6343 // check if multiple families involved
6344 boolean multipleFamilies = false;
6345 byte[] family = null;
6346 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6347 byte[] fam = storeDescriptor.getFamilyName().toByteArray();
6348 if (family == null) {
6349 family = fam;
6350 } else if (!Bytes.equals(family, fam)) {
6351 multipleFamilies = true;
6352 break;
6356 startBulkRegionOperation(multipleFamilies);
6357 try {
6358 // we will use writestate as a coarse-grain lock for all the replay events
6359 synchronized (writestate) {
6360 // Replication can deliver events out of order when primary region moves or the region
6361 // server crashes, since there is no coordination between replication of different wal files
6362 // belonging to different region servers. We have to safe guard against this case by using
6363 // region open event's seqid. Since this is the first event that the region puts (after
6364 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6365 // smaller than this seqId
6366 if (bulkLoadEvent.getBulkloadSeqNum() >= 0
6367 && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) {
6368 LOG.warn(getRegionInfo().getEncodedName() + " : "
6369 + "Skipping replaying bulkload event :"
6370 + TextFormat.shortDebugString(bulkLoadEvent)
6371 + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
6372 + " =" + lastReplayedOpenRegionSeqId);
6374 return;
6377 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6378 // stores of primary may be different now
6379 family = storeDescriptor.getFamilyName().toByteArray();
6380 HStore store = getStore(family);
6381 if (store == null) {
6382 LOG.warn(getRegionInfo().getEncodedName() + " : "
6383 + "Received a bulk load marker from primary, but the family is not found. "
6384 + "Ignoring. StoreDescriptor:" + storeDescriptor);
6385 continue;
6388 List<String> storeFiles = storeDescriptor.getStoreFileList();
6389 for (String storeFile : storeFiles) {
6390 StoreFileInfo storeFileInfo = null;
6391 try {
6392 storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile);
6393 store.bulkLoadHFile(storeFileInfo);
6394 } catch(FileNotFoundException ex) {
6395 LOG.warn(getRegionInfo().getEncodedName() + " : "
6396 + ((storeFileInfo != null) ? storeFileInfo.toString() :
6397 (new Path(Bytes.toString(family), storeFile)).toString())
6398 + " doesn't exist any more. Skip loading the file");
6403 if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
6404 mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
6406 } finally {
6407 closeBulkRegionOperation();
6412 * Replay the batch mutate for secondary replica.
6413 * <p/>
6414 * We will directly apply the cells to the memstore. This is because:
6415 * <ol>
6416 * <li>All the cells are gotten from {@link WALEdit}, so we only have {@link Put} and
6417 * {@link Delete} here</li>
6418 * <li>The replay is single threaded, we do not need to acquire row lock, as the region is read
6419 * only so no one else can write it.</li>
6420 * <li>We do not need to write WAL.</li>
6421 * <li>We will advance MVCC in the caller directly.</li>
6422 * </ol>
6424 private void replayWALBatchMutate(Map<byte[], List<Cell>> family2Cells) throws IOException {
6425 startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
6426 try {
6427 for (Map.Entry<byte[], List<Cell>> entry : family2Cells.entrySet()) {
6428 applyToMemStore(getStore(entry.getKey()), entry.getValue(), false, memStoreSizing);
6430 } finally {
6431 closeRegionOperation(Operation.REPLAY_BATCH_MUTATE);
6436 * Replay the meta edits, i.e, flush marker, compaction marker, bulk load marker, region event
6437 * marker, etc.
6438 * <p/>
6439 * For all events other than start flush, we will just call {@link #refreshStoreFiles()} as the
6440 * logic is straight-forward and robust. For start flush, we need to snapshot the memstore, so
6441 * later {@link #refreshStoreFiles()} call could drop the snapshot, otherwise we may run out of
6442 * memory.
6444 private void replayWALMetaEdit(Cell cell) throws IOException {
6445 startRegionOperation(Operation.REPLAY_EVENT);
6446 try {
6447 FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(cell);
6448 if (flushDesc != null) {
6449 switch (flushDesc.getAction()) {
6450 case START_FLUSH:
6451 // for start flush, we need to take a snapshot of the current memstore
6452 synchronized (writestate) {
6453 if (!writestate.flushing) {
6454 this.writestate.flushing = true;
6455 } else {
6456 // usually this should not happen but let's make the code more robust, it is not a
6457 // big deal to just ignore it, the refreshStoreFiles call should have the ability to
6458 // clean up the inconsistent state.
6459 LOG.debug("NOT flushing {} as already flushing", getRegionInfo());
6460 break;
6463 MonitoredTask status =
6464 TaskMonitor.get().createStatus("Preparing flush " + getRegionInfo());
6465 Collection<HStore> storesToFlush = getStoresToFlush(flushDesc);
6466 try {
6467 PrepareFlushResult prepareResult =
6468 internalPrepareFlushCache(null, flushDesc.getFlushSequenceNumber(), storesToFlush,
6469 status, false, FlushLifeCycleTracker.DUMMY);
6470 if (prepareResult.result == null) {
6471 // save the PrepareFlushResult so that we can use it later from commit flush
6472 this.prepareFlushResult = prepareResult;
6473 status.markComplete("Flush prepare successful");
6474 if (LOG.isDebugEnabled()) {
6475 LOG.debug("{} prepared flush with seqId: {}", getRegionInfo(),
6476 flushDesc.getFlushSequenceNumber());
6478 } else {
6479 // special case empty memstore. We will still save the flush result in this case,
6480 // since our memstore is empty, but the primary is still flushing
6481 if (prepareResult.getResult()
6482 .getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
6483 this.prepareFlushResult = prepareResult;
6484 if (LOG.isDebugEnabled()) {
6485 LOG.debug("{} prepared empty flush with seqId: {}", getRegionInfo(),
6486 flushDesc.getFlushSequenceNumber());
6489 status.abort("Flush prepare failed with " + prepareResult.result);
6490 // nothing much to do. prepare flush failed because of some reason.
6492 } finally {
6493 status.cleanup();
6495 break;
6496 case ABORT_FLUSH:
6497 // do nothing, an abort flush means the source region server will crash itself, after
6498 // the primary region online, it will send us an open region marker, then we can clean
6499 // up the memstore.
6500 synchronized (writestate) {
6501 writestate.flushing = false;
6503 break;
6504 case COMMIT_FLUSH:
6505 case CANNOT_FLUSH:
6506 // just call refreshStoreFiles
6507 refreshStoreFiles();
6508 logRegionFiles();
6509 synchronized (writestate) {
6510 writestate.flushing = false;
6512 break;
6513 default:
6514 LOG.warn("{} received a flush event with unknown action: {}", getRegionInfo(),
6515 TextFormat.shortDebugString(flushDesc));
6517 } else {
6518 // for all other region events, we will do a refreshStoreFiles
6519 refreshStoreFiles();
6520 logRegionFiles();
6522 } finally {
6523 closeRegionOperation(Operation.REPLAY_EVENT);
6528 * Replay remote wal entry sent by primary replica.
6529 * <p/>
6530 * Should only call this method on secondary replicas.
6532 void replayWALEntry(WALEntry entry, CellScanner cells) throws IOException {
6533 long timeout = -1L;
6534 Optional<RpcCall> call = RpcServer.getCurrentCall();
6535 if (call.isPresent()) {
6536 long deadline = call.get().getDeadline();
6537 if (deadline < Long.MAX_VALUE) {
6538 timeout = deadline - EnvironmentEdgeManager.currentTime();
6539 if (timeout <= 0) {
6540 throw new TimeoutIOException("Timeout while replaying edits for " + getRegionInfo());
6544 if (timeout > 0) {
6545 try {
6546 if (!replayLock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
6547 throw new TimeoutIOException(
6548 "Timeout while waiting for lock when replaying edits for " + getRegionInfo());
6550 } catch (InterruptedException e) {
6551 throw throwOnInterrupt(e);
6553 } else {
6554 replayLock.lock();
6556 try {
6557 int count = entry.getAssociatedCellCount();
6558 long sequenceId = entry.getKey().getLogSequenceNumber();
6559 if (lastReplayedSequenceId >= sequenceId) {
6560 // we have already replayed this edit, skip
6561 // remember to advance the CellScanner, as we may have multiple WALEntries, we may still
6562 // need apply later WALEntries
6563 for (int i = 0; i < count; i++) {
6564 // Throw index out of bounds if our cell count is off
6565 if (!cells.advance()) {
6566 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
6569 return;
6571 Map<byte[], List<Cell>> family2Cells = new TreeMap<>(Bytes.BYTES_COMPARATOR);
6572 for (int i = 0; i < count; i++) {
6573 // Throw index out of bounds if our cell count is off
6574 if (!cells.advance()) {
6575 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
6577 Cell cell = cells.current();
6578 if (WALEdit.isMetaEditFamily(cell)) {
6579 // If there is meta edit, i.e, we have done flush/compaction/open, then we need to apply
6580 // the previous cells first, and then replay the special meta edit. The meta edit is like
6581 // a barrier, We need to keep the order. For example, the flush marker will contain a
6582 // flush sequence number, which makes us possible to drop memstore content, but if we
6583 // apply some edits which have greater sequence id first, then we can not drop the
6584 // memstore content when replaying the flush marker, which is not good as we could run out
6585 // of memory.
6586 // And usually, a meta edit will have a special WALEntry for it, so this is just a safe
6587 // guard logic to make sure we do not break things in the worst case.
6588 if (!family2Cells.isEmpty()) {
6589 replayWALBatchMutate(family2Cells);
6590 family2Cells.clear();
6592 replayWALMetaEdit(cell);
6593 } else {
6594 family2Cells
6595 .computeIfAbsent(CellUtil.cloneFamily(cell), k -> new ArrayList<>())
6596 .add(cell);
6599 // do not forget to apply the remaining cells
6600 if (!family2Cells.isEmpty()) {
6601 replayWALBatchMutate(family2Cells);
6603 mvcc.advanceTo(sequenceId);
6604 lastReplayedSequenceId = sequenceId;
6605 } finally {
6606 replayLock.unlock();
6611 * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult
6613 private void dropPrepareFlushIfPossible() {
6614 if (writestate.flushing) {
6615 boolean canDrop = true;
6616 if (prepareFlushResult.storeFlushCtxs != null) {
6617 for (Entry<byte[], StoreFlushContext> entry : prepareFlushResult.storeFlushCtxs
6618 .entrySet()) {
6619 HStore store = getStore(entry.getKey());
6620 if (store == null) {
6621 continue;
6623 if (store.getSnapshotSize().getDataSize() > 0) {
6624 canDrop = false;
6625 break;
6630 // this means that all the stores in the region has finished flushing, but the WAL marker
6631 // may not have been written or we did not receive it yet.
6632 if (canDrop) {
6633 writestate.flushing = false;
6634 this.prepareFlushResult = null;
6639 @Override
6640 public boolean refreshStoreFiles() throws IOException {
6641 return refreshStoreFiles(false);
6644 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
6645 justification = "Notify is about post replay. Intentional")
6646 protected boolean refreshStoreFiles(boolean force) throws IOException {
6647 if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6648 return false; // if primary nothing to do
6651 if (LOG.isDebugEnabled()) {
6652 LOG.debug(getRegionInfo().getEncodedName() + " : "
6653 + "Refreshing store files to see whether we can free up memstore");
6656 long totalFreedDataSize = 0;
6658 long smallestSeqIdInStores = Long.MAX_VALUE;
6660 startRegionOperation(); // obtain region close lock
6661 try {
6662 Map<HStore, Long> map = new HashMap<>();
6663 synchronized (writestate) {
6664 for (HStore store : stores.values()) {
6665 // TODO: some stores might see new data from flush, while others do not which
6666 // MIGHT break atomic edits across column families.
6667 long maxSeqIdBefore = store.getMaxSequenceId().orElse(0L);
6669 // refresh the store files. This is similar to observing a region open wal marker.
6670 store.refreshStoreFiles();
6672 long storeSeqId = store.getMaxSequenceId().orElse(0L);
6673 if (storeSeqId < smallestSeqIdInStores) {
6674 smallestSeqIdInStores = storeSeqId;
6677 // see whether we can drop the memstore or the snapshot
6678 if (storeSeqId > maxSeqIdBefore) {
6679 if (writestate.flushing) {
6680 // only drop memstore snapshots if they are smaller than last flush for the store
6681 if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
6682 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
6683 null : this.prepareFlushResult.storeFlushCtxs.get(
6684 store.getColumnFamilyDescriptor().getName());
6685 if (ctx != null) {
6686 MemStoreSize mss = store.getFlushableSize();
6687 ctx.abort();
6688 this.decrMemStoreSize(mss);
6689 this.prepareFlushResult.storeFlushCtxs.
6690 remove(store.getColumnFamilyDescriptor().getName());
6691 totalFreedDataSize += mss.getDataSize();
6696 map.put(store, storeSeqId);
6700 // if all stores ended up dropping their snapshots, we can safely drop the
6701 // prepareFlushResult
6702 dropPrepareFlushIfPossible();
6704 // advance the mvcc read point so that the new flushed files are visible.
6705 // either greater than flush seq number or they were already picked up via flush.
6706 for (HStore s : stores.values()) {
6707 mvcc.advanceTo(s.getMaxMemStoreTS().orElse(0L));
6711 // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely
6712 // skip all edits that are to be replayed in the future with that has a smaller seqId
6713 // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits
6714 // that we have picked the flush files for
6715 if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
6716 this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
6719 if (!map.isEmpty()) {
6720 for (Map.Entry<HStore, Long> entry : map.entrySet()) {
6721 // Drop the memstore contents if they are now smaller than the latest seen flushed file
6722 totalFreedDataSize += dropMemStoreContentsForSeqId(entry.getValue(), entry.getKey())
6723 .getDataSize();
6726 // C. Finally notify anyone waiting on memstore to clear:
6727 // e.g. checkResources().
6728 synchronized (this) {
6729 notifyAll(); // FindBugs NN_NAKED_NOTIFY
6731 return totalFreedDataSize > 0;
6732 } finally {
6733 closeRegionOperation();
6737 private void logRegionFiles() {
6738 if (LOG.isTraceEnabled()) {
6739 LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
6740 stores.values().stream().filter(s -> s.getStorefiles() != null)
6741 .flatMap(s -> s.getStorefiles().stream())
6742 .forEachOrdered(sf -> LOG.trace(getRegionInfo().getEncodedName() + " : " + sf));
6746 /** Checks whether the given regionName is either equal to our region, or that
6747 * the regionName is the primary region to our corresponding range for the secondary replica.
6749 private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
6750 throws WrongRegionException {
6751 if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
6752 return;
6755 if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) &&
6756 Bytes.equals(encodedRegionName,
6757 this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) {
6758 return;
6761 throw new WrongRegionException(exceptionMsg + payload
6762 + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
6763 + " does not match this region: " + this.getRegionInfo());
6767 * Used by tests
6768 * @param s Store to add edit too.
6769 * @param cell Cell to add.
6771 protected void restoreEdit(HStore s, Cell cell, MemStoreSizing memstoreAccounting) {
6772 s.add(cell, memstoreAccounting);
6776 * make sure have been through lease recovery before get file status, so the file length can be
6777 * trusted.
6778 * @param p File to check.
6779 * @return True if file was zero-length (and if so, we'll delete it in here).
6780 * @throws IOException
6782 private static boolean isZeroLengthThenDelete(final FileSystem fs, final FileStatus stat,
6783 final Path p) throws IOException {
6784 if (stat.getLen() > 0) {
6785 return false;
6787 LOG.warn("File " + p + " is zero-length, deleting.");
6788 fs.delete(p, false);
6789 return true;
6792 protected HStore instantiateHStore(final ColumnFamilyDescriptor family, boolean warmup)
6793 throws IOException {
6794 if (family.isMobEnabled()) {
6795 if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
6796 throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS +
6797 " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY +
6798 " accordingly.");
6800 return new HMobStore(this, family, this.conf, warmup);
6802 return new HStore(this, family, this.conf, warmup);
6805 @Override
6806 public HStore getStore(byte[] column) {
6807 return this.stores.get(column);
6811 * Return HStore instance. Does not do any copy: as the number of store is limited, we iterate on
6812 * the list.
6814 private HStore getStore(Cell cell) {
6815 return stores.entrySet().stream().filter(e -> CellUtil.matchingFamily(cell, e.getKey()))
6816 .map(e -> e.getValue()).findFirst().orElse(null);
6819 @Override
6820 public List<HStore> getStores() {
6821 return new ArrayList<>(stores.values());
6824 @Override
6825 public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException {
6826 List<String> storeFileNames = new ArrayList<>();
6827 synchronized (closeLock) {
6828 for (byte[] column : columns) {
6829 HStore store = this.stores.get(column);
6830 if (store == null) {
6831 throw new IllegalArgumentException(
6832 "No column family : " + new String(column, StandardCharsets.UTF_8) + " available");
6834 Collection<HStoreFile> storeFiles = store.getStorefiles();
6835 if (storeFiles == null) {
6836 continue;
6838 for (HStoreFile storeFile : storeFiles) {
6839 storeFileNames.add(storeFile.getPath().toString());
6842 logRegionFiles();
6845 return storeFileNames;
6848 //////////////////////////////////////////////////////////////////////////////
6849 // Support code
6850 //////////////////////////////////////////////////////////////////////////////
6852 /** Make sure this is a valid row for the HRegion */
6853 void checkRow(byte[] row, String op) throws IOException {
6854 if (!rowIsInRange(getRegionInfo(), row)) {
6855 throw new WrongRegionException("Requested row out of range for " +
6856 op + " on HRegion " + this + ", startKey='" +
6857 Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" +
6858 Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" +
6859 Bytes.toStringBinary(row) + "'");
6865 * Get an exclusive ( write lock ) lock on a given row.
6866 * @param row Which row to lock.
6867 * @return A locked RowLock. The lock is exclusive and already aqquired.
6868 * @throws IOException
6870 public RowLock getRowLock(byte[] row) throws IOException {
6871 return getRowLock(row, false);
6874 @Override
6875 public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
6876 checkRow(row, "row lock");
6877 return getRowLock(row, readLock, null);
6880 Span createRegionSpan(String name) {
6881 return TraceUtil.createSpan(name).setAttribute(REGION_NAMES_KEY,
6882 Collections.singletonList(getRegionInfo().getRegionNameAsString()));
6885 // will be override in tests
6886 protected RowLock getRowLockInternal(byte[] row, boolean readLock, RowLock prevRowLock)
6887 throws IOException {
6888 // create an object to use a a key in the row lock map
6889 HashedBytes rowKey = new HashedBytes(row);
6891 RowLockContext rowLockContext = null;
6892 RowLockImpl result = null;
6894 boolean success = false;
6895 try {
6896 // Keep trying until we have a lock or error out.
6897 // TODO: do we need to add a time component here?
6898 while (result == null) {
6899 rowLockContext = computeIfAbsent(lockedRows, rowKey, () -> new RowLockContext(rowKey));
6900 // Now try an get the lock.
6901 // This can fail as
6902 if (readLock) {
6903 // For read lock, if the caller has locked the same row previously, it will not try
6904 // to acquire the same read lock. It simply returns the previous row lock.
6905 RowLockImpl prevRowLockImpl = (RowLockImpl)prevRowLock;
6906 if ((prevRowLockImpl != null) && (prevRowLockImpl.getLock() ==
6907 rowLockContext.readWriteLock.readLock())) {
6908 success = true;
6909 return prevRowLock;
6911 result = rowLockContext.newReadLock();
6912 } else {
6913 result = rowLockContext.newWriteLock();
6917 int timeout = rowLockWaitDuration;
6918 boolean reachDeadlineFirst = false;
6919 Optional<RpcCall> call = RpcServer.getCurrentCall();
6920 if (call.isPresent()) {
6921 long deadline = call.get().getDeadline();
6922 if (deadline < Long.MAX_VALUE) {
6923 int timeToDeadline = (int) (deadline - EnvironmentEdgeManager.currentTime());
6924 if (timeToDeadline <= this.rowLockWaitDuration) {
6925 reachDeadlineFirst = true;
6926 timeout = timeToDeadline;
6931 if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
6932 String message = "Timed out waiting for lock for row: " + rowKey + " in region "
6933 + getRegionInfo().getEncodedName();
6934 if (reachDeadlineFirst) {
6935 throw new TimeoutIOException(message);
6936 } else {
6937 // If timeToDeadline is larger than rowLockWaitDuration, we can not drop the request.
6938 throw new IOException(message);
6941 rowLockContext.setThreadName(Thread.currentThread().getName());
6942 success = true;
6943 return result;
6944 } catch (InterruptedException ie) {
6945 if (LOG.isDebugEnabled()) {
6946 LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey,
6947 getRegionInfo().getRegionNameAsString());
6949 throw throwOnInterrupt(ie);
6950 } catch (Error error) {
6951 // The maximum lock count for read lock is 64K (hardcoded), when this maximum count
6952 // is reached, it will throw out an Error. This Error needs to be caught so it can
6953 // go ahead to process the minibatch with lock acquired.
6954 LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row),
6955 getRegionInfo().getRegionNameAsString(), error);
6956 IOException ioe = new IOException(error);
6957 throw ioe;
6958 } finally {
6959 // Clean up the counts just in case this was the thing keeping the context alive.
6960 if (!success && rowLockContext != null) {
6961 rowLockContext.cleanUp();
6966 private RowLock getRowLock(byte[] row, boolean readLock, final RowLock prevRowLock)
6967 throws IOException {
6968 return TraceUtil.trace(() -> getRowLockInternal(row, readLock, prevRowLock),
6969 () -> createRegionSpan("Region.getRowLock").setAttribute(ROW_LOCK_READ_LOCK_KEY,
6970 readLock));
6973 private void releaseRowLocks(List<RowLock> rowLocks) {
6974 if (rowLocks != null) {
6975 for (RowLock rowLock : rowLocks) {
6976 rowLock.release();
6978 rowLocks.clear();
6982 public int getReadLockCount() {
6983 return lock.getReadLockCount();
6986 public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
6987 return lockedRows;
6990 class RowLockContext {
6991 private final HashedBytes row;
6992 final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
6993 final AtomicBoolean usable = new AtomicBoolean(true);
6994 final AtomicInteger count = new AtomicInteger(0);
6995 final Object lock = new Object();
6996 private String threadName;
6998 RowLockContext(HashedBytes row) {
6999 this.row = row;
7002 RowLockImpl newWriteLock() {
7003 Lock l = readWriteLock.writeLock();
7004 return getRowLock(l);
7006 RowLockImpl newReadLock() {
7007 Lock l = readWriteLock.readLock();
7008 return getRowLock(l);
7011 private RowLockImpl getRowLock(Lock l) {
7012 count.incrementAndGet();
7013 synchronized (lock) {
7014 if (usable.get()) {
7015 return new RowLockImpl(this, l);
7016 } else {
7017 return null;
7022 void cleanUp() {
7023 long c = count.decrementAndGet();
7024 if (c <= 0) {
7025 synchronized (lock) {
7026 if (count.get() <= 0 && usable.get()){ // Don't attempt to remove row if already removed
7027 usable.set(false);
7028 RowLockContext removed = lockedRows.remove(row);
7029 assert removed == this: "we should never remove a different context";
7035 public void setThreadName(String threadName) {
7036 this.threadName = threadName;
7039 @Override
7040 public String toString() {
7041 return "RowLockContext{" +
7042 "row=" + row +
7043 ", readWriteLock=" + readWriteLock +
7044 ", count=" + count +
7045 ", threadName=" + threadName +
7046 '}';
7051 * Class used to represent a lock on a row.
7053 public static class RowLockImpl implements RowLock {
7054 private final RowLockContext context;
7055 private final Lock lock;
7057 public RowLockImpl(RowLockContext context, Lock lock) {
7058 this.context = context;
7059 this.lock = lock;
7062 public Lock getLock() {
7063 return lock;
7066 public RowLockContext getContext() {
7067 return context;
7070 @Override
7071 public void release() {
7072 lock.unlock();
7073 context.cleanUp();
7076 @Override
7077 public String toString() {
7078 return "RowLockImpl{" +
7079 "context=" + context +
7080 ", lock=" + lock +
7081 '}';
7086 * Determines whether multiple column families are present
7087 * Precondition: familyPaths is not null
7089 * @param familyPaths List of (column family, hfilePath)
7091 private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
7092 boolean multipleFamilies = false;
7093 byte[] family = null;
7094 for (Pair<byte[], String> pair : familyPaths) {
7095 byte[] fam = pair.getFirst();
7096 if (family == null) {
7097 family = fam;
7098 } else if (!Bytes.equals(family, fam)) {
7099 multipleFamilies = true;
7100 break;
7103 return multipleFamilies;
7107 * Attempts to atomically load a group of hfiles. This is critical for loading
7108 * rows with multiple column families atomically.
7110 * @param familyPaths List of Pair&lt;byte[] column family, String hfilePath&gt;
7111 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
7112 * file about to be bulk loaded
7113 * @param assignSeqId
7114 * @return Map from family to List of store file paths if successful, null if failed recoverably
7115 * @throws IOException if failed unrecoverably.
7117 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
7118 BulkLoadListener bulkLoadListener) throws IOException {
7119 return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false,
7120 null, true);
7124 * Listener class to enable callers of
7125 * bulkLoadHFile() to perform any necessary
7126 * pre/post processing of a given bulkload call
7128 public interface BulkLoadListener {
7130 * Called before an HFile is actually loaded
7131 * @param family family being loaded to
7132 * @param srcPath path of HFile
7133 * @return final path to be used for actual loading
7134 * @throws IOException
7136 String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile, String customStaging)
7137 throws IOException;
7140 * Called after a successful HFile load
7141 * @param family family being loaded to
7142 * @param srcPath path of HFile
7143 * @throws IOException
7145 void doneBulkLoad(byte[] family, String srcPath) throws IOException;
7148 * Called after a failed HFile load
7149 * @param family family being loaded to
7150 * @param srcPath path of HFile
7151 * @throws IOException
7153 void failedBulkLoad(byte[] family, String srcPath) throws IOException;
7157 * Attempts to atomically load a group of hfiles. This is critical for loading
7158 * rows with multiple column families atomically.
7160 * @param familyPaths List of Pair&lt;byte[] column family, String hfilePath&gt;
7161 * @param assignSeqId
7162 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
7163 * file about to be bulk loaded
7164 * @param copyFile always copy hfiles if true
7165 * @param clusterIds ids from clusters that had already handled the given bulkload event.
7166 * @return Map from family to List of store file paths if successful, null if failed recoverably
7167 * @throws IOException if failed unrecoverably.
7169 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
7170 boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile,
7171 List<String> clusterIds, boolean replicate) throws IOException {
7172 long seqId = -1;
7173 Map<byte[], List<Path>> storeFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
7174 Map<String, Long> storeFilesSizes = new HashMap<>();
7175 Preconditions.checkNotNull(familyPaths);
7176 // we need writeLock for multi-family bulk load
7177 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
7178 boolean isSuccessful = false;
7179 try {
7180 this.writeRequestsCount.increment();
7182 // There possibly was a split that happened between when the split keys
7183 // were gathered and before the HRegion's write lock was taken. We need
7184 // to validate the HFile region before attempting to bulk load all of them
7185 IOException ioException = null;
7186 List<Pair<byte[], String>> failures = new ArrayList<>();
7187 for (Pair<byte[], String> p : familyPaths) {
7188 byte[] familyName = p.getFirst();
7189 String path = p.getSecond();
7191 HStore store = getStore(familyName);
7192 if (store == null) {
7193 ioException = new org.apache.hadoop.hbase.DoNotRetryIOException(
7194 "No such column family " + Bytes.toStringBinary(familyName));
7195 } else {
7196 try {
7197 store.assertBulkLoadHFileOk(new Path(path));
7198 } catch (WrongRegionException wre) {
7199 // recoverable (file doesn't fit in region)
7200 failures.add(p);
7201 } catch (IOException ioe) {
7202 // unrecoverable (hdfs problem)
7203 ioException = ioe;
7207 // validation failed because of some sort of IO problem.
7208 if (ioException != null) {
7209 LOG.error("There was IO error when checking if the bulk load is ok in region {}.", this,
7210 ioException);
7211 throw ioException;
7214 // validation failed, bail out before doing anything permanent.
7215 if (failures.size() != 0) {
7216 StringBuilder list = new StringBuilder();
7217 for (Pair<byte[], String> p : failures) {
7218 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
7219 .append(p.getSecond());
7221 // problem when validating
7222 LOG.warn("There was a recoverable bulk load failure likely due to a split. These (family,"
7223 + " HFile) pairs were not loaded: {}, in region {}", list.toString(), this);
7224 return null;
7227 // We need to assign a sequential ID that's in between two memstores in order to preserve
7228 // the guarantee that all the edits lower than the highest sequential ID from all the
7229 // HFiles are flushed on disk. See HBASE-10958. The sequence id returned when we flush is
7230 // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is
7231 // a sequence id that we can be sure is beyond the last hfile written).
7232 if (assignSeqId) {
7233 FlushResult fs = flushcache(true, false, FlushLifeCycleTracker.DUMMY);
7234 if (fs.isFlushSucceeded()) {
7235 seqId = ((FlushResultImpl)fs).flushSequenceId;
7236 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
7237 seqId = ((FlushResultImpl)fs).flushSequenceId;
7238 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) {
7239 // CANNOT_FLUSH may mean that a flush is already on-going
7240 // we need to wait for that flush to complete
7241 waitForFlushes();
7242 } else {
7243 throw new IOException("Could not bulk load with an assigned sequential ID because the "+
7244 "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason);
7248 Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath =
7249 new TreeMap<>(Bytes.BYTES_COMPARATOR);
7250 for (Pair<byte[], String> p : familyPaths) {
7251 byte[] familyName = p.getFirst();
7252 String path = p.getSecond();
7253 HStore store = getStore(familyName);
7254 if (!familyWithFinalPath.containsKey(familyName)) {
7255 familyWithFinalPath.put(familyName, new ArrayList<>());
7257 List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName);
7258 String finalPath = path;
7259 try {
7260 boolean reqTmp = store.storeEngine.requireWritingToTmpDirFirst();
7261 if (bulkLoadListener != null) {
7262 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile,
7263 reqTmp ? null : regionDir.toString());
7265 Pair<Path, Path> pair = null;
7266 if (reqTmp) {
7267 pair = store.preBulkLoadHFile(finalPath, seqId);
7269 else {
7270 Path livePath = new Path(finalPath);
7271 pair = new Pair<>(livePath, livePath);
7273 lst.add(pair);
7274 } catch (IOException ioe) {
7275 // A failure here can cause an atomicity violation that we currently
7276 // cannot recover from since it is likely a failed HDFS operation.
7278 LOG.error("There was a partial failure due to IO when attempting to" +
7279 " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
7280 if (bulkLoadListener != null) {
7281 try {
7282 bulkLoadListener.failedBulkLoad(familyName, finalPath);
7283 } catch (Exception ex) {
7284 LOG.error("Error while calling failedBulkLoad for family " +
7285 Bytes.toString(familyName) + " with path " + path, ex);
7288 throw ioe;
7292 if (this.getCoprocessorHost() != null) {
7293 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7294 this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue());
7297 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7298 byte[] familyName = entry.getKey();
7299 for (Pair<Path, Path> p : entry.getValue()) {
7300 String path = p.getFirst().toString();
7301 Path commitedStoreFile = p.getSecond();
7302 HStore store = getStore(familyName);
7303 try {
7304 store.bulkLoadHFile(familyName, path, commitedStoreFile);
7305 // Note the size of the store file
7306 try {
7307 FileSystem fs = commitedStoreFile.getFileSystem(baseConf);
7308 storeFilesSizes.put(commitedStoreFile.getName(), fs.getFileStatus(commitedStoreFile)
7309 .getLen());
7310 } catch (IOException e) {
7311 LOG.warn("Failed to find the size of hfile " + commitedStoreFile, e);
7312 storeFilesSizes.put(commitedStoreFile.getName(), 0L);
7315 if(storeFiles.containsKey(familyName)) {
7316 storeFiles.get(familyName).add(commitedStoreFile);
7317 } else {
7318 List<Path> storeFileNames = new ArrayList<>();
7319 storeFileNames.add(commitedStoreFile);
7320 storeFiles.put(familyName, storeFileNames);
7322 if (bulkLoadListener != null) {
7323 bulkLoadListener.doneBulkLoad(familyName, path);
7325 } catch (IOException ioe) {
7326 // A failure here can cause an atomicity violation that we currently
7327 // cannot recover from since it is likely a failed HDFS operation.
7329 // TODO Need a better story for reverting partial failures due to HDFS.
7330 LOG.error("There was a partial failure due to IO when attempting to" +
7331 " load " + Bytes.toString(familyName) + " : " + p.getSecond(), ioe);
7332 if (bulkLoadListener != null) {
7333 try {
7334 bulkLoadListener.failedBulkLoad(familyName, path);
7335 } catch (Exception ex) {
7336 LOG.error("Error while calling failedBulkLoad for family " +
7337 Bytes.toString(familyName) + " with path " + path, ex);
7340 throw ioe;
7345 isSuccessful = true;
7346 if (conf.getBoolean(COMPACTION_AFTER_BULKLOAD_ENABLE, false)) {
7347 // request compaction
7348 familyWithFinalPath.keySet().forEach(family -> {
7349 HStore store = getStore(family);
7350 try {
7351 if (this.rsServices != null && store.needsCompaction()) {
7352 this.rsServices.getCompactionRequestor().requestSystemCompaction(this, store,
7353 "bulkload hfiles request compaction", true);
7354 LOG.info("Request compaction for region {} family {} after bulk load",
7355 this.getRegionInfo().getEncodedName(), store.getColumnFamilyName());
7357 } catch (IOException e) {
7358 LOG.error("bulkload hfiles request compaction error ", e);
7362 } finally {
7363 if (wal != null && !storeFiles.isEmpty()) {
7364 // Write a bulk load event for hfiles that are loaded
7365 try {
7366 WALProtos.BulkLoadDescriptor loadDescriptor =
7367 ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
7368 UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()),
7369 storeFiles, storeFilesSizes, seqId, clusterIds, replicate);
7370 WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(),
7371 loadDescriptor, mvcc, regionReplicationSink.orElse(null));
7372 } catch (IOException ioe) {
7373 if (this.rsServices != null) {
7374 // Have to abort region server because some hfiles has been loaded but we can't write
7375 // the event into WAL
7376 isSuccessful = false;
7377 this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
7382 closeBulkRegionOperation();
7384 return isSuccessful ? storeFiles : null;
7387 @Override
7388 public boolean equals(Object o) {
7389 return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
7390 ((HRegion) o).getRegionInfo().getRegionName());
7393 @Override
7394 public int hashCode() {
7395 return Bytes.hashCode(getRegionInfo().getRegionName());
7398 @Override
7399 public String toString() {
7400 return getRegionInfo().getRegionNameAsString();
7403 // Utility methods
7405 * A utility method to create new instances of HRegion based on the {@link HConstants#REGION_IMPL}
7406 * configuration property.
7407 * @param tableDir qualified path of directory where region should be located, usually the table
7408 * directory.
7409 * @param wal The WAL is the outbound log for any updates to the HRegion The wal file is a logfile
7410 * from the previous execution that's custom-computed for this HRegion. The HRegionServer
7411 * computes and sorts the appropriate wal info for this HRegion. If there is a previous
7412 * file (implying that the HRegion has been written-to before), then read it from the
7413 * supplied path.
7414 * @param fs is the filesystem.
7415 * @param conf is global configuration settings.
7416 * @param regionInfo - RegionInfo that describes the region is new), then read them from the
7417 * supplied path.
7418 * @param htd the table descriptor
7419 * @return the new instance
7421 public static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
7422 Configuration conf, RegionInfo regionInfo, final TableDescriptor htd,
7423 RegionServerServices rsServices) {
7424 try {
7425 @SuppressWarnings("unchecked")
7426 Class<? extends HRegion> regionClass =
7427 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
7429 Constructor<? extends HRegion> c =
7430 regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
7431 Configuration.class, RegionInfo.class, TableDescriptor.class,
7432 RegionServerServices.class);
7434 return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
7435 } catch (Throwable e) {
7436 // todo: what should I throw here?
7437 throw new IllegalStateException("Could not instantiate a region instance.", e);
7442 * Convenience method creating new HRegions. Used by createTable.
7443 * @param info Info for region to create.
7444 * @param rootDir Root directory for HBase instance
7445 * @param wal shared WAL
7446 * @param initialize - true to initialize the region
7447 * @return new HRegion
7449 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7450 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7451 final boolean initialize) throws IOException {
7452 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, initialize, null);
7456 * Convenience method creating new HRegions. Used by createTable.
7457 * @param info Info for region to create.
7458 * @param rootDir Root directory for HBase instance
7459 * @param wal shared WAL
7460 * @param initialize - true to initialize the region
7461 * @param rsRpcServices An interface we can request flushes against.
7462 * @return new HRegion
7464 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7465 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7466 final boolean initialize, RegionServerServices rsRpcServices) throws IOException {
7467 LOG.info("creating " + info + ", tableDescriptor="
7468 + (hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir);
7469 createRegionDir(conf, info, rootDir);
7470 FileSystem fs = rootDir.getFileSystem(conf);
7471 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7472 HRegion region =
7473 HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, rsRpcServices);
7474 if (initialize) {
7475 region.initialize(null);
7477 return region;
7481 * Create a region under the given table directory.
7483 public static HRegion createHRegion(Configuration conf, RegionInfo regionInfo, FileSystem fs,
7484 Path tableDir, TableDescriptor tableDesc) throws IOException {
7485 LOG.info("Creating {}, tableDescriptor={}, under table dir {}", regionInfo, tableDesc,
7486 tableDir);
7487 HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo);
7488 HRegion region = HRegion.newHRegion(tableDir, null, fs, conf, regionInfo, tableDesc, null);
7489 return region;
7493 * Create the region directory in the filesystem.
7495 public static HRegionFileSystem createRegionDir(Configuration configuration, RegionInfo ri,
7496 Path rootDir)
7497 throws IOException {
7498 FileSystem fs = rootDir.getFileSystem(configuration);
7499 Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable());
7500 // If directory already exists, will log warning and keep going. Will try to create
7501 // .regioninfo. If one exists, will overwrite.
7502 return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri);
7505 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7506 final Configuration conf,
7507 final TableDescriptor hTableDescriptor,
7508 final WAL wal)
7509 throws IOException {
7510 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
7515 * Open a Region.
7516 * @param info Info for region to be opened.
7517 * @param wal WAL for region to use. This method will call
7518 * WAL#setSequenceNumber(long) passing the result of the call to
7519 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7520 * up. HRegionStore does this every time it opens a new region.
7521 * @return new HRegion
7523 * @throws IOException
7525 public static HRegion openHRegion(final RegionInfo info,
7526 final TableDescriptor htd, final WAL wal,
7527 final Configuration conf)
7528 throws IOException {
7529 return openHRegion(info, htd, wal, conf, null, null);
7533 * Open a Region.
7534 * @param info Info for region to be opened
7535 * @param htd the table descriptor
7536 * @param wal WAL for region to use. This method will call
7537 * WAL#setSequenceNumber(long) passing the result of the call to
7538 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7539 * up. HRegionStore does this every time it opens a new region.
7540 * @param conf The Configuration object to use.
7541 * @param rsServices An interface we can request flushes against.
7542 * @param reporter An interface we can report progress against.
7543 * @return new HRegion
7545 * @throws IOException
7547 public static HRegion openHRegion(final RegionInfo info,
7548 final TableDescriptor htd, final WAL wal, final Configuration conf,
7549 final RegionServerServices rsServices,
7550 final CancelableProgressable reporter)
7551 throws IOException {
7552 return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
7556 * Open a Region.
7557 * @param rootDir Root directory for HBase instance
7558 * @param info Info for region to be opened.
7559 * @param htd the table descriptor
7560 * @param wal WAL for region to use. This method will call
7561 * WAL#setSequenceNumber(long) passing the result of the call to
7562 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7563 * up. HRegionStore does this every time it opens a new region.
7564 * @param conf The Configuration object to use.
7565 * @return new HRegion
7566 * @throws IOException
7568 public static HRegion openHRegion(Path rootDir, final RegionInfo info,
7569 final TableDescriptor htd, final WAL wal, final Configuration conf)
7570 throws IOException {
7571 return openHRegion(rootDir, info, htd, wal, conf, null, null);
7575 * Open a Region.
7576 * @param rootDir Root directory for HBase instance
7577 * @param info Info for region to be opened.
7578 * @param htd the table descriptor
7579 * @param wal WAL for region to use. This method will call
7580 * WAL#setSequenceNumber(long) passing the result of the call to
7581 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7582 * up. HRegionStore does this every time it opens a new region.
7583 * @param conf The Configuration object to use.
7584 * @param rsServices An interface we can request flushes against.
7585 * @param reporter An interface we can report progress against.
7586 * @return new HRegion
7587 * @throws IOException
7589 public static HRegion openHRegion(final Path rootDir, final RegionInfo info,
7590 final TableDescriptor htd, final WAL wal, final Configuration conf,
7591 final RegionServerServices rsServices,
7592 final CancelableProgressable reporter)
7593 throws IOException {
7594 FileSystem fs = null;
7595 if (rsServices != null) {
7596 fs = rsServices.getFileSystem();
7598 if (fs == null) {
7599 fs = rootDir.getFileSystem(conf);
7601 return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
7605 * Open a Region.
7606 * @param conf The Configuration object to use.
7607 * @param fs Filesystem to use
7608 * @param rootDir Root directory for HBase instance
7609 * @param info Info for region to be opened.
7610 * @param htd the table descriptor
7611 * @param wal WAL for region to use. This method will call
7612 * WAL#setSequenceNumber(long) passing the result of the call to
7613 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7614 * up. HRegionStore does this every time it opens a new region.
7615 * @return new HRegion
7617 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7618 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal)
7619 throws IOException {
7620 return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
7624 * Open a Region.
7625 * @param conf The Configuration object to use.
7626 * @param fs Filesystem to use
7627 * @param rootDir Root directory for HBase instance
7628 * @param info Info for region to be opened.
7629 * @param htd the table descriptor
7630 * @param wal WAL for region to use. This method will call
7631 * WAL#setSequenceNumber(long) passing the result of the call to
7632 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7633 * up. HRegionStore does this every time it opens a new region.
7634 * @param rsServices An interface we can request flushes against.
7635 * @param reporter An interface we can report progress against.
7636 * @return new HRegion
7638 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7639 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7640 final RegionServerServices rsServices, final CancelableProgressable reporter)
7641 throws IOException {
7642 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7643 return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter);
7647 * Open a Region.
7648 * @param conf The Configuration object to use.
7649 * @param fs Filesystem to use
7650 * @param info Info for region to be opened.
7651 * @param htd the table descriptor
7652 * @param wal WAL for region to use. This method will call
7653 * WAL#setSequenceNumber(long) passing the result of the call to
7654 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7655 * up. HRegionStore does this every time it opens a new region.
7656 * @param rsServices An interface we can request flushes against.
7657 * @param reporter An interface we can report progress against.
7658 * @return new HRegion
7659 * @throws NullPointerException if {@code info} is {@code null}
7661 public static HRegion openHRegionFromTableDir(final Configuration conf, final FileSystem fs,
7662 final Path tableDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7663 final RegionServerServices rsServices, final CancelableProgressable reporter)
7664 throws IOException {
7665 Objects.requireNonNull(info, "RegionInfo cannot be null");
7666 LOG.debug("Opening region: {}", info);
7667 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
7668 return r.openHRegion(reporter);
7671 public NavigableMap<byte[], Integer> getReplicationScope() {
7672 return this.replicationScope;
7676 * Useful when reopening a closed region (normally for unit tests)
7677 * @param other original object
7678 * @param reporter An interface we can report progress against.
7679 * @return new HRegion
7681 public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
7682 throws IOException {
7683 HRegionFileSystem regionFs = other.getRegionFileSystem();
7684 HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
7685 other.baseConf, other.getRegionInfo(), other.getTableDescriptor(), null);
7686 return r.openHRegion(reporter);
7689 public static Region openHRegion(final Region other, final CancelableProgressable reporter)
7690 throws IOException {
7691 return openHRegion((HRegion)other, reporter);
7695 * Open HRegion.
7696 * <p/>
7697 * Calls initialize and sets sequenceId.
7698 * @return Returns <code>this</code>
7700 private HRegion openHRegion(final CancelableProgressable reporter) throws IOException {
7701 try {
7702 // Refuse to open the region if we are missing local compression support
7703 TableDescriptorChecker.checkCompression(htableDescriptor);
7704 // Refuse to open the region if encryption configuration is incorrect or
7705 // codec support is missing
7706 LOG.debug("checking encryption for " + this.getRegionInfo().getEncodedName());
7707 TableDescriptorChecker.checkEncryption(conf, htableDescriptor);
7708 // Refuse to open the region if a required class cannot be loaded
7709 LOG.debug("checking classloading for " + this.getRegionInfo().getEncodedName());
7710 TableDescriptorChecker.checkClassLoading(conf, htableDescriptor);
7711 this.openSeqNum = initialize(reporter);
7712 this.mvcc.advanceTo(openSeqNum);
7713 // The openSeqNum must be increased every time when a region is assigned, as we rely on it to
7714 // determine whether a region has been successfully reopened. So here we always write open
7715 // marker, even if the table is read only.
7716 if (wal != null && getRegionServerServices() != null &&
7717 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
7718 writeRegionOpenMarker(wal, openSeqNum);
7720 } catch (Throwable t) {
7721 // By coprocessor path wrong region will open failed,
7722 // MetricsRegionWrapperImpl is already init and not close,
7723 // add region close when open failed
7724 try {
7725 // It is not required to write sequence id file when region open is failed.
7726 // Passing true to skip the sequence id file write.
7727 this.close(true);
7728 } catch (Throwable e) {
7729 LOG.warn("Open region: {} failed. Try close region but got exception ", this.getRegionInfo(),
7732 throw t;
7734 return this;
7738 * Open a Region on a read-only file-system (like hdfs snapshots)
7739 * @param conf The Configuration object to use.
7740 * @param fs Filesystem to use
7741 * @param info Info for region to be opened.
7742 * @param htd the table descriptor
7743 * @return new HRegion
7744 * @throws NullPointerException if {@code info} is {@code null}
7746 public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs,
7747 final Path tableDir, RegionInfo info, final TableDescriptor htd) throws IOException {
7748 Objects.requireNonNull(info, "RegionInfo cannot be null");
7749 if (LOG.isDebugEnabled()) {
7750 LOG.debug("Opening region (readOnly filesystem): " + info);
7752 if (info.getReplicaId() <= 0) {
7753 info = RegionReplicaUtil.getRegionInfoForReplica(info, 1);
7755 HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null);
7756 r.writestate.setReadOnly(true);
7757 return r.openHRegion(null);
7760 public static void warmupHRegion(final RegionInfo info,
7761 final TableDescriptor htd, final WAL wal, final Configuration conf,
7762 final RegionServerServices rsServices,
7763 final CancelableProgressable reporter)
7764 throws IOException {
7766 Objects.requireNonNull(info, "RegionInfo cannot be null");
7767 LOG.debug("Warmup {}", info);
7768 Path rootDir = CommonFSUtils.getRootDir(conf);
7769 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7770 FileSystem fs = null;
7771 if (rsServices != null) {
7772 fs = rsServices.getFileSystem();
7774 if (fs == null) {
7775 fs = rootDir.getFileSystem(conf);
7777 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
7778 r.initializeWarmup(reporter);
7782 * Computes the Path of the HRegion
7784 * @param tabledir qualified path for table
7785 * @param name ENCODED region name
7786 * @return Path of HRegion directory
7787 * @deprecated For tests only; to be removed.
7789 @Deprecated
7790 public static Path getRegionDir(final Path tabledir, final String name) {
7791 return new Path(tabledir, name);
7795 * Determines if the specified row is within the row range specified by the
7796 * specified RegionInfo
7798 * @param info RegionInfo that specifies the row range
7799 * @param row row to be checked
7800 * @return true if the row is within the range specified by the RegionInfo
7802 public static boolean rowIsInRange(RegionInfo info, final byte [] row) {
7803 return ((info.getStartKey().length == 0) ||
7804 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
7805 ((info.getEndKey().length == 0) ||
7806 (Bytes.compareTo(info.getEndKey(), row) > 0));
7809 public static boolean rowIsInRange(RegionInfo info, final byte [] row, final int offset,
7810 final short length) {
7811 return ((info.getStartKey().length == 0) ||
7812 (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
7813 row, offset, length) <= 0)) &&
7814 ((info.getEndKey().length == 0) ||
7815 (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
7818 @Override
7819 public Result get(final Get get) throws IOException {
7820 prepareGet(get);
7821 List<Cell> results = get(get, true);
7822 boolean stale = this.getRegionInfo().getReplicaId() != 0;
7823 return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
7826 void prepareGet(final Get get) throws IOException {
7827 checkRow(get.getRow(), "Get");
7828 // Verify families are all valid
7829 if (get.hasFamilies()) {
7830 for (byte[] family : get.familySet()) {
7831 checkFamily(family);
7833 } else { // Adding all families to scanner
7834 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
7835 get.addFamily(family);
7840 @Override
7841 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
7842 return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
7845 private List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7846 throws IOException {
7847 return TraceUtil.trace(() -> getInternal(get, withCoprocessor, nonceGroup, nonce),
7848 () -> createRegionSpan("Region.get"));
7851 private List<Cell> getInternal(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7852 throws IOException {
7853 List<Cell> results = new ArrayList<>();
7854 long before = EnvironmentEdgeManager.currentTime();
7856 // pre-get CP hook
7857 if (withCoprocessor && (coprocessorHost != null)) {
7858 if (coprocessorHost.preGet(get, results)) {
7859 metricsUpdateForGet(results, before);
7860 return results;
7863 Scan scan = new Scan(get);
7864 if (scan.getLoadColumnFamiliesOnDemandValue() == null) {
7865 scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault());
7867 try (RegionScanner scanner = getScanner(scan, null, nonceGroup, nonce)) {
7868 List<Cell> tmp = new ArrayList<>();
7869 scanner.next(tmp);
7870 // Copy EC to heap, then close the scanner.
7871 // This can be an EXPENSIVE call. It may make an extra copy from offheap to onheap buffers.
7872 // See more details in HBASE-26036.
7873 for (Cell cell : tmp) {
7874 results.add(
7875 CellUtil.cloneIfNecessary(cell));
7879 // post-get CP hook
7880 if (withCoprocessor && (coprocessorHost != null)) {
7881 coprocessorHost.postGet(get, results);
7884 metricsUpdateForGet(results, before);
7886 return results;
7889 void metricsUpdateForGet(List<Cell> results, long before) {
7890 if (this.metricsRegion != null) {
7891 this.metricsRegion.updateGet(EnvironmentEdgeManager.currentTime() - before);
7893 if (this.rsServices != null && this.rsServices.getMetrics() != null) {
7894 rsServices.getMetrics().updateReadQueryMeter(getRegionInfo().getTable(), 1);
7899 @Override
7900 public Result mutateRow(RowMutations rm) throws IOException {
7901 return mutateRow(rm, HConstants.NO_NONCE, HConstants.NO_NONCE);
7904 public Result mutateRow(RowMutations rm, long nonceGroup, long nonce) throws IOException {
7905 final List<Mutation> m = rm.getMutations();
7906 OperationStatus[] statuses = batchMutate(m.toArray(new Mutation[0]), true, nonceGroup, nonce);
7908 List<Result> results = new ArrayList<>();
7909 for (OperationStatus status : statuses) {
7910 if (status.getResult() != null) {
7911 results.add(status.getResult());
7915 if (results.isEmpty()) {
7916 return null;
7919 // Merge the results of the Increment/Append operations
7920 List<Cell> cells = new ArrayList<>();
7921 for (Result result : results) {
7922 if (result.rawCells() != null) {
7923 cells.addAll(Arrays.asList(result.rawCells()));
7926 return Result.create(cells);
7930 * Perform atomic (all or none) mutations within the region.
7931 * @param mutations The list of mutations to perform.
7932 * <code>mutations</code> can contain operations for multiple rows.
7933 * Caller has to ensure that all rows are contained in this region.
7934 * @param rowsToLock Rows to lock
7935 * @param nonceGroup Optional nonce group of the operation (client Id)
7936 * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
7937 * If multiple rows are locked care should be taken that
7938 * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
7939 * @throws IOException
7941 @Override
7942 public void mutateRowsWithLocks(Collection<Mutation> mutations,
7943 Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
7944 batchMutate(new MutationBatchOperation(this, mutations.toArray(new Mutation[mutations.size()]),
7945 true, nonceGroup, nonce) {
7946 @Override
7947 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch(
7948 List<RowLock> acquiredRowLocks) throws IOException {
7949 RowLock prevRowLock = null;
7950 for (byte[] row : rowsToLock) {
7951 try {
7952 RowLock rowLock = region.getRowLock(row, false, prevRowLock); // write lock
7953 if (rowLock != prevRowLock) {
7954 acquiredRowLocks.add(rowLock);
7955 prevRowLock = rowLock;
7957 } catch (IOException ioe) {
7958 LOG.warn("Failed getting lock, row={}, in region {}", Bytes.toStringBinary(row), this,
7959 ioe);
7960 throw ioe;
7963 return createMiniBatch(size(), size());
7969 * @return statistics about the current load of the region
7971 public ClientProtos.RegionLoadStats getLoadStatistics() {
7972 if (!regionStatsEnabled) {
7973 return null;
7975 ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
7976 stats.setMemStoreLoad((int) (Math.min(100,
7977 (this.memStoreSizing.getMemStoreSize().getHeapSize() * 100) / this.memstoreFlushSize)));
7978 if (rsServices.getHeapMemoryManager() != null) {
7979 // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM,
7980 // so we could just do the calculation below and we'll get a 0.
7981 // treating it as a special case analogous to no HMM instead so that it can be
7982 // programatically treated different from using <1% of heap.
7983 final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent();
7984 if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) {
7985 stats.setHeapOccupancy((int)(occupancy * 100));
7988 stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 ? 100
7989 : rsServices.getCompactionPressure() * 100));
7990 return stats.build();
7993 @Override
7994 public Result append(Append append) throws IOException {
7995 return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
7998 public Result append(Append append, long nonceGroup, long nonce) throws IOException {
7999 return TraceUtil.trace(() -> {
8000 checkReadOnly();
8001 checkResources();
8002 startRegionOperation(Operation.APPEND);
8003 try {
8004 // All edits for the given row (across all column families) must happen atomically.
8005 return mutate(append, true, nonceGroup, nonce).getResult();
8006 } finally {
8007 closeRegionOperation(Operation.APPEND);
8009 }, () -> createRegionSpan("Region.append"));
8012 @Override
8013 public Result increment(Increment increment) throws IOException {
8014 return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
8017 public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException {
8018 return TraceUtil.trace(() -> {
8019 checkReadOnly();
8020 checkResources();
8021 startRegionOperation(Operation.INCREMENT);
8022 try {
8023 // All edits for the given row (across all column families) must happen atomically.
8024 return mutate(increment, true, nonceGroup, nonce).getResult();
8025 } finally {
8026 closeRegionOperation(Operation.INCREMENT);
8028 }, () -> createRegionSpan("Region.increment"));
8032 * @return writeEntry associated with this append
8034 private WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds,
8035 long now, long nonceGroup, long nonce, long origLogSeqNum) throws IOException {
8036 Preconditions.checkArgument(walEdit != null && !walEdit.isEmpty(),
8037 "WALEdit is null or empty!");
8038 Preconditions.checkArgument(!walEdit.isReplay() || origLogSeqNum != SequenceId.NO_SEQUENCE_ID,
8039 "Invalid replay sequence Id for replay WALEdit!");
8040 // Using default cluster id, as this can only happen in the originating cluster.
8041 // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey
8042 // here instead of WALKeyImpl directly to support legacy coprocessors.
8043 WALKeyImpl walKey = walEdit.isReplay()?
8044 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
8045 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
8046 nonceGroup, nonce, mvcc) :
8047 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
8048 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
8049 nonceGroup, nonce, mvcc, this.getReplicationScope());
8050 if (walEdit.isReplay()) {
8051 walKey.setOrigLogSeqNum(origLogSeqNum);
8053 //don't call the coproc hook for writes to the WAL caused by
8054 //system lifecycle events like flushes or compactions
8055 if (this.coprocessorHost != null && !walEdit.isMetaEdit()) {
8056 this.coprocessorHost.preWALAppend(walKey, walEdit);
8058 ServerCall<?> rpcCall = RpcServer.getCurrentServerCallWithCellScanner().orElse(null);
8059 try {
8060 long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit);
8061 WriteEntry writeEntry = walKey.getWriteEntry();
8062 regionReplicationSink.ifPresent(sink -> writeEntry.attachCompletionAction(() -> {
8063 sink.add(walKey, walEdit, rpcCall);
8064 }));
8065 // Call sync on our edit.
8066 if (txid != 0) {
8067 sync(txid, durability);
8069 return writeEntry;
8070 } catch (IOException ioe) {
8071 if (walKey.getWriteEntry() != null) {
8072 mvcc.complete(walKey.getWriteEntry());
8074 throw ioe;
8079 public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HRegion.class, false);
8081 // woefully out of date - currently missing:
8082 // 1 x HashMap - coprocessorServiceHandlers
8083 // 6 x LongAdder - numMutationsWithoutWAL, dataInMemoryWithoutWAL,
8084 // checkAndMutateChecksPassed, checkAndMutateChecksFailed, readRequestsCount,
8085 // writeRequestsCount, cpRequestsCount
8086 // 1 x HRegion$WriteState - writestate
8087 // 1 x RegionCoprocessorHost - coprocessorHost
8088 // 1 x RegionSplitPolicy - splitPolicy
8089 // 1 x MetricsRegion - metricsRegion
8090 // 1 x MetricsRegionWrapperImpl - metricsRegionWrapper
8091 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
8092 ClassSize.OBJECT + // closeLock
8093 (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
8094 (3 * ClassSize.ATOMIC_LONG) + // numPutsWithoutWAL, dataInMemoryWithoutWAL,
8095 // compactionsFailed
8096 (3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, scannerReadPoints, regionLockHolders
8097 WriteState.HEAP_SIZE + // writestate
8098 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
8099 (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
8100 MultiVersionConcurrencyControl.FIXED_SIZE // mvcc
8101 + 2 * ClassSize.TREEMAP // maxSeqIdInStores, replicationScopes
8102 + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress
8103 + ClassSize.STORE_SERVICES // store services
8104 + StoreHotnessProtector.FIXED_SIZE
8107 @Override
8108 public long heapSize() {
8109 // this does not take into account row locks, recent flushes, mvcc entries, and more
8110 return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum();
8114 * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to be
8115 * available for handling {@link #execService(RpcController, CoprocessorServiceCall)} calls.
8116 * <p/>
8117 * Only a single instance may be registered per region for a given {@link Service} subclass (the
8118 * instances are keyed on {@link ServiceDescriptor#getFullName()}.. After the first registration,
8119 * subsequent calls with the same service name will fail with a return value of {@code false}.
8120 * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint
8121 * @return {@code true} if the registration was successful, {@code false} otherwise
8123 public boolean registerService(Service instance) {
8124 // No stacking of instances is allowed for a single service name
8125 ServiceDescriptor serviceDesc = instance.getDescriptorForType();
8126 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
8127 if (coprocessorServiceHandlers.containsKey(serviceName)) {
8128 LOG.error("Coprocessor service {} already registered, rejecting request from {} in region {}",
8129 serviceName, instance, this);
8130 return false;
8133 coprocessorServiceHandlers.put(serviceName, instance);
8134 if (LOG.isDebugEnabled()) {
8135 LOG.debug("Registered coprocessor service: region=" +
8136 Bytes.toStringBinary(getRegionInfo().getRegionName()) + " service=" + serviceName);
8138 return true;
8142 * Executes a single protocol buffer coprocessor endpoint {@link Service} method using
8143 * the registered protocol handlers. {@link Service} implementations must be registered via the
8144 * {@link #registerService(Service)}
8145 * method before they are available.
8147 * @param controller an {@code RpcContoller} implementation to pass to the invoked service
8148 * @param call a {@code CoprocessorServiceCall} instance identifying the service, method,
8149 * and parameters for the method invocation
8150 * @return a protocol buffer {@code Message} instance containing the method's result
8151 * @throws IOException if no registered service handler is found or an error
8152 * occurs during the invocation
8153 * @see #registerService(Service)
8155 public Message execService(RpcController controller, CoprocessorServiceCall call)
8156 throws IOException {
8157 String serviceName = call.getServiceName();
8158 Service service = coprocessorServiceHandlers.get(serviceName);
8159 if (service == null) {
8160 throw new UnknownProtocolException(null, "No registered coprocessor service found for " +
8161 serviceName + " in region " + Bytes.toStringBinary(getRegionInfo().getRegionName()));
8163 ServiceDescriptor serviceDesc = service.getDescriptorForType();
8165 cpRequestsCount.increment();
8166 String methodName = call.getMethodName();
8167 MethodDescriptor methodDesc =
8168 CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
8170 Message.Builder builder =
8171 service.getRequestPrototype(methodDesc).newBuilderForType();
8173 ProtobufUtil.mergeFrom(builder,
8174 call.getRequest().toByteArray());
8175 Message request =
8176 CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
8178 if (coprocessorHost != null) {
8179 request = coprocessorHost.preEndpointInvocation(service, methodName, request);
8182 final Message.Builder responseBuilder =
8183 service.getResponsePrototype(methodDesc).newBuilderForType();
8184 service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
8185 @Override
8186 public void run(Message message) {
8187 if (message != null) {
8188 responseBuilder.mergeFrom(message);
8193 if (coprocessorHost != null) {
8194 coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder);
8196 IOException exception =
8197 org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.getControllerException(controller);
8198 if (exception != null) {
8199 throw exception;
8202 return responseBuilder.build();
8205 public Optional<byte[]> checkSplit() {
8206 return checkSplit(false);
8210 * Return the split point. An empty result indicates the region isn't splittable.
8212 public Optional<byte[]> checkSplit(boolean force) {
8213 // Can't split META
8214 if (this.getRegionInfo().isMetaRegion()) {
8215 return Optional.empty();
8218 // Can't split a region that is closing.
8219 if (this.isClosing()) {
8220 return Optional.empty();
8223 if (!force && !splitPolicy.shouldSplit()) {
8224 return Optional.empty();
8227 byte[] ret = splitPolicy.getSplitPoint();
8228 if (ret != null && ret.length > 0) {
8229 ret = splitRestriction.getRestrictedSplitPoint(ret);
8232 if (ret != null) {
8233 try {
8234 checkRow(ret, "calculated split");
8235 } catch (IOException e) {
8236 LOG.error("Ignoring invalid split for region {}", this, e);
8237 return Optional.empty();
8239 return Optional.of(ret);
8240 } else {
8241 return Optional.empty();
8246 * @return The priority that this region should have in the compaction queue
8248 public int getCompactPriority() {
8249 if (checkSplit().isPresent() && conf.getBoolean(SPLIT_IGNORE_BLOCKING_ENABLED_KEY, false)) {
8250 // if a region should split, split it before compact
8251 return Store.PRIORITY_USER;
8253 return stores.values().stream().mapToInt(HStore::getCompactPriority).min()
8254 .orElse(Store.NO_PRIORITY);
8257 /** @return the coprocessor host */
8258 public RegionCoprocessorHost getCoprocessorHost() {
8259 return coprocessorHost;
8262 /** @param coprocessorHost the new coprocessor host */
8263 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
8264 this.coprocessorHost = coprocessorHost;
8267 @Override
8268 public void startRegionOperation() throws IOException {
8269 startRegionOperation(Operation.ANY);
8272 @Override
8273 public void startRegionOperation(Operation op) throws IOException {
8274 boolean isInterruptableOp = false;
8275 switch (op) {
8276 case GET: // interruptible read operations
8277 case SCAN:
8278 isInterruptableOp = true;
8279 checkReadsEnabled();
8280 break;
8281 case INCREMENT: // interruptible write operations
8282 case APPEND:
8283 case PUT:
8284 case DELETE:
8285 case BATCH_MUTATE:
8286 case CHECK_AND_MUTATE:
8287 isInterruptableOp = true;
8288 break;
8289 default: // all others
8290 break;
8292 if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION
8293 || op == Operation.COMPACT_REGION || op == Operation.COMPACT_SWITCH) {
8294 // split, merge or compact region doesn't need to check the closing/closed state or lock the
8295 // region
8296 return;
8298 if (this.closing.get()) {
8299 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8301 lock(lock.readLock());
8302 // Update regionLockHolders ONLY for any startRegionOperation call that is invoked from
8303 // an RPC handler
8304 Thread thisThread = Thread.currentThread();
8305 if (isInterruptableOp) {
8306 regionLockHolders.put(thisThread, true);
8308 if (this.closed.get()) {
8309 lock.readLock().unlock();
8310 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8312 // The unit for snapshot is a region. So, all stores for this region must be
8313 // prepared for snapshot operation before proceeding.
8314 if (op == Operation.SNAPSHOT) {
8315 stores.values().forEach(HStore::preSnapshotOperation);
8317 try {
8318 if (coprocessorHost != null) {
8319 coprocessorHost.postStartRegionOperation(op);
8321 } catch (Exception e) {
8322 if (isInterruptableOp) {
8323 // would be harmless to remove what we didn't add but we know by 'isInterruptableOp'
8324 // if we added this thread to regionLockHolders
8325 regionLockHolders.remove(thisThread);
8327 lock.readLock().unlock();
8328 throw new IOException(e);
8332 @Override
8333 public void closeRegionOperation() throws IOException {
8334 closeRegionOperation(Operation.ANY);
8337 @Override
8338 public void closeRegionOperation(Operation operation) throws IOException {
8339 if (operation == Operation.SNAPSHOT) {
8340 stores.values().forEach(HStore::postSnapshotOperation);
8342 Thread thisThread = Thread.currentThread();
8343 regionLockHolders.remove(thisThread);
8344 lock.readLock().unlock();
8345 if (coprocessorHost != null) {
8346 coprocessorHost.postCloseRegionOperation(operation);
8351 * This method needs to be called before any public call that reads or
8352 * modifies stores in bulk. It has to be called just before a try.
8353 * #closeBulkRegionOperation needs to be called in the try's finally block
8354 * Acquires a writelock and checks if the region is closing or closed.
8355 * @throws NotServingRegionException when the region is closing or closed
8356 * @throws RegionTooBusyException if failed to get the lock in time
8357 * @throws InterruptedIOException if interrupted while waiting for a lock
8359 private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException {
8360 if (this.closing.get()) {
8361 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8363 if (writeLockNeeded) lock(lock.writeLock());
8364 else lock(lock.readLock());
8365 if (this.closed.get()) {
8366 if (writeLockNeeded) lock.writeLock().unlock();
8367 else lock.readLock().unlock();
8368 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8370 regionLockHolders.put(Thread.currentThread(), true);
8374 * Closes the lock. This needs to be called in the finally block corresponding
8375 * to the try block of #startRegionOperation
8377 private void closeBulkRegionOperation(){
8378 regionLockHolders.remove(Thread.currentThread());
8379 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
8380 else lock.readLock().unlock();
8384 * Update LongAdders for number of puts without wal and the size of possible data loss.
8385 * These information are exposed by the region server metrics.
8387 private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) {
8388 numMutationsWithoutWAL.increment();
8389 if (numMutationsWithoutWAL.sum() <= 1) {
8390 LOG.info("writing data to region " + this +
8391 " with WAL disabled. Data may be lost in the event of a crash.");
8394 long mutationSize = 0;
8395 for (List<Cell> cells: familyMap.values()) {
8396 // Optimization: 'foreach' loop is not used. See:
8397 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
8398 assert cells instanceof RandomAccess;
8399 int listSize = cells.size();
8400 for (int i=0; i < listSize; i++) {
8401 Cell cell = cells.get(i);
8402 mutationSize += cell.getSerializedSize();
8406 dataInMemoryWithoutWAL.add(mutationSize);
8409 private void lock(final Lock lock) throws IOException {
8410 lock(lock, 1);
8414 * Try to acquire a lock. Throw RegionTooBusyException
8415 * if failed to get the lock in time. Throw InterruptedIOException
8416 * if interrupted while waiting for the lock.
8418 private void lock(final Lock lock, final int multiplier) throws IOException {
8419 try {
8420 final long waitTime = Math.min(maxBusyWaitDuration,
8421 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
8422 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
8423 // Don't print millis. Message is used as a key over in
8424 // RetriesExhaustedWithDetailsException processing.
8425 final String regionName =
8426 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
8427 final String serverName = this.getRegionServerServices() == null ?
8428 "unknown" : (this.getRegionServerServices().getServerName() == null ?
8429 "unknown" : this.getRegionServerServices().getServerName().toString());
8430 RegionTooBusyException rtbe = new RegionTooBusyException(
8431 "Failed to obtain lock; regionName=" + regionName + ", server=" + serverName);
8432 LOG.warn("Region is too busy to allow lock acquisition.", rtbe);
8433 throw rtbe;
8435 } catch (InterruptedException ie) {
8436 if (LOG.isDebugEnabled()) {
8437 LOG.debug("Interrupted while waiting for a lock in region {}", this);
8439 throw throwOnInterrupt(ie);
8444 * Calls sync with the given transaction ID
8445 * @param txid should sync up to which transaction
8446 * @throws IOException If anything goes wrong with DFS
8448 private void sync(long txid, Durability durability) throws IOException {
8449 if (this.getRegionInfo().isMetaRegion()) {
8450 this.wal.sync(txid);
8451 } else {
8452 switch(durability) {
8453 case USE_DEFAULT:
8454 // do what table defaults to
8455 if (shouldSyncWAL()) {
8456 this.wal.sync(txid);
8458 break;
8459 case SKIP_WAL:
8460 // nothing do to
8461 break;
8462 case ASYNC_WAL:
8463 // nothing do to
8464 break;
8465 case SYNC_WAL:
8466 this.wal.sync(txid, false);
8467 break;
8468 case FSYNC_WAL:
8469 this.wal.sync(txid, true);
8470 break;
8471 default:
8472 throw new RuntimeException("Unknown durability " + durability);
8478 * Check whether we should sync the wal from the table's durability settings
8480 private boolean shouldSyncWAL() {
8481 return regionDurability.ordinal() > Durability.ASYNC_WAL.ordinal();
8484 /** @return the latest sequence number that was read from storage when this region was opened */
8485 public long getOpenSeqNum() {
8486 return this.openSeqNum;
8489 @Override
8490 public Map<byte[], Long> getMaxStoreSeqId() {
8491 return this.maxSeqIdInStores;
8494 public long getOldestSeqIdOfStore(byte[] familyName) {
8495 return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
8498 @Override
8499 public CompactionState getCompactionState() {
8500 boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0;
8501 return (hasMajor ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR)
8502 : (hasMinor ? CompactionState.MINOR : CompactionState.NONE));
8505 public void reportCompactionRequestStart(boolean isMajor){
8506 (isMajor ? majorInProgress : minorInProgress).incrementAndGet();
8509 public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) {
8510 int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet();
8512 // metrics
8513 compactionsFinished.increment();
8514 compactionNumFilesCompacted.add(numFiles);
8515 compactionNumBytesCompacted.add(filesSizeCompacted);
8517 assert newValue >= 0;
8520 public void reportCompactionRequestFailure() {
8521 compactionsFailed.increment();
8524 public void incrementCompactionsQueuedCount() {
8525 compactionsQueued.increment();
8528 public void decrementCompactionsQueuedCount() {
8529 compactionsQueued.decrement();
8532 public void incrementFlushesQueuedCount() {
8533 flushesQueued.increment();
8536 protected void decrementFlushesQueuedCount() {
8537 flushesQueued.decrement();
8541 * If a handler thread is eligible for interrupt, make it ineligible. Should be paired
8542 * with {{@link #enableInterrupts()}.
8544 void disableInterrupts() {
8545 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> false);
8549 * If a handler thread was made ineligible for interrupt via {{@link #disableInterrupts()},
8550 * make it eligible again. No-op if interrupts are already enabled.
8552 void enableInterrupts() {
8553 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> true);
8557 * Interrupt any region options that have acquired the region lock via
8558 * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)},
8559 * or {@link #startBulkRegionOperation(boolean)}.
8561 private void interruptRegionOperations() {
8562 for (Map.Entry<Thread, Boolean> entry: regionLockHolders.entrySet()) {
8563 // An entry in this map will have a boolean value indicating if it is currently
8564 // eligible for interrupt; if so, we should interrupt it.
8565 if (entry.getValue().booleanValue()) {
8566 entry.getKey().interrupt();
8572 * Check thread interrupt status and throw an exception if interrupted.
8573 * @throws NotServingRegionException if region is closing
8574 * @throws InterruptedIOException if interrupted but region is not closing
8576 // Package scope for tests
8577 void checkInterrupt() throws NotServingRegionException, InterruptedIOException {
8578 if (Thread.interrupted()) {
8579 if (this.closing.get()) {
8580 throw new NotServingRegionException(
8581 getRegionInfo().getRegionNameAsString() + " is closing");
8583 throw new InterruptedIOException();
8588 * Throw the correct exception upon interrupt
8589 * @param t cause
8591 // Package scope for tests
8592 IOException throwOnInterrupt(Throwable t) {
8593 if (this.closing.get()) {
8594 return (NotServingRegionException) new NotServingRegionException(
8595 getRegionInfo().getRegionNameAsString() + " is closing")
8596 .initCause(t);
8598 return (InterruptedIOException) new InterruptedIOException().initCause(t);
8602 * {@inheritDoc}
8604 @Override
8605 public void onConfigurationChange(Configuration conf) {
8606 this.storeHotnessProtector.update(conf);
8607 // update coprocessorHost if the configuration has changed.
8608 if (CoprocessorConfigurationUtil.checkConfigurationChange(getReadOnlyConfiguration(), conf,
8609 CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
8610 CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY)) {
8611 LOG.info("Update the system coprocessors because the configuration has changed");
8612 decorateRegionConfiguration(conf);
8613 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
8618 * {@inheritDoc}
8620 @Override
8621 public void registerChildren(ConfigurationManager manager) {
8622 configurationManager = manager;
8623 stores.values().forEach(manager::registerObserver);
8627 * {@inheritDoc}
8629 @Override
8630 public void deregisterChildren(ConfigurationManager manager) {
8631 stores.values().forEach(configurationManager::deregisterObserver);
8634 @Override
8635 public CellComparator getCellComparator() {
8636 return cellComparator;
8639 public long getMemStoreFlushSize() {
8640 return this.memstoreFlushSize;
8644 //// method for debugging tests
8645 void throwException(String title, String regionName) {
8646 StringBuilder buf = new StringBuilder();
8647 buf.append(title + ", ");
8648 buf.append(getRegionInfo().toString());
8649 buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " ");
8650 buf.append("stores: ");
8651 for (HStore s : stores.values()) {
8652 buf.append(s.getColumnFamilyDescriptor().getNameAsString());
8653 buf.append(" size: ");
8654 buf.append(s.getMemStoreSize().getDataSize());
8655 buf.append(" ");
8657 buf.append("end-of-stores");
8658 buf.append(", memstore size ");
8659 buf.append(getMemStoreDataSize());
8660 if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) {
8661 throw new RuntimeException(buf.toString());
8665 @Override
8666 public void requestCompaction(String why, int priority, boolean major,
8667 CompactionLifeCycleTracker tracker) throws IOException {
8668 if (major) {
8669 stores.values().forEach(HStore::triggerMajorCompaction);
8671 rsServices.getCompactionRequestor().requestCompaction(this, why, priority, tracker,
8672 RpcServer.getRequestUser().orElse(null));
8675 @Override
8676 public void requestCompaction(byte[] family, String why, int priority, boolean major,
8677 CompactionLifeCycleTracker tracker) throws IOException {
8678 HStore store = stores.get(family);
8679 if (store == null) {
8680 throw new NoSuchColumnFamilyException("column family " + Bytes.toString(family) +
8681 " does not exist in region " + getRegionInfo().getRegionNameAsString());
8683 if (major) {
8684 store.triggerMajorCompaction();
8686 rsServices.getCompactionRequestor().requestCompaction(this, store, why, priority, tracker,
8687 RpcServer.getRequestUser().orElse(null));
8690 private void requestFlushIfNeeded() throws RegionTooBusyException {
8691 if(isFlushSize(this.memStoreSizing.getMemStoreSize())) {
8692 requestFlush();
8696 private void requestFlush() {
8697 if (this.rsServices == null) {
8698 return;
8700 requestFlush0(FlushLifeCycleTracker.DUMMY);
8703 private void requestFlush0(FlushLifeCycleTracker tracker) {
8704 boolean shouldFlush = false;
8705 synchronized (writestate) {
8706 if (!this.writestate.isFlushRequested()) {
8707 shouldFlush = true;
8708 writestate.flushRequested = true;
8711 if (shouldFlush) {
8712 // Make request outside of synchronize block; HBASE-818.
8713 this.rsServices.getFlushRequester().requestFlush(this, tracker);
8714 if (LOG.isDebugEnabled()) {
8715 LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
8717 } else {
8718 tracker.notExecuted("Flush already requested on " + this);
8722 @Override
8723 public void requestFlush(FlushLifeCycleTracker tracker) throws IOException {
8724 requestFlush0(tracker);
8728 * This method modifies the region's configuration in order to inject replication-related
8729 * features
8730 * @param conf region configurations
8732 private static void decorateRegionConfiguration(Configuration conf) {
8733 if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
8734 String plugins = conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,"");
8735 String replicationCoprocessorClass = ReplicationObserver.class.getCanonicalName();
8736 if (!plugins.contains(replicationCoprocessorClass)) {
8737 conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
8738 (plugins.equals("") ? "" : (plugins + ",")) + replicationCoprocessorClass);
8743 public Optional<RegionReplicationSink> getRegionReplicationSink() {
8744 return regionReplicationSink;
8747 public void addReadRequestsCount(long readRequestsCount) {
8748 this.readRequestsCount.add(readRequestsCount);
8751 public void addWriteRequestsCount(long writeRequestsCount) {
8752 this.writeRequestsCount.add(writeRequestsCount);
8755 @RestrictedApi(explanation = "Should only be called in tests", link = "",
8756 allowedOnPath = ".*/src/test/.*")
8757 boolean isReadsEnabled() {
8758 return this.writestate.readsEnabled;