HBASE-26413 Limit the total size of buffered region replication entries (#3844)
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / HRegion.java
blob2b337ecc3957ac0fb26ba3c1446584d45c439a57
1 /*
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.regionserver;
20 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
21 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
22 import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.REGION_NAMES_KEY;
23 import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.ROW_LOCK_READ_LOCK_KEY;
24 import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
26 import edu.umd.cs.findbugs.annotations.Nullable;
27 import io.opentelemetry.api.trace.Span;
28 import java.io.EOFException;
29 import java.io.FileNotFoundException;
30 import java.io.IOException;
31 import java.io.InterruptedIOException;
32 import java.lang.reflect.Constructor;
33 import java.nio.ByteBuffer;
34 import java.nio.charset.StandardCharsets;
35 import java.text.ParseException;
36 import java.util.ArrayList;
37 import java.util.Arrays;
38 import java.util.Collection;
39 import java.util.Collections;
40 import java.util.HashMap;
41 import java.util.HashSet;
42 import java.util.Iterator;
43 import java.util.List;
44 import java.util.Map;
45 import java.util.Map.Entry;
46 import java.util.NavigableMap;
47 import java.util.NavigableSet;
48 import java.util.Objects;
49 import java.util.Optional;
50 import java.util.RandomAccess;
51 import java.util.Set;
52 import java.util.TreeMap;
53 import java.util.UUID;
54 import java.util.concurrent.Callable;
55 import java.util.concurrent.CompletionService;
56 import java.util.concurrent.ConcurrentHashMap;
57 import java.util.concurrent.ConcurrentMap;
58 import java.util.concurrent.ConcurrentSkipListMap;
59 import java.util.concurrent.ExecutionException;
60 import java.util.concurrent.ExecutorCompletionService;
61 import java.util.concurrent.Future;
62 import java.util.concurrent.ThreadFactory;
63 import java.util.concurrent.ThreadPoolExecutor;
64 import java.util.concurrent.TimeUnit;
65 import java.util.concurrent.atomic.AtomicBoolean;
66 import java.util.concurrent.atomic.AtomicInteger;
67 import java.util.concurrent.atomic.LongAdder;
68 import java.util.concurrent.locks.Lock;
69 import java.util.concurrent.locks.ReadWriteLock;
70 import java.util.concurrent.locks.ReentrantReadWriteLock;
71 import java.util.function.Function;
72 import java.util.stream.Collectors;
73 import java.util.stream.Stream;
75 import org.apache.hadoop.conf.Configuration;
76 import org.apache.hadoop.fs.FileStatus;
77 import org.apache.hadoop.fs.FileSystem;
78 import org.apache.hadoop.fs.LocatedFileStatus;
79 import org.apache.hadoop.fs.Path;
80 import org.apache.hadoop.hbase.ByteBufferExtendedCell;
81 import org.apache.hadoop.hbase.Cell;
82 import org.apache.hadoop.hbase.CellBuilderType;
83 import org.apache.hadoop.hbase.CellComparator;
84 import org.apache.hadoop.hbase.CellComparatorImpl;
85 import org.apache.hadoop.hbase.CellScanner;
86 import org.apache.hadoop.hbase.CellUtil;
87 import org.apache.hadoop.hbase.CompareOperator;
88 import org.apache.hadoop.hbase.CompoundConfiguration;
89 import org.apache.hadoop.hbase.DoNotRetryIOException;
90 import org.apache.hadoop.hbase.DroppedSnapshotException;
91 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
92 import org.apache.hadoop.hbase.HConstants;
93 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
94 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
95 import org.apache.hadoop.hbase.KeyValue;
96 import org.apache.hadoop.hbase.MetaCellComparator;
97 import org.apache.hadoop.hbase.NamespaceDescriptor;
98 import org.apache.hadoop.hbase.NotServingRegionException;
99 import org.apache.hadoop.hbase.PrivateCellUtil;
100 import org.apache.hadoop.hbase.RegionTooBusyException;
101 import org.apache.hadoop.hbase.Tag;
102 import org.apache.hadoop.hbase.TagUtil;
103 import org.apache.hadoop.hbase.client.Append;
104 import org.apache.hadoop.hbase.client.CheckAndMutate;
105 import org.apache.hadoop.hbase.client.CheckAndMutateResult;
106 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
107 import org.apache.hadoop.hbase.client.CompactionState;
108 import org.apache.hadoop.hbase.client.Delete;
109 import org.apache.hadoop.hbase.client.Durability;
110 import org.apache.hadoop.hbase.client.Get;
111 import org.apache.hadoop.hbase.client.Increment;
112 import org.apache.hadoop.hbase.client.IsolationLevel;
113 import org.apache.hadoop.hbase.client.Mutation;
114 import org.apache.hadoop.hbase.client.Put;
115 import org.apache.hadoop.hbase.client.RegionInfo;
116 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
117 import org.apache.hadoop.hbase.client.Result;
118 import org.apache.hadoop.hbase.client.Row;
119 import org.apache.hadoop.hbase.client.RowMutations;
120 import org.apache.hadoop.hbase.client.Scan;
121 import org.apache.hadoop.hbase.client.TableDescriptor;
122 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
123 import org.apache.hadoop.hbase.conf.ConfigurationManager;
124 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
125 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
126 import org.apache.hadoop.hbase.coprocessor.ReadOnlyConfiguration;
127 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
128 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
129 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
130 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
131 import org.apache.hadoop.hbase.filter.BinaryComparator;
132 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
133 import org.apache.hadoop.hbase.filter.Filter;
134 import org.apache.hadoop.hbase.io.HFileLink;
135 import org.apache.hadoop.hbase.io.HeapSize;
136 import org.apache.hadoop.hbase.io.TimeRange;
137 import org.apache.hadoop.hbase.io.hfile.BlockCache;
138 import org.apache.hadoop.hbase.io.hfile.HFile;
139 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
140 import org.apache.hadoop.hbase.ipc.RpcCall;
141 import org.apache.hadoop.hbase.ipc.RpcServer;
142 import org.apache.hadoop.hbase.ipc.ServerCall;
143 import org.apache.hadoop.hbase.mob.MobFileCache;
144 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
145 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
146 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
147 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
148 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
149 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
150 import org.apache.hadoop.hbase.regionserver.compactions.ForbidMajorCompactionChecker;
151 import org.apache.hadoop.hbase.regionserver.regionreplication.RegionReplicationSink;
152 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
153 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
154 import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector;
155 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
156 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
157 import org.apache.hadoop.hbase.replication.ReplicationUtils;
158 import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver;
159 import org.apache.hadoop.hbase.security.User;
160 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
161 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
162 import org.apache.hadoop.hbase.trace.TraceUtil;
163 import org.apache.hadoop.hbase.util.Bytes;
164 import org.apache.hadoop.hbase.util.CancelableProgressable;
165 import org.apache.hadoop.hbase.util.ClassSize;
166 import org.apache.hadoop.hbase.util.CommonFSUtils;
167 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
168 import org.apache.hadoop.hbase.util.FSUtils;
169 import org.apache.hadoop.hbase.util.HashedBytes;
170 import org.apache.hadoop.hbase.util.NonceKey;
171 import org.apache.hadoop.hbase.util.Pair;
172 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
173 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
174 import org.apache.hadoop.hbase.util.Threads;
175 import org.apache.hadoop.hbase.wal.WAL;
176 import org.apache.hadoop.hbase.wal.WALEdit;
177 import org.apache.hadoop.hbase.wal.WALFactory;
178 import org.apache.hadoop.hbase.wal.WALKey;
179 import org.apache.hadoop.hbase.wal.WALKeyImpl;
180 import org.apache.hadoop.hbase.wal.WALSplitUtil;
181 import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay;
182 import org.apache.hadoop.util.StringUtils;
183 import org.apache.yetus.audience.InterfaceAudience;
184 import org.slf4j.Logger;
185 import org.slf4j.LoggerFactory;
187 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
188 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
189 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
190 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
191 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
192 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
193 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
194 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
195 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
196 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
197 import org.apache.hbase.thirdparty.com.google.protobuf.Service;
198 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
199 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
200 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
202 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
203 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
204 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
205 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
206 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
207 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
208 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
209 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
210 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
211 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
212 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
213 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
214 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
215 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
218 * Regions store data for a certain region of a table. It stores all columns
219 * for each row. A given table consists of one or more Regions.
221 * <p>An Region is defined by its table and its key extent.
223 * <p>Locking at the Region level serves only one purpose: preventing the
224 * region from being closed (and consequently split) while other operations
225 * are ongoing. Each row level operation obtains both a row lock and a region
226 * read lock for the duration of the operation. While a scanner is being
227 * constructed, getScanner holds a read lock. If the scanner is successfully
228 * constructed, it holds a read lock until it is closed. A close takes out a
229 * write lock and consequently will block for ongoing operations and will block
230 * new operations from starting while the close is in progress.
232 @SuppressWarnings("deprecation")
233 @InterfaceAudience.Private
234 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
235 private static final Logger LOG = LoggerFactory.getLogger(HRegion.class);
237 public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
238 "hbase.hregion.scan.loadColumnFamiliesOnDemand";
240 public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
241 public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
243 public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE =
244 "hbase.regionserver.minibatch.size";
245 public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000;
247 public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync";
248 public static final boolean DEFAULT_WAL_HSYNC = false;
250 /** Parameter name for compaction after bulkload */
251 public static final String COMPACTION_AFTER_BULKLOAD_ENABLE =
252 "hbase.compaction.after.bulkload.enable";
255 * This is for for using HRegion as a local storage, where we may put the recovered edits in a
256 * special place. Once this is set, we will only replay the recovered edits under this directory
257 * and ignore the original replay directory configs.
259 public static final String SPECIAL_RECOVERED_EDITS_DIR =
260 "hbase.hregion.special.recovered.edits.dir";
263 * Whether to use {@link MetaCellComparator} even if we are not meta region. Used when creating
264 * master local region.
266 public static final String USE_META_CELL_COMPARATOR = "hbase.region.use.meta.cell.comparator";
268 public static final boolean DEFAULT_USE_META_CELL_COMPARATOR = false;
270 final AtomicBoolean closed = new AtomicBoolean(false);
272 /* Closing can take some time; use the closing flag if there is stuff we don't
273 * want to do while in closing state; e.g. like offer this region up to the
274 * master as a region to close if the carrying regionserver is overloaded.
275 * Once set, it is never cleared.
277 final AtomicBoolean closing = new AtomicBoolean(false);
280 * The max sequence id of flushed data on this region. There is no edit in memory that is
281 * less that this sequence id.
283 private volatile long maxFlushedSeqId = HConstants.NO_SEQNUM;
286 * Record the sequence id of last flush operation. Can be in advance of
287 * {@link #maxFlushedSeqId} when flushing a single column family. In this case,
288 * {@link #maxFlushedSeqId} will be older than the oldest edit in memory.
290 private volatile long lastFlushOpSeqId = HConstants.NO_SEQNUM;
293 * The sequence id of the last replayed open region event from the primary region. This is used
294 * to skip entries before this due to the possibility of replay edits coming out of order from
295 * replication.
297 protected volatile long lastReplayedOpenRegionSeqId = -1L;
298 protected volatile long lastReplayedCompactionSeqId = -1L;
300 //////////////////////////////////////////////////////////////////////////////
301 // Members
302 //////////////////////////////////////////////////////////////////////////////
304 // map from a locked row to the context for that lock including:
305 // - CountDownLatch for threads waiting on that row
306 // - the thread that owns the lock (allow reentrancy)
307 // - reference count of (reentrant) locks held by the thread
308 // - the row itself
309 private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows =
310 new ConcurrentHashMap<>();
312 protected final Map<byte[], HStore> stores =
313 new ConcurrentSkipListMap<>(Bytes.BYTES_RAWCOMPARATOR);
315 // TODO: account for each registered handler in HeapSize computation
316 private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
318 // Track data size in all memstores
319 private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();
320 RegionServicesForStores regionServicesForStores;
322 // Debug possible data loss due to WAL off
323 final LongAdder numMutationsWithoutWAL = new LongAdder();
324 final LongAdder dataInMemoryWithoutWAL = new LongAdder();
326 // Debug why CAS operations are taking a while.
327 final LongAdder checkAndMutateChecksPassed = new LongAdder();
328 final LongAdder checkAndMutateChecksFailed = new LongAdder();
330 // Number of requests
331 // Count rows for scan
332 final LongAdder readRequestsCount = new LongAdder();
333 final LongAdder cpRequestsCount = new LongAdder();
334 final LongAdder filteredReadRequestsCount = new LongAdder();
335 // Count rows for multi row mutations
336 final LongAdder writeRequestsCount = new LongAdder();
338 // Number of requests blocked by memstore size.
339 private final LongAdder blockedRequestsCount = new LongAdder();
341 // Compaction LongAdders
342 final LongAdder compactionsFinished = new LongAdder();
343 final LongAdder compactionsFailed = new LongAdder();
344 final LongAdder compactionNumFilesCompacted = new LongAdder();
345 final LongAdder compactionNumBytesCompacted = new LongAdder();
346 final LongAdder compactionsQueued = new LongAdder();
347 final LongAdder flushesQueued = new LongAdder();
349 private BlockCache blockCache;
350 private MobFileCache mobFileCache;
351 private final WAL wal;
352 private final HRegionFileSystem fs;
353 protected final Configuration conf;
354 private final Configuration baseConf;
355 private final int rowLockWaitDuration;
356 static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
358 private Path regionDir;
359 private FileSystem walFS;
361 // set to true if the region is restored from snapshot
362 private boolean isRestoredRegion = false;
364 public void setRestoredRegion(boolean restoredRegion) {
365 isRestoredRegion = restoredRegion;
368 // The internal wait duration to acquire a lock before read/update
369 // from the region. It is not per row. The purpose of this wait time
370 // is to avoid waiting a long time while the region is busy, so that
371 // we can release the IPC handler soon enough to improve the
372 // availability of the region server. It can be adjusted by
373 // tuning configuration "hbase.busy.wait.duration".
374 final long busyWaitDuration;
375 static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
377 // If updating multiple rows in one call, wait longer,
378 // i.e. waiting for busyWaitDuration * # of rows. However,
379 // we can limit the max multiplier.
380 final int maxBusyWaitMultiplier;
382 // Max busy wait duration. There is no point to wait longer than the RPC
383 // purge timeout, when a RPC call will be terminated by the RPC engine.
384 final long maxBusyWaitDuration;
386 // Max cell size. If nonzero, the maximum allowed size for any given cell
387 // in bytes
388 final long maxCellSize;
390 // Number of mutations for minibatch processing.
391 private final int miniBatchSize;
393 final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
396 * The sequence ID that was enLongAddered when this region was opened.
398 private long openSeqNum = HConstants.NO_SEQNUM;
401 * The default setting for whether to enable on-demand CF loading for
402 * scan requests to this region. Requests can override it.
404 private boolean isLoadingCfsOnDemandDefault = false;
406 private final AtomicInteger majorInProgress = new AtomicInteger(0);
407 private final AtomicInteger minorInProgress = new AtomicInteger(0);
410 // Context: During replay we want to ensure that we do not lose any data. So, we
411 // have to be conservative in how we replay wals. For each store, we calculate
412 // the maxSeqId up to which the store was flushed. And, skip the edits which
413 // are equal to or lower than maxSeqId for each store.
414 // The following map is populated when opening the region
415 Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
417 /** Saved state from replaying prepare flush cache */
418 private PrepareFlushResult prepareFlushResult = null;
420 private volatile ConfigurationManager configurationManager;
422 // Used for testing.
423 private volatile Long timeoutForWriteLock = null;
425 private final CellComparator cellComparator;
428 * @return The smallest mvcc readPoint across all the scanners in this
429 * region. Writes older than this readPoint, are included in every
430 * read operation.
432 public long getSmallestReadPoint() {
433 long minimumReadPoint;
434 // We need to ensure that while we are calculating the smallestReadPoint
435 // no new RegionScanners can grab a readPoint that we are unaware of.
436 // We achieve this by synchronizing on the scannerReadPoints object.
437 synchronized (scannerReadPoints) {
438 minimumReadPoint = mvcc.getReadPoint();
439 for (Long readPoint : this.scannerReadPoints.values()) {
440 if (readPoint < minimumReadPoint) {
441 minimumReadPoint = readPoint;
445 return minimumReadPoint;
449 * Data structure of write state flags used coordinating flushes,
450 * compactions and closes.
452 static class WriteState {
453 // Set while a memstore flush is happening.
454 volatile boolean flushing = false;
455 // Set when a flush has been requested.
456 volatile boolean flushRequested = false;
457 // Number of compactions running.
458 AtomicInteger compacting = new AtomicInteger(0);
459 // Gets set in close. If set, cannot compact or flush again.
460 volatile boolean writesEnabled = true;
461 // Set if region is read-only
462 volatile boolean readOnly = false;
463 // whether the reads are enabled. This is different than readOnly, because readOnly is
464 // static in the lifetime of the region, while readsEnabled is dynamic
465 volatile boolean readsEnabled = true;
468 * Set flags that make this region read-only.
470 * @param onOff flip value for region r/o setting
472 synchronized void setReadOnly(final boolean onOff) {
473 this.writesEnabled = !onOff;
474 this.readOnly = onOff;
477 boolean isReadOnly() {
478 return this.readOnly;
481 boolean isFlushRequested() {
482 return this.flushRequested;
485 void setReadsEnabled(boolean readsEnabled) {
486 this.readsEnabled = readsEnabled;
489 static final long HEAP_SIZE = ClassSize.align(
490 ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
494 * Objects from this class are created when flushing to describe all the different states that
495 * that method ends up in. The Result enum describes those states. The sequence id should only
496 * be specified if the flush was successful, and the failure message should only be specified
497 * if it didn't flush.
499 public static class FlushResultImpl implements FlushResult {
500 final Result result;
501 final String failureReason;
502 final long flushSequenceId;
503 final boolean wroteFlushWalMarker;
506 * Convenience constructor to use when the flush is successful, the failure message is set to
507 * null.
508 * @param result Expecting FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_COMPACTION_NEEDED.
509 * @param flushSequenceId Generated sequence id that comes right after the edits in the
510 * memstores.
512 FlushResultImpl(Result result, long flushSequenceId) {
513 this(result, flushSequenceId, null, false);
514 assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
515 .FLUSHED_COMPACTION_NEEDED;
519 * Convenience constructor to use when we cannot flush.
520 * @param result Expecting CANNOT_FLUSH_MEMSTORE_EMPTY or CANNOT_FLUSH.
521 * @param failureReason Reason why we couldn't flush.
523 FlushResultImpl(Result result, String failureReason, boolean wroteFlushMarker) {
524 this(result, -1, failureReason, wroteFlushMarker);
525 assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH;
529 * Constructor with all the parameters.
530 * @param result Any of the Result.
531 * @param flushSequenceId Generated sequence id if the memstores were flushed else -1.
532 * @param failureReason Reason why we couldn't flush, or null.
534 FlushResultImpl(Result result, long flushSequenceId, String failureReason,
535 boolean wroteFlushMarker) {
536 this.result = result;
537 this.flushSequenceId = flushSequenceId;
538 this.failureReason = failureReason;
539 this.wroteFlushWalMarker = wroteFlushMarker;
543 * Convenience method, the equivalent of checking if result is
544 * FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_NO_COMPACTION_NEEDED.
545 * @return true if the memstores were flushed, else false.
547 @Override
548 public boolean isFlushSucceeded() {
549 return result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
550 .FLUSHED_COMPACTION_NEEDED;
554 * Convenience method, the equivalent of checking if result is FLUSHED_COMPACTION_NEEDED.
555 * @return True if the flush requested a compaction, else false (doesn't even mean it flushed).
557 @Override
558 public boolean isCompactionNeeded() {
559 return result == Result.FLUSHED_COMPACTION_NEEDED;
562 @Override
563 public String toString() {
564 return new StringBuilder()
565 .append("flush result:").append(result).append(", ")
566 .append("failureReason:").append(failureReason).append(",")
567 .append("flush seq id").append(flushSequenceId).toString();
570 @Override
571 public Result getResult() {
572 return result;
576 /** A result object from prepare flush cache stage */
577 static class PrepareFlushResult {
578 final FlushResultImpl result; // indicating a failure result from prepare
579 final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
580 final TreeMap<byte[], List<Path>> committedFiles;
581 final TreeMap<byte[], MemStoreSize> storeFlushableSize;
582 final long startTime;
583 final long flushOpSeqId;
584 final long flushedSeqId;
585 final MemStoreSizing totalFlushableSize;
587 /** Constructs an early exit case */
588 PrepareFlushResult(FlushResultImpl result, long flushSeqId) {
589 this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, MemStoreSizing.DUD);
592 /** Constructs a successful prepare flush result */
593 PrepareFlushResult(
594 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
595 TreeMap<byte[], List<Path>> committedFiles,
596 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId,
597 long flushedSeqId, MemStoreSizing totalFlushableSize) {
598 this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
599 flushSeqId, flushedSeqId, totalFlushableSize);
602 private PrepareFlushResult(
603 FlushResultImpl result,
604 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
605 TreeMap<byte[], List<Path>> committedFiles,
606 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId,
607 long flushedSeqId, MemStoreSizing totalFlushableSize) {
608 this.result = result;
609 this.storeFlushCtxs = storeFlushCtxs;
610 this.committedFiles = committedFiles;
611 this.storeFlushableSize = storeFlushableSize;
612 this.startTime = startTime;
613 this.flushOpSeqId = flushSeqId;
614 this.flushedSeqId = flushedSeqId;
615 this.totalFlushableSize = totalFlushableSize;
618 public FlushResult getResult() {
619 return this.result;
624 * A class that tracks exceptions that have been observed in one batch. Not thread safe.
626 static class ObservedExceptionsInBatch {
627 private boolean wrongRegion = false;
628 private boolean failedSanityCheck = false;
629 private boolean wrongFamily = false;
632 * @return If a {@link WrongRegionException} has been observed.
634 boolean hasSeenWrongRegion() {
635 return wrongRegion;
639 * Records that a {@link WrongRegionException} has been observed.
641 void sawWrongRegion() {
642 wrongRegion = true;
646 * @return If a {@link FailedSanityCheckException} has been observed.
648 boolean hasSeenFailedSanityCheck() {
649 return failedSanityCheck;
653 * Records that a {@link FailedSanityCheckException} has been observed.
655 void sawFailedSanityCheck() {
656 failedSanityCheck = true;
660 * @return If a {@link NoSuchColumnFamilyException} has been observed.
662 boolean hasSeenNoSuchFamily() {
663 return wrongFamily;
667 * Records that a {@link NoSuchColumnFamilyException} has been observed.
669 void sawNoSuchFamily() {
670 wrongFamily = true;
674 final WriteState writestate = new WriteState();
676 long memstoreFlushSize;
677 final long timestampSlop;
679 // Last flush time for each Store. Useful when we are flushing for each column
680 private final ConcurrentMap<HStore, Long> lastStoreFlushTimeMap = new ConcurrentHashMap<>();
682 protected RegionServerServices rsServices;
683 private RegionServerAccounting rsAccounting;
684 private long flushCheckInterval;
685 // flushPerChanges is to prevent too many changes in memstore
686 private long flushPerChanges;
687 private long blockingMemStoreSize;
688 // Used to guard closes
689 final ReentrantReadWriteLock lock;
690 // Used to track interruptible holders of the region lock. Currently that is only RPC handler
691 // threads. Boolean value in map determines if lock holder can be interrupted, normally true,
692 // but may be false when thread is transiting a critical section.
693 final ConcurrentHashMap<Thread, Boolean> regionLockHolders;
695 // Stop updates lock
696 private final ReentrantReadWriteLock updatesLock = new ReentrantReadWriteLock();
698 private final MultiVersionConcurrencyControl mvcc;
700 // Coprocessor host
701 private RegionCoprocessorHost coprocessorHost;
703 private TableDescriptor htableDescriptor = null;
704 private RegionSplitPolicy splitPolicy;
705 private RegionSplitRestriction splitRestriction;
706 private FlushPolicy flushPolicy;
708 private final MetricsRegion metricsRegion;
709 private final MetricsRegionWrapperImpl metricsRegionWrapper;
710 private final Durability regionDurability;
711 private final boolean regionStatsEnabled;
712 // Stores the replication scope of the various column families of the table
713 // that has non-default scope
714 private final NavigableMap<byte[], Integer> replicationScope = new TreeMap<>(
715 Bytes.BYTES_COMPARATOR);
717 private final StoreHotnessProtector storeHotnessProtector;
719 private Optional<RegionReplicationSink> regionReplicationSink = Optional.empty();
722 * HRegion constructor. This constructor should only be used for testing and
723 * extensions. Instances of HRegion should be instantiated with the
724 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
726 * @param tableDir qualified path of directory where region should be located,
727 * usually the table directory.
728 * @param wal The WAL is the outbound log for any updates to the HRegion
729 * The wal file is a logfile from the previous execution that's
730 * custom-computed for this HRegion. The HRegionServer computes and sorts the
731 * appropriate wal info for this HRegion. If there is a previous wal file
732 * (implying that the HRegion has been written-to before), then read it from
733 * the supplied path.
734 * @param fs is the filesystem.
735 * @param confParam is global configuration settings.
736 * @param regionInfo - RegionInfo that describes the region
737 * is new), then read them from the supplied path.
738 * @param htd the table descriptor
739 * @param rsServices reference to {@link RegionServerServices} or null
740 * @deprecated Use other constructors.
742 @Deprecated
743 public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
744 final Configuration confParam, final RegionInfo regionInfo,
745 final TableDescriptor htd, final RegionServerServices rsServices) {
746 this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
747 wal, confParam, htd, rsServices);
751 * HRegion constructor. This constructor should only be used for testing and
752 * extensions. Instances of HRegion should be instantiated with the
753 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
755 * @param fs is the filesystem.
756 * @param wal The WAL is the outbound log for any updates to the HRegion
757 * The wal file is a logfile from the previous execution that's
758 * custom-computed for this HRegion. The HRegionServer computes and sorts the
759 * appropriate wal info for this HRegion. If there is a previous wal file
760 * (implying that the HRegion has been written-to before), then read it from
761 * the supplied path.
762 * @param confParam is global configuration settings.
763 * @param htd the table descriptor
764 * @param rsServices reference to {@link RegionServerServices} or null
766 public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam,
767 final TableDescriptor htd, final RegionServerServices rsServices) {
768 if (htd == null) {
769 throw new IllegalArgumentException("Need table descriptor");
772 if (confParam instanceof CompoundConfiguration) {
773 throw new IllegalArgumentException("Need original base configuration");
776 this.wal = wal;
777 this.fs = fs;
778 this.mvcc = new MultiVersionConcurrencyControl(getRegionInfo().getShortNameToLog());
780 // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
781 this.baseConf = confParam;
782 this.conf = new CompoundConfiguration().add(confParam).addBytesMap(htd.getValues());
783 this.cellComparator = htd.isMetaTable() ||
784 conf.getBoolean(USE_META_CELL_COMPARATOR, DEFAULT_USE_META_CELL_COMPARATOR) ?
785 MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR;
786 this.lock = new ReentrantReadWriteLock(conf.getBoolean(FAIR_REENTRANT_CLOSE_LOCK,
787 DEFAULT_FAIR_REENTRANT_CLOSE_LOCK));
788 this.regionLockHolders = new ConcurrentHashMap<>();
789 this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL,
790 DEFAULT_CACHE_FLUSH_INTERVAL);
791 this.flushPerChanges = conf.getLong(MEMSTORE_FLUSH_PER_CHANGES, DEFAULT_FLUSH_PER_CHANGES);
792 if (this.flushPerChanges > MAX_FLUSH_PER_CHANGES) {
793 throw new IllegalArgumentException(MEMSTORE_FLUSH_PER_CHANGES + " can not exceed "
794 + MAX_FLUSH_PER_CHANGES);
796 int tmpRowLockDuration = conf.getInt("hbase.rowlock.wait.duration",
797 DEFAULT_ROWLOCK_WAIT_DURATION);
798 if (tmpRowLockDuration <= 0) {
799 LOG.info("Found hbase.rowlock.wait.duration set to {}. values <= 0 will cause all row " +
800 "locking to fail. Treating it as 1ms to avoid region failure.", tmpRowLockDuration);
801 tmpRowLockDuration = 1;
803 this.rowLockWaitDuration = tmpRowLockDuration;
805 this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
806 this.htableDescriptor = htd;
807 Set<byte[]> families = this.htableDescriptor.getColumnFamilyNames();
808 for (byte[] family : families) {
809 if (!replicationScope.containsKey(family)) {
810 int scope = htd.getColumnFamily(family).getScope();
811 // Only store those families that has NON-DEFAULT scope
812 if (scope != REPLICATION_SCOPE_LOCAL) {
813 // Do a copy before storing it here.
814 replicationScope.put(Bytes.copy(family), scope);
819 this.rsServices = rsServices;
820 if (this.rsServices != null) {
821 this.blockCache = rsServices.getBlockCache().orElse(null);
822 this.mobFileCache = rsServices.getMobFileCache().orElse(null);
824 this.regionServicesForStores = new RegionServicesForStores(this, rsServices);
826 setHTableSpecificConf();
827 this.scannerReadPoints = new ConcurrentHashMap<>();
829 this.busyWaitDuration = conf.getLong(
830 "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
831 this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
832 if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
833 throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
834 + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
835 + maxBusyWaitMultiplier + "). Their product should be positive");
837 this.maxBusyWaitDuration = conf.getLong("hbase.ipc.client.call.purge.timeout",
838 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
841 * timestamp.slop provides a server-side constraint on the timestamp. This
842 * assumes that you base your TS around EnvironmentEdgeManager.currentTime(). In this case,
843 * throw an error to the user if the user-specified TS is newer than now +
844 * slop. LATEST_TIMESTAMP == don't use this functionality
846 this.timestampSlop = conf.getLong(
847 "hbase.hregion.keyvalue.timestamp.slop.millisecs",
848 HConstants.LATEST_TIMESTAMP);
850 this.storeHotnessProtector = new StoreHotnessProtector(this, conf);
852 boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC);
854 * This is the global default value for durability. All tables/mutations not defining a
855 * durability or using USE_DEFAULT will default to this value.
857 Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL;
858 this.regionDurability =
859 this.htableDescriptor.getDurability() == Durability.USE_DEFAULT ? defaultDurability :
860 this.htableDescriptor.getDurability();
862 decorateRegionConfiguration(conf);
863 if (rsServices != null) {
864 this.rsAccounting = this.rsServices.getRegionServerAccounting();
865 // don't initialize coprocessors if not running within a regionserver
866 // TODO: revisit if coprocessors should load in other cases
867 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
868 this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
869 this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper, conf);
870 } else {
871 this.metricsRegionWrapper = null;
872 this.metricsRegion = null;
874 if (LOG.isDebugEnabled()) {
875 // Write out region name, its encoded name and storeHotnessProtector as string.
876 LOG.debug("Instantiated " + this +"; "+ storeHotnessProtector.toString());
879 configurationManager = null;
881 // disable stats tracking system tables, but check the config for everything else
882 this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals(
883 NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) ?
884 false :
885 conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
886 HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
888 this.maxCellSize = conf.getLong(HBASE_MAX_CELL_SIZE_KEY, DEFAULT_MAX_CELL_SIZE);
889 this.miniBatchSize = conf.getInt(HBASE_REGIONSERVER_MINIBATCH_SIZE,
890 DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE);
892 // recover the metrics of read and write requests count if they were retained
893 if (rsServices != null && rsServices.getRegionServerAccounting() != null) {
894 Pair<Long, Long> retainedRWRequestsCnt = rsServices.getRegionServerAccounting()
895 .getRetainedRegionRWRequestsCnt().get(getRegionInfo().getEncodedName());
896 if (retainedRWRequestsCnt != null) {
897 this.addReadRequestsCount(retainedRWRequestsCnt.getFirst());
898 this.addWriteRequestsCount(retainedRWRequestsCnt.getSecond());
899 // remove them since won't use again
900 rsServices.getRegionServerAccounting().getRetainedRegionRWRequestsCnt()
901 .remove(getRegionInfo().getEncodedName());
906 private void setHTableSpecificConf() {
907 if (this.htableDescriptor == null) {
908 return;
910 long flushSize = this.htableDescriptor.getMemStoreFlushSize();
912 if (flushSize <= 0) {
913 flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
914 TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE);
916 this.memstoreFlushSize = flushSize;
917 long mult = conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,
918 HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER);
919 this.blockingMemStoreSize = this.memstoreFlushSize * mult;
923 * Initialize this region.
924 * Used only by tests and SplitTransaction to reopen the region.
925 * You should use createHRegion() or openHRegion()
926 * @return What the next sequence (edit) id should be.
927 * @throws IOException e
928 * @deprecated use HRegion.createHRegion() or HRegion.openHRegion()
930 @Deprecated
931 public long initialize() throws IOException {
932 return initialize(null);
936 * Initialize this region.
938 * @param reporter Tickle every so often if initialize is taking a while.
939 * @return What the next sequence (edit) id should be.
940 * @throws IOException e
942 long initialize(final CancelableProgressable reporter) throws IOException {
944 //Refuse to open the region if there is no column family in the table
945 if (htableDescriptor.getColumnFamilyCount() == 0) {
946 throw new DoNotRetryIOException("Table " + htableDescriptor.getTableName().getNameAsString()+
947 " should have at least one column family.");
950 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
951 status.enableStatusJournal(true);
952 long nextSeqId = -1;
953 try {
954 nextSeqId = initializeRegionInternals(reporter, status);
955 return nextSeqId;
956 } catch (IOException e) {
957 LOG.warn("Failed initialize of region= {}, starting to roll back memstore",
958 getRegionInfo().getRegionNameAsString(), e);
959 // global memstore size will be decreased when dropping memstore
960 try {
961 //drop the memory used by memstore if open region fails
962 dropMemStoreContents();
963 } catch (IOException ioE) {
964 if (conf.getBoolean(MemStoreLAB.USEMSLAB_KEY, MemStoreLAB.USEMSLAB_DEFAULT)) {
965 LOG.warn("Failed drop memstore of region= {}, "
966 + "some chunks may not released forever since MSLAB is enabled",
967 getRegionInfo().getRegionNameAsString());
971 throw e;
972 } finally {
973 // nextSeqid will be -1 if the initialization fails.
974 // At least it will be 0 otherwise.
975 if (nextSeqId == -1) {
976 status.abort("Exception during region " + getRegionInfo().getRegionNameAsString() +
977 " initialization.");
979 if (LOG.isDebugEnabled()) {
980 LOG.debug("Region open journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
981 status.prettyPrintJournal());
983 status.cleanup();
987 private long initializeRegionInternals(final CancelableProgressable reporter,
988 final MonitoredTask status) throws IOException {
989 if (coprocessorHost != null) {
990 status.setStatus("Running coprocessor pre-open hook");
991 coprocessorHost.preOpen();
994 // Write HRI to a file in case we need to recover hbase:meta
995 // Only the primary replica should write .regioninfo
996 if (this.getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
997 status.setStatus("Writing region info on filesystem");
998 fs.checkRegionInfoOnFilesystem();
1001 // Initialize all the HStores
1002 status.setStatus("Initializing all the Stores");
1003 long maxSeqId = initializeStores(reporter, status);
1004 this.mvcc.advanceTo(maxSeqId);
1005 if (!isRestoredRegion && ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
1006 Collection<HStore> stores = this.stores.values();
1007 try {
1008 // update the stores that we are replaying
1009 LOG.debug("replaying wal for " + this.getRegionInfo().getEncodedName());
1010 stores.forEach(HStore::startReplayingFromWAL);
1011 // Recover any edits if available.
1012 maxSeqId = Math.max(maxSeqId,
1013 replayRecoveredEditsIfAny(maxSeqIdInStores, reporter, status));
1014 // Recover any hfiles if available
1015 maxSeqId = Math.max(maxSeqId, loadRecoveredHFilesIfAny(stores));
1016 // Make sure mvcc is up to max.
1017 this.mvcc.advanceTo(maxSeqId);
1018 } finally {
1019 LOG.debug("stopping wal replay for " + this.getRegionInfo().getEncodedName());
1020 // update the stores that we are done replaying
1021 stores.forEach(HStore::stopReplayingFromWAL);
1024 this.lastReplayedOpenRegionSeqId = maxSeqId;
1026 this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this));
1027 this.writestate.flushRequested = false;
1028 this.writestate.compacting.set(0);
1030 if (this.writestate.writesEnabled) {
1031 LOG.debug("Cleaning up temporary data for " + this.getRegionInfo().getEncodedName());
1032 // Remove temporary data left over from old regions
1033 status.setStatus("Cleaning up temporary data from old regions");
1034 fs.cleanupTempDir();
1037 // Initialize split policy
1038 this.splitPolicy = RegionSplitPolicy.create(this, conf);
1040 // Initialize split restriction
1041 splitRestriction = RegionSplitRestriction.create(getTableDescriptor(), conf);
1043 // Initialize flush policy
1044 this.flushPolicy = FlushPolicyFactory.create(this, conf);
1046 long lastFlushTime = EnvironmentEdgeManager.currentTime();
1047 for (HStore store: stores.values()) {
1048 this.lastStoreFlushTimeMap.put(store, lastFlushTime);
1051 // Use maximum of log sequenceid or that which was found in stores
1052 // (particularly if no recovered edits, seqid will be -1).
1053 long nextSeqId = maxSeqId + 1;
1054 if (!isRestoredRegion) {
1055 // always get openSeqNum from the default replica, even if we are secondary replicas
1056 long maxSeqIdFromFile = WALSplitUtil.getMaxRegionSequenceId(conf,
1057 RegionReplicaUtil.getRegionInfoForDefaultReplica(getRegionInfo()), this::getFilesystem,
1058 this::getWalFileSystem);
1059 nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
1060 // The openSeqNum will always be increase even for read only region, as we rely on it to
1061 // determine whether a region has been successfully reopened, so here we always need to update
1062 // the max sequence id file.
1063 if (RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
1064 LOG.debug("writing seq id for {}", this.getRegionInfo().getEncodedName());
1065 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
1066 nextSeqId - 1);
1067 // This means we have replayed all the recovered edits and also written out the max sequence
1068 // id file, let's delete the wrong directories introduced in HBASE-20734, see HBASE-22617
1069 // for more details.
1070 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf,
1071 getRegionInfo().getTable(), getRegionInfo().getEncodedName());
1072 FileSystem walFs = getWalFileSystem();
1073 if (walFs.exists(wrongRegionWALDir)) {
1074 if (!walFs.delete(wrongRegionWALDir, true)) {
1075 LOG.debug("Failed to clean up wrong region WAL directory {}", wrongRegionWALDir);
1081 LOG.info("Opened {}; next sequenceid={}; {}, {}", this.getRegionInfo().getShortNameToLog(),
1082 nextSeqId, this.splitPolicy, this.flushPolicy);
1084 // A region can be reopened if failed a split; reset flags
1085 this.closing.set(false);
1086 this.closed.set(false);
1088 if (coprocessorHost != null) {
1089 LOG.debug("Running coprocessor post-open hooks for " + this.getRegionInfo().getEncodedName());
1090 status.setStatus("Running coprocessor post-open hooks");
1091 coprocessorHost.postOpen();
1093 initializeRegionReplicationSink(reporter, status);
1094 status.markComplete("Region opened successfully");
1095 return nextSeqId;
1098 private void initializeRegionReplicationSink(CancelableProgressable reporter,
1099 MonitoredTask status) {
1100 RegionServerServices rss = getRegionServerServices();
1101 TableDescriptor td = getTableDescriptor();
1102 int regionReplication = td.getRegionReplication();
1103 RegionInfo regionInfo = getRegionInfo();
1104 if (regionReplication <= 1 || !RegionReplicaUtil.isDefaultReplica(regionInfo) ||
1105 !ServerRegionReplicaUtil.isRegionReplicaReplicationEnabled(conf, regionInfo.getTable()) ||
1106 rss == null) {
1107 regionReplicationSink = Optional.empty();
1108 return;
1110 status.setStatus("Initializaing region replication sink");
1111 regionReplicationSink = Optional.of(new RegionReplicationSink(conf, regionInfo, td,
1112 rss.getRegionReplicationBufferManager(), () -> rss.getFlushRequester().requestFlush(this,
1113 new ArrayList<>(td.getColumnFamilyNames()), FlushLifeCycleTracker.DUMMY),
1114 rss.getAsyncClusterConnection()));
1118 * Open all Stores.
1119 * @param reporter
1120 * @param status
1121 * @return Highest sequenceId found out in a Store.
1122 * @throws IOException
1124 private long initializeStores(CancelableProgressable reporter, MonitoredTask status)
1125 throws IOException {
1126 return initializeStores(reporter, status, false);
1129 private long initializeStores(CancelableProgressable reporter, MonitoredTask status,
1130 boolean warmup) throws IOException {
1131 // Load in all the HStores.
1132 long maxSeqId = -1;
1133 // initialized to -1 so that we pick up MemstoreTS from column families
1134 long maxMemstoreTS = -1;
1136 if (htableDescriptor.getColumnFamilyCount() != 0) {
1137 // initialize the thread pool for opening stores in parallel.
1138 ThreadPoolExecutor storeOpenerThreadPool =
1139 getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog());
1140 CompletionService<HStore> completionService = new ExecutorCompletionService<>(storeOpenerThreadPool);
1142 // initialize each store in parallel
1143 for (final ColumnFamilyDescriptor family : htableDescriptor.getColumnFamilies()) {
1144 status.setStatus("Instantiating store for column family " + family);
1145 completionService.submit(new Callable<HStore>() {
1146 @Override
1147 public HStore call() throws IOException {
1148 return instantiateHStore(family, warmup);
1152 boolean allStoresOpened = false;
1153 boolean hasSloppyStores = false;
1154 try {
1155 for (int i = 0; i < htableDescriptor.getColumnFamilyCount(); i++) {
1156 Future<HStore> future = completionService.take();
1157 HStore store = future.get();
1158 this.stores.put(store.getColumnFamilyDescriptor().getName(), store);
1159 if (store.isSloppyMemStore()) {
1160 hasSloppyStores = true;
1163 long storeMaxSequenceId = store.getMaxSequenceId().orElse(0L);
1164 maxSeqIdInStores.put(Bytes.toBytes(store.getColumnFamilyName()),
1165 storeMaxSequenceId);
1166 if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) {
1167 maxSeqId = storeMaxSequenceId;
1169 long maxStoreMemstoreTS = store.getMaxMemStoreTS().orElse(0L);
1170 if (maxStoreMemstoreTS > maxMemstoreTS) {
1171 maxMemstoreTS = maxStoreMemstoreTS;
1174 allStoresOpened = true;
1175 if(hasSloppyStores) {
1176 htableDescriptor = TableDescriptorBuilder.newBuilder(htableDescriptor)
1177 .setFlushPolicyClassName(FlushNonSloppyStoresFirstPolicy.class.getName())
1178 .build();
1179 LOG.info("Setting FlushNonSloppyStoresFirstPolicy for the region=" + this);
1181 } catch (InterruptedException e) {
1182 throw throwOnInterrupt(e);
1183 } catch (ExecutionException e) {
1184 throw new IOException(e.getCause());
1185 } finally {
1186 storeOpenerThreadPool.shutdownNow();
1187 if (!allStoresOpened) {
1188 // something went wrong, close all opened stores
1189 LOG.error("Could not initialize all stores for the region=" + this);
1190 for (HStore store : this.stores.values()) {
1191 try {
1192 store.close();
1193 } catch (IOException e) {
1194 LOG.warn("close store {} failed in region {}", store.toString(), this, e);
1200 return Math.max(maxSeqId, maxMemstoreTS + 1);
1203 private void initializeWarmup(final CancelableProgressable reporter) throws IOException {
1204 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
1205 // Initialize all the HStores
1206 status.setStatus("Warmup all stores of " + this.getRegionInfo().getRegionNameAsString());
1207 try {
1208 initializeStores(reporter, status, true);
1209 } finally {
1210 status.markComplete("Warmed up " + this.getRegionInfo().getRegionNameAsString());
1215 * @return Map of StoreFiles by column family
1217 private NavigableMap<byte[], List<Path>> getStoreFiles() {
1218 NavigableMap<byte[], List<Path>> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1219 for (HStore store : stores.values()) {
1220 Collection<HStoreFile> storeFiles = store.getStorefiles();
1221 if (storeFiles == null) {
1222 continue;
1224 List<Path> storeFileNames = new ArrayList<>();
1225 for (HStoreFile storeFile : storeFiles) {
1226 storeFileNames.add(storeFile.getPath());
1228 allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames);
1230 return allStoreFiles;
1233 protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
1234 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1235 RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
1236 RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId,
1237 getRegionServerServices().getServerName(), storeFiles);
1238 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionOpenDesc,
1239 mvcc, regionReplicationSink.orElse(null));
1242 private void writeRegionCloseMarker(WAL wal) throws IOException {
1243 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1244 RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor(
1245 RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(),
1246 getRegionServerServices().getServerName(), storeFiles);
1247 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionEventDesc,
1248 mvcc, null);
1250 // Store SeqId in WAL FileSystem when a region closes
1251 // checking region folder exists is due to many tests which delete the table folder while a
1252 // table is still online
1253 if (getWalFileSystem().exists(getWALRegionDir())) {
1254 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
1255 mvcc.getReadPoint());
1260 * @return True if this region has references.
1262 public boolean hasReferences() {
1263 return stores.values().stream().anyMatch(HStore::hasReferences);
1266 public void blockUpdates() {
1267 this.updatesLock.writeLock().lock();
1270 public void unblockUpdates() {
1271 this.updatesLock.writeLock().unlock();
1274 public HDFSBlocksDistribution getHDFSBlocksDistribution() {
1275 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1276 stores.values().stream().filter(s -> s.getStorefiles() != null)
1277 .flatMap(s -> s.getStorefiles().stream()).map(HStoreFile::getHDFSBlockDistribution)
1278 .forEachOrdered(hdfsBlocksDistribution::add);
1279 return hdfsBlocksDistribution;
1283 * This is a helper function to compute HDFS block distribution on demand
1284 * @param conf configuration
1285 * @param tableDescriptor TableDescriptor of the table
1286 * @param regionInfo encoded name of the region
1287 * @return The HDFS blocks distribution for the given region.
1289 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
1290 TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
1291 Path tablePath =
1292 CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), tableDescriptor.getTableName());
1293 return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
1297 * This is a helper function to compute HDFS block distribution on demand
1298 * @param conf configuration
1299 * @param tableDescriptor TableDescriptor of the table
1300 * @param regionInfo encoded name of the region
1301 * @param tablePath the table directory
1302 * @return The HDFS blocks distribution for the given region.
1303 * @throws IOException
1305 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
1306 TableDescriptor tableDescriptor, RegionInfo regionInfo, Path tablePath) throws IOException {
1307 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1308 FileSystem fs = tablePath.getFileSystem(conf);
1310 HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
1311 for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()) {
1312 List<LocatedFileStatus> locatedFileStatusList = HRegionFileSystem
1313 .getStoreFilesLocatedStatus(regionFs, family.getNameAsString(), true);
1314 if (locatedFileStatusList == null) {
1315 continue;
1318 for (LocatedFileStatus status : locatedFileStatusList) {
1319 Path p = status.getPath();
1320 if (StoreFileInfo.isReference(p) || HFileLink.isHFileLink(p)) {
1321 // Only construct StoreFileInfo object if its not a hfile, save obj
1322 // creation
1323 StoreFileInfo storeFileInfo = new StoreFileInfo(conf, fs, status);
1324 hdfsBlocksDistribution.add(storeFileInfo
1325 .computeHDFSBlocksDistribution(fs));
1326 } else if (StoreFileInfo.isHFile(p)) {
1327 // If its a HFile, then lets just add to the block distribution
1328 // lets not create more objects here, not even another HDFSBlocksDistribution
1329 FSUtils.addToHDFSBlocksDistribution(hdfsBlocksDistribution,
1330 status.getBlockLocations());
1331 } else {
1332 throw new IOException("path=" + p
1333 + " doesn't look like a valid StoreFile");
1337 return hdfsBlocksDistribution;
1341 * Increase the size of mem store in this region and the size of global mem
1342 * store
1344 private void incMemStoreSize(MemStoreSize mss) {
1345 incMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(),
1346 mss.getCellsCount());
1349 void incMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta,
1350 int cellsCountDelta) {
1351 if (this.rsAccounting != null) {
1352 rsAccounting.incGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta);
1354 long dataSize = this.memStoreSizing.incMemStoreSize(dataSizeDelta, heapSizeDelta,
1355 offHeapSizeDelta, cellsCountDelta);
1356 checkNegativeMemStoreDataSize(dataSize, dataSizeDelta);
1359 void decrMemStoreSize(MemStoreSize mss) {
1360 decrMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(),
1361 mss.getCellsCount());
1364 private void decrMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta,
1365 int cellsCountDelta) {
1366 if (this.rsAccounting != null) {
1367 rsAccounting.decGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta);
1369 long dataSize = this.memStoreSizing.decMemStoreSize(dataSizeDelta, heapSizeDelta,
1370 offHeapSizeDelta, cellsCountDelta);
1371 checkNegativeMemStoreDataSize(dataSize, -dataSizeDelta);
1374 private void checkNegativeMemStoreDataSize(long memStoreDataSize, long delta) {
1375 // This is extremely bad if we make memStoreSizing negative. Log as much info on the offending
1376 // caller as possible. (memStoreSizing might be a negative value already -- freeing memory)
1377 if (memStoreDataSize < 0) {
1378 LOG.error("Asked to modify this region's (" + this.toString()
1379 + ") memStoreSizing to a negative value which is incorrect. Current memStoreSizing="
1380 + (memStoreDataSize - delta) + ", delta=" + delta, new Exception());
1384 @Override
1385 public RegionInfo getRegionInfo() {
1386 return this.fs.getRegionInfo();
1390 * @return Instance of {@link RegionServerServices} used by this HRegion.
1391 * Can be null.
1393 RegionServerServices getRegionServerServices() {
1394 return this.rsServices;
1397 @Override
1398 public long getReadRequestsCount() {
1399 return readRequestsCount.sum();
1402 @Override
1403 public long getCpRequestsCount() {
1404 return cpRequestsCount.sum();
1407 @Override
1408 public long getFilteredReadRequestsCount() {
1409 return filteredReadRequestsCount.sum();
1412 @Override
1413 public long getWriteRequestsCount() {
1414 return writeRequestsCount.sum();
1417 @Override
1418 public long getMemStoreDataSize() {
1419 return memStoreSizing.getDataSize();
1422 @Override
1423 public long getMemStoreHeapSize() {
1424 return memStoreSizing.getHeapSize();
1427 @Override
1428 public long getMemStoreOffHeapSize() {
1429 return memStoreSizing.getOffHeapSize();
1432 /** @return store services for this region, to access services required by store level needs */
1433 public RegionServicesForStores getRegionServicesForStores() {
1434 return regionServicesForStores;
1437 @Override
1438 public long getNumMutationsWithoutWAL() {
1439 return numMutationsWithoutWAL.sum();
1442 @Override
1443 public long getDataInMemoryWithoutWAL() {
1444 return dataInMemoryWithoutWAL.sum();
1447 @Override
1448 public long getBlockedRequestsCount() {
1449 return blockedRequestsCount.sum();
1452 @Override
1453 public long getCheckAndMutateChecksPassed() {
1454 return checkAndMutateChecksPassed.sum();
1457 @Override
1458 public long getCheckAndMutateChecksFailed() {
1459 return checkAndMutateChecksFailed.sum();
1462 // TODO Needs to check whether we should expose our metrics system to CPs. If CPs themselves doing
1463 // the op and bypassing the core, this might be needed? Should be stop supporting the bypass
1464 // feature?
1465 public MetricsRegion getMetrics() {
1466 return metricsRegion;
1469 @Override
1470 public boolean isClosed() {
1471 return this.closed.get();
1474 @Override
1475 public boolean isClosing() {
1476 return this.closing.get();
1479 @Override
1480 public boolean isReadOnly() {
1481 return this.writestate.isReadOnly();
1484 @Override
1485 public boolean isAvailable() {
1486 return !isClosed() && !isClosing();
1489 @Override
1490 public boolean isSplittable() {
1491 return splitPolicy.canSplit();
1494 @Override
1495 public boolean isMergeable() {
1496 if (!isAvailable()) {
1497 LOG.debug("Region " + this
1498 + " is not mergeable because it is closing or closed");
1499 return false;
1501 if (hasReferences()) {
1502 LOG.debug("Region " + this
1503 + " is not mergeable because it has references");
1504 return false;
1507 return true;
1510 public boolean areWritesEnabled() {
1511 synchronized(this.writestate) {
1512 return this.writestate.writesEnabled;
1516 public MultiVersionConcurrencyControl getMVCC() {
1517 return mvcc;
1520 @Override
1521 public long getMaxFlushedSeqId() {
1522 return maxFlushedSeqId;
1526 * @return readpoint considering given IsolationLevel. Pass {@code null} for default
1528 public long getReadPoint(IsolationLevel isolationLevel) {
1529 if (isolationLevel != null && isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
1530 // This scan can read even uncommitted transactions
1531 return Long.MAX_VALUE;
1533 return mvcc.getReadPoint();
1536 public boolean isLoadingCfsOnDemandDefault() {
1537 return this.isLoadingCfsOnDemandDefault;
1541 * Close down this HRegion. Flush the cache, shut down each HStore, don't
1542 * service any more calls.
1544 * <p>This method could take some time to execute, so don't call it from a
1545 * time-sensitive thread.
1547 * @return Vector of all the storage files that the HRegion's component
1548 * HStores make use of. It's a list of all StoreFile objects. Returns empty
1549 * vector if already closed and null if judged that it should not close.
1551 * @throws IOException e
1552 * @throws DroppedSnapshotException Thrown when replay of wal is required
1553 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1554 * caller MUST abort after this.
1556 public Map<byte[], List<HStoreFile>> close() throws IOException {
1557 return close(false);
1560 private final Object closeLock = new Object();
1562 /** Conf key for fair locking policy */
1563 public static final String FAIR_REENTRANT_CLOSE_LOCK =
1564 "hbase.regionserver.fair.region.close.lock";
1565 public static final boolean DEFAULT_FAIR_REENTRANT_CLOSE_LOCK = true;
1566 /** Conf key for the periodic flush interval */
1567 public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL =
1568 "hbase.regionserver.optionalcacheflushinterval";
1569 /** Default interval for the memstore flush */
1570 public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
1571 /** Default interval for System tables memstore flush */
1572 public static final int SYSTEM_CACHE_FLUSH_INTERVAL = 300000; // 5 minutes
1574 /** Conf key to force a flush if there are already enough changes for one region in memstore */
1575 public static final String MEMSTORE_FLUSH_PER_CHANGES =
1576 "hbase.regionserver.flush.per.changes";
1577 public static final long DEFAULT_FLUSH_PER_CHANGES = 30000000; // 30 millions
1579 * The following MAX_FLUSH_PER_CHANGES is large enough because each KeyValue has 20+ bytes
1580 * overhead. Therefore, even 1G empty KVs occupy at least 20GB memstore size for a single region
1582 public static final long MAX_FLUSH_PER_CHANGES = 1000000000; // 1G
1584 public static final String CLOSE_WAIT_ABORT = "hbase.regionserver.close.wait.abort";
1585 public static final boolean DEFAULT_CLOSE_WAIT_ABORT = true;
1586 public static final String CLOSE_WAIT_TIME = "hbase.regionserver.close.wait.time.ms";
1587 public static final long DEFAULT_CLOSE_WAIT_TIME = 60000; // 1 minute
1588 public static final String CLOSE_WAIT_INTERVAL = "hbase.regionserver.close.wait.interval.ms";
1589 public static final long DEFAULT_CLOSE_WAIT_INTERVAL = 10000; // 10 seconds
1591 public Map<byte[], List<HStoreFile>> close(boolean abort) throws IOException {
1592 return close(abort, false);
1596 * Close down this HRegion. Flush the cache unless abort parameter is true,
1597 * Shut down each HStore, don't service any more calls.
1599 * This method could take some time to execute, so don't call it from a
1600 * time-sensitive thread.
1602 * @param abort true if server is aborting (only during testing)
1603 * @param ignoreStatus true if ignore the status (wont be showed on task list)
1604 * @return Vector of all the storage files that the HRegion's component
1605 * HStores make use of. It's a list of StoreFile objects. Can be null if
1606 * we are not to close at this time or we are already closed.
1608 * @throws IOException e
1609 * @throws DroppedSnapshotException Thrown when replay of wal is required
1610 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1611 * caller MUST abort after this.
1613 public Map<byte[], List<HStoreFile>> close(boolean abort, boolean ignoreStatus)
1614 throws IOException {
1615 // Only allow one thread to close at a time. Serialize them so dual
1616 // threads attempting to close will run up against each other.
1617 MonitoredTask status = TaskMonitor.get().createStatus(
1618 "Closing region " + this.getRegionInfo().getEncodedName() +
1619 (abort ? " due to abort" : ""), ignoreStatus);
1620 status.enableStatusJournal(true);
1621 status.setStatus("Waiting for close lock");
1622 try {
1623 synchronized (closeLock) {
1624 return doClose(abort, status);
1626 } finally {
1627 if (LOG.isDebugEnabled()) {
1628 LOG.debug("Region close journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
1629 status.prettyPrintJournal());
1631 status.cleanup();
1636 * Exposed for some very specific unit tests.
1638 public void setClosing(boolean closing) {
1639 this.closing.set(closing);
1643 * The {@link HRegion#doClose} will block forever if someone tries proving the dead lock via the unit test.
1644 * Instead of blocking, the {@link HRegion#doClose} will throw exception if you set the timeout.
1645 * @param timeoutForWriteLock the second time to wait for the write lock in {@link HRegion#doClose}
1647 public void setTimeoutForWriteLock(long timeoutForWriteLock) {
1648 assert timeoutForWriteLock >= 0;
1649 this.timeoutForWriteLock = timeoutForWriteLock;
1652 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
1653 justification="I think FindBugs is confused")
1654 private Map<byte[], List<HStoreFile>> doClose(boolean abort, MonitoredTask status)
1655 throws IOException {
1656 if (isClosed()) {
1657 LOG.warn("Region " + this + " already closed");
1658 return null;
1661 if (coprocessorHost != null) {
1662 status.setStatus("Running coprocessor pre-close hooks");
1663 this.coprocessorHost.preClose(abort);
1665 status.setStatus("Disabling compacts and flushes for region");
1666 boolean canFlush = true;
1667 synchronized (writestate) {
1668 // Disable compacting and flushing by background threads for this
1669 // region.
1670 canFlush = !writestate.readOnly;
1671 writestate.writesEnabled = false;
1672 LOG.debug("Closing {}, disabling compactions & flushes",
1673 this.getRegionInfo().getEncodedName());
1674 waitForFlushesAndCompactions();
1676 // If we were not just flushing, is it worth doing a preflush...one
1677 // that will clear out of the bulk of the memstore before we put up
1678 // the close flag?
1679 if (!abort && worthPreFlushing() && canFlush) {
1680 status.setStatus("Pre-flushing region before close");
1681 LOG.info("Running close preflush of {}", this.getRegionInfo().getEncodedName());
1682 try {
1683 internalFlushcache(status);
1684 } catch (IOException ioe) {
1685 // Failed to flush the region. Keep going.
1686 status.setStatus("Failed pre-flush " + this + "; " + ioe.getMessage());
1690 // Set the closing flag
1691 // From this point new arrivals at the region lock will get NSRE.
1693 this.closing.set(true);
1694 LOG.info("Closing region {}", this);
1696 // Acquire the close lock
1698 // The configuration parameter CLOSE_WAIT_ABORT is overloaded to enable both
1699 // the new regionserver abort condition and interrupts for running requests.
1700 // If CLOSE_WAIT_ABORT is not enabled there is no change from earlier behavior,
1701 // we will not attempt to interrupt threads servicing requests nor crash out
1702 // the regionserver if something remains stubborn.
1704 final boolean canAbort = conf.getBoolean(CLOSE_WAIT_ABORT, DEFAULT_CLOSE_WAIT_ABORT);
1705 boolean useTimedWait = false;
1706 if (timeoutForWriteLock != null && timeoutForWriteLock != Long.MAX_VALUE) {
1707 // convert legacy use of timeoutForWriteLock in seconds to new use in millis
1708 timeoutForWriteLock = TimeUnit.SECONDS.toMillis(timeoutForWriteLock);
1709 useTimedWait = true;
1710 } else if (canAbort) {
1711 timeoutForWriteLock = conf.getLong(CLOSE_WAIT_TIME, DEFAULT_CLOSE_WAIT_TIME);
1712 useTimedWait = true;
1714 if (LOG.isDebugEnabled()) {
1715 LOG.debug((useTimedWait ? "Time limited wait" : "Waiting without time limit") +
1716 " for close lock on " + this);
1718 final long closeWaitInterval = conf.getLong(CLOSE_WAIT_INTERVAL, DEFAULT_CLOSE_WAIT_INTERVAL);
1719 long elapsedWaitTime = 0;
1720 if (useTimedWait) {
1721 // Sanity check configuration
1722 long remainingWaitTime = timeoutForWriteLock;
1723 if (remainingWaitTime < closeWaitInterval) {
1724 LOG.warn("Time limit for close wait of " + timeoutForWriteLock +
1725 " ms is less than the configured lock acquisition wait interval " +
1726 closeWaitInterval + " ms, using wait interval as time limit");
1727 remainingWaitTime = closeWaitInterval;
1729 boolean acquired = false;
1730 do {
1731 long start = EnvironmentEdgeManager.currentTime();
1732 try {
1733 acquired = lock.writeLock().tryLock(Math.min(remainingWaitTime, closeWaitInterval),
1734 TimeUnit.MILLISECONDS);
1735 } catch (InterruptedException e) {
1736 // Interrupted waiting for close lock. More likely the server is shutting down, not
1737 // normal operation, so aborting upon interrupt while waiting on this lock would not
1738 // provide much value. Throw an IOE (as IIOE) like we would in the case where we
1739 // fail to acquire the lock.
1740 String msg = "Interrupted while waiting for close lock on " + this;
1741 LOG.warn(msg, e);
1742 throw (InterruptedIOException) new InterruptedIOException(msg).initCause(e);
1744 long elapsed = EnvironmentEdgeManager.currentTime() - start;
1745 elapsedWaitTime += elapsed;
1746 remainingWaitTime -= elapsed;
1747 if (canAbort && !acquired && remainingWaitTime > 0) {
1748 // Before we loop to wait again, interrupt all region operations that might
1749 // still be in progress, to encourage them to break out of waiting states or
1750 // inner loops, throw an exception to clients, and release the read lock via
1751 // endRegionOperation.
1752 if (LOG.isDebugEnabled()) {
1753 LOG.debug("Interrupting region operations after waiting for close lock for " +
1754 elapsedWaitTime + " ms on " + this + ", " + remainingWaitTime +
1755 " ms remaining");
1757 interruptRegionOperations();
1759 } while (!acquired && remainingWaitTime > 0);
1761 // If we fail to acquire the lock, trigger an abort if we can; otherwise throw an IOE
1762 // to let the caller know we could not proceed with the close.
1763 if (!acquired) {
1764 String msg = "Failed to acquire close lock on " + this + " after waiting " +
1765 elapsedWaitTime + " ms";
1766 LOG.error(msg);
1767 if (canAbort) {
1768 // If we failed to acquire the write lock, abort the server
1769 rsServices.abort(msg, null);
1771 throw new IOException(msg);
1774 } else {
1776 long start = EnvironmentEdgeManager.currentTime();
1777 lock.writeLock().lock();
1778 elapsedWaitTime = EnvironmentEdgeManager.currentTime() - start;
1782 if (LOG.isDebugEnabled()) {
1783 LOG.debug("Acquired close lock on " + this + " after waiting " +
1784 elapsedWaitTime + " ms");
1787 status.setStatus("Disabling writes for close");
1788 try {
1789 if (this.isClosed()) {
1790 status.abort("Already got closed by another process");
1791 // SplitTransaction handles the null
1792 return null;
1794 LOG.debug("Updates disabled for region " + this);
1795 // Don't flush the cache if we are aborting
1796 if (!abort && canFlush) {
1797 int failedfFlushCount = 0;
1798 int flushCount = 0;
1799 long tmp = 0;
1800 long remainingSize = this.memStoreSizing.getDataSize();
1801 while (remainingSize > 0) {
1802 try {
1803 internalFlushcache(status);
1804 if(flushCount >0) {
1805 LOG.info("Running extra flush, " + flushCount +
1806 " (carrying snapshot?) " + this);
1808 flushCount++;
1809 tmp = this.memStoreSizing.getDataSize();
1810 if (tmp >= remainingSize) {
1811 failedfFlushCount++;
1813 remainingSize = tmp;
1814 if (failedfFlushCount > 5) {
1815 // If we failed 5 times and are unable to clear memory, abort
1816 // so we do not lose data
1817 throw new DroppedSnapshotException("Failed clearing memory after " +
1818 flushCount + " attempts on region: " +
1819 Bytes.toStringBinary(getRegionInfo().getRegionName()));
1821 } catch (IOException ioe) {
1822 status.setStatus("Failed flush " + this + ", putting online again");
1823 synchronized (writestate) {
1824 writestate.writesEnabled = true;
1826 // Have to throw to upper layers. I can't abort server from here.
1827 throw ioe;
1832 Map<byte[], List<HStoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1833 if (!stores.isEmpty()) {
1834 // initialize the thread pool for closing stores in parallel.
1835 ThreadPoolExecutor storeCloserThreadPool =
1836 getStoreOpenAndCloseThreadPool("StoreCloser-" +
1837 getRegionInfo().getRegionNameAsString());
1838 CompletionService<Pair<byte[], Collection<HStoreFile>>> completionService =
1839 new ExecutorCompletionService<>(storeCloserThreadPool);
1841 // close each store in parallel
1842 for (HStore store : stores.values()) {
1843 MemStoreSize mss = store.getFlushableSize();
1844 if (!(abort || mss.getDataSize() == 0 || writestate.readOnly)) {
1845 if (getRegionServerServices() != null) {
1846 getRegionServerServices().abort("Assertion failed while closing store "
1847 + getRegionInfo().getRegionNameAsString() + " " + store
1848 + ". flushableSize expected=0, actual={" + mss
1849 + "}. Current memStoreSize=" + this.memStoreSizing.getMemStoreSize() +
1850 ". Maybe a coprocessor "
1851 + "operation failed and left the memstore in a partially updated state.", null);
1854 completionService
1855 .submit(new Callable<Pair<byte[], Collection<HStoreFile>>>() {
1856 @Override
1857 public Pair<byte[], Collection<HStoreFile>> call() throws IOException {
1858 return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close());
1862 try {
1863 for (int i = 0; i < stores.size(); i++) {
1864 Future<Pair<byte[], Collection<HStoreFile>>> future = completionService.take();
1865 Pair<byte[], Collection<HStoreFile>> storeFiles = future.get();
1866 List<HStoreFile> familyFiles = result.get(storeFiles.getFirst());
1867 if (familyFiles == null) {
1868 familyFiles = new ArrayList<>();
1869 result.put(storeFiles.getFirst(), familyFiles);
1871 familyFiles.addAll(storeFiles.getSecond());
1873 } catch (InterruptedException e) {
1874 throw throwOnInterrupt(e);
1875 } catch (ExecutionException e) {
1876 Throwable cause = e.getCause();
1877 if (cause instanceof IOException) {
1878 throw (IOException) cause;
1880 throw new IOException(cause);
1881 } finally {
1882 storeCloserThreadPool.shutdownNow();
1886 status.setStatus("Writing region close event to WAL");
1887 // Always write close marker to wal even for read only table. This is not a big problem as we
1888 // do not write any data into the region; it is just a meta edit in the WAL file.
1889 if (!abort && wal != null && getRegionServerServices() != null &&
1890 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
1891 writeRegionCloseMarker(wal);
1893 if (regionReplicationSink.isPresent()) {
1894 // stop replicating to secondary replicas
1895 RegionReplicationSink sink = regionReplicationSink.get();
1896 sink.stop();
1897 try {
1898 regionReplicationSink.get().waitUntilStopped();
1899 } catch (InterruptedException e) {
1900 throw throwOnInterrupt(e);
1903 this.closed.set(true);
1904 if (!canFlush) {
1905 decrMemStoreSize(this.memStoreSizing.getMemStoreSize());
1906 } else if (this.memStoreSizing.getDataSize() != 0) {
1907 LOG.error("Memstore data size is {} in region {}", this.memStoreSizing.getDataSize(), this);
1909 if (coprocessorHost != null) {
1910 status.setStatus("Running coprocessor post-close hooks");
1911 this.coprocessorHost.postClose(abort);
1913 if (this.metricsRegion != null) {
1914 this.metricsRegion.close();
1916 if (this.metricsRegionWrapper != null) {
1917 Closeables.close(this.metricsRegionWrapper, true);
1919 status.markComplete("Closed");
1920 LOG.info("Closed {}", this);
1921 return result;
1922 } finally {
1923 lock.writeLock().unlock();
1927 /** Wait for all current flushes and compactions of the region to complete */
1928 // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for
1929 // Phoenix needs.
1930 public void waitForFlushesAndCompactions() {
1931 synchronized (writestate) {
1932 if (this.writestate.readOnly) {
1933 // we should not wait for replayed flushed if we are read only (for example in case the
1934 // region is a secondary replica).
1935 return;
1937 boolean interrupted = false;
1938 try {
1939 while (writestate.compacting.get() > 0 || writestate.flushing) {
1940 LOG.debug("waiting for " + writestate.compacting + " compactions"
1941 + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
1942 try {
1943 writestate.wait();
1944 } catch (InterruptedException iex) {
1945 // essentially ignore and propagate the interrupt back up
1946 LOG.warn("Interrupted while waiting in region {}", this);
1947 interrupted = true;
1948 break;
1951 } finally {
1952 if (interrupted) {
1953 Thread.currentThread().interrupt();
1960 * Wait for all current flushes of the region to complete
1962 public void waitForFlushes() {
1963 waitForFlushes(0);// Unbound wait
1966 @Override
1967 public boolean waitForFlushes(long timeout) {
1968 synchronized (writestate) {
1969 if (this.writestate.readOnly) {
1970 // we should not wait for replayed flushed if we are read only (for example in case the
1971 // region is a secondary replica).
1972 return true;
1974 if (!writestate.flushing) return true;
1975 long start = EnvironmentEdgeManager.currentTime();
1976 long duration = 0;
1977 boolean interrupted = false;
1978 LOG.debug("waiting for cache flush to complete for region " + this);
1979 try {
1980 while (writestate.flushing) {
1981 if (timeout > 0 && duration >= timeout) break;
1982 try {
1983 long toWait = timeout == 0 ? 0 : (timeout - duration);
1984 writestate.wait(toWait);
1985 } catch (InterruptedException iex) {
1986 // essentially ignore and propagate the interrupt back up
1987 LOG.warn("Interrupted while waiting in region {}", this);
1988 interrupted = true;
1989 break;
1990 } finally {
1991 duration = EnvironmentEdgeManager.currentTime() - start;
1994 } finally {
1995 if (interrupted) {
1996 Thread.currentThread().interrupt();
1999 LOG.debug("Waited {} ms for region {} flush to complete", duration, this);
2000 return !(writestate.flushing);
2004 @Override
2005 public Configuration getReadOnlyConfiguration() {
2006 return new ReadOnlyConfiguration(this.conf);
2009 private ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
2010 final String threadNamePrefix) {
2011 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
2012 int maxThreads = Math.min(numStores,
2013 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
2014 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
2015 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
2018 ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
2019 final String threadNamePrefix) {
2020 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
2021 int maxThreads = Math.max(1,
2022 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
2023 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
2024 / numStores);
2025 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
2028 private static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
2029 final String threadNamePrefix) {
2030 return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
2031 new ThreadFactory() {
2032 private int count = 1;
2034 @Override
2035 public Thread newThread(Runnable r) {
2036 return new Thread(r, threadNamePrefix + "-" + count++);
2042 * @return True if its worth doing a flush before we put up the close flag.
2044 private boolean worthPreFlushing() {
2045 return this.memStoreSizing.getDataSize() >
2046 this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
2049 //////////////////////////////////////////////////////////////////////////////
2050 // HRegion accessors
2051 //////////////////////////////////////////////////////////////////////////////
2053 @Override
2054 public TableDescriptor getTableDescriptor() {
2055 return this.htableDescriptor;
2058 public void setTableDescriptor(TableDescriptor desc) {
2059 htableDescriptor = desc;
2062 /** @return WAL in use for this region */
2063 public WAL getWAL() {
2064 return this.wal;
2067 public BlockCache getBlockCache() {
2068 return this.blockCache;
2072 * Only used for unit test which doesn't start region server.
2074 public void setBlockCache(BlockCache blockCache) {
2075 this.blockCache = blockCache;
2078 public MobFileCache getMobFileCache() {
2079 return this.mobFileCache;
2083 * Only used for unit test which doesn't start region server.
2085 public void setMobFileCache(MobFileCache mobFileCache) {
2086 this.mobFileCache = mobFileCache;
2090 * @return split policy for this region.
2092 RegionSplitPolicy getSplitPolicy() {
2093 return this.splitPolicy;
2097 * A split takes the config from the parent region & passes it to the daughter
2098 * region's constructor. If 'conf' was passed, you would end up using the HTD
2099 * of the parent region in addition to the new daughter HTD. Pass 'baseConf'
2100 * to the daughter regions to avoid this tricky dedupe problem.
2101 * @return Configuration object
2103 Configuration getBaseConf() {
2104 return this.baseConf;
2107 /** @return {@link FileSystem} being used by this region */
2108 public FileSystem getFilesystem() {
2109 return fs.getFileSystem();
2112 /** @return the {@link HRegionFileSystem} used by this region */
2113 public HRegionFileSystem getRegionFileSystem() {
2114 return this.fs;
2117 /** @return the WAL {@link HRegionFileSystem} used by this region */
2118 HRegionWALFileSystem getRegionWALFileSystem() throws IOException {
2119 return new HRegionWALFileSystem(conf, getWalFileSystem(),
2120 CommonFSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
2123 /** @return the WAL {@link FileSystem} being used by this region */
2124 FileSystem getWalFileSystem() throws IOException {
2125 if (walFS == null) {
2126 walFS = CommonFSUtils.getWALFileSystem(conf);
2128 return walFS;
2132 * @return the Region directory under WALRootDirectory
2133 * @throws IOException if there is an error getting WALRootDir
2135 public Path getWALRegionDir() throws IOException {
2136 if (regionDir == null) {
2137 regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
2138 getRegionInfo().getEncodedName());
2140 return regionDir;
2143 @Override
2144 public long getEarliestFlushTimeForAllStores() {
2145 return Collections.min(lastStoreFlushTimeMap.values());
2148 @Override
2149 public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
2150 long result = Long.MAX_VALUE;
2151 for (HStore store : stores.values()) {
2152 Collection<HStoreFile> storeFiles = store.getStorefiles();
2153 if (storeFiles == null) {
2154 continue;
2156 for (HStoreFile file : storeFiles) {
2157 StoreFileReader sfReader = file.getReader();
2158 if (sfReader == null) {
2159 continue;
2161 HFile.Reader reader = sfReader.getHFileReader();
2162 if (reader == null) {
2163 continue;
2165 if (majorCompactionOnly) {
2166 byte[] val = reader.getHFileInfo().get(MAJOR_COMPACTION_KEY);
2167 if (val == null || !Bytes.toBoolean(val)) {
2168 continue;
2171 result = Math.min(result, reader.getFileContext().getFileCreateTime());
2174 return result == Long.MAX_VALUE ? 0 : result;
2177 RegionLoad.Builder setCompleteSequenceId(RegionLoad.Builder regionLoadBldr) {
2178 long lastFlushOpSeqIdLocal = this.lastFlushOpSeqId;
2179 byte[] encodedRegionName = this.getRegionInfo().getEncodedNameAsBytes();
2180 regionLoadBldr.clearStoreCompleteSequenceId();
2181 for (byte[] familyName : this.stores.keySet()) {
2182 long earliest = this.wal.getEarliestMemStoreSeqNum(encodedRegionName, familyName);
2183 // Subtract - 1 to go earlier than the current oldest, unflushed edit in memstore; this will
2184 // give us a sequence id that is for sure flushed. We want edit replay to start after this
2185 // sequence id in this region. If NO_SEQNUM, use the regions maximum flush id.
2186 long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1;
2187 regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId.newBuilder()
2188 .setFamilyName(UnsafeByteOperations.unsafeWrap(familyName)).setSequenceId(csid).build());
2190 return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId());
2193 //////////////////////////////////////////////////////////////////////////////
2194 // HRegion maintenance.
2196 // These methods are meant to be called periodically by the HRegionServer for
2197 // upkeep.
2198 //////////////////////////////////////////////////////////////////////////////
2200 * Do preparation for pending compaction.
2201 * @throws IOException
2203 protected void doRegionCompactionPrep() throws IOException {
2207 * Synchronously compact all stores in the region.
2208 * <p>This operation could block for a long time, so don't call it from a
2209 * time-sensitive thread.
2210 * <p>Note that no locks are taken to prevent possible conflicts between
2211 * compaction and splitting activities. The regionserver does not normally compact
2212 * and split in parallel. However by calling this method you may introduce
2213 * unexpected and unhandled concurrency. Don't do this unless you know what
2214 * you are doing.
2216 * @param majorCompaction True to force a major compaction regardless of thresholds
2217 * @throws IOException
2219 public void compact(boolean majorCompaction) throws IOException {
2220 if (majorCompaction) {
2221 stores.values().forEach(HStore::triggerMajorCompaction);
2223 for (HStore s : stores.values()) {
2224 Optional<CompactionContext> compaction = s.requestCompaction();
2225 if (compaction.isPresent()) {
2226 ThroughputController controller = null;
2227 if (rsServices != null) {
2228 controller = CompactionThroughputControllerFactory.create(rsServices, conf);
2230 if (controller == null) {
2231 controller = NoLimitThroughputController.INSTANCE;
2233 compact(compaction.get(), s, controller, null);
2239 * This is a helper function that compact all the stores synchronously.
2240 * <p>
2241 * It is used by utilities and testing
2243 public void compactStores() throws IOException {
2244 for (HStore s : stores.values()) {
2245 Optional<CompactionContext> compaction = s.requestCompaction();
2246 if (compaction.isPresent()) {
2247 compact(compaction.get(), s, NoLimitThroughputController.INSTANCE, null);
2253 * This is a helper function that compact the given store.
2254 * <p>
2255 * It is used by utilities and testing
2257 void compactStore(byte[] family, ThroughputController throughputController) throws IOException {
2258 HStore s = getStore(family);
2259 Optional<CompactionContext> compaction = s.requestCompaction();
2260 if (compaction.isPresent()) {
2261 compact(compaction.get(), s, throughputController, null);
2266 * Called by compaction thread and after region is opened to compact the
2267 * HStores if necessary.
2269 * <p>This operation could block for a long time, so don't call it from a
2270 * time-sensitive thread.
2272 * Note that no locking is necessary at this level because compaction only
2273 * conflicts with a region split, and that cannot happen because the region
2274 * server does them sequentially and not in parallel.
2276 * @param compaction Compaction details, obtained by requestCompaction()
2277 * @param throughputController
2278 * @return whether the compaction completed
2280 public boolean compact(CompactionContext compaction, HStore store,
2281 ThroughputController throughputController) throws IOException {
2282 return compact(compaction, store, throughputController, null);
2285 private boolean shouldForbidMajorCompaction() {
2286 if (rsServices != null && rsServices.getReplicationSourceService() != null) {
2287 return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
2288 .checkState(getRegionInfo().getTable(), ForbidMajorCompactionChecker.get());
2290 return false;
2294 * We are trying to remove / relax the region read lock for compaction.
2295 * Let's see what are the potential race conditions among the operations (user scan,
2296 * region split, region close and region bulk load).
2298 * user scan ---> region read lock
2299 * region split --> region close first --> region write lock
2300 * region close --> region write lock
2301 * region bulk load --> region write lock
2303 * read lock is compatible with read lock. ---> no problem with user scan/read
2304 * region bulk load does not cause problem for compaction (no consistency problem, store lock
2305 * will help the store file accounting).
2306 * They can run almost concurrently at the region level.
2308 * The only remaining race condition is between the region close and compaction.
2309 * So we will evaluate, below, how region close intervenes with compaction if compaction does
2310 * not acquire region read lock.
2312 * Here are the steps for compaction:
2313 * 1. obtain list of StoreFile's
2314 * 2. create StoreFileScanner's based on list from #1
2315 * 3. perform compaction and save resulting files under tmp dir
2316 * 4. swap in compacted files
2318 * #1 is guarded by store lock. This patch does not change this --> no worse or better
2319 * For #2, we obtain smallest read point (for region) across all the Scanners (for both default
2320 * compactor and stripe compactor).
2321 * The read points are for user scans. Region keeps the read points for all currently open
2322 * user scanners.
2323 * Compaction needs to know the smallest read point so that during re-write of the hfiles,
2324 * it can remove the mvcc points for the cells if their mvccs are older than the smallest
2325 * since they are not needed anymore.
2326 * This will not conflict with compaction.
2327 * For #3, it can be performed in parallel to other operations.
2328 * For #4 bulk load and compaction don't conflict with each other on the region level
2329 * (for multi-family atomicy).
2330 * Region close and compaction are guarded pretty well by the 'writestate'.
2331 * In HRegion#doClose(), we have :
2332 * synchronized (writestate) {
2333 * // Disable compacting and flushing by background threads for this
2334 * // region.
2335 * canFlush = !writestate.readOnly;
2336 * writestate.writesEnabled = false;
2337 * LOG.debug("Closing " + this + ": disabling compactions & flushes");
2338 * waitForFlushesAndCompactions();
2340 * waitForFlushesAndCompactions() would wait for writestate.compacting to come down to 0.
2341 * and in HRegion.compact()
2342 * try {
2343 * synchronized (writestate) {
2344 * if (writestate.writesEnabled) {
2345 * wasStateSet = true;
2346 * ++writestate.compacting;
2347 * } else {
2348 * String msg = "NOT compacting region " + this + ". Writes disabled.";
2349 * LOG.info(msg);
2350 * status.abort(msg);
2351 * return false;
2354 * Also in compactor.performCompaction():
2355 * check periodically to see if a system stop is requested
2356 * if (closeChecker != null && closeChecker.isTimeLimit(store, now)) {
2357 * progress.cancel();
2358 * return false;
2360 * if (closeChecker != null && closeChecker.isSizeLimit(store, len)) {
2361 * progress.cancel();
2362 * return false;
2365 public boolean compact(CompactionContext compaction, HStore store,
2366 ThroughputController throughputController, User user) throws IOException {
2367 assert compaction != null && compaction.hasSelection();
2368 assert !compaction.getRequest().getFiles().isEmpty();
2369 if (this.closing.get() || this.closed.get()) {
2370 LOG.debug("Skipping compaction on " + this + " because closing/closed");
2371 store.cancelRequestedCompaction(compaction);
2372 return false;
2375 if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) {
2376 LOG.warn("Skipping major compaction on " + this
2377 + " because this cluster is transiting sync replication state"
2378 + " from STANDBY to DOWNGRADE_ACTIVE");
2379 store.cancelRequestedCompaction(compaction);
2380 return false;
2383 MonitoredTask status = null;
2384 boolean requestNeedsCancellation = true;
2385 try {
2386 byte[] cf = Bytes.toBytes(store.getColumnFamilyName());
2387 if (stores.get(cf) != store) {
2388 LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this
2389 + " has been re-instantiated, cancel this compaction request. "
2390 + " It may be caused by the roll back of split transaction");
2391 return false;
2394 status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this);
2395 status.enableStatusJournal(false);
2396 if (this.closed.get()) {
2397 String msg = "Skipping compaction on " + this + " because closed";
2398 LOG.debug(msg);
2399 status.abort(msg);
2400 return false;
2402 boolean wasStateSet = false;
2403 try {
2404 synchronized (writestate) {
2405 if (writestate.writesEnabled) {
2406 wasStateSet = true;
2407 writestate.compacting.incrementAndGet();
2408 } else {
2409 String msg = "NOT compacting region " + this + ". Writes disabled.";
2410 LOG.info(msg);
2411 status.abort(msg);
2412 return false;
2415 LOG.info("Starting compaction of {} in {}{}", store, this,
2416 (compaction.getRequest().isOffPeak()?" as an off-peak compaction":""));
2417 doRegionCompactionPrep();
2418 try {
2419 status.setStatus("Compacting store " + store);
2420 // We no longer need to cancel the request on the way out of this
2421 // method because Store#compact will clean up unconditionally
2422 requestNeedsCancellation = false;
2423 store.compact(compaction, throughputController, user);
2424 } catch (InterruptedIOException iioe) {
2425 String msg = "region " + this + " compaction interrupted";
2426 LOG.info(msg, iioe);
2427 status.abort(msg);
2428 return false;
2430 } finally {
2431 if (wasStateSet) {
2432 synchronized (writestate) {
2433 writestate.compacting.decrementAndGet();
2434 if (writestate.compacting.get() <= 0) {
2435 writestate.notifyAll();
2440 status.markComplete("Compaction complete");
2441 return true;
2442 } finally {
2443 if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction);
2444 if (status != null) {
2445 LOG.debug("Compaction status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2446 status.prettyPrintJournal());
2447 status.cleanup();
2453 * Flush the cache.
2455 * <p>When this method is called the cache will be flushed unless:
2456 * <ol>
2457 * <li>the cache is empty</li>
2458 * <li>the region is closed.</li>
2459 * <li>a flush is already in progress</li>
2460 * <li>writes are disabled</li>
2461 * </ol>
2463 * <p>This method may block for some time, so it should not be called from a
2464 * time-sensitive thread.
2465 * @param flushAllStores whether we want to force a flush of all stores
2466 * @return FlushResult indicating whether the flush was successful or not and if
2467 * the region needs compacting
2469 * @throws IOException general io exceptions
2470 * because a snapshot was not properly persisted.
2472 // TODO HBASE-18905. We might have to expose a requestFlush API for CPs
2473 public FlushResult flush(boolean flushAllStores) throws IOException {
2474 return flushcache(flushAllStores, false, FlushLifeCycleTracker.DUMMY);
2477 public interface FlushResult {
2478 enum Result {
2479 FLUSHED_NO_COMPACTION_NEEDED,
2480 FLUSHED_COMPACTION_NEEDED,
2481 // Special case where a flush didn't run because there's nothing in the memstores. Used when
2482 // bulk loading to know when we can still load even if a flush didn't happen.
2483 CANNOT_FLUSH_MEMSTORE_EMPTY,
2484 CANNOT_FLUSH
2487 /** @return the detailed result code */
2488 Result getResult();
2490 /** @return true if the memstores were flushed, else false */
2491 boolean isFlushSucceeded();
2493 /** @return True if the flush requested a compaction, else false */
2494 boolean isCompactionNeeded();
2497 public FlushResultImpl flushcache(boolean flushAllStores, boolean writeFlushRequestWalMarker,
2498 FlushLifeCycleTracker tracker) throws IOException {
2499 List<byte[]> families = null;
2500 if (flushAllStores) {
2501 families = new ArrayList<>();
2502 families.addAll(this.getTableDescriptor().getColumnFamilyNames());
2504 return this.flushcache(families, writeFlushRequestWalMarker, tracker);
2508 * Flush the cache.
2510 * When this method is called the cache will be flushed unless:
2511 * <ol>
2512 * <li>the cache is empty</li>
2513 * <li>the region is closed.</li>
2514 * <li>a flush is already in progress</li>
2515 * <li>writes are disabled</li>
2516 * </ol>
2518 * <p>This method may block for some time, so it should not be called from a
2519 * time-sensitive thread.
2520 * @param families stores of region to flush.
2521 * @param writeFlushRequestWalMarker whether to write the flush request marker to WAL
2522 * @param tracker used to track the life cycle of this flush
2523 * @return whether the flush is success and whether the region needs compacting
2525 * @throws IOException general io exceptions
2526 * @throws DroppedSnapshotException Thrown when replay of wal is required
2527 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
2528 * caller MUST abort after this.
2530 public FlushResultImpl flushcache(List<byte[]> families,
2531 boolean writeFlushRequestWalMarker, FlushLifeCycleTracker tracker) throws IOException {
2532 // fail-fast instead of waiting on the lock
2533 if (this.closing.get()) {
2534 String msg = "Skipping flush on " + this + " because closing";
2535 LOG.debug(msg);
2536 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2538 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
2539 status.enableStatusJournal(false);
2540 status.setStatus("Acquiring readlock on region");
2541 // block waiting for the lock for flushing cache
2542 lock.readLock().lock();
2543 boolean flushed = true;
2544 try {
2545 if (this.closed.get()) {
2546 String msg = "Skipping flush on " + this + " because closed";
2547 LOG.debug(msg);
2548 status.abort(msg);
2549 flushed = false;
2550 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2552 if (coprocessorHost != null) {
2553 status.setStatus("Running coprocessor pre-flush hooks");
2554 coprocessorHost.preFlush(tracker);
2556 // TODO: this should be managed within memstore with the snapshot, updated only after flush
2557 // successful
2558 if (numMutationsWithoutWAL.sum() > 0) {
2559 numMutationsWithoutWAL.reset();
2560 dataInMemoryWithoutWAL.reset();
2562 synchronized (writestate) {
2563 if (!writestate.flushing && writestate.writesEnabled) {
2564 this.writestate.flushing = true;
2565 } else {
2566 String msg = "NOT flushing " + this + " as " + (writestate.flushing ? "already flushing"
2567 : "writes are not enabled");
2568 LOG.debug(msg);
2569 status.abort(msg);
2570 flushed = false;
2571 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2575 try {
2576 // The reason that we do not always use flushPolicy is, when the flush is
2577 // caused by logRoller, we should select stores which must be flushed
2578 // rather than could be flushed.
2579 Collection<HStore> specificStoresToFlush = null;
2580 if (families != null) {
2581 specificStoresToFlush = getSpecificStores(families);
2582 } else {
2583 specificStoresToFlush = flushPolicy.selectStoresToFlush();
2585 FlushResultImpl fs =
2586 internalFlushcache(specificStoresToFlush, status, writeFlushRequestWalMarker, tracker);
2588 if (coprocessorHost != null) {
2589 status.setStatus("Running post-flush coprocessor hooks");
2590 coprocessorHost.postFlush(tracker);
2593 if(fs.isFlushSucceeded()) {
2594 flushesQueued.reset();
2597 status.markComplete("Flush successful " + fs.toString());
2598 return fs;
2599 } finally {
2600 synchronized (writestate) {
2601 writestate.flushing = false;
2602 this.writestate.flushRequested = false;
2603 writestate.notifyAll();
2606 } finally {
2607 lock.readLock().unlock();
2608 if (flushed) {
2609 // Don't log this journal stuff if no flush -- confusing.
2610 LOG.debug("Flush status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2611 status.prettyPrintJournal());
2613 status.cleanup();
2618 * get stores which matches the specified families
2620 * @return the stores need to be flushed.
2622 private Collection<HStore> getSpecificStores(List<byte[]> families) {
2623 Collection<HStore> specificStoresToFlush = new ArrayList<>();
2624 for (byte[] family : families) {
2625 specificStoresToFlush.add(stores.get(family));
2627 return specificStoresToFlush;
2631 * Should the store be flushed because it is old enough.
2632 * <p>
2633 * Every FlushPolicy should call this to determine whether a store is old enough to flush (except
2634 * that you always flush all stores). Otherwise the method will always
2635 * returns true which will make a lot of flush requests.
2637 boolean shouldFlushStore(HStore store) {
2638 long earliest = this.wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
2639 store.getColumnFamilyDescriptor().getName()) - 1;
2640 if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
2641 if (LOG.isDebugEnabled()) {
2642 LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " +
2643 getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest +
2644 " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint());
2646 return true;
2648 if (this.flushCheckInterval <= 0) {
2649 return false;
2651 long now = EnvironmentEdgeManager.currentTime();
2652 if (store.timeOfOldestEdit() < now - this.flushCheckInterval) {
2653 if (LOG.isDebugEnabled()) {
2654 LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of " +
2655 getRegionInfo().getEncodedName() + " because time of oldest edit=" +
2656 store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now);
2658 return true;
2660 return false;
2664 * Should the memstore be flushed now
2666 boolean shouldFlush(final StringBuilder whyFlush) {
2667 whyFlush.setLength(0);
2668 // This is a rough measure.
2669 if (this.maxFlushedSeqId > 0
2670 && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())) {
2671 whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush");
2672 return true;
2674 long modifiedFlushCheckInterval = flushCheckInterval;
2675 if (getRegionInfo().getTable().isSystemTable() &&
2676 getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
2677 modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
2679 if (modifiedFlushCheckInterval <= 0) { //disabled
2680 return false;
2682 long now = EnvironmentEdgeManager.currentTime();
2683 //if we flushed in the recent past, we don't need to do again now
2684 if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) {
2685 return false;
2687 //since we didn't flush in the recent past, flush now if certain conditions
2688 //are met. Return true on first such memstore hit.
2689 for (HStore s : stores.values()) {
2690 if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
2691 // we have an old enough edit in the memstore, flush
2692 whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
2693 return true;
2696 return false;
2700 * Flushing all stores.
2701 * @see #internalFlushcache(Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2703 private FlushResult internalFlushcache(MonitoredTask status) throws IOException {
2704 return internalFlushcache(stores.values(), status, false, FlushLifeCycleTracker.DUMMY);
2708 * Flushing given stores.
2709 * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2711 private FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status,
2712 boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException {
2713 return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush, status,
2714 writeFlushWalMarker, tracker);
2718 * Flush the memstore. Flushing the memstore is a little tricky. We have a lot of updates in the
2719 * memstore, all of which have also been written to the wal. We need to write those updates in the
2720 * memstore out to disk, while being able to process reads/writes as much as possible during the
2721 * flush operation.
2722 * <p>
2723 * This method may block for some time. Every time you call it, we up the regions sequence id even
2724 * if we don't flush; i.e. the returned region id will be at least one larger than the last edit
2725 * applied to this region. The returned id does not refer to an actual edit. The returned id can
2726 * be used for say installing a bulk loaded file just ahead of the last hfile that was the result
2727 * of this flush, etc.
2728 * @param wal Null if we're NOT to go via wal.
2729 * @param myseqid The seqid to use if <code>wal</code> is null writing out flush file.
2730 * @param storesToFlush The list of stores to flush.
2731 * @return object describing the flush's state
2732 * @throws IOException general io exceptions
2733 * @throws DroppedSnapshotException Thrown when replay of WAL is required.
2735 protected FlushResultImpl internalFlushcache(WAL wal, long myseqid,
2736 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2737 FlushLifeCycleTracker tracker) throws IOException {
2738 PrepareFlushResult result =
2739 internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker, tracker);
2740 if (result.result == null) {
2741 return internalFlushCacheAndCommit(wal, status, result, storesToFlush);
2742 } else {
2743 return result.result; // early exit due to failure from prepare stage
2747 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
2748 justification="FindBugs seems confused about trxId")
2749 protected PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid,
2750 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2751 FlushLifeCycleTracker tracker) throws IOException {
2752 if (this.rsServices != null && this.rsServices.isAborted()) {
2753 // Don't flush when server aborting, it's unsafe
2754 throw new IOException("Aborting flush because server is aborted...");
2756 final long startTime = EnvironmentEdgeManager.currentTime();
2757 // If nothing to flush, return, but return with a valid unused sequenceId.
2758 // Its needed by bulk upload IIRC. It flushes until no edits in memory so it can insert a
2759 // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs
2760 // to no other that it can use to associate with the bulk load. Hence this little dance below
2761 // to go get one.
2762 if (this.memStoreSizing.getDataSize() <= 0) {
2763 // Take an update lock so no edits can come into memory just yet.
2764 this.updatesLock.writeLock().lock();
2765 WriteEntry writeEntry = null;
2766 try {
2767 if (this.memStoreSizing.getDataSize() <= 0) {
2768 // Presume that if there are still no edits in the memstore, then there are no edits for
2769 // this region out in the WAL subsystem so no need to do any trickery clearing out
2770 // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
2771 // sure just beyond the last appended region edit and not associated with any edit
2772 // (useful as marker when bulk loading, etc.).
2773 if (wal != null) {
2774 writeEntry = mvcc.begin();
2775 long flushOpSeqId = writeEntry.getWriteNumber();
2776 FlushResultImpl flushResult =
2777 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId,
2778 "Nothing to flush", writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
2779 mvcc.completeAndWait(writeEntry);
2780 // Set to null so we don't complete it again down in finally block.
2781 writeEntry = null;
2782 return new PrepareFlushResult(flushResult, myseqid);
2783 } else {
2784 return new PrepareFlushResult(new FlushResultImpl(
2785 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", false), myseqid);
2788 } finally {
2789 if (writeEntry != null) {
2790 // If writeEntry is non-null, this operation failed; the mvcc transaction failed...
2791 // but complete it anyways so it doesn't block the mvcc queue.
2792 mvcc.complete(writeEntry);
2794 this.updatesLock.writeLock().unlock();
2797 logFatLineOnFlush(storesToFlush, myseqid);
2798 // Stop updates while we snapshot the memstore of all of these regions' stores. We only have
2799 // to do this for a moment. It is quick. We also set the memstore size to zero here before we
2800 // allow updates again so its value will represent the size of the updates received
2801 // during flush
2803 // We have to take an update lock during snapshot, or else a write could end up in both snapshot
2804 // and memstore (makes it difficult to do atomic rows then)
2805 status.setStatus("Obtaining lock to block concurrent updates");
2806 // block waiting for the lock for internal flush
2807 this.updatesLock.writeLock().lock();
2808 status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName());
2809 MemStoreSizing totalSizeOfFlushableStores = new NonThreadSafeMemStoreSizing();
2811 Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>();
2812 for (HStore store : storesToFlush) {
2813 flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(),
2814 store.preFlushSeqIDEstimation());
2817 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2818 TreeMap<byte[], List<Path>> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2819 TreeMap<byte[], MemStoreSize> storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2820 // The sequence id of this flush operation which is used to log FlushMarker and pass to
2821 // createFlushContext to use as the store file's sequence id. It can be in advance of edits
2822 // still in the memstore, edits that are in other column families yet to be flushed.
2823 long flushOpSeqId = HConstants.NO_SEQNUM;
2824 // The max flushed sequence id after this flush operation completes. All edits in memstore
2825 // will be in advance of this sequence id.
2826 long flushedSeqId = HConstants.NO_SEQNUM;
2827 byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes();
2828 try {
2829 if (wal != null) {
2830 Long earliestUnflushedSequenceIdForTheRegion =
2831 wal.startCacheFlush(encodedRegionName, flushedFamilyNamesToSeq);
2832 if (earliestUnflushedSequenceIdForTheRegion == null) {
2833 // This should never happen. This is how startCacheFlush signals flush cannot proceed.
2834 String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
2835 status.setStatus(msg);
2836 return new PrepareFlushResult(
2837 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false),
2838 myseqid);
2840 flushOpSeqId = getNextSequenceId(wal);
2841 // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
2842 flushedSeqId =
2843 earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM?
2844 flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
2845 } else {
2846 // use the provided sequence Id as WAL is not being used for this flush.
2847 flushedSeqId = flushOpSeqId = myseqid;
2850 for (HStore s : storesToFlush) {
2851 storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(),
2852 s.createFlushContext(flushOpSeqId, tracker));
2853 // for writing stores to WAL
2854 committedFiles.put(s.getColumnFamilyDescriptor().getName(), null);
2857 // write the snapshot start to WAL
2858 if (wal != null && !writestate.readOnly) {
2859 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
2860 getRegionInfo(), flushOpSeqId, committedFiles);
2861 // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH
2862 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false,
2863 mvcc, null);
2866 // Prepare flush (take a snapshot)
2867 storeFlushCtxs.forEach((name, flush) -> {
2868 MemStoreSize snapshotSize = flush.prepare();
2869 totalSizeOfFlushableStores.incMemStoreSize(snapshotSize);
2870 storeFlushableSize.put(name, snapshotSize);
2872 } catch (IOException ex) {
2873 doAbortFlushToWAL(wal, flushOpSeqId, committedFiles);
2874 throw ex;
2875 } finally {
2876 this.updatesLock.writeLock().unlock();
2878 String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " +
2879 "flushsize=" + totalSizeOfFlushableStores;
2880 status.setStatus(s);
2881 doSyncOfUnflushedWALChanges(wal, getRegionInfo());
2882 return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
2883 flushOpSeqId, flushedSeqId, totalSizeOfFlushableStores);
2887 * Utility method broken out of internalPrepareFlushCache so that method is smaller.
2889 private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) {
2890 if (!LOG.isInfoEnabled()) {
2891 return;
2893 // Log a fat line detailing what is being flushed.
2894 StringBuilder perCfExtras = null;
2895 if (!isAllFamilies(storesToFlush)) {
2896 perCfExtras = new StringBuilder();
2897 for (HStore store: storesToFlush) {
2898 MemStoreSize mss = store.getFlushableSize();
2899 perCfExtras.append("; ").append(store.getColumnFamilyName());
2900 perCfExtras.append("={dataSize=")
2901 .append(StringUtils.byteDesc(mss.getDataSize()));
2902 perCfExtras.append(", heapSize=")
2903 .append(StringUtils.byteDesc(mss.getHeapSize()));
2904 perCfExtras.append(", offHeapSize=")
2905 .append(StringUtils.byteDesc(mss.getOffHeapSize()));
2906 perCfExtras.append("}");
2909 MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
2910 LOG.info("Flushing " + this.getRegionInfo().getEncodedName() + " " +
2911 storesToFlush.size() + "/" + stores.size() + " column families," +
2912 " dataSize=" + StringUtils.byteDesc(mss.getDataSize()) +
2913 " heapSize=" + StringUtils.byteDesc(mss.getHeapSize()) +
2914 ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
2915 ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId));
2918 private void doAbortFlushToWAL(final WAL wal, final long flushOpSeqId,
2919 final Map<byte[], List<Path>> committedFiles) {
2920 if (wal == null) return;
2921 try {
2922 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2923 getRegionInfo(), flushOpSeqId, committedFiles);
2924 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false, mvcc,
2925 null);
2926 } catch (Throwable t) {
2927 LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL: {} in "
2928 + " region {}", StringUtils.stringifyException(t), this);
2929 // ignore this since we will be aborting the RS with DSE.
2931 // we have called wal.startCacheFlush(), now we have to abort it
2932 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2936 * Sync unflushed WAL changes. See HBASE-8208 for details
2938 private static void doSyncOfUnflushedWALChanges(final WAL wal, final RegionInfo hri)
2939 throws IOException {
2940 if (wal == null) {
2941 return;
2943 try {
2944 wal.sync(); // ensure that flush marker is sync'ed
2945 } catch (IOException ioe) {
2946 wal.abortCacheFlush(hri.getEncodedNameAsBytes());
2947 throw ioe;
2952 * @return True if passed Set is all families in the region.
2954 private boolean isAllFamilies(Collection<HStore> families) {
2955 return families == null || this.stores.size() == families.size();
2959 * Writes a marker to WAL indicating a flush is requested but cannot be complete due to various
2960 * reasons. Ignores exceptions from WAL. Returns whether the write succeeded.
2961 * @param wal
2962 * @return whether WAL write was successful
2964 private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker) {
2965 if (writeFlushWalMarker && wal != null && !writestate.readOnly) {
2966 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH,
2967 getRegionInfo(), -1, new TreeMap<>(Bytes.BYTES_COMPARATOR));
2968 try {
2969 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc,
2970 null);
2971 return true;
2972 } catch (IOException e) {
2973 LOG.warn(getRegionInfo().getEncodedName() + " : "
2974 + "Received exception while trying to write the flush request to wal", e);
2977 return false;
2980 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
2981 justification="Intentional; notify is about completed flush")
2982 FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status,
2983 PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException {
2984 // prepare flush context is carried via PrepareFlushResult
2985 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs;
2986 TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles;
2987 long startTime = prepareResult.startTime;
2988 long flushOpSeqId = prepareResult.flushOpSeqId;
2989 long flushedSeqId = prepareResult.flushedSeqId;
2991 String s = "Flushing stores of " + this;
2992 status.setStatus(s);
2993 if (LOG.isTraceEnabled()) LOG.trace(s);
2995 // Any failure from here on out will be catastrophic requiring server
2996 // restart so wal content can be replayed and put back into the memstore.
2997 // Otherwise, the snapshot content while backed up in the wal, it will not
2998 // be part of the current running servers state.
2999 boolean compactionRequested = false;
3000 long flushedOutputFileSize = 0;
3001 try {
3002 // A. Flush memstore to all the HStores.
3003 // Keep running vector of all store files that includes both old and the
3004 // just-made new flush store file. The new flushed file is still in the
3005 // tmp directory.
3007 for (StoreFlushContext flush : storeFlushCtxs.values()) {
3008 flush.flushCache(status);
3011 // Switch snapshot (in memstore) -> new hfile (thus causing
3012 // all the store scanners to reset/reseek).
3013 for (Map.Entry<byte[], StoreFlushContext> flushEntry : storeFlushCtxs.entrySet()) {
3014 StoreFlushContext sfc = flushEntry.getValue();
3015 boolean needsCompaction = sfc.commit(status);
3016 if (needsCompaction) {
3017 compactionRequested = true;
3019 byte[] storeName = flushEntry.getKey();
3020 List<Path> storeCommittedFiles = sfc.getCommittedFiles();
3021 committedFiles.put(storeName, storeCommittedFiles);
3022 // Flush committed no files, indicating flush is empty or flush was canceled
3023 if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
3024 MemStoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName);
3025 prepareResult.totalFlushableSize.decMemStoreSize(storeFlushableSize);
3027 flushedOutputFileSize += sfc.getOutputFileSize();
3029 storeFlushCtxs.clear();
3031 // Set down the memstore size by amount of flush.
3032 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3033 this.decrMemStoreSize(mss);
3035 // Increase the size of this Region for the purposes of quota. Noop if quotas are disabled.
3036 // During startup, quota manager may not be initialized yet.
3037 if (rsServices != null) {
3038 RegionServerSpaceQuotaManager quotaManager = rsServices.getRegionServerSpaceQuotaManager();
3039 if (quotaManager != null) {
3040 quotaManager.getRegionSizeStore().incrementRegionSize(
3041 this.getRegionInfo(), flushedOutputFileSize);
3045 if (wal != null) {
3046 // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
3047 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
3048 getRegionInfo(), flushOpSeqId, committedFiles);
3049 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc,
3050 regionReplicationSink.orElse(null));
3052 } catch (Throwable t) {
3053 // An exception here means that the snapshot was not persisted.
3054 // The wal needs to be replayed so its content is restored to memstore.
3055 // Currently, only a server restart will do this.
3056 // We used to only catch IOEs but its possible that we'd get other
3057 // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
3058 // all and sundry.
3059 if (wal != null) {
3060 try {
3061 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
3062 getRegionInfo(), flushOpSeqId, committedFiles);
3063 WALUtil.writeFlushMarker(wal, this.replicationScope, getRegionInfo(), desc, false, mvcc,
3064 null);
3065 } catch (Throwable ex) {
3066 LOG.warn(getRegionInfo().getEncodedName() + " : "
3067 + "failed writing ABORT_FLUSH marker to WAL", ex);
3068 // ignore this since we will be aborting the RS with DSE.
3070 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
3072 DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
3073 Bytes.toStringBinary(getRegionInfo().getRegionName()), t);
3074 status.abort("Flush failed: " + StringUtils.stringifyException(t));
3076 // Callers for flushcache() should catch DroppedSnapshotException and abort the region server.
3077 // However, since we may have the region read lock, we cannot call close(true) here since
3078 // we cannot promote to a write lock. Instead we are setting closing so that all other region
3079 // operations except for close will be rejected.
3080 this.closing.set(true);
3082 if (rsServices != null) {
3083 // This is a safeguard against the case where the caller fails to explicitly handle aborting
3084 rsServices.abort("Replay of WAL required. Forcing server shutdown", dse);
3087 throw dse;
3090 // If we get to here, the HStores have been written.
3091 if (wal != null) {
3092 wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes(), flushedSeqId);
3095 // Record latest flush time
3096 for (HStore store: storesToFlush) {
3097 this.lastStoreFlushTimeMap.put(store, startTime);
3100 this.maxFlushedSeqId = flushedSeqId;
3101 this.lastFlushOpSeqId = flushOpSeqId;
3103 // C. Finally notify anyone waiting on memstore to clear:
3104 // e.g. checkResources().
3105 synchronized (this) {
3106 notifyAll(); // FindBugs NN_NAKED_NOTIFY
3109 long time = EnvironmentEdgeManager.currentTime() - startTime;
3110 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3111 long memstoresize = this.memStoreSizing.getMemStoreSize().getDataSize();
3112 String msg = "Finished flush of"
3113 + " dataSize ~" + StringUtils.byteDesc(mss.getDataSize()) + "/" + mss.getDataSize()
3114 + ", heapSize ~" + StringUtils.byteDesc(mss.getHeapSize()) + "/" + mss.getHeapSize()
3115 + ", currentSize=" + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
3116 + " for " + this.getRegionInfo().getEncodedName() + " in " + time + "ms, sequenceid="
3117 + flushOpSeqId + ", compaction requested=" + compactionRequested
3118 + ((wal == null) ? "; wal=null" : "");
3119 LOG.info(msg);
3120 status.setStatus(msg);
3122 if (rsServices != null && rsServices.getMetrics() != null) {
3123 rsServices.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(),
3124 time,
3125 mss.getDataSize(), flushedOutputFileSize);
3128 return new FlushResultImpl(compactionRequested ?
3129 FlushResult.Result.FLUSHED_COMPACTION_NEEDED :
3130 FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId);
3134 * Method to safely get the next sequence number.
3135 * @return Next sequence number unassociated with any actual edit.
3136 * @throws IOException
3138 protected long getNextSequenceId(final WAL wal) throws IOException {
3139 WriteEntry we = mvcc.begin();
3140 mvcc.completeAndWait(we);
3141 return we.getWriteNumber();
3144 //////////////////////////////////////////////////////////////////////////////
3145 // get() methods for client use.
3146 //////////////////////////////////////////////////////////////////////////////
3148 @Override
3149 public RegionScannerImpl getScanner(Scan scan) throws IOException {
3150 return getScanner(scan, null);
3153 @Override
3154 public RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners)
3155 throws IOException {
3156 return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE);
3159 private RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
3160 long nonceGroup, long nonce) throws IOException {
3161 return TraceUtil.trace(() -> {
3162 startRegionOperation(Operation.SCAN);
3163 try {
3164 // Verify families are all valid
3165 if (!scan.hasFamilies()) {
3166 // Adding all families to scanner
3167 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
3168 scan.addFamily(family);
3170 } else {
3171 for (byte[] family : scan.getFamilyMap().keySet()) {
3172 checkFamily(family);
3175 return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
3176 } finally {
3177 closeRegionOperation(Operation.SCAN);
3179 }, () -> createRegionSpan("Region.getScanner"));
3182 protected RegionScannerImpl instantiateRegionScanner(Scan scan,
3183 List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException {
3184 if (scan.isReversed()) {
3185 if (scan.getFilter() != null) {
3186 scan.getFilter().setReversed(true);
3188 return new ReversedRegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3190 return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3194 * Prepare a delete for a row mutation processor
3195 * @param delete The passed delete is modified by this method. WARNING!
3197 private void prepareDelete(Delete delete) throws IOException {
3198 // Check to see if this is a deleteRow insert
3199 if(delete.getFamilyCellMap().isEmpty()){
3200 for(byte [] family : this.htableDescriptor.getColumnFamilyNames()){
3201 // Don't eat the timestamp
3202 delete.addFamily(family, delete.getTimestamp());
3204 } else {
3205 for(byte [] family : delete.getFamilyCellMap().keySet()) {
3206 if(family == null) {
3207 throw new NoSuchColumnFamilyException("Empty family is invalid");
3209 checkFamily(family, delete.getDurability());
3214 @Override
3215 public void delete(Delete delete) throws IOException {
3216 TraceUtil.trace(() -> {
3217 checkReadOnly();
3218 checkResources();
3219 startRegionOperation(Operation.DELETE);
3220 try {
3221 // All edits for the given row (across all column families) must happen atomically.
3222 return mutate(delete);
3223 } finally {
3224 closeRegionOperation(Operation.DELETE);
3226 }, () -> createRegionSpan("Region.delete"));
3230 * Set up correct timestamps in the KVs in Delete object.
3231 * <p/>
3232 * Caller should have the row and region locks.
3234 private void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap,
3235 byte[] byteNow) throws IOException {
3236 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3238 byte[] family = e.getKey();
3239 List<Cell> cells = e.getValue();
3240 assert cells instanceof RandomAccess;
3242 Map<byte[], Integer> kvCount = new TreeMap<>(Bytes.BYTES_COMPARATOR);
3243 int listSize = cells.size();
3244 for (int i=0; i < listSize; i++) {
3245 Cell cell = cells.get(i);
3246 // Check if time is LATEST, change to time of most recent addition if so
3247 // This is expensive.
3248 if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP
3249 && PrivateCellUtil.isDeleteType(cell)) {
3250 byte[] qual = CellUtil.cloneQualifier(cell);
3252 Integer count = kvCount.get(qual);
3253 if (count == null) {
3254 kvCount.put(qual, 1);
3255 } else {
3256 kvCount.put(qual, count + 1);
3258 count = kvCount.get(qual);
3260 Get get = new Get(CellUtil.cloneRow(cell));
3261 get.readVersions(count);
3262 get.addColumn(family, qual);
3263 if (coprocessorHost != null) {
3264 if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
3265 byteNow, get)) {
3266 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3268 } else {
3269 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3271 } else {
3272 PrivateCellUtil.updateLatestStamp(cell, byteNow);
3278 private void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
3279 throws IOException {
3280 try (RegionScanner scanner = getScanner(new Scan(get))) {
3281 // NOTE: Please don't use HRegion.get() instead,
3282 // because it will copy cells to heap. See HBASE-26036
3283 List<Cell> result = new ArrayList<>();
3284 scanner.next(result);
3286 if (result.size() < count) {
3287 // Nothing to delete
3288 PrivateCellUtil.updateLatestStamp(cell, byteNow);
3289 return;
3291 if (result.size() > count) {
3292 throw new RuntimeException("Unexpected size: " + result.size());
3294 Cell getCell = result.get(count - 1);
3295 PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
3299 @Override
3300 public void put(Put put) throws IOException {
3301 TraceUtil.trace(() -> {
3302 checkReadOnly();
3304 // Do a rough check that we have resources to accept a write. The check is
3305 // 'rough' in that between the resource check and the call to obtain a
3306 // read lock, resources may run out. For now, the thought is that this
3307 // will be extremely rare; we'll deal with it when it happens.
3308 checkResources();
3309 startRegionOperation(Operation.PUT);
3310 try {
3311 // All edits for the given row (across all column families) must happen atomically.
3312 return mutate(put);
3313 } finally {
3314 closeRegionOperation(Operation.PUT);
3316 }, () -> createRegionSpan("Region.put"));
3320 * Class that tracks the progress of a batch operations, accumulating status codes and tracking
3321 * the index at which processing is proceeding. These batch operations may get split into
3322 * mini-batches for processing.
3324 private abstract static class BatchOperation<T> {
3325 protected final T[] operations;
3326 protected final OperationStatus[] retCodeDetails;
3327 protected final WALEdit[] walEditsFromCoprocessors;
3328 // reference family cell maps directly so coprocessors can mutate them if desired
3329 protected final Map<byte[], List<Cell>>[] familyCellMaps;
3330 // For Increment/Append operations
3331 protected final Result[] results;
3333 protected final HRegion region;
3334 protected int nextIndexToProcess = 0;
3335 protected final ObservedExceptionsInBatch observedExceptions;
3336 //Durability of the batch (highest durability of all operations)
3337 protected Durability durability;
3338 protected boolean atomic = false;
3340 public BatchOperation(final HRegion region, T[] operations) {
3341 this.operations = operations;
3342 this.retCodeDetails = new OperationStatus[operations.length];
3343 Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
3344 this.walEditsFromCoprocessors = new WALEdit[operations.length];
3345 familyCellMaps = new Map[operations.length];
3346 this.results = new Result[operations.length];
3348 this.region = region;
3349 observedExceptions = new ObservedExceptionsInBatch();
3350 durability = Durability.USE_DEFAULT;
3354 * Visitor interface for batch operations
3356 @FunctionalInterface
3357 interface Visitor {
3359 * @param index operation index
3360 * @return If true continue visiting remaining entries, break otherwise
3362 boolean visit(int index) throws IOException;
3366 * Helper method for visiting pending/ all batch operations
3368 public void visitBatchOperations(boolean pendingOnly, int lastIndexExclusive, Visitor visitor)
3369 throws IOException {
3370 assert lastIndexExclusive <= this.size();
3371 for (int i = nextIndexToProcess; i < lastIndexExclusive; i++) {
3372 if (!pendingOnly || isOperationPending(i)) {
3373 if (!visitor.visit(i)) {
3374 break;
3380 public abstract Mutation getMutation(int index);
3382 public abstract long getNonceGroup(int index);
3384 public abstract long getNonce(int index);
3387 * This method is potentially expensive and useful mostly for non-replay CP path.
3389 public abstract Mutation[] getMutationsForCoprocs();
3391 public abstract boolean isInReplay();
3393 public abstract long getOrigLogSeqNum();
3395 public abstract void startRegionOperation() throws IOException;
3397 public abstract void closeRegionOperation() throws IOException;
3400 * Validates each mutation and prepares a batch for write. If necessary (non-replay case), runs
3401 * CP prePut()/preDelete()/preIncrement()/preAppend() hooks for all mutations in a batch. This
3402 * is intended to operate on entire batch and will be called from outside of class to check
3403 * and prepare batch. This can be implemented by calling helper method
3404 * {@link #checkAndPrepareMutation(int, long)} in a 'for' loop over mutations.
3406 public abstract void checkAndPrepare() throws IOException;
3409 * Implement any Put request specific check and prepare logic here. Please refer to
3410 * {@link #checkAndPrepareMutation(Mutation, long)} for how its used.
3412 protected abstract void checkAndPreparePut(final Put p) throws IOException;
3415 * If necessary, calls preBatchMutate() CP hook for a mini-batch and updates metrics, cell
3416 * count, tags and timestamp for all cells of all operations in a mini-batch.
3418 public abstract void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation>
3419 miniBatchOp, long timestamp, final List<RowLock> acquiredRowLocks) throws IOException;
3422 * Write mini-batch operations to MemStore
3424 public abstract WriteEntry writeMiniBatchOperationsToMemStore(
3425 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
3426 throws IOException;
3428 protected void writeMiniBatchOperationsToMemStore(
3429 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final long writeNumber)
3430 throws IOException {
3431 MemStoreSizing memStoreAccounting = new NonThreadSafeMemStoreSizing();
3432 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
3433 // We need to update the sequence id for following reasons.
3434 // 1) If the op is in replay mode, FSWALEntry#stampRegionSequenceId won't stamp sequence id.
3435 // 2) If no WAL, FSWALEntry won't be used
3436 // we use durability of the original mutation for the mutation passed by CP.
3437 if (isInReplay() || getMutation(index).getDurability() == Durability.SKIP_WAL) {
3438 region.updateSequenceId(familyCellMaps[index].values(), writeNumber);
3440 applyFamilyMapToMemStore(familyCellMaps[index], memStoreAccounting);
3441 return true;
3443 // update memStore size
3444 region.incMemStoreSize(memStoreAccounting.getDataSize(), memStoreAccounting.getHeapSize(),
3445 memStoreAccounting.getOffHeapSize(), memStoreAccounting.getCellsCount());
3448 public boolean isDone() {
3449 return nextIndexToProcess == operations.length;
3452 public int size() {
3453 return operations.length;
3456 public boolean isOperationPending(int index) {
3457 return retCodeDetails[index].getOperationStatusCode() == OperationStatusCode.NOT_RUN;
3460 public List<UUID> getClusterIds() {
3461 assert size() != 0;
3462 return getMutation(0).getClusterIds();
3465 boolean isAtomic() {
3466 return atomic;
3470 * Helper method that checks and prepares only one mutation. This can be used to implement
3471 * {@link #checkAndPrepare()} for entire Batch.
3472 * NOTE: As CP prePut()/preDelete()/preIncrement()/preAppend() hooks may modify mutations,
3473 * this method should be called after prePut()/preDelete()/preIncrement()/preAppend() CP hooks
3474 * are run for the mutation
3476 protected void checkAndPrepareMutation(Mutation mutation, final long timestamp)
3477 throws IOException {
3478 region.checkRow(mutation.getRow(), "batchMutate");
3479 if (mutation instanceof Put) {
3480 // Check the families in the put. If bad, skip this one.
3481 checkAndPreparePut((Put) mutation);
3482 region.checkTimestamps(mutation.getFamilyCellMap(), timestamp);
3483 } else if (mutation instanceof Delete) {
3484 region.prepareDelete((Delete) mutation);
3485 } else if (mutation instanceof Increment || mutation instanceof Append) {
3486 region.checkFamilies(mutation.getFamilyCellMap().keySet(), mutation.getDurability());
3490 protected void checkAndPrepareMutation(int index, long timestamp) throws IOException {
3491 Mutation mutation = getMutation(index);
3492 try {
3493 this.checkAndPrepareMutation(mutation, timestamp);
3495 if (mutation instanceof Put || mutation instanceof Delete) {
3496 // store the family map reference to allow for mutations
3497 familyCellMaps[index] = mutation.getFamilyCellMap();
3500 // store durability for the batch (highest durability of all operations in the batch)
3501 Durability tmpDur = region.getEffectiveDurability(mutation.getDurability());
3502 if (tmpDur.ordinal() > durability.ordinal()) {
3503 durability = tmpDur;
3505 } catch (NoSuchColumnFamilyException nscfe) {
3506 final String msg = "No such column family in batch mutation in region " + this;
3507 if (observedExceptions.hasSeenNoSuchFamily()) {
3508 LOG.warn(msg + nscfe.getMessage());
3509 } else {
3510 LOG.warn(msg, nscfe);
3511 observedExceptions.sawNoSuchFamily();
3513 retCodeDetails[index] = new OperationStatus(
3514 OperationStatusCode.BAD_FAMILY, nscfe.getMessage());
3515 if (isAtomic()) { // fail, atomic means all or none
3516 throw nscfe;
3518 } catch (FailedSanityCheckException fsce) {
3519 final String msg = "Batch Mutation did not pass sanity check in region " + this;
3520 if (observedExceptions.hasSeenFailedSanityCheck()) {
3521 LOG.warn(msg + fsce.getMessage());
3522 } else {
3523 LOG.warn(msg, fsce);
3524 observedExceptions.sawFailedSanityCheck();
3526 retCodeDetails[index] = new OperationStatus(
3527 OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
3528 if (isAtomic()) {
3529 throw fsce;
3531 } catch (WrongRegionException we) {
3532 final String msg = "Batch mutation had a row that does not belong to this region " + this;
3533 if (observedExceptions.hasSeenWrongRegion()) {
3534 LOG.warn(msg + we.getMessage());
3535 } else {
3536 LOG.warn(msg, we);
3537 observedExceptions.sawWrongRegion();
3539 retCodeDetails[index] = new OperationStatus(
3540 OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
3541 if (isAtomic()) {
3542 throw we;
3548 * Creates Mini-batch of all operations [nextIndexToProcess, lastIndexExclusive) for which
3549 * a row lock can be acquired. All mutations with locked rows are considered to be
3550 * In-progress operations and hence the name {@link MiniBatchOperationInProgress}. Mini batch
3551 * is window over {@link BatchOperation} and contains contiguous pending operations.
3553 * @param acquiredRowLocks keeps track of rowLocks acquired.
3555 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch(
3556 List<RowLock> acquiredRowLocks) throws IOException {
3557 int readyToWriteCount = 0;
3558 int lastIndexExclusive = 0;
3559 RowLock prevRowLock = null;
3560 for (; lastIndexExclusive < size(); lastIndexExclusive++) {
3561 // It reaches the miniBatchSize, stop here and process the miniBatch
3562 // This only applies to non-atomic batch operations.
3563 if (!isAtomic() && (readyToWriteCount == region.miniBatchSize)) {
3564 break;
3567 if (!isOperationPending(lastIndexExclusive)) {
3568 continue;
3571 // HBASE-19389 Limit concurrency of put with dense (hundreds) columns to avoid exhausting
3572 // RS handlers, covering both MutationBatchOperation and ReplayBatchOperation
3573 // The BAD_FAMILY/SANITY_CHECK_FAILURE cases are handled in checkAndPrepare phase and won't
3574 // pass the isOperationPending check
3575 Map<byte[], List<Cell>> curFamilyCellMap =
3576 getMutation(lastIndexExclusive).getFamilyCellMap();
3577 try {
3578 // start the protector before acquiring row lock considering performance, and will finish
3579 // it when encountering exception
3580 region.storeHotnessProtector.start(curFamilyCellMap);
3581 } catch (RegionTooBusyException rtbe) {
3582 region.storeHotnessProtector.finish(curFamilyCellMap);
3583 if (isAtomic()) {
3584 throw rtbe;
3586 retCodeDetails[lastIndexExclusive] =
3587 new OperationStatus(OperationStatusCode.STORE_TOO_BUSY, rtbe.getMessage());
3588 continue;
3591 Mutation mutation = getMutation(lastIndexExclusive);
3592 // If we haven't got any rows in our batch, we should block to get the next one.
3593 RowLock rowLock = null;
3594 boolean throwException = false;
3595 try {
3596 // if atomic then get exclusive lock, else shared lock
3597 rowLock = region.getRowLock(mutation.getRow(), !isAtomic(), prevRowLock);
3598 } catch (TimeoutIOException | InterruptedIOException e) {
3599 // NOTE: We will retry when other exceptions, but we should stop if we receive
3600 // TimeoutIOException or InterruptedIOException as operation has timed out or
3601 // interrupted respectively.
3602 throwException = true;
3603 throw e;
3604 } catch (IOException ioe) {
3605 LOG.warn("Failed getting lock, row={}, in region {}",
3606 Bytes.toStringBinary(mutation.getRow()), this, ioe);
3607 if (isAtomic()) { // fail, atomic means all or none
3608 throwException = true;
3609 throw ioe;
3611 } catch (Throwable throwable) {
3612 throwException = true;
3613 throw throwable;
3614 } finally {
3615 if (throwException) {
3616 region.storeHotnessProtector.finish(curFamilyCellMap);
3619 if (rowLock == null) {
3620 // We failed to grab another lock
3621 if (isAtomic()) {
3622 region.storeHotnessProtector.finish(curFamilyCellMap);
3623 throw new IOException("Can't apply all operations atomically!");
3625 break; // Stop acquiring more rows for this batch
3626 } else {
3627 if (rowLock != prevRowLock) {
3628 // It is a different row now, add this to the acquiredRowLocks and
3629 // set prevRowLock to the new returned rowLock
3630 acquiredRowLocks.add(rowLock);
3631 prevRowLock = rowLock;
3635 readyToWriteCount++;
3637 return createMiniBatch(lastIndexExclusive, readyToWriteCount);
3640 protected MiniBatchOperationInProgress<Mutation> createMiniBatch(final int lastIndexExclusive,
3641 final int readyToWriteCount) {
3642 return new MiniBatchOperationInProgress<>(getMutationsForCoprocs(), retCodeDetails,
3643 walEditsFromCoprocessors, nextIndexToProcess, lastIndexExclusive, readyToWriteCount);
3647 * Builds separate WALEdit per nonce by applying input mutations. If WALEdits from CP are
3648 * present, they are merged to result WALEdit.
3650 public List<Pair<NonceKey, WALEdit>> buildWALEdits(
3651 final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
3652 List<Pair<NonceKey, WALEdit>> walEdits = new ArrayList<>();
3654 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), new Visitor() {
3655 private Pair<NonceKey, WALEdit> curWALEditForNonce;
3657 @Override
3658 public boolean visit(int index) throws IOException {
3659 Mutation m = getMutation(index);
3660 // we use durability of the original mutation for the mutation passed by CP.
3661 if (region.getEffectiveDurability(m.getDurability()) == Durability.SKIP_WAL) {
3662 region.recordMutationWithoutWal(m.getFamilyCellMap());
3663 return true;
3666 // the batch may contain multiple nonce keys (replay case). If so, write WALEdit for each.
3667 // Given how nonce keys are originally written, these should be contiguous.
3668 // They don't have to be, it will still work, just write more WALEdits than needed.
3669 long nonceGroup = getNonceGroup(index);
3670 long nonce = getNonce(index);
3671 if (curWALEditForNonce == null ||
3672 curWALEditForNonce.getFirst().getNonceGroup() != nonceGroup ||
3673 curWALEditForNonce.getFirst().getNonce() != nonce) {
3674 curWALEditForNonce = new Pair<>(new NonceKey(nonceGroup, nonce),
3675 new WALEdit(miniBatchOp.getCellCount(), isInReplay()));
3676 walEdits.add(curWALEditForNonce);
3678 WALEdit walEdit = curWALEditForNonce.getSecond();
3680 // Add WAL edits from CPs.
3681 WALEdit fromCP = walEditsFromCoprocessors[index];
3682 if (fromCP != null) {
3683 for (Cell cell : fromCP.getCells()) {
3684 walEdit.add(cell);
3687 walEdit.add(familyCellMaps[index]);
3689 return true;
3692 return walEdits;
3696 * This method completes mini-batch operations by calling postBatchMutate() CP hook (if
3697 * required) and completing mvcc.
3699 public void completeMiniBatchOperations(
3700 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
3701 throws IOException {
3702 if (writeEntry != null) {
3703 region.mvcc.completeAndWait(writeEntry);
3707 public void doPostOpCleanupForMiniBatch(
3708 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WALEdit walEdit,
3709 boolean success) throws IOException {
3710 doFinishHotnessProtector(miniBatchOp);
3713 private void doFinishHotnessProtector(
3714 final MiniBatchOperationInProgress<Mutation> miniBatchOp) {
3715 // check and return if the protector is not enabled
3716 if (!region.storeHotnessProtector.isEnable()) {
3717 return;
3719 // miniBatchOp is null, if and only if lockRowsAndBuildMiniBatch throwing exception.
3720 // This case was handled.
3721 if (miniBatchOp == null) {
3722 return;
3725 final int finalLastIndexExclusive = miniBatchOp.getLastIndexExclusive();
3727 for (int i = nextIndexToProcess; i < finalLastIndexExclusive; i++) {
3728 switch (retCodeDetails[i].getOperationStatusCode()) {
3729 case SUCCESS:
3730 case FAILURE:
3731 region.storeHotnessProtector.finish(getMutation(i).getFamilyCellMap());
3732 break;
3733 default:
3734 // do nothing
3735 // We won't start the protector for NOT_RUN/BAD_FAMILY/SANITY_CHECK_FAILURE and the
3736 // STORE_TOO_BUSY case is handled in StoreHotnessProtector#start
3737 break;
3743 * Atomically apply the given map of family->edits to the memstore.
3744 * This handles the consistency control on its own, but the caller
3745 * should already have locked updatesLock.readLock(). This also does
3746 * <b>not</b> check the families for validity.
3748 * @param familyMap Map of Cells by family
3750 protected void applyFamilyMapToMemStore(Map<byte[], List<Cell>> familyMap,
3751 MemStoreSizing memstoreAccounting) throws IOException {
3752 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3753 byte[] family = e.getKey();
3754 List<Cell> cells = e.getValue();
3755 assert cells instanceof RandomAccess;
3756 region.applyToMemStore(region.getStore(family), cells, false, memstoreAccounting);
3763 * Batch of mutation operations. Base class is shared with {@link ReplayBatchOperation} as most of
3764 * the logic is same.
3766 private static class MutationBatchOperation extends BatchOperation<Mutation> {
3768 // For nonce operations
3769 private long nonceGroup;
3770 private long nonce;
3771 protected boolean canProceed;
3773 public MutationBatchOperation(final HRegion region, Mutation[] operations, boolean atomic,
3774 long nonceGroup, long nonce) {
3775 super(region, operations);
3776 this.atomic = atomic;
3777 this.nonceGroup = nonceGroup;
3778 this.nonce = nonce;
3781 @Override
3782 public Mutation getMutation(int index) {
3783 return this.operations[index];
3786 @Override
3787 public long getNonceGroup(int index) {
3788 return nonceGroup;
3791 @Override
3792 public long getNonce(int index) {
3793 return nonce;
3796 @Override
3797 public Mutation[] getMutationsForCoprocs() {
3798 return this.operations;
3801 @Override
3802 public boolean isInReplay() {
3803 return false;
3806 @Override
3807 public long getOrigLogSeqNum() {
3808 return SequenceId.NO_SEQUENCE_ID;
3811 @Override
3812 public void startRegionOperation() throws IOException {
3813 region.startRegionOperation(Operation.BATCH_MUTATE);
3816 @Override
3817 public void closeRegionOperation() throws IOException {
3818 region.closeRegionOperation(Operation.BATCH_MUTATE);
3821 @Override
3822 public void checkAndPreparePut(Put p) throws IOException {
3823 region.checkFamilies(p.getFamilyCellMap().keySet(), p.getDurability());
3826 @Override
3827 public void checkAndPrepare() throws IOException {
3828 // index 0: puts, index 1: deletes, index 2: increments, index 3: append
3829 final int[] metrics = {0, 0, 0, 0};
3831 visitBatchOperations(true, this.size(), new Visitor() {
3832 private long now = EnvironmentEdgeManager.currentTime();
3833 private WALEdit walEdit;
3834 @Override
3835 public boolean visit(int index) throws IOException {
3836 // Run coprocessor pre hook outside of locks to avoid deadlock
3837 if (region.coprocessorHost != null) {
3838 if (walEdit == null) {
3839 walEdit = new WALEdit();
3841 callPreMutateCPHook(index, walEdit, metrics);
3842 if (!walEdit.isEmpty()) {
3843 walEditsFromCoprocessors[index] = walEdit;
3844 walEdit = null;
3847 if (isOperationPending(index)) {
3848 // TODO: Currently validation is done with current time before acquiring locks and
3849 // updates are done with different timestamps after acquiring locks. This behavior is
3850 // inherited from the code prior to this change. Can this be changed?
3851 checkAndPrepareMutation(index, now);
3853 return true;
3857 // FIXME: we may update metrics twice! here for all operations bypassed by CP and later in
3858 // normal processing.
3859 // Update metrics in same way as it is done when we go the normal processing route (we now
3860 // update general metrics though a Coprocessor did the work).
3861 if (region.metricsRegion != null) {
3862 if (metrics[0] > 0) {
3863 // There were some Puts in the batch.
3864 region.metricsRegion.updatePut();
3866 if (metrics[1] > 0) {
3867 // There were some Deletes in the batch.
3868 region.metricsRegion.updateDelete();
3870 if (metrics[2] > 0) {
3871 // There were some Increment in the batch.
3872 region.metricsRegion.updateIncrement();
3874 if (metrics[3] > 0) {
3875 // There were some Append in the batch.
3876 region.metricsRegion.updateAppend();
3881 @Override
3882 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
3883 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
3884 // For nonce operations
3885 canProceed = startNonceOperation();
3887 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
3888 Mutation mutation = getMutation(index);
3889 if (mutation instanceof Put) {
3890 HRegion.updateCellTimestamps(familyCellMaps[index].values(), Bytes.toBytes(timestamp));
3891 miniBatchOp.incrementNumOfPuts();
3892 } else if (mutation instanceof Delete) {
3893 region.prepareDeleteTimestamps(mutation, familyCellMaps[index],
3894 Bytes.toBytes(timestamp));
3895 miniBatchOp.incrementNumOfDeletes();
3896 } else if (mutation instanceof Increment || mutation instanceof Append) {
3897 boolean returnResults;
3898 if (mutation instanceof Increment) {
3899 returnResults = ((Increment) mutation).isReturnResults();
3900 } else {
3901 returnResults = ((Append) mutation).isReturnResults();
3904 // For nonce operations
3905 if (!canProceed) {
3906 Result result;
3907 if (returnResults) {
3908 // convert duplicate increment/append to get
3909 List<Cell> results = region.get(toGet(mutation), false, nonceGroup, nonce);
3910 result = Result.create(results);
3911 } else {
3912 result = Result.EMPTY_RESULT;
3914 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
3915 return true;
3918 Result result = null;
3919 if (region.coprocessorHost != null) {
3920 if (mutation instanceof Increment) {
3921 result = region.coprocessorHost.preIncrementAfterRowLock((Increment) mutation);
3922 } else {
3923 result = region.coprocessorHost.preAppendAfterRowLock((Append) mutation);
3926 if (result != null) {
3927 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS,
3928 returnResults ? result : Result.EMPTY_RESULT);
3929 return true;
3932 List<Cell> results = returnResults ? new ArrayList<>(mutation.size()) : null;
3933 familyCellMaps[index] = reckonDeltas(mutation, results, timestamp);
3934 this.results[index] = results != null ? Result.create(results) : Result.EMPTY_RESULT;
3936 if (mutation instanceof Increment) {
3937 miniBatchOp.incrementNumOfIncrements();
3938 } else {
3939 miniBatchOp.incrementNumOfAppends();
3942 region.rewriteCellTags(familyCellMaps[index], mutation);
3944 // update cell count
3945 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
3946 for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
3947 miniBatchOp.addCellCount(cells.size());
3951 WALEdit fromCP = walEditsFromCoprocessors[index];
3952 if (fromCP != null) {
3953 miniBatchOp.addCellCount(fromCP.size());
3955 return true;
3958 if (region.coprocessorHost != null) {
3959 // calling the pre CP hook for batch mutation
3960 region.coprocessorHost.preBatchMutate(miniBatchOp);
3961 checkAndMergeCPMutations(miniBatchOp, acquiredRowLocks, timestamp);
3966 * Starts the nonce operation for a mutation, if needed.
3967 * @return whether to proceed this mutation.
3969 private boolean startNonceOperation() throws IOException {
3970 if (region.rsServices == null || region.rsServices.getNonceManager() == null
3971 || nonce == HConstants.NO_NONCE) {
3972 return true;
3974 boolean canProceed;
3975 try {
3976 canProceed = region.rsServices.getNonceManager()
3977 .startOperation(nonceGroup, nonce, region.rsServices);
3978 } catch (InterruptedException ex) {
3979 throw new InterruptedIOException("Nonce start operation interrupted");
3981 return canProceed;
3985 * Ends nonce operation for a mutation, if needed.
3986 * @param success Whether the operation for this nonce has succeeded.
3988 private void endNonceOperation(boolean success) {
3989 if (region.rsServices != null && region.rsServices.getNonceManager() != null
3990 && nonce != HConstants.NO_NONCE) {
3991 region.rsServices.getNonceManager().endOperation(nonceGroup, nonce, success);
3995 private static Get toGet(final Mutation mutation) throws IOException {
3996 assert mutation instanceof Increment || mutation instanceof Append;
3997 Get get = new Get(mutation.getRow());
3998 CellScanner cellScanner = mutation.cellScanner();
3999 while (!cellScanner.advance()) {
4000 Cell cell = cellScanner.current();
4001 get.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell));
4003 if (mutation instanceof Increment) {
4004 // Increment
4005 Increment increment = (Increment) mutation;
4006 get.setTimeRange(increment.getTimeRange().getMin(), increment.getTimeRange().getMax());
4007 } else {
4008 // Append
4009 Append append = (Append) mutation;
4010 get.setTimeRange(append.getTimeRange().getMin(), append.getTimeRange().getMax());
4012 for (Entry<String, byte[]> entry : mutation.getAttributesMap().entrySet()) {
4013 get.setAttribute(entry.getKey(), entry.getValue());
4015 return get;
4018 private Map<byte[], List<Cell>> reckonDeltas(Mutation mutation, List<Cell> results,
4019 long now) throws IOException {
4020 assert mutation instanceof Increment || mutation instanceof Append;
4021 Map<byte[], List<Cell>> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR);
4022 // Process a Store/family at a time.
4023 for (Map.Entry<byte [], List<Cell>> entry: mutation.getFamilyCellMap().entrySet()) {
4024 final byte[] columnFamilyName = entry.getKey();
4025 List<Cell> deltas = entry.getValue();
4026 // Reckon for the Store what to apply to WAL and MemStore.
4027 List<Cell> toApply = reckonDeltasByStore(region.stores.get(columnFamilyName), mutation,
4028 now, deltas, results);
4029 if (!toApply.isEmpty()) {
4030 for (Cell cell : toApply) {
4031 HStore store = region.getStore(cell);
4032 if (store == null) {
4033 region.checkFamily(CellUtil.cloneFamily(cell));
4034 } else {
4035 ret.computeIfAbsent(store.getColumnFamilyDescriptor().getName(),
4036 key -> new ArrayList<>()).add(cell);
4041 return ret;
4045 * Reckon the Cells to apply to WAL, memstore, and to return to the Client in passed
4046 * column family/Store.
4048 * Does Get of current value and then adds passed in deltas for this Store returning the
4049 * result.
4051 * @param mutation The encompassing Mutation object
4052 * @param deltas Changes to apply to this Store; either increment amount or data to append
4053 * @param results In here we accumulate all the Cells we are to return to the client. If null,
4054 * client doesn't want results returned.
4055 * @return Resulting Cells after <code>deltas</code> have been applied to current
4056 * values. Side effect is our filling out of the <code>results</code> List.
4058 private List<Cell> reckonDeltasByStore(HStore store, Mutation mutation, long now,
4059 List<Cell> deltas, List<Cell> results) throws IOException {
4060 assert mutation instanceof Increment || mutation instanceof Append;
4061 byte[] columnFamily = store.getColumnFamilyDescriptor().getName();
4062 List<Pair<Cell, Cell>> cellPairs = new ArrayList<>(deltas.size());
4064 // Sort the cells so that they match the order that they appear in the Get results.
4065 // Otherwise, we won't be able to find the existing values if the cells are not specified
4066 // in order by the client since cells are in an array list.
4067 deltas.sort(store.getComparator());
4069 // Get previous values for all columns in this family.
4070 Get get = new Get(mutation.getRow());
4071 for (Cell cell: deltas) {
4072 get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
4074 TimeRange tr;
4075 if (mutation instanceof Increment) {
4076 tr = ((Increment) mutation).getTimeRange();
4077 } else {
4078 tr = ((Append) mutation).getTimeRange();
4081 if (tr != null) {
4082 get.setTimeRange(tr.getMin(), tr.getMax());
4085 try (RegionScanner scanner = region.getScanner(new Scan(get))) {
4086 // NOTE: Please don't use HRegion.get() instead,
4087 // because it will copy cells to heap. See HBASE-26036
4088 List<Cell> currentValues = new ArrayList<>();
4089 scanner.next(currentValues);
4090 // Iterate the input columns and update existing values if they were found, otherwise
4091 // add new column initialized to the delta amount
4092 int currentValuesIndex = 0;
4093 for (int i = 0; i < deltas.size(); i++) {
4094 Cell delta = deltas.get(i);
4095 Cell currentValue = null;
4096 if (currentValuesIndex < currentValues.size() && CellUtil
4097 .matchingQualifier(currentValues.get(currentValuesIndex), delta)) {
4098 currentValue = currentValues.get(currentValuesIndex);
4099 if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) {
4100 currentValuesIndex++;
4103 // Switch on whether this an increment or an append building the new Cell to apply.
4104 Cell newCell;
4105 if (mutation instanceof Increment) {
4106 long deltaAmount = getLongValue(delta);
4107 final long newValue = currentValue == null ? deltaAmount :
4108 getLongValue(currentValue) + deltaAmount;
4109 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4110 (oldCell) -> Bytes.toBytes(newValue));
4111 } else {
4112 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4113 (oldCell) -> ByteBuffer.wrap(new byte[delta.getValueLength() +
4114 oldCell.getValueLength()])
4115 .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength())
4116 .put(delta.getValueArray(), delta.getValueOffset(), delta.getValueLength())
4117 .array());
4119 if (region.maxCellSize > 0) {
4120 int newCellSize = PrivateCellUtil.estimatedSerializedSizeOf(newCell);
4121 if (newCellSize > region.maxCellSize) {
4122 String msg =
4123 "Cell with size " + newCellSize + " exceeds limit of " + region.maxCellSize +
4124 " bytes in region " + this;
4125 LOG.debug(msg);
4126 throw new DoNotRetryIOException(msg);
4129 cellPairs.add(new Pair<>(currentValue, newCell));
4130 // Add to results to get returned to the Client. If null, cilent does not want results.
4131 if (results != null) {
4132 results.add(newCell);
4135 // Give coprocessors a chance to update the new cells before apply to WAL or memstore
4136 if (region.coprocessorHost != null) {
4137 // Here the operation must be increment or append.
4138 cellPairs = mutation instanceof Increment ?
4139 region.coprocessorHost.postIncrementBeforeWAL(mutation, cellPairs) :
4140 region.coprocessorHost.postAppendBeforeWAL(mutation, cellPairs);
4143 return cellPairs.stream().map(Pair::getSecond).collect(Collectors.toList());
4146 private static Cell reckonDelta(final Cell delta, final Cell currentCell,
4147 final byte[] columnFamily, final long now, Mutation mutation,
4148 Function<Cell, byte[]> supplier) throws IOException {
4149 // Forward any tags found on the delta.
4150 List<Tag> tags = TagUtil.carryForwardTags(delta);
4151 if (currentCell != null) {
4152 tags = TagUtil.carryForwardTags(tags, currentCell);
4153 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4154 byte[] newValue = supplier.apply(currentCell);
4155 return ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
4156 .setRow(mutation.getRow(), 0, mutation.getRow().length)
4157 .setFamily(columnFamily, 0, columnFamily.length)
4158 // copy the qualifier if the cell is located in shared memory.
4159 .setQualifier(CellUtil.cloneQualifier(delta))
4160 .setTimestamp(Math.max(currentCell.getTimestamp() + 1, now))
4161 .setType(KeyValue.Type.Put.getCode())
4162 .setValue(newValue, 0, newValue.length)
4163 .setTags(TagUtil.fromList(tags))
4164 .build();
4165 } else {
4166 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4167 PrivateCellUtil.updateLatestStamp(delta, now);
4168 return CollectionUtils.isEmpty(tags) ? delta : PrivateCellUtil.createCell(delta, tags);
4173 * @return Get the long out of the passed in Cell
4175 private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
4176 int len = cell.getValueLength();
4177 if (len != Bytes.SIZEOF_LONG) {
4178 // throw DoNotRetryIOException instead of IllegalArgumentException
4179 throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
4181 return PrivateCellUtil.getValueAsLong(cell);
4184 @Override
4185 public List<Pair<NonceKey, WALEdit>> buildWALEdits(final MiniBatchOperationInProgress<Mutation>
4186 miniBatchOp) throws IOException {
4187 List<Pair<NonceKey, WALEdit>> walEdits = super.buildWALEdits(miniBatchOp);
4188 // for MutationBatchOperation, more than one nonce is not allowed
4189 if (walEdits.size() > 1) {
4190 throw new IOException("Found multiple nonce keys per batch!");
4192 return walEdits;
4195 @Override
4196 public WriteEntry writeMiniBatchOperationsToMemStore(
4197 final MiniBatchOperationInProgress<Mutation> miniBatchOp, @Nullable WriteEntry writeEntry)
4198 throws IOException {
4199 if (writeEntry == null) {
4200 writeEntry = region.mvcc.begin();
4202 super.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry.getWriteNumber());
4203 return writeEntry;
4206 @Override
4207 public void completeMiniBatchOperations(
4208 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4209 throws IOException {
4210 // TODO: can it be done after completing mvcc?
4211 // calling the post CP hook for batch mutation
4212 if (region.coprocessorHost != null) {
4213 region.coprocessorHost.postBatchMutate(miniBatchOp);
4215 super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4217 if (nonce != HConstants.NO_NONCE) {
4218 if (region.rsServices != null && region.rsServices.getNonceManager() != null) {
4219 region.rsServices.getNonceManager()
4220 .addMvccToOperationContext(nonceGroup, nonce, writeEntry.getWriteNumber());
4225 @Override
4226 public void doPostOpCleanupForMiniBatch(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4227 final WALEdit walEdit, boolean success) throws IOException {
4229 super.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, success);
4230 if (miniBatchOp != null) {
4231 // synced so that the coprocessor contract is adhered to.
4232 if (region.coprocessorHost != null) {
4233 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4234 // only for successful puts/deletes/increments/appends
4235 if (retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.SUCCESS) {
4236 Mutation m = getMutation(i);
4237 if (m instanceof Put) {
4238 region.coprocessorHost.postPut((Put) m, walEdit);
4239 } else if (m instanceof Delete) {
4240 region.coprocessorHost.postDelete((Delete) m, walEdit);
4241 } else if (m instanceof Increment) {
4242 Result result = region.getCoprocessorHost().postIncrement((Increment) m,
4243 results[i], walEdit);
4244 if (result != results[i]) {
4245 retCodeDetails[i] =
4246 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4248 } else if (m instanceof Append) {
4249 Result result = region.getCoprocessorHost().postAppend((Append) m, results[i],
4250 walEdit);
4251 if (result != results[i]) {
4252 retCodeDetails[i] =
4253 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4257 return true;
4261 // For nonce operations
4262 if (canProceed && nonce != HConstants.NO_NONCE) {
4263 boolean[] areAllIncrementsAndAppendsSuccessful = new boolean[]{true};
4264 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4265 Mutation mutation = getMutation(i);
4266 if (mutation instanceof Increment || mutation instanceof Append) {
4267 if (retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
4268 areAllIncrementsAndAppendsSuccessful[0] = false;
4269 return false;
4272 return true;
4274 endNonceOperation(areAllIncrementsAndAppendsSuccessful[0]);
4277 // See if the column families were consistent through the whole thing.
4278 // if they were then keep them. If they were not then pass a null.
4279 // null will be treated as unknown.
4280 // Total time taken might be involving Puts, Deletes, Increments and Appends.
4281 // Split the time for puts and deletes based on the total number of Puts, Deletes,
4282 // Increments and Appends.
4283 if (region.metricsRegion != null) {
4284 if (miniBatchOp.getNumOfPuts() > 0) {
4285 // There were some Puts in the batch.
4286 region.metricsRegion.updatePut();
4288 if (miniBatchOp.getNumOfDeletes() > 0) {
4289 // There were some Deletes in the batch.
4290 region.metricsRegion.updateDelete();
4292 if (miniBatchOp.getNumOfIncrements() > 0) {
4293 // There were some Increments in the batch.
4294 region.metricsRegion.updateIncrement();
4296 if (miniBatchOp.getNumOfAppends() > 0) {
4297 // There were some Appends in the batch.
4298 region.metricsRegion.updateAppend();
4303 if (region.coprocessorHost != null) {
4304 // call the coprocessor hook to do any finalization steps after the put is done
4305 region.coprocessorHost.postBatchMutateIndispensably(
4306 miniBatchOp != null ? miniBatchOp : createMiniBatch(size(), 0), success);
4311 * Runs prePut/preDelete/preIncrement/preAppend coprocessor hook for input mutation in a batch
4312 * @param metrics Array of 2 ints. index 0: count of puts, index 1: count of deletes, index 2:
4313 * count of increments and 3: count of appends
4315 private void callPreMutateCPHook(int index, final WALEdit walEdit, final int[] metrics)
4316 throws IOException {
4317 Mutation m = getMutation(index);
4318 if (m instanceof Put) {
4319 if (region.coprocessorHost.prePut((Put) m, walEdit)) {
4320 // pre hook says skip this Put
4321 // mark as success and skip in doMiniBatchMutation
4322 metrics[0]++;
4323 retCodeDetails[index] = OperationStatus.SUCCESS;
4325 } else if (m instanceof Delete) {
4326 Delete curDel = (Delete) m;
4327 if (curDel.getFamilyCellMap().isEmpty()) {
4328 // handle deleting a row case
4329 // TODO: prepareDelete() has been called twice, before and after preDelete() CP hook.
4330 // Can this be avoided?
4331 region.prepareDelete(curDel);
4333 if (region.coprocessorHost.preDelete(curDel, walEdit)) {
4334 // pre hook says skip this Delete
4335 // mark as success and skip in doMiniBatchMutation
4336 metrics[1]++;
4337 retCodeDetails[index] = OperationStatus.SUCCESS;
4339 } else if (m instanceof Increment) {
4340 Increment increment = (Increment) m;
4341 Result result = region.coprocessorHost.preIncrement(increment, walEdit);
4342 if (result != null) {
4343 // pre hook says skip this Increment
4344 // mark as success and skip in doMiniBatchMutation
4345 metrics[2]++;
4346 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4348 } else if (m instanceof Append) {
4349 Append append = (Append) m;
4350 Result result = region.coprocessorHost.preAppend(append, walEdit);
4351 if (result != null) {
4352 // pre hook says skip this Append
4353 // mark as success and skip in doMiniBatchMutation
4354 metrics[3]++;
4355 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4357 } else {
4358 String msg = "Put/Delete/Increment/Append mutations only supported in a batch";
4359 retCodeDetails[index] = new OperationStatus(OperationStatusCode.FAILURE, msg);
4360 if (isAtomic()) { // fail, atomic means all or none
4361 throw new IOException(msg);
4366 // TODO Support Increment/Append operations
4367 private void checkAndMergeCPMutations(final MiniBatchOperationInProgress<Mutation> miniBatchOp,
4368 final List<RowLock> acquiredRowLocks, final long timestamp) throws IOException {
4369 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), (int i) -> {
4370 // we pass (i - firstIndex) below since the call expects a relative index
4371 Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - nextIndexToProcess);
4372 if (cpMutations == null) {
4373 return true;
4375 // Else Coprocessor added more Mutations corresponding to the Mutation at this index.
4376 Mutation mutation = getMutation(i);
4377 for (Mutation cpMutation : cpMutations) {
4378 this.checkAndPrepareMutation(cpMutation, timestamp);
4380 // Acquire row locks. If not, the whole batch will fail.
4381 acquiredRowLocks.add(region.getRowLock(cpMutation.getRow(), true, null));
4383 // Returned mutations from coprocessor correspond to the Mutation at index i. We can
4384 // directly add the cells from those mutations to the familyMaps of this mutation.
4385 Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap();
4386 region.rewriteCellTags(cpFamilyMap, mutation);
4387 // will get added to the memStore later
4388 mergeFamilyMaps(familyCellMaps[i], cpFamilyMap);
4390 // The durability of returned mutation is replaced by the corresponding mutation.
4391 // If the corresponding mutation contains the SKIP_WAL, we shouldn't count the
4392 // cells of returned mutation.
4393 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
4394 for (List<Cell> cells : cpFamilyMap.values()) {
4395 miniBatchOp.addCellCount(cells.size());
4399 return true;
4403 private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
4404 Map<byte[], List<Cell>> toBeMerged) {
4405 for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
4406 List<Cell> cells = familyMap.get(entry.getKey());
4407 if (cells == null) {
4408 familyMap.put(entry.getKey(), entry.getValue());
4409 } else {
4410 cells.addAll(entry.getValue());
4417 * Batch of mutations for replay. Base class is shared with {@link MutationBatchOperation} as most
4418 * of the logic is same.
4420 private static final class ReplayBatchOperation extends BatchOperation<MutationReplay> {
4422 private long origLogSeqNum = 0;
4424 public ReplayBatchOperation(final HRegion region, MutationReplay[] operations,
4425 long origLogSeqNum) {
4426 super(region, operations);
4427 this.origLogSeqNum = origLogSeqNum;
4430 @Override
4431 public Mutation getMutation(int index) {
4432 return this.operations[index].mutation;
4435 @Override
4436 public long getNonceGroup(int index) {
4437 return this.operations[index].nonceGroup;
4440 @Override
4441 public long getNonce(int index) {
4442 return this.operations[index].nonce;
4445 @Override
4446 public Mutation[] getMutationsForCoprocs() {
4447 return null;
4450 @Override
4451 public boolean isInReplay() {
4452 return true;
4455 @Override
4456 public long getOrigLogSeqNum() {
4457 return this.origLogSeqNum;
4460 @Override
4461 public void startRegionOperation() throws IOException {
4462 region.startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4465 @Override
4466 public void closeRegionOperation() throws IOException {
4467 region.closeRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4471 * During replay, there could exist column families which are removed between region server
4472 * failure and replay
4474 @Override
4475 protected void checkAndPreparePut(Put p) throws IOException {
4476 Map<byte[], List<Cell>> familyCellMap = p.getFamilyCellMap();
4477 List<byte[]> nonExistentList = null;
4478 for (byte[] family : familyCellMap.keySet()) {
4479 if (!region.htableDescriptor.hasColumnFamily(family)) {
4480 if (nonExistentList == null) {
4481 nonExistentList = new ArrayList<>();
4483 nonExistentList.add(family);
4486 if (nonExistentList != null) {
4487 for (byte[] family : nonExistentList) {
4488 // Perhaps schema was changed between crash and replay
4489 LOG.info("No family for {} omit from reply in region {}.", Bytes.toString(family), this);
4490 familyCellMap.remove(family);
4495 @Override
4496 public void checkAndPrepare() throws IOException {
4497 long now = EnvironmentEdgeManager.currentTime();
4498 visitBatchOperations(true, this.size(), (int index) -> {
4499 checkAndPrepareMutation(index, now);
4500 return true;
4504 @Override
4505 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4506 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
4507 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
4508 // update cell count
4509 for (List<Cell> cells : getMutation(index).getFamilyCellMap().values()) {
4510 miniBatchOp.addCellCount(cells.size());
4512 return true;
4516 @Override
4517 public WriteEntry writeMiniBatchOperationsToMemStore(
4518 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4519 throws IOException {
4520 super.writeMiniBatchOperationsToMemStore(miniBatchOp, getOrigLogSeqNum());
4521 return writeEntry;
4524 @Override
4525 public void completeMiniBatchOperations(
4526 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4527 throws IOException {
4528 super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4529 region.mvcc.advanceTo(getOrigLogSeqNum());
4533 public OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic, long nonceGroup,
4534 long nonce) throws IOException {
4535 // As it stands, this is used for 3 things
4536 // * batchMutate with single mutation - put/delete/increment/append, separate or from
4537 // checkAndMutate.
4538 // * coprocessor calls (see ex. BulkDeleteEndpoint).
4539 // So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd...
4540 return batchMutate(new MutationBatchOperation(this, mutations, atomic, nonceGroup, nonce));
4543 @Override
4544 public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
4545 // If the mutations has any Increment/Append operations, we need to do batchMutate atomically
4546 boolean atomic =
4547 Arrays.stream(mutations).anyMatch(m -> m instanceof Increment || m instanceof Append);
4548 return batchMutate(mutations, atomic);
4551 OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic) throws IOException {
4552 return TraceUtil.trace(
4553 () -> batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE),
4554 () -> createRegionSpan("Region.batchMutate"));
4557 public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
4558 throws IOException {
4559 if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo())
4560 && replaySeqId < lastReplayedOpenRegionSeqId) {
4561 // if it is a secondary replica we should ignore these entries silently
4562 // since they are coming out of order
4563 if (LOG.isTraceEnabled()) {
4564 LOG.trace(getRegionInfo().getEncodedName() + " : "
4565 + "Skipping " + mutations.length + " mutations with replaySeqId=" + replaySeqId
4566 + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId);
4567 for (MutationReplay mut : mutations) {
4568 LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation);
4572 OperationStatus[] statuses = new OperationStatus[mutations.length];
4573 for (int i = 0; i < statuses.length; i++) {
4574 statuses[i] = OperationStatus.SUCCESS;
4576 return statuses;
4578 return batchMutate(new ReplayBatchOperation(this, mutations, replaySeqId));
4582 * Perform a batch of mutations.
4583 * <p/>
4584 * Operations in a batch are stored with highest durability specified of for all operations in a
4585 * batch, except for {@link Durability#SKIP_WAL}.
4586 * <p/>
4587 * This function is called from {@link #batchReplay(WALSplitUtil.MutationReplay[], long)} with
4588 * {@link ReplayBatchOperation} instance and {@link #batchMutate(Mutation[])} with
4589 * {@link MutationBatchOperation} instance as an argument. As the processing of replay batch and
4590 * mutation batch is very similar, lot of code is shared by providing generic methods in base
4591 * class {@link BatchOperation}. The logic for this method and
4592 * {@link #doMiniBatchMutate(BatchOperation)} is implemented using methods in base class which are
4593 * overridden by derived classes to implement special behavior.
4594 * @param batchOp contains the list of mutations
4595 * @return an array of OperationStatus which internally contains the OperationStatusCode and the
4596 * exceptionMessage if any.
4597 * @throws IOException if an IO problem is encountered
4599 private OperationStatus[] batchMutate(BatchOperation<?> batchOp) throws IOException {
4600 boolean initialized = false;
4601 batchOp.startRegionOperation();
4602 try {
4603 while (!batchOp.isDone()) {
4604 if (!batchOp.isInReplay()) {
4605 checkReadOnly();
4607 checkResources();
4609 if (!initialized) {
4610 this.writeRequestsCount.add(batchOp.size());
4611 // validate and prepare batch for write, for MutationBatchOperation it also calls CP
4612 // prePut()/preDelete()/preIncrement()/preAppend() hooks
4613 batchOp.checkAndPrepare();
4614 initialized = true;
4616 doMiniBatchMutate(batchOp);
4617 requestFlushIfNeeded();
4619 } finally {
4620 if (rsServices != null && rsServices.getMetrics() != null) {
4621 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
4622 getTableName(), batchOp.size());
4624 batchOp.closeRegionOperation();
4626 return batchOp.retCodeDetails;
4630 * Called to do a piece of the batch that came in to {@link #batchMutate(Mutation[])}
4631 * In here we also handle replay of edits on region recover. Also gets change in size brought
4632 * about by applying {@code batchOp}.
4634 private void doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException {
4635 boolean success = false;
4636 WALEdit walEdit = null;
4637 WriteEntry writeEntry = null;
4638 boolean locked = false;
4639 // We try to set up a batch in the range [batchOp.nextIndexToProcess,lastIndexExclusive)
4640 MiniBatchOperationInProgress<Mutation> miniBatchOp = null;
4641 /** Keep track of the locks we hold so we can release them in finally clause */
4642 List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size());
4644 // Check for thread interrupt status in case we have been signaled from
4645 // #interruptRegionOperation.
4646 checkInterrupt();
4648 try {
4649 // STEP 1. Try to acquire as many locks as we can and build mini-batch of operations with
4650 // locked rows
4651 miniBatchOp = batchOp.lockRowsAndBuildMiniBatch(acquiredRowLocks);
4653 // We've now grabbed as many mutations off the list as we can
4654 // Ensure we acquire at least one.
4655 if (miniBatchOp.getReadyToWriteCount() <= 0) {
4656 // Nothing to put/delete/increment/append -- an exception in the above such as
4657 // NoSuchColumnFamily?
4658 return;
4661 // Check for thread interrupt status in case we have been signaled from
4662 // #interruptRegionOperation. Do it before we take the lock and disable interrupts for
4663 // the WAL append.
4664 checkInterrupt();
4666 lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount());
4667 locked = true;
4669 // From this point until memstore update this operation should not be interrupted.
4670 disableInterrupts();
4672 // STEP 2. Update mini batch of all operations in progress with LATEST_TIMESTAMP timestamp
4673 // We should record the timestamp only after we have acquired the rowLock,
4674 // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer
4675 // timestamp
4677 long now = EnvironmentEdgeManager.currentTime();
4678 batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks);
4680 // STEP 3. Build WAL edit
4682 List<Pair<NonceKey, WALEdit>> walEdits = batchOp.buildWALEdits(miniBatchOp);
4684 // STEP 4. Append the WALEdits to WAL and sync.
4686 for(Iterator<Pair<NonceKey, WALEdit>> it = walEdits.iterator(); it.hasNext();) {
4687 Pair<NonceKey, WALEdit> nonceKeyWALEditPair = it.next();
4688 walEdit = nonceKeyWALEditPair.getSecond();
4689 NonceKey nonceKey = nonceKeyWALEditPair.getFirst();
4691 if (walEdit != null && !walEdit.isEmpty()) {
4692 writeEntry = doWALAppend(walEdit, batchOp.durability, batchOp.getClusterIds(), now,
4693 nonceKey.getNonceGroup(), nonceKey.getNonce(), batchOp.getOrigLogSeqNum());
4696 // Complete mvcc for all but last writeEntry (for replay case)
4697 if (it.hasNext() && writeEntry != null) {
4698 mvcc.complete(writeEntry);
4699 writeEntry = null;
4703 // STEP 5. Write back to memStore
4704 // NOTE: writeEntry can be null here
4705 writeEntry = batchOp.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry);
4707 // STEP 6. Complete MiniBatchOperations: If required calls postBatchMutate() CP hook and
4708 // complete mvcc for last writeEntry
4709 batchOp.completeMiniBatchOperations(miniBatchOp, writeEntry);
4710 writeEntry = null;
4711 success = true;
4712 } finally {
4713 // Call complete rather than completeAndWait because we probably had error if walKey != null
4714 if (writeEntry != null) mvcc.complete(writeEntry);
4716 if (locked) {
4717 this.updatesLock.readLock().unlock();
4719 releaseRowLocks(acquiredRowLocks);
4721 enableInterrupts();
4723 final int finalLastIndexExclusive =
4724 miniBatchOp != null ? miniBatchOp.getLastIndexExclusive() : batchOp.size();
4725 final boolean finalSuccess = success;
4726 batchOp.visitBatchOperations(true, finalLastIndexExclusive,
4727 (int i) -> {
4728 Mutation mutation = batchOp.getMutation(i);
4729 if (mutation instanceof Increment || mutation instanceof Append) {
4730 if (finalSuccess) {
4731 batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.SUCCESS,
4732 batchOp.results[i]);
4733 } else {
4734 batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
4736 } else {
4737 batchOp.retCodeDetails[i] =
4738 finalSuccess ? OperationStatus.SUCCESS : OperationStatus.FAILURE;
4740 return true;
4743 batchOp.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, finalSuccess);
4745 batchOp.nextIndexToProcess = finalLastIndexExclusive;
4750 * Returns effective durability from the passed durability and
4751 * the table descriptor.
4753 private Durability getEffectiveDurability(Durability d) {
4754 return d == Durability.USE_DEFAULT ? this.regionDurability : d;
4757 @Override
4758 @Deprecated
4759 public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
4760 ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException {
4761 CheckAndMutate checkAndMutate;
4762 try {
4763 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row)
4764 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange);
4765 if (mutation instanceof Put) {
4766 checkAndMutate = builder.build((Put) mutation);
4767 } else if (mutation instanceof Delete) {
4768 checkAndMutate = builder.build((Delete) mutation);
4769 } else {
4770 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass()
4771 .getSimpleName().toUpperCase());
4773 } catch (IllegalArgumentException e) {
4774 throw new DoNotRetryIOException(e.getMessage());
4776 return checkAndMutate(checkAndMutate).isSuccess();
4779 @Override
4780 @Deprecated
4781 public boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation)
4782 throws IOException {
4783 CheckAndMutate checkAndMutate;
4784 try {
4785 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row).ifMatches(filter)
4786 .timeRange(timeRange);
4787 if (mutation instanceof Put) {
4788 checkAndMutate = builder.build((Put) mutation);
4789 } else if (mutation instanceof Delete) {
4790 checkAndMutate = builder.build((Delete) mutation);
4791 } else {
4792 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass()
4793 .getSimpleName().toUpperCase());
4795 } catch (IllegalArgumentException e) {
4796 throw new DoNotRetryIOException(e.getMessage());
4798 return checkAndMutate(checkAndMutate).isSuccess();
4801 @Override
4802 @Deprecated
4803 public boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
4804 ByteArrayComparable comparator, TimeRange timeRange, RowMutations rm) throws IOException {
4805 CheckAndMutate checkAndMutate;
4806 try {
4807 checkAndMutate = CheckAndMutate.newBuilder(row)
4808 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange).build(rm);
4809 } catch (IllegalArgumentException e) {
4810 throw new DoNotRetryIOException(e.getMessage());
4812 return checkAndMutate(checkAndMutate).isSuccess();
4815 @Override
4816 @Deprecated
4817 public boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations rm)
4818 throws IOException {
4819 CheckAndMutate checkAndMutate;
4820 try {
4821 checkAndMutate = CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange)
4822 .build(rm);
4823 } catch (IllegalArgumentException e) {
4824 throw new DoNotRetryIOException(e.getMessage());
4826 return checkAndMutate(checkAndMutate).isSuccess();
4829 @Override
4830 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException {
4831 return checkAndMutate(checkAndMutate, HConstants.NO_NONCE, HConstants.NO_NONCE);
4834 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long nonceGroup,
4835 long nonce) throws IOException {
4836 return TraceUtil.trace(() -> checkAndMutateInternal(checkAndMutate, nonceGroup, nonce),
4837 () -> createRegionSpan("Region.checkAndMutate"));
4840 private CheckAndMutateResult checkAndMutateInternal(CheckAndMutate checkAndMutate,
4841 long nonceGroup, long nonce) throws IOException {
4842 byte[] row = checkAndMutate.getRow();
4843 Filter filter = null;
4844 byte[] family = null;
4845 byte[] qualifier = null;
4846 CompareOperator op = null;
4847 ByteArrayComparable comparator = null;
4848 if (checkAndMutate.hasFilter()) {
4849 filter = checkAndMutate.getFilter();
4850 } else {
4851 family = checkAndMutate.getFamily();
4852 qualifier = checkAndMutate.getQualifier();
4853 op = checkAndMutate.getCompareOp();
4854 comparator = new BinaryComparator(checkAndMutate.getValue());
4856 TimeRange timeRange = checkAndMutate.getTimeRange();
4858 Mutation mutation = null;
4859 RowMutations rowMutations = null;
4860 if (checkAndMutate.getAction() instanceof Mutation) {
4861 mutation = (Mutation) checkAndMutate.getAction();
4862 } else {
4863 rowMutations = (RowMutations) checkAndMutate.getAction();
4866 if (mutation != null) {
4867 checkMutationType(mutation);
4868 checkRow(mutation, row);
4869 } else {
4870 checkRow(rowMutations, row);
4872 checkReadOnly();
4873 // TODO, add check for value length also move this check to the client
4874 checkResources();
4875 startRegionOperation();
4876 try {
4877 Get get = new Get(row);
4878 if (family != null) {
4879 checkFamily(family);
4880 get.addColumn(family, qualifier);
4882 if (filter != null) {
4883 get.setFilter(filter);
4885 if (timeRange != null) {
4886 get.setTimeRange(timeRange.getMin(), timeRange.getMax());
4888 // Lock row - note that doBatchMutate will relock this row if called
4889 checkRow(row, "doCheckAndRowMutate");
4890 RowLock rowLock = getRowLock(get.getRow(), false, null);
4891 try {
4892 if (this.getCoprocessorHost() != null) {
4893 CheckAndMutateResult result =
4894 getCoprocessorHost().preCheckAndMutateAfterRowLock(checkAndMutate);
4895 if (result != null) {
4896 return result;
4900 // NOTE: We used to wait here until mvcc caught up: mvcc.await();
4901 // Supposition is that now all changes are done under row locks, then when we go to read,
4902 // we'll get the latest on this row.
4903 boolean matches = false;
4904 long cellTs = 0;
4905 try (RegionScanner scanner = getScanner(new Scan(get))) {
4906 // NOTE: Please don't use HRegion.get() instead,
4907 // because it will copy cells to heap. See HBASE-26036
4908 List<Cell> result = new ArrayList<>(1);
4909 scanner.next(result);
4910 if (filter != null) {
4911 if (!result.isEmpty()) {
4912 matches = true;
4913 cellTs = result.get(0).getTimestamp();
4915 } else {
4916 boolean valueIsNull =
4917 comparator.getValue() == null || comparator.getValue().length == 0;
4918 if (result.isEmpty() && valueIsNull) {
4919 matches = true;
4920 } else if (result.size() > 0 && result.get(0).getValueLength() == 0 && valueIsNull) {
4921 matches = true;
4922 cellTs = result.get(0).getTimestamp();
4923 } else if (result.size() == 1 && !valueIsNull) {
4924 Cell kv = result.get(0);
4925 cellTs = kv.getTimestamp();
4926 int compareResult = PrivateCellUtil.compareValue(kv, comparator);
4927 matches = matches(op, compareResult);
4932 // If matches, perform the mutation or the rowMutations
4933 if (matches) {
4934 // We have acquired the row lock already. If the system clock is NOT monotonically
4935 // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
4936 // larger timestamp than what was observed via Get. doBatchMutate already does this, but
4937 // there is no way to pass the cellTs. See HBASE-14054.
4938 long now = EnvironmentEdgeManager.currentTime();
4939 long ts = Math.max(now, cellTs); // ensure write is not eclipsed
4940 byte[] byteTs = Bytes.toBytes(ts);
4941 if (mutation != null) {
4942 if (mutation instanceof Put) {
4943 updateCellTimestamps(mutation.getFamilyCellMap().values(), byteTs);
4945 // And else 'delete' is not needed since it already does a second get, and sets the
4946 // timestamp from get (see prepareDeleteTimestamps).
4947 } else {
4948 for (Mutation m : rowMutations.getMutations()) {
4949 if (m instanceof Put) {
4950 updateCellTimestamps(m.getFamilyCellMap().values(), byteTs);
4953 // And else 'delete' is not needed since it already does a second get, and sets the
4954 // timestamp from get (see prepareDeleteTimestamps).
4956 // All edits for the given row (across all column families) must happen atomically.
4957 Result r;
4958 if (mutation != null) {
4959 r = mutate(mutation, true, nonceGroup, nonce).getResult();
4960 } else {
4961 r = mutateRow(rowMutations, nonceGroup, nonce);
4963 this.checkAndMutateChecksPassed.increment();
4964 return new CheckAndMutateResult(true, r);
4966 this.checkAndMutateChecksFailed.increment();
4967 return new CheckAndMutateResult(false, null);
4968 } finally {
4969 rowLock.release();
4971 } finally {
4972 closeRegionOperation();
4976 private void checkMutationType(final Mutation mutation) throws DoNotRetryIOException {
4977 if (!(mutation instanceof Put) && !(mutation instanceof Delete) &&
4978 !(mutation instanceof Increment) && !(mutation instanceof Append)) {
4979 throw new org.apache.hadoop.hbase.DoNotRetryIOException(
4980 "Action must be Put or Delete or Increment or Delete");
4984 private void checkRow(final Row action, final byte[] row)
4985 throws DoNotRetryIOException {
4986 if (!Bytes.equals(row, action.getRow())) {
4987 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's getRow must match");
4991 private boolean matches(final CompareOperator op, final int compareResult) {
4992 boolean matches = false;
4993 switch (op) {
4994 case LESS:
4995 matches = compareResult < 0;
4996 break;
4997 case LESS_OR_EQUAL:
4998 matches = compareResult <= 0;
4999 break;
5000 case EQUAL:
5001 matches = compareResult == 0;
5002 break;
5003 case NOT_EQUAL:
5004 matches = compareResult != 0;
5005 break;
5006 case GREATER_OR_EQUAL:
5007 matches = compareResult >= 0;
5008 break;
5009 case GREATER:
5010 matches = compareResult > 0;
5011 break;
5012 default:
5013 throw new RuntimeException("Unknown Compare op " + op.name());
5015 return matches;
5018 private OperationStatus mutate(Mutation mutation) throws IOException {
5019 return mutate(mutation, false);
5022 private OperationStatus mutate(Mutation mutation, boolean atomic) throws IOException {
5023 return mutate(mutation, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE);
5026 private OperationStatus mutate(Mutation mutation, boolean atomic, long nonceGroup, long nonce)
5027 throws IOException {
5028 OperationStatus[] status =
5029 this.batchMutate(new Mutation[] { mutation }, atomic, nonceGroup, nonce);
5030 if (status[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
5031 throw new FailedSanityCheckException(status[0].getExceptionMsg());
5032 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
5033 throw new NoSuchColumnFamilyException(status[0].getExceptionMsg());
5034 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.STORE_TOO_BUSY)) {
5035 throw new RegionTooBusyException(status[0].getExceptionMsg());
5037 return status[0];
5041 * Complete taking the snapshot on the region. Writes the region info and adds references to the
5042 * working snapshot directory.
5044 * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
5045 * arg. (In the future other cancellable HRegion methods could eventually add a
5046 * {@link ForeignExceptionSnare}, or we could do something fancier).
5048 * @param desc snapshot description object
5049 * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to
5050 * bail out. This is allowed to be null and will just be ignored in that case.
5051 * @throws IOException if there is an external or internal error causing the snapshot to fail
5053 public void addRegionToSnapshot(SnapshotDescription desc,
5054 ForeignExceptionSnare exnSnare) throws IOException {
5055 Path rootDir = CommonFSUtils.getRootDir(conf);
5056 Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
5058 SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
5059 snapshotDir, desc, exnSnare);
5060 manifest.addRegion(this);
5063 private void updateSequenceId(final Iterable<List<Cell>> cellItr, final long sequenceId)
5064 throws IOException {
5065 for (List<Cell> cells: cellItr) {
5066 if (cells == null) return;
5067 for (Cell cell : cells) {
5068 PrivateCellUtil.setSequenceId(cell, sequenceId);
5074 * Replace any cell timestamps set to {@link org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP}
5075 * provided current timestamp.
5076 * @param cellItr
5077 * @param now
5079 private static void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
5080 throws IOException {
5081 for (List<Cell> cells: cellItr) {
5082 if (cells == null) continue;
5083 // Optimization: 'foreach' loop is not used. See:
5084 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5085 assert cells instanceof RandomAccess;
5086 int listSize = cells.size();
5087 for (int i = 0; i < listSize; i++) {
5088 PrivateCellUtil.updateLatestStamp(cells.get(i), now);
5094 * Possibly rewrite incoming cell tags.
5096 private void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
5097 // Check if we have any work to do and early out otherwise
5098 // Update these checks as more logic is added here
5099 if (m.getTTL() == Long.MAX_VALUE) {
5100 return;
5103 // From this point we know we have some work to do
5104 for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
5105 List<Cell> cells = e.getValue();
5106 assert cells instanceof RandomAccess;
5107 int listSize = cells.size();
5108 for (int i = 0; i < listSize; i++) {
5109 Cell cell = cells.get(i);
5110 List<Tag> newTags = TagUtil.carryForwardTags(null, cell);
5111 newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL());
5112 // Rewrite the cell with the updated set of tags
5113 cells.set(i, PrivateCellUtil.createCell(cell, newTags));
5119 * Check if resources to support an update.
5120 * <p/>
5121 * We throw RegionTooBusyException if above memstore limit and expect client to retry using some
5122 * kind of backoff
5124 private void checkResources() throws RegionTooBusyException {
5125 // If catalog region, do not impose resource constraints or block updates.
5126 if (this.getRegionInfo().isMetaRegion()) {
5127 return;
5130 MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
5131 if (mss.getHeapSize() + mss.getOffHeapSize() > this.blockingMemStoreSize) {
5132 blockedRequestsCount.increment();
5133 requestFlush();
5134 // Don't print current limit because it will vary too much. The message is used as a key
5135 // over in RetriesExhaustedWithDetailsException processing.
5136 final String regionName =
5137 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getEncodedName();
5138 final String serverName = this.getRegionServerServices() == null ?
5139 "unknown" : (this.getRegionServerServices().getServerName() == null ? "unknown" :
5140 this.getRegionServerServices().getServerName().toString());
5141 RegionTooBusyException rtbe = new RegionTooBusyException(
5142 "Over memstore limit=" + org.apache.hadoop.hbase.procedure2.util.StringUtils
5143 .humanSize(this.blockingMemStoreSize) + ", regionName=" + regionName + ", server="
5144 + serverName);
5145 LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe);
5146 throw rtbe;
5151 * @throws IOException Throws exception if region is in read-only mode.
5153 private void checkReadOnly() throws IOException {
5154 if (isReadOnly()) {
5155 throw new DoNotRetryIOException("region is read only");
5159 private void checkReadsEnabled() throws IOException {
5160 if (!this.writestate.readsEnabled) {
5161 throw new IOException(getRegionInfo().getEncodedName()
5162 + ": The region's reads are disabled. Cannot serve the request");
5166 public void setReadsEnabled(boolean readsEnabled) {
5167 if (readsEnabled && !this.writestate.readsEnabled) {
5168 LOG.info("Enabling reads for {}", getRegionInfo().getEncodedName());
5170 this.writestate.setReadsEnabled(readsEnabled);
5174 * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be
5175 * set; when set we will run operations that make sense in the increment/append scenario
5176 * but that do not make sense otherwise.
5178 private void applyToMemStore(HStore store, List<Cell> cells, boolean delta,
5179 MemStoreSizing memstoreAccounting) throws IOException {
5180 // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!!
5181 boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
5182 if (upsert) {
5183 store.upsert(cells, getSmallestReadPoint(), memstoreAccounting);
5184 } else {
5185 store.add(cells, memstoreAccounting);
5189 private void checkFamilies(Collection<byte[]> families, Durability durability)
5190 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5191 for (byte[] family : families) {
5192 checkFamily(family, durability);
5196 private void checkFamily(final byte[] family, Durability durability)
5197 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5198 checkFamily(family);
5199 if (durability.equals(Durability.SKIP_WAL)
5200 && htableDescriptor.getColumnFamily(family).getScope()
5201 != HConstants.REPLICATION_SCOPE_LOCAL) {
5202 throw new InvalidMutationDurabilityException(
5203 "Mutation's durability is SKIP_WAL but table's column family " + Bytes.toString(family)
5204 + " need replication");
5208 private void checkFamily(final byte[] family) throws NoSuchColumnFamilyException {
5209 if (!this.htableDescriptor.hasColumnFamily(family)) {
5210 throw new NoSuchColumnFamilyException(
5211 "Column family " + Bytes.toString(family) + " does not exist in region " + this
5212 + " in table " + this.htableDescriptor);
5217 * Check the collection of families for valid timestamps
5218 * @param familyMap
5219 * @param now current timestamp
5220 * @throws FailedSanityCheckException
5222 public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
5223 throws FailedSanityCheckException {
5224 if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
5225 return;
5227 long maxTs = now + timestampSlop;
5228 for (List<Cell> kvs : familyMap.values()) {
5229 // Optimization: 'foreach' loop is not used. See:
5230 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5231 assert kvs instanceof RandomAccess;
5232 int listSize = kvs.size();
5233 for (int i=0; i < listSize; i++) {
5234 Cell cell = kvs.get(i);
5235 // see if the user-side TS is out of range. latest = server-side
5236 long ts = cell.getTimestamp();
5237 if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
5238 throw new FailedSanityCheckException("Timestamp for KV out of range "
5239 + cell + " (too.new=" + timestampSlop + ")");
5246 * @param size
5247 * @return True if size is over the flush threshold
5249 private boolean isFlushSize(MemStoreSize size) {
5250 return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize();
5253 private void deleteRecoveredEdits(FileSystem fs, Iterable<Path> files) throws IOException {
5254 for (Path file : files) {
5255 if (!fs.delete(file, false)) {
5256 LOG.error("Failed delete of {}", file);
5257 } else {
5258 LOG.debug("Deleted recovered.edits file={}", file);
5264 * Read the edits put under this region by wal splitting process. Put
5265 * the recovered edits back up into this region.
5267 * <p>We can ignore any wal message that has a sequence ID that's equal to or
5268 * lower than minSeqId. (Because we know such messages are already
5269 * reflected in the HFiles.)
5271 * <p>While this is running we are putting pressure on memory yet we are
5272 * outside of our usual accounting because we are not yet an onlined region
5273 * (this stuff is being run as part of Region initialization). This means
5274 * that if we're up against global memory limits, we'll not be flagged to flush
5275 * because we are not online. We can't be flushed by usual mechanisms anyways;
5276 * we're not yet online so our relative sequenceids are not yet aligned with
5277 * WAL sequenceids -- not till we come up online, post processing of split
5278 * edits.
5280 * <p>But to help relieve memory pressure, at least manage our own heap size
5281 * flushing if are in excess of per-region limits. Flushing, though, we have
5282 * to be careful and avoid using the regionserver/wal sequenceid. Its running
5283 * on a different line to whats going on in here in this region context so if we
5284 * crashed replaying these edits, but in the midst had a flush that used the
5285 * regionserver wal with a sequenceid in excess of whats going on in here
5286 * in this region and with its split editlogs, then we could miss edits the
5287 * next time we go to recover. So, we have to flush inline, using seqids that
5288 * make sense in a this single region context only -- until we online.
5290 * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of
5291 * the maxSeqId for the store to be applied, else its skipped.
5292 * @return the sequence id of the last edit added to this region out of the
5293 * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
5295 long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
5296 final CancelableProgressable reporter, final MonitoredTask status) throws IOException {
5297 long minSeqIdForTheRegion = -1;
5298 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
5299 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
5300 minSeqIdForTheRegion = maxSeqIdInStore;
5303 long seqId = minSeqIdForTheRegion;
5304 String specialRecoveredEditsDirStr = conf.get(SPECIAL_RECOVERED_EDITS_DIR);
5305 if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) {
5306 FileSystem walFS = getWalFileSystem();
5307 FileSystem rootFS = getFilesystem();
5308 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
5309 getRegionInfo().getEncodedName());
5310 Path regionWALDir = getWALRegionDir();
5311 Path regionDir =
5312 FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo());
5314 // We made a mistake in HBASE-20734 so we need to do this dirty hack...
5315 NavigableSet<Path> filesUnderWrongRegionWALDir =
5316 WALSplitUtil.getSplitEditFilesSorted(walFS, wrongRegionWALDir);
5317 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
5318 filesUnderWrongRegionWALDir, reporter, regionDir));
5319 // This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear
5320 // under the root dir even if walDir is set.
5321 NavigableSet<Path> filesUnderRootDir = Collections.emptyNavigableSet();
5322 if (!regionWALDir.equals(regionDir)) {
5323 filesUnderRootDir = WALSplitUtil.getSplitEditFilesSorted(rootFS, regionDir);
5324 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS,
5325 filesUnderRootDir, reporter, regionDir));
5328 NavigableSet<Path> files = WALSplitUtil.getSplitEditFilesSorted(walFS, regionWALDir);
5329 seqId = Math.max(seqId,
5330 replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, files, reporter, regionWALDir));
5331 if (seqId > minSeqIdForTheRegion) {
5332 // Then we added some edits to memory. Flush and cleanup split edit files.
5333 internalFlushcache(null, seqId, stores.values(), status, false,
5334 FlushLifeCycleTracker.DUMMY);
5336 // Now delete the content of recovered edits. We're done w/ them.
5337 if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
5338 // For debugging data loss issues!
5339 // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
5340 // column family. Have to fake out file type too by casting our recovered.edits as
5341 // storefiles
5342 String fakeFamilyName = WALSplitUtil.getRegionDirRecoveredEditsDir(regionWALDir).getName();
5343 Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size());
5344 for (Path file : files) {
5345 fakeStoreFiles.add(new HStoreFile(walFS, file, this.conf, null, null, true));
5347 getRegionWALFileSystem().archiveRecoveredEdits(fakeFamilyName, fakeStoreFiles);
5348 } else {
5349 deleteRecoveredEdits(walFS, Iterables.concat(files, filesUnderWrongRegionWALDir));
5350 deleteRecoveredEdits(rootFS, filesUnderRootDir);
5352 } else {
5353 Path recoveredEditsDir = new Path(specialRecoveredEditsDirStr);
5354 FileSystem fs = recoveredEditsDir.getFileSystem(conf);
5355 FileStatus[] files = fs.listStatus(recoveredEditsDir);
5356 LOG.debug("Found {} recovered edits file(s) under {}", files == null ? 0 : files.length,
5357 recoveredEditsDir);
5358 if (files != null) {
5359 for (FileStatus file : files) {
5360 // it is safe to trust the zero-length in this case because we've been through rename and
5361 // lease recovery in the above.
5362 if (isZeroLengthThenDelete(fs, file, file.getPath())) {
5363 continue;
5365 seqId =
5366 Math.max(seqId, replayRecoveredEdits(file.getPath(), maxSeqIdInStores, reporter, fs));
5369 if (seqId > minSeqIdForTheRegion) {
5370 // Then we added some edits to memory. Flush and cleanup split edit files.
5371 internalFlushcache(null, seqId, stores.values(), status, false,
5372 FlushLifeCycleTracker.DUMMY);
5374 deleteRecoveredEdits(fs,
5375 Stream.of(files).map(FileStatus::getPath).collect(Collectors.toList()));
5378 return seqId;
5381 private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
5382 final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir)
5383 throws IOException {
5384 long seqid = minSeqIdForTheRegion;
5385 if (LOG.isDebugEnabled()) {
5386 LOG.debug("Found " + (files == null ? 0 : files.size())
5387 + " recovered edits file(s) under " + regionDir);
5390 if (files == null || files.isEmpty()) {
5391 return minSeqIdForTheRegion;
5394 for (Path edits: files) {
5395 if (edits == null || !fs.exists(edits)) {
5396 LOG.warn("Null or non-existent edits file: " + edits);
5397 continue;
5399 if (isZeroLengthThenDelete(fs, fs.getFileStatus(edits), edits)) {
5400 continue;
5403 long maxSeqId;
5404 String fileName = edits.getName();
5405 maxSeqId = Math.abs(Long.parseLong(fileName));
5406 if (maxSeqId <= minSeqIdForTheRegion) {
5407 if (LOG.isDebugEnabled()) {
5408 String msg = "Maximum sequenceid for this wal is " + maxSeqId
5409 + " and minimum sequenceid for the region " + this + " is " + minSeqIdForTheRegion
5410 + ", skipped the whole file, path=" + edits;
5411 LOG.debug(msg);
5413 continue;
5416 try {
5417 // replay the edits. Replay can return -1 if everything is skipped, only update
5418 // if seqId is greater
5419 seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs));
5420 } catch (IOException e) {
5421 handleException(fs, edits, e);
5424 return seqid;
5427 private void handleException(FileSystem fs, Path edits, IOException e) throws IOException {
5428 boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
5429 conf.getBoolean("hbase.skip.errors", HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
5430 if (conf.get("hbase.skip.errors") != null) {
5431 LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use "
5432 + HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
5434 if (skipErrors) {
5435 Path p = WALSplitUtil.moveAsideBadEditsFile(fs, edits);
5436 LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + "=true so continuing. Renamed "
5437 + edits + " as " + p,
5439 } else {
5440 throw e;
5445 * @param edits File of recovered edits.
5446 * @param maxSeqIdInStores Maximum sequenceid found in each store. Edits in wal must be larger
5447 * than this to be replayed for each store.
5448 * @return the sequence id of the last edit added to this region out of the recovered edits log or
5449 * <code>minSeqId</code> if nothing added from editlogs.
5451 private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores,
5452 final CancelableProgressable reporter, FileSystem fs) throws IOException {
5453 String msg = "Replaying edits from " + edits;
5454 LOG.info(msg);
5455 MonitoredTask status = TaskMonitor.get().createStatus(msg);
5457 status.setStatus("Opening recovered edits");
5458 WAL.Reader reader = null;
5459 try {
5460 reader = WALFactory.createReader(fs, edits, conf);
5461 long currentEditSeqId = -1;
5462 long currentReplaySeqId = -1;
5463 long firstSeqIdInLog = -1;
5464 long skippedEdits = 0;
5465 long editsCount = 0;
5466 long intervalEdits = 0;
5467 WAL.Entry entry;
5468 HStore store = null;
5469 boolean reported_once = false;
5470 ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
5472 try {
5473 // How many edits seen before we check elapsed time
5474 int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
5475 // How often to send a progress report (default 1/2 master timeout)
5476 int period = this.conf.getInt("hbase.hstore.report.period", 300000);
5477 long lastReport = EnvironmentEdgeManager.currentTime();
5479 if (coprocessorHost != null) {
5480 coprocessorHost.preReplayWALs(this.getRegionInfo(), edits);
5483 while ((entry = reader.next()) != null) {
5484 WALKey key = entry.getKey();
5485 WALEdit val = entry.getEdit();
5487 if (ng != null) { // some test, or nonces disabled
5488 ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
5491 if (reporter != null) {
5492 intervalEdits += val.size();
5493 if (intervalEdits >= interval) {
5494 // Number of edits interval reached
5495 intervalEdits = 0;
5496 long cur = EnvironmentEdgeManager.currentTime();
5497 if (lastReport + period <= cur) {
5498 status.setStatus("Replaying edits..." +
5499 " skipped=" + skippedEdits +
5500 " edits=" + editsCount);
5501 // Timeout reached
5502 if(!reporter.progress()) {
5503 msg = "Progressable reporter failed, stopping replay for region " + this;
5504 LOG.warn(msg);
5505 status.abort(msg);
5506 throw new IOException(msg);
5508 reported_once = true;
5509 lastReport = cur;
5514 if (firstSeqIdInLog == -1) {
5515 firstSeqIdInLog = key.getSequenceId();
5517 if (currentEditSeqId > key.getSequenceId()) {
5518 // when this condition is true, it means we have a serious defect because we need to
5519 // maintain increasing SeqId for WAL edits per region
5520 LOG.error(getRegionInfo().getEncodedName() + " : "
5521 + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
5522 + "; edit=" + val);
5523 } else {
5524 currentEditSeqId = key.getSequenceId();
5526 currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
5527 key.getOrigLogSeqNum() : currentEditSeqId;
5529 // Start coprocessor replay here. The coprocessor is for each WALEdit
5530 // instead of a KeyValue.
5531 if (coprocessorHost != null) {
5532 status.setStatus("Running pre-WAL-restore hook in coprocessors");
5533 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
5534 // if bypass this wal entry, ignore it ...
5535 continue;
5538 boolean checkRowWithinBoundary = false;
5539 // Check this edit is for this region.
5540 if (!Bytes.equals(key.getEncodedRegionName(),
5541 this.getRegionInfo().getEncodedNameAsBytes())) {
5542 checkRowWithinBoundary = true;
5545 boolean flush = false;
5546 MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
5547 for (Cell cell: val.getCells()) {
5548 // Check this edit is for me. Also, guard against writing the special
5549 // METACOLUMN info such as HBASE::CACHEFLUSH entries
5550 if (WALEdit.isMetaEditFamily(cell)) {
5551 // if region names don't match, skipp replaying compaction marker
5552 if (!checkRowWithinBoundary) {
5553 //this is a special edit, we should handle it
5554 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
5555 if (compaction != null) {
5556 //replay the compaction
5557 replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
5560 skippedEdits++;
5561 continue;
5563 // Figure which store the edit is meant for.
5564 if (store == null || !CellUtil.matchingFamily(cell,
5565 store.getColumnFamilyDescriptor().getName())) {
5566 store = getStore(cell);
5568 if (store == null) {
5569 // This should never happen. Perhaps schema was changed between
5570 // crash and redeploy?
5571 LOG.warn("No family for cell {} in region {}", cell, this);
5572 skippedEdits++;
5573 continue;
5575 if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
5576 cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
5577 LOG.warn("Row of {} is not within region boundary for region {}", cell, this);
5578 skippedEdits++;
5579 continue;
5581 // Now, figure if we should skip this edit.
5582 if (key.getSequenceId() <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor()
5583 .getName())) {
5584 skippedEdits++;
5585 continue;
5587 PrivateCellUtil.setSequenceId(cell, currentReplaySeqId);
5589 restoreEdit(store, cell, memStoreSizing);
5590 editsCount++;
5592 MemStoreSize mss = memStoreSizing.getMemStoreSize();
5593 incMemStoreSize(mss);
5594 flush = isFlushSize(this.memStoreSizing.getMemStoreSize());
5595 if (flush) {
5596 internalFlushcache(null, currentEditSeqId, stores.values(), status, false,
5597 FlushLifeCycleTracker.DUMMY);
5600 if (coprocessorHost != null) {
5601 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
5605 if (coprocessorHost != null) {
5606 coprocessorHost.postReplayWALs(this.getRegionInfo(), edits);
5608 } catch (EOFException eof) {
5609 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5610 msg = "EnLongAddered EOF. Most likely due to Master failure during "
5611 + "wal splitting, so we have this data in another edit. Continuing, but renaming "
5612 + edits + " as " + p + " for region " + this;
5613 LOG.warn(msg, eof);
5614 status.abort(msg);
5615 } catch (IOException ioe) {
5616 // If the IOE resulted from bad file format,
5617 // then this problem is idempotent and retrying won't help
5618 if (ioe.getCause() instanceof ParseException) {
5619 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5620 msg = "File corruption enLongAddered! " +
5621 "Continuing, but renaming " + edits + " as " + p;
5622 LOG.warn(msg, ioe);
5623 status.setStatus(msg);
5624 } else {
5625 status.abort(StringUtils.stringifyException(ioe));
5626 // other IO errors may be transient (bad network connection,
5627 // checksum exception on one datanode, etc). throw & retry
5628 throw ioe;
5631 if (reporter != null && !reported_once) {
5632 reporter.progress();
5634 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
5635 ", firstSequenceIdInLog=" + firstSeqIdInLog +
5636 ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
5637 status.markComplete(msg);
5638 LOG.debug(msg);
5639 return currentEditSeqId;
5640 } finally {
5641 status.cleanup();
5642 if (reader != null) {
5643 reader.close();
5649 * Call to complete a compaction. Its for the case where we find in the WAL a compaction
5650 * that was not finished. We could find one recovering a WAL after a regionserver crash.
5651 * See HBASE-2331.
5653 void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
5654 boolean removeFiles, long replaySeqId)
5655 throws IOException {
5656 try {
5657 checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
5658 "Compaction marker from WAL ", compaction);
5659 } catch (WrongRegionException wre) {
5660 if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5661 // skip the compaction marker since it is not for this region
5662 return;
5664 throw wre;
5667 synchronized (writestate) {
5668 if (replaySeqId < lastReplayedOpenRegionSeqId) {
5669 LOG.warn(getRegionInfo().getEncodedName() + " : "
5670 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
5671 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5672 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
5673 return;
5675 if (replaySeqId < lastReplayedCompactionSeqId) {
5676 LOG.warn(getRegionInfo().getEncodedName() + " : "
5677 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
5678 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5679 + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId);
5680 return;
5681 } else {
5682 lastReplayedCompactionSeqId = replaySeqId;
5685 if (LOG.isDebugEnabled()) {
5686 LOG.debug(getRegionInfo().getEncodedName() + " : "
5687 + "Replaying compaction marker " + TextFormat.shortDebugString(compaction)
5688 + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId="
5689 + lastReplayedOpenRegionSeqId);
5692 startRegionOperation(Operation.REPLAY_EVENT);
5693 try {
5694 HStore store = this.getStore(compaction.getFamilyName().toByteArray());
5695 if (store == null) {
5696 LOG.warn(getRegionInfo().getEncodedName() + " : "
5697 + "Found Compaction WAL edit for deleted family:"
5698 + Bytes.toString(compaction.getFamilyName().toByteArray()));
5699 return;
5701 store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
5702 logRegionFiles();
5703 } catch (FileNotFoundException ex) {
5704 LOG.warn(getRegionInfo().getEncodedName() + " : "
5705 + "At least one of the store files in compaction: "
5706 + TextFormat.shortDebugString(compaction)
5707 + " doesn't exist any more. Skip loading the file(s)", ex);
5708 } finally {
5709 closeRegionOperation(Operation.REPLAY_EVENT);
5714 void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
5715 checkTargetRegion(flush.getEncodedRegionName().toByteArray(),
5716 "Flush marker from WAL ", flush);
5718 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5719 return; // if primary nothing to do
5722 if (LOG.isDebugEnabled()) {
5723 LOG.debug(getRegionInfo().getEncodedName() + " : "
5724 + "Replaying flush marker " + TextFormat.shortDebugString(flush));
5727 startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close
5728 try {
5729 FlushAction action = flush.getAction();
5730 switch (action) {
5731 case START_FLUSH:
5732 replayWALFlushStartMarker(flush);
5733 break;
5734 case COMMIT_FLUSH:
5735 replayWALFlushCommitMarker(flush);
5736 break;
5737 case ABORT_FLUSH:
5738 replayWALFlushAbortMarker(flush);
5739 break;
5740 case CANNOT_FLUSH:
5741 replayWALFlushCannotFlushMarker(flush, replaySeqId);
5742 break;
5743 default:
5744 LOG.warn(getRegionInfo().getEncodedName() + " : " +
5745 "Received a flush event with unknown action, ignoring. " +
5746 TextFormat.shortDebugString(flush));
5747 break;
5750 logRegionFiles();
5751 } finally {
5752 closeRegionOperation(Operation.REPLAY_EVENT);
5756 /** Replay the flush marker from primary region by creating a corresponding snapshot of
5757 * the store memstores, only if the memstores do not have a higher seqId from an earlier wal
5758 * edit (because the events may be coming out of order).
5760 PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
5761 long flushSeqId = flush.getFlushSequenceNumber();
5763 HashSet<HStore> storesToFlush = new HashSet<>();
5764 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
5765 byte[] family = storeFlush.getFamilyName().toByteArray();
5766 HStore store = getStore(family);
5767 if (store == null) {
5768 LOG.warn(getRegionInfo().getEncodedName() + " : "
5769 + "Received a flush start marker from primary, but the family is not found. Ignoring"
5770 + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
5771 continue;
5773 storesToFlush.add(store);
5776 MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
5778 // we will use writestate as a coarse-grain lock for all the replay events
5779 // (flush, compaction, region open etc)
5780 synchronized (writestate) {
5781 try {
5782 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
5783 LOG.warn(getRegionInfo().getEncodedName() + " : "
5784 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5785 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5786 + " of " + lastReplayedOpenRegionSeqId);
5787 return null;
5789 if (numMutationsWithoutWAL.sum() > 0) {
5790 numMutationsWithoutWAL.reset();
5791 dataInMemoryWithoutWAL.reset();
5794 if (!writestate.flushing) {
5795 // we do not have an active snapshot and corresponding this.prepareResult. This means
5796 // we can just snapshot our memstores and continue as normal.
5798 // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal
5799 PrepareFlushResult prepareResult = internalPrepareFlushCache(null, flushSeqId,
5800 storesToFlush, status, false, FlushLifeCycleTracker.DUMMY);
5801 if (prepareResult.result == null) {
5802 // save the PrepareFlushResult so that we can use it later from commit flush
5803 this.writestate.flushing = true;
5804 this.prepareFlushResult = prepareResult;
5805 status.markComplete("Flush prepare successful");
5806 if (LOG.isDebugEnabled()) {
5807 LOG.debug(getRegionInfo().getEncodedName() + " : "
5808 + " Prepared flush with seqId:" + flush.getFlushSequenceNumber());
5810 } else {
5811 // special case empty memstore. We will still save the flush result in this case, since
5812 // our memstore ie empty, but the primary is still flushing
5813 if (prepareResult.getResult().getResult() ==
5814 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
5815 this.writestate.flushing = true;
5816 this.prepareFlushResult = prepareResult;
5817 if (LOG.isDebugEnabled()) {
5818 LOG.debug(getRegionInfo().getEncodedName() + " : "
5819 + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
5822 status.abort("Flush prepare failed with " + prepareResult.result);
5823 // nothing much to do. prepare flush failed because of some reason.
5825 return prepareResult;
5826 } else {
5827 // we already have an active snapshot.
5828 if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
5829 // They define the same flush. Log and continue.
5830 LOG.warn(getRegionInfo().getEncodedName() + " : "
5831 + "Received a flush prepare marker with the same seqId: " +
5832 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5833 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5834 // ignore
5835 } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
5836 // We received a flush with a smaller seqNum than what we have prepared. We can only
5837 // ignore this prepare flush request.
5838 LOG.warn(getRegionInfo().getEncodedName() + " : "
5839 + "Received a flush prepare marker with a smaller seqId: " +
5840 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5841 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5842 // ignore
5843 } else {
5844 // We received a flush with a larger seqNum than what we have prepared
5845 LOG.warn(getRegionInfo().getEncodedName() + " : "
5846 + "Received a flush prepare marker with a larger seqId: " +
5847 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5848 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5849 // We do not have multiple active snapshots in the memstore or a way to merge current
5850 // memstore snapshot with the contents and resnapshot for now. We cannot take
5851 // another snapshot and drop the previous one because that will cause temporary
5852 // data loss in the secondary. So we ignore this for now, deferring the resolution
5853 // to happen when we see the corresponding flush commit marker. If we have a memstore
5854 // snapshot with x, and later received another prepare snapshot with y (where x < y),
5855 // when we see flush commit for y, we will drop snapshot for x, and can also drop all
5856 // the memstore edits if everything in memstore is < y. This is the usual case for
5857 // RS crash + recovery where we might see consequtive prepare flush wal markers.
5858 // Otherwise, this will cause more memory to be used in secondary replica until a
5859 // further prapare + commit flush is seen and replayed.
5862 } finally {
5863 status.cleanup();
5864 writestate.notifyAll();
5867 return null;
5870 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
5871 justification="Intentional; post memstore flush")
5872 void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
5873 MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
5875 // check whether we have the memstore snapshot with the corresponding seqId. Replay to
5876 // secondary region replicas are in order, except for when the region moves or then the
5877 // region server crashes. In those cases, we may receive replay requests out of order from
5878 // the original seqIds.
5879 synchronized (writestate) {
5880 try {
5881 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
5882 LOG.warn(getRegionInfo().getEncodedName() + " : "
5883 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5884 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5885 + " of " + lastReplayedOpenRegionSeqId);
5886 return;
5889 if (writestate.flushing) {
5890 PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
5891 if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
5892 if (LOG.isDebugEnabled()) {
5893 LOG.debug(getRegionInfo().getEncodedName() + " : "
5894 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5895 + " and a previous prepared snapshot was found");
5897 // This is the regular case where we received commit flush after prepare flush
5898 // corresponding to the same seqId.
5899 replayFlushInStores(flush, prepareFlushResult, true);
5901 // Set down the memstore size by amount of flush.
5902 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
5903 this.prepareFlushResult = null;
5904 writestate.flushing = false;
5905 } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
5906 // This should not happen normally. However, lets be safe and guard against these cases
5907 // we received a flush commit with a smaller seqId than what we have prepared
5908 // we will pick the flush file up from this commit (if we have not seen it), but we
5909 // will not drop the memstore
5910 LOG.warn(getRegionInfo().getEncodedName() + " : "
5911 + "Received a flush commit marker with smaller seqId: "
5912 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
5913 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
5914 +" prepared memstore snapshot");
5915 replayFlushInStores(flush, prepareFlushResult, false);
5917 // snapshot is not dropped, so memstore sizes should not be decremented
5918 // we still have the prepared snapshot, flushing should still be true
5919 } else {
5920 // This should not happen normally. However, lets be safe and guard against these cases
5921 // we received a flush commit with a larger seqId than what we have prepared
5922 // we will pick the flush file for this. We will also obtain the updates lock and
5923 // look for contents of the memstore to see whether we have edits after this seqId.
5924 // If not, we will drop all the memstore edits and the snapshot as well.
5925 LOG.warn(getRegionInfo().getEncodedName() + " : "
5926 + "Received a flush commit marker with larger seqId: "
5927 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " +
5928 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
5929 +" memstore snapshot");
5931 replayFlushInStores(flush, prepareFlushResult, true);
5933 // Set down the memstore size by amount of flush.
5934 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
5936 // Inspect the memstore contents to see whether the memstore contains only edits
5937 // with seqId smaller than the flush seqId. If so, we can discard those edits.
5938 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5940 this.prepareFlushResult = null;
5941 writestate.flushing = false;
5943 // If we were waiting for observing a flush or region opening event for not showing
5944 // partial data after a secondary region crash, we can allow reads now. We can only make
5945 // sure that we are not showing partial data (for example skipping some previous edits)
5946 // until we observe a full flush start and flush commit. So if we were not able to find
5947 // a previous flush we will not enable reads now.
5948 this.setReadsEnabled(true);
5949 } else {
5950 LOG.warn(getRegionInfo().getEncodedName() + " : "
5951 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5952 + ", but no previous prepared snapshot was found");
5953 // There is no corresponding prepare snapshot from before.
5954 // We will pick up the new flushed file
5955 replayFlushInStores(flush, null, false);
5957 // Inspect the memstore contents to see whether the memstore contains only edits
5958 // with seqId smaller than the flush seqId. If so, we can discard those edits.
5959 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5962 status.markComplete("Flush commit successful");
5964 // Update the last flushed sequence id for region.
5965 this.maxFlushedSeqId = flush.getFlushSequenceNumber();
5967 // advance the mvcc read point so that the new flushed file is visible.
5968 mvcc.advanceTo(flush.getFlushSequenceNumber());
5970 } catch (FileNotFoundException ex) {
5971 LOG.warn(getRegionInfo().getEncodedName() + " : "
5972 + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
5973 + " doesn't exist any more. Skip loading the file(s)", ex);
5975 finally {
5976 status.cleanup();
5977 writestate.notifyAll();
5981 // C. Finally notify anyone waiting on memstore to clear:
5982 // e.g. checkResources().
5983 synchronized (this) {
5984 notifyAll(); // FindBugs NN_NAKED_NOTIFY
5989 * Replays the given flush descriptor by opening the flush files in stores and dropping the
5990 * memstore snapshots if requested.
5991 * @param flush
5992 * @param prepareFlushResult
5993 * @param dropMemstoreSnapshot
5994 * @throws IOException
5996 private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
5997 boolean dropMemstoreSnapshot)
5998 throws IOException {
5999 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
6000 byte[] family = storeFlush.getFamilyName().toByteArray();
6001 HStore store = getStore(family);
6002 if (store == null) {
6003 LOG.warn(getRegionInfo().getEncodedName() + " : "
6004 + "Received a flush commit marker from primary, but the family is not found."
6005 + "Ignoring StoreFlushDescriptor:" + storeFlush);
6006 continue;
6008 List<String> flushFiles = storeFlush.getFlushOutputList();
6009 StoreFlushContext ctx = null;
6010 long startTime = EnvironmentEdgeManager.currentTime();
6011 if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
6012 ctx = store.createFlushContext(flush.getFlushSequenceNumber(), FlushLifeCycleTracker.DUMMY);
6013 } else {
6014 ctx = prepareFlushResult.storeFlushCtxs.get(family);
6015 startTime = prepareFlushResult.startTime;
6018 if (ctx == null) {
6019 LOG.warn(getRegionInfo().getEncodedName() + " : "
6020 + "Unexpected: flush commit marker received from store "
6021 + Bytes.toString(family) + " but no associated flush context. Ignoring");
6022 continue;
6025 ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush
6027 // Record latest flush time
6028 this.lastStoreFlushTimeMap.put(store, startTime);
6032 private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException {
6033 Path regionDir = fs.getRegionDir();
6034 long maxSeqId = -1;
6035 for (HStore store : stores) {
6036 String familyName = store.getColumnFamilyName();
6037 FileStatus[] files =
6038 WALSplitUtil.getRecoveredHFiles(fs.getFileSystem(), regionDir, familyName);
6039 if (files != null && files.length != 0) {
6040 for (FileStatus file : files) {
6041 Path filePath = file.getPath();
6042 // If file length is zero then delete it
6043 if (isZeroLengthThenDelete(fs.getFileSystem(), file, filePath)) {
6044 continue;
6046 try {
6047 HStoreFile storefile = store.tryCommitRecoveredHFile(file.getPath());
6048 maxSeqId = Math.max(maxSeqId, storefile.getReader().getSequenceID());
6049 } catch (IOException e) {
6050 handleException(fs.getFileSystem(), filePath, e);
6051 continue;
6054 if (this.rsServices != null && store.needsCompaction()) {
6055 this.rsServices.getCompactionRequestor()
6056 .requestCompaction(this, store, "load recovered hfiles request compaction",
6057 Store.PRIORITY_USER + 1, CompactionLifeCycleTracker.DUMMY, null);
6061 return maxSeqId;
6065 * Be careful, this method will drop all data in the memstore of this region.
6066 * Currently, this method is used to drop memstore to prevent memory leak
6067 * when replaying recovered.edits while opening region.
6069 private MemStoreSize dropMemStoreContents() throws IOException {
6070 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6071 this.updatesLock.writeLock().lock();
6072 try {
6073 for (HStore s : stores.values()) {
6074 MemStoreSize memStoreSize = doDropStoreMemStoreContentsForSeqId(s, HConstants.NO_SEQNUM);
6075 LOG.info("Drop memstore for Store " + s.getColumnFamilyName() + " in region "
6076 + this.getRegionInfo().getRegionNameAsString()
6077 + " , dropped memstoresize: [" + memStoreSize + " }");
6078 totalFreedSize.incMemStoreSize(memStoreSize);
6080 return totalFreedSize.getMemStoreSize();
6081 } finally {
6082 this.updatesLock.writeLock().unlock();
6087 * Drops the memstore contents after replaying a flush descriptor or region open event replay
6088 * if the memstore edits have seqNums smaller than the given seq id
6089 * @throws IOException
6091 private MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException {
6092 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6093 this.updatesLock.writeLock().lock();
6094 try {
6096 long currentSeqId = mvcc.getReadPoint();
6097 if (seqId >= currentSeqId) {
6098 // then we can drop the memstore contents since everything is below this seqId
6099 LOG.info(getRegionInfo().getEncodedName() + " : "
6100 + "Dropping memstore contents as well since replayed flush seqId: "
6101 + seqId + " is greater than current seqId:" + currentSeqId);
6103 // Prepare flush (take a snapshot) and then abort (drop the snapshot)
6104 if (store == null) {
6105 for (HStore s : stores.values()) {
6106 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(s, currentSeqId));
6108 } else {
6109 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store, currentSeqId));
6111 } else {
6112 LOG.info(getRegionInfo().getEncodedName() + " : "
6113 + "Not dropping memstore contents since replayed flush seqId: "
6114 + seqId + " is smaller than current seqId:" + currentSeqId);
6116 } finally {
6117 this.updatesLock.writeLock().unlock();
6119 return totalFreedSize.getMemStoreSize();
6122 private MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId)
6123 throws IOException {
6124 MemStoreSize flushableSize = s.getFlushableSize();
6125 this.decrMemStoreSize(flushableSize);
6126 StoreFlushContext ctx = s.createFlushContext(currentSeqId, FlushLifeCycleTracker.DUMMY);
6127 ctx.prepare();
6128 ctx.abort();
6129 return flushableSize;
6132 private void replayWALFlushAbortMarker(FlushDescriptor flush) {
6133 // nothing to do for now. A flush abort will cause a RS abort which means that the region
6134 // will be opened somewhere else later. We will see the region open event soon, and replaying
6135 // that will drop the snapshot
6138 private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
6139 synchronized (writestate) {
6140 if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
6141 LOG.warn(getRegionInfo().getEncodedName() + " : "
6142 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
6143 + " because its sequence id " + replaySeqId + " is smaller than this regions "
6144 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
6145 return;
6148 // If we were waiting for observing a flush or region opening event for not showing partial
6149 // data after a secondary region crash, we can allow reads now. This event means that the
6150 // primary was not able to flush because memstore is empty when we requested flush. By the
6151 // time we observe this, we are guaranteed to have up to date seqId with our previous
6152 // assignment.
6153 this.setReadsEnabled(true);
6157 PrepareFlushResult getPrepareFlushResult() {
6158 return prepareFlushResult;
6161 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
6162 justification="Intentional; cleared the memstore")
6163 void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
6164 checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
6165 "RegionEvent marker from WAL ", regionEvent);
6167 startRegionOperation(Operation.REPLAY_EVENT);
6168 try {
6169 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6170 return; // if primary nothing to do
6173 if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
6174 // nothing to do on REGION_CLOSE for now.
6175 return;
6177 if (regionEvent.getEventType() != EventType.REGION_OPEN) {
6178 LOG.warn(getRegionInfo().getEncodedName() + " : "
6179 + "Unknown region event received, ignoring :"
6180 + TextFormat.shortDebugString(regionEvent));
6181 return;
6184 if (LOG.isDebugEnabled()) {
6185 LOG.debug(getRegionInfo().getEncodedName() + " : "
6186 + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent));
6189 // we will use writestate as a coarse-grain lock for all the replay events
6190 synchronized (writestate) {
6191 // Replication can deliver events out of order when primary region moves or the region
6192 // server crashes, since there is no coordination between replication of different wal files
6193 // belonging to different region servers. We have to safe guard against this case by using
6194 // region open event's seqid. Since this is the first event that the region puts (after
6195 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6196 // smaller than this seqId
6197 if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
6198 this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
6199 } else {
6200 LOG.warn(getRegionInfo().getEncodedName() + " : "
6201 + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent)
6202 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
6203 + " of " + lastReplayedOpenRegionSeqId);
6204 return;
6207 // region open lists all the files that the region has at the time of the opening. Just pick
6208 // all the files and drop prepared flushes and empty memstores
6209 for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
6210 // stores of primary may be different now
6211 byte[] family = storeDescriptor.getFamilyName().toByteArray();
6212 HStore store = getStore(family);
6213 if (store == null) {
6214 LOG.warn(getRegionInfo().getEncodedName() + " : "
6215 + "Received a region open marker from primary, but the family is not found. "
6216 + "Ignoring. StoreDescriptor:" + storeDescriptor);
6217 continue;
6220 long storeSeqId = store.getMaxSequenceId().orElse(0L);
6221 List<String> storeFiles = storeDescriptor.getStoreFileList();
6222 try {
6223 store.refreshStoreFiles(storeFiles); // replace the files with the new ones
6224 } catch (FileNotFoundException ex) {
6225 LOG.warn(getRegionInfo().getEncodedName() + " : "
6226 + "At least one of the store files: " + storeFiles
6227 + " doesn't exist any more. Skip loading the file(s)", ex);
6228 continue;
6230 if (store.getMaxSequenceId().orElse(0L) != storeSeqId) {
6231 // Record latest flush time if we picked up new files
6232 lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
6235 if (writestate.flushing) {
6236 // only drop memstore snapshots if they are smaller than last flush for the store
6237 if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
6238 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
6239 null : this.prepareFlushResult.storeFlushCtxs.get(family);
6240 if (ctx != null) {
6241 MemStoreSize mss = store.getFlushableSize();
6242 ctx.abort();
6243 this.decrMemStoreSize(mss);
6244 this.prepareFlushResult.storeFlushCtxs.remove(family);
6249 // Drop the memstore contents if they are now smaller than the latest seen flushed file
6250 dropMemStoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
6251 if (storeSeqId > this.maxFlushedSeqId) {
6252 this.maxFlushedSeqId = storeSeqId;
6256 // if all stores ended up dropping their snapshots, we can safely drop the
6257 // prepareFlushResult
6258 dropPrepareFlushIfPossible();
6260 // advance the mvcc read point so that the new flushed file is visible.
6261 mvcc.await();
6263 // If we were waiting for observing a flush or region opening event for not showing partial
6264 // data after a secondary region crash, we can allow reads now.
6265 this.setReadsEnabled(true);
6267 // C. Finally notify anyone waiting on memstore to clear:
6268 // e.g. checkResources().
6269 synchronized (this) {
6270 notifyAll(); // FindBugs NN_NAKED_NOTIFY
6273 logRegionFiles();
6274 } finally {
6275 closeRegionOperation(Operation.REPLAY_EVENT);
6279 void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
6280 checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
6281 "BulkLoad marker from WAL ", bulkLoadEvent);
6283 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6284 return; // if primary nothing to do
6287 if (LOG.isDebugEnabled()) {
6288 LOG.debug(getRegionInfo().getEncodedName() + " : "
6289 + "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent));
6291 // check if multiple families involved
6292 boolean multipleFamilies = false;
6293 byte[] family = null;
6294 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6295 byte[] fam = storeDescriptor.getFamilyName().toByteArray();
6296 if (family == null) {
6297 family = fam;
6298 } else if (!Bytes.equals(family, fam)) {
6299 multipleFamilies = true;
6300 break;
6304 startBulkRegionOperation(multipleFamilies);
6305 try {
6306 // we will use writestate as a coarse-grain lock for all the replay events
6307 synchronized (writestate) {
6308 // Replication can deliver events out of order when primary region moves or the region
6309 // server crashes, since there is no coordination between replication of different wal files
6310 // belonging to different region servers. We have to safe guard against this case by using
6311 // region open event's seqid. Since this is the first event that the region puts (after
6312 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6313 // smaller than this seqId
6314 if (bulkLoadEvent.getBulkloadSeqNum() >= 0
6315 && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) {
6316 LOG.warn(getRegionInfo().getEncodedName() + " : "
6317 + "Skipping replaying bulkload event :"
6318 + TextFormat.shortDebugString(bulkLoadEvent)
6319 + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
6320 + " =" + lastReplayedOpenRegionSeqId);
6322 return;
6325 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6326 // stores of primary may be different now
6327 family = storeDescriptor.getFamilyName().toByteArray();
6328 HStore store = getStore(family);
6329 if (store == null) {
6330 LOG.warn(getRegionInfo().getEncodedName() + " : "
6331 + "Received a bulk load marker from primary, but the family is not found. "
6332 + "Ignoring. StoreDescriptor:" + storeDescriptor);
6333 continue;
6336 List<String> storeFiles = storeDescriptor.getStoreFileList();
6337 for (String storeFile : storeFiles) {
6338 StoreFileInfo storeFileInfo = null;
6339 try {
6340 storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile);
6341 store.bulkLoadHFile(storeFileInfo);
6342 } catch(FileNotFoundException ex) {
6343 LOG.warn(getRegionInfo().getEncodedName() + " : "
6344 + ((storeFileInfo != null) ? storeFileInfo.toString() :
6345 (new Path(Bytes.toString(family), storeFile)).toString())
6346 + " doesn't exist any more. Skip loading the file");
6351 if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
6352 mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
6354 } finally {
6355 closeBulkRegionOperation();
6360 * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult
6362 private void dropPrepareFlushIfPossible() {
6363 if (writestate.flushing) {
6364 boolean canDrop = true;
6365 if (prepareFlushResult.storeFlushCtxs != null) {
6366 for (Entry<byte[], StoreFlushContext> entry : prepareFlushResult.storeFlushCtxs
6367 .entrySet()) {
6368 HStore store = getStore(entry.getKey());
6369 if (store == null) {
6370 continue;
6372 if (store.getSnapshotSize().getDataSize() > 0) {
6373 canDrop = false;
6374 break;
6379 // this means that all the stores in the region has finished flushing, but the WAL marker
6380 // may not have been written or we did not receive it yet.
6381 if (canDrop) {
6382 writestate.flushing = false;
6383 this.prepareFlushResult = null;
6388 @Override
6389 public boolean refreshStoreFiles() throws IOException {
6390 return refreshStoreFiles(false);
6393 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
6394 justification = "Notify is about post replay. Intentional")
6395 protected boolean refreshStoreFiles(boolean force) throws IOException {
6396 if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6397 return false; // if primary nothing to do
6400 if (LOG.isDebugEnabled()) {
6401 LOG.debug(getRegionInfo().getEncodedName() + " : "
6402 + "Refreshing store files to see whether we can free up memstore");
6405 long totalFreedDataSize = 0;
6407 long smallestSeqIdInStores = Long.MAX_VALUE;
6409 startRegionOperation(); // obtain region close lock
6410 try {
6411 Map<HStore, Long> map = new HashMap<>();
6412 synchronized (writestate) {
6413 for (HStore store : stores.values()) {
6414 // TODO: some stores might see new data from flush, while others do not which
6415 // MIGHT break atomic edits across column families.
6416 long maxSeqIdBefore = store.getMaxSequenceId().orElse(0L);
6418 // refresh the store files. This is similar to observing a region open wal marker.
6419 store.refreshStoreFiles();
6421 long storeSeqId = store.getMaxSequenceId().orElse(0L);
6422 if (storeSeqId < smallestSeqIdInStores) {
6423 smallestSeqIdInStores = storeSeqId;
6426 // see whether we can drop the memstore or the snapshot
6427 if (storeSeqId > maxSeqIdBefore) {
6428 if (writestate.flushing) {
6429 // only drop memstore snapshots if they are smaller than last flush for the store
6430 if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
6431 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
6432 null : this.prepareFlushResult.storeFlushCtxs.get(
6433 store.getColumnFamilyDescriptor().getName());
6434 if (ctx != null) {
6435 MemStoreSize mss = store.getFlushableSize();
6436 ctx.abort();
6437 this.decrMemStoreSize(mss);
6438 this.prepareFlushResult.storeFlushCtxs.
6439 remove(store.getColumnFamilyDescriptor().getName());
6440 totalFreedDataSize += mss.getDataSize();
6445 map.put(store, storeSeqId);
6449 // if all stores ended up dropping their snapshots, we can safely drop the
6450 // prepareFlushResult
6451 dropPrepareFlushIfPossible();
6453 // advance the mvcc read point so that the new flushed files are visible.
6454 // either greater than flush seq number or they were already picked up via flush.
6455 for (HStore s : stores.values()) {
6456 mvcc.advanceTo(s.getMaxMemStoreTS().orElse(0L));
6460 // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely
6461 // skip all edits that are to be replayed in the future with that has a smaller seqId
6462 // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits
6463 // that we have picked the flush files for
6464 if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
6465 this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
6468 if (!map.isEmpty()) {
6469 for (Map.Entry<HStore, Long> entry : map.entrySet()) {
6470 // Drop the memstore contents if they are now smaller than the latest seen flushed file
6471 totalFreedDataSize += dropMemStoreContentsForSeqId(entry.getValue(), entry.getKey())
6472 .getDataSize();
6475 // C. Finally notify anyone waiting on memstore to clear:
6476 // e.g. checkResources().
6477 synchronized (this) {
6478 notifyAll(); // FindBugs NN_NAKED_NOTIFY
6480 return totalFreedDataSize > 0;
6481 } finally {
6482 closeRegionOperation();
6486 private void logRegionFiles() {
6487 if (LOG.isTraceEnabled()) {
6488 LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
6489 stores.values().stream().filter(s -> s.getStorefiles() != null)
6490 .flatMap(s -> s.getStorefiles().stream())
6491 .forEachOrdered(sf -> LOG.trace(getRegionInfo().getEncodedName() + " : " + sf));
6495 /** Checks whether the given regionName is either equal to our region, or that
6496 * the regionName is the primary region to our corresponding range for the secondary replica.
6498 private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
6499 throws WrongRegionException {
6500 if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
6501 return;
6504 if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) &&
6505 Bytes.equals(encodedRegionName,
6506 this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) {
6507 return;
6510 throw new WrongRegionException(exceptionMsg + payload
6511 + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
6512 + " does not match this region: " + this.getRegionInfo());
6516 * Used by tests
6517 * @param s Store to add edit too.
6518 * @param cell Cell to add.
6520 protected void restoreEdit(HStore s, Cell cell, MemStoreSizing memstoreAccounting) {
6521 s.add(cell, memstoreAccounting);
6525 * make sure have been through lease recovery before get file status, so the file length can be
6526 * trusted.
6527 * @param p File to check.
6528 * @return True if file was zero-length (and if so, we'll delete it in here).
6529 * @throws IOException
6531 private static boolean isZeroLengthThenDelete(final FileSystem fs, final FileStatus stat,
6532 final Path p) throws IOException {
6533 if (stat.getLen() > 0) {
6534 return false;
6536 LOG.warn("File " + p + " is zero-length, deleting.");
6537 fs.delete(p, false);
6538 return true;
6541 protected HStore instantiateHStore(final ColumnFamilyDescriptor family, boolean warmup)
6542 throws IOException {
6543 if (family.isMobEnabled()) {
6544 if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
6545 throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS +
6546 " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY +
6547 " accordingly.");
6549 return new HMobStore(this, family, this.conf, warmup);
6551 return new HStore(this, family, this.conf, warmup);
6554 @Override
6555 public HStore getStore(byte[] column) {
6556 return this.stores.get(column);
6560 * Return HStore instance. Does not do any copy: as the number of store is limited, we iterate on
6561 * the list.
6563 private HStore getStore(Cell cell) {
6564 return stores.entrySet().stream().filter(e -> CellUtil.matchingFamily(cell, e.getKey()))
6565 .map(e -> e.getValue()).findFirst().orElse(null);
6568 @Override
6569 public List<HStore> getStores() {
6570 return new ArrayList<>(stores.values());
6573 @Override
6574 public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException {
6575 List<String> storeFileNames = new ArrayList<>();
6576 synchronized (closeLock) {
6577 for (byte[] column : columns) {
6578 HStore store = this.stores.get(column);
6579 if (store == null) {
6580 throw new IllegalArgumentException(
6581 "No column family : " + new String(column, StandardCharsets.UTF_8) + " available");
6583 Collection<HStoreFile> storeFiles = store.getStorefiles();
6584 if (storeFiles == null) {
6585 continue;
6587 for (HStoreFile storeFile : storeFiles) {
6588 storeFileNames.add(storeFile.getPath().toString());
6591 logRegionFiles();
6594 return storeFileNames;
6597 //////////////////////////////////////////////////////////////////////////////
6598 // Support code
6599 //////////////////////////////////////////////////////////////////////////////
6601 /** Make sure this is a valid row for the HRegion */
6602 void checkRow(byte[] row, String op) throws IOException {
6603 if (!rowIsInRange(getRegionInfo(), row)) {
6604 throw new WrongRegionException("Requested row out of range for " +
6605 op + " on HRegion " + this + ", startKey='" +
6606 Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" +
6607 Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" +
6608 Bytes.toStringBinary(row) + "'");
6614 * Get an exclusive ( write lock ) lock on a given row.
6615 * @param row Which row to lock.
6616 * @return A locked RowLock. The lock is exclusive and already aqquired.
6617 * @throws IOException
6619 public RowLock getRowLock(byte[] row) throws IOException {
6620 return getRowLock(row, false);
6623 @Override
6624 public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
6625 checkRow(row, "row lock");
6626 return getRowLock(row, readLock, null);
6629 Span createRegionSpan(String name) {
6630 return TraceUtil.createSpan(name).setAttribute(REGION_NAMES_KEY,
6631 Collections.singletonList(getRegionInfo().getRegionNameAsString()));
6634 // will be override in tests
6635 protected RowLock getRowLockInternal(byte[] row, boolean readLock, RowLock prevRowLock)
6636 throws IOException {
6637 // create an object to use a a key in the row lock map
6638 HashedBytes rowKey = new HashedBytes(row);
6640 RowLockContext rowLockContext = null;
6641 RowLockImpl result = null;
6643 boolean success = false;
6644 try {
6645 // Keep trying until we have a lock or error out.
6646 // TODO: do we need to add a time component here?
6647 while (result == null) {
6648 rowLockContext = computeIfAbsent(lockedRows, rowKey, () -> new RowLockContext(rowKey));
6649 // Now try an get the lock.
6650 // This can fail as
6651 if (readLock) {
6652 // For read lock, if the caller has locked the same row previously, it will not try
6653 // to acquire the same read lock. It simply returns the previous row lock.
6654 RowLockImpl prevRowLockImpl = (RowLockImpl)prevRowLock;
6655 if ((prevRowLockImpl != null) && (prevRowLockImpl.getLock() ==
6656 rowLockContext.readWriteLock.readLock())) {
6657 success = true;
6658 return prevRowLock;
6660 result = rowLockContext.newReadLock();
6661 } else {
6662 result = rowLockContext.newWriteLock();
6666 int timeout = rowLockWaitDuration;
6667 boolean reachDeadlineFirst = false;
6668 Optional<RpcCall> call = RpcServer.getCurrentCall();
6669 if (call.isPresent()) {
6670 long deadline = call.get().getDeadline();
6671 if (deadline < Long.MAX_VALUE) {
6672 int timeToDeadline = (int) (deadline - EnvironmentEdgeManager.currentTime());
6673 if (timeToDeadline <= this.rowLockWaitDuration) {
6674 reachDeadlineFirst = true;
6675 timeout = timeToDeadline;
6680 if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
6681 String message = "Timed out waiting for lock for row: " + rowKey + " in region "
6682 + getRegionInfo().getEncodedName();
6683 if (reachDeadlineFirst) {
6684 throw new TimeoutIOException(message);
6685 } else {
6686 // If timeToDeadline is larger than rowLockWaitDuration, we can not drop the request.
6687 throw new IOException(message);
6690 rowLockContext.setThreadName(Thread.currentThread().getName());
6691 success = true;
6692 return result;
6693 } catch (InterruptedException ie) {
6694 if (LOG.isDebugEnabled()) {
6695 LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey,
6696 getRegionInfo().getRegionNameAsString());
6698 throw throwOnInterrupt(ie);
6699 } catch (Error error) {
6700 // The maximum lock count for read lock is 64K (hardcoded), when this maximum count
6701 // is reached, it will throw out an Error. This Error needs to be caught so it can
6702 // go ahead to process the minibatch with lock acquired.
6703 LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row),
6704 getRegionInfo().getRegionNameAsString(), error);
6705 IOException ioe = new IOException(error);
6706 throw ioe;
6707 } finally {
6708 // Clean up the counts just in case this was the thing keeping the context alive.
6709 if (!success && rowLockContext != null) {
6710 rowLockContext.cleanUp();
6715 private RowLock getRowLock(byte[] row, boolean readLock, final RowLock prevRowLock)
6716 throws IOException {
6717 return TraceUtil.trace(() -> getRowLockInternal(row, readLock, prevRowLock),
6718 () -> createRegionSpan("Region.getRowLock").setAttribute(ROW_LOCK_READ_LOCK_KEY,
6719 readLock));
6722 private void releaseRowLocks(List<RowLock> rowLocks) {
6723 if (rowLocks != null) {
6724 for (RowLock rowLock : rowLocks) {
6725 rowLock.release();
6727 rowLocks.clear();
6731 public int getReadLockCount() {
6732 return lock.getReadLockCount();
6735 public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
6736 return lockedRows;
6739 class RowLockContext {
6740 private final HashedBytes row;
6741 final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
6742 final AtomicBoolean usable = new AtomicBoolean(true);
6743 final AtomicInteger count = new AtomicInteger(0);
6744 final Object lock = new Object();
6745 private String threadName;
6747 RowLockContext(HashedBytes row) {
6748 this.row = row;
6751 RowLockImpl newWriteLock() {
6752 Lock l = readWriteLock.writeLock();
6753 return getRowLock(l);
6755 RowLockImpl newReadLock() {
6756 Lock l = readWriteLock.readLock();
6757 return getRowLock(l);
6760 private RowLockImpl getRowLock(Lock l) {
6761 count.incrementAndGet();
6762 synchronized (lock) {
6763 if (usable.get()) {
6764 return new RowLockImpl(this, l);
6765 } else {
6766 return null;
6771 void cleanUp() {
6772 long c = count.decrementAndGet();
6773 if (c <= 0) {
6774 synchronized (lock) {
6775 if (count.get() <= 0 && usable.get()){ // Don't attempt to remove row if already removed
6776 usable.set(false);
6777 RowLockContext removed = lockedRows.remove(row);
6778 assert removed == this: "we should never remove a different context";
6784 public void setThreadName(String threadName) {
6785 this.threadName = threadName;
6788 @Override
6789 public String toString() {
6790 return "RowLockContext{" +
6791 "row=" + row +
6792 ", readWriteLock=" + readWriteLock +
6793 ", count=" + count +
6794 ", threadName=" + threadName +
6795 '}';
6800 * Class used to represent a lock on a row.
6802 public static class RowLockImpl implements RowLock {
6803 private final RowLockContext context;
6804 private final Lock lock;
6806 public RowLockImpl(RowLockContext context, Lock lock) {
6807 this.context = context;
6808 this.lock = lock;
6811 public Lock getLock() {
6812 return lock;
6815 public RowLockContext getContext() {
6816 return context;
6819 @Override
6820 public void release() {
6821 lock.unlock();
6822 context.cleanUp();
6825 @Override
6826 public String toString() {
6827 return "RowLockImpl{" +
6828 "context=" + context +
6829 ", lock=" + lock +
6830 '}';
6835 * Determines whether multiple column families are present
6836 * Precondition: familyPaths is not null
6838 * @param familyPaths List of (column family, hfilePath)
6840 private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
6841 boolean multipleFamilies = false;
6842 byte[] family = null;
6843 for (Pair<byte[], String> pair : familyPaths) {
6844 byte[] fam = pair.getFirst();
6845 if (family == null) {
6846 family = fam;
6847 } else if (!Bytes.equals(family, fam)) {
6848 multipleFamilies = true;
6849 break;
6852 return multipleFamilies;
6856 * Attempts to atomically load a group of hfiles. This is critical for loading
6857 * rows with multiple column families atomically.
6859 * @param familyPaths List of Pair&lt;byte[] column family, String hfilePath&gt;
6860 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
6861 * file about to be bulk loaded
6862 * @param assignSeqId
6863 * @return Map from family to List of store file paths if successful, null if failed recoverably
6864 * @throws IOException if failed unrecoverably.
6866 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
6867 BulkLoadListener bulkLoadListener) throws IOException {
6868 return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false,
6869 null, true);
6873 * Listener class to enable callers of
6874 * bulkLoadHFile() to perform any necessary
6875 * pre/post processing of a given bulkload call
6877 public interface BulkLoadListener {
6879 * Called before an HFile is actually loaded
6880 * @param family family being loaded to
6881 * @param srcPath path of HFile
6882 * @return final path to be used for actual loading
6883 * @throws IOException
6885 String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile)
6886 throws IOException;
6889 * Called after a successful HFile load
6890 * @param family family being loaded to
6891 * @param srcPath path of HFile
6892 * @throws IOException
6894 void doneBulkLoad(byte[] family, String srcPath) throws IOException;
6897 * Called after a failed HFile load
6898 * @param family family being loaded to
6899 * @param srcPath path of HFile
6900 * @throws IOException
6902 void failedBulkLoad(byte[] family, String srcPath) throws IOException;
6906 * Attempts to atomically load a group of hfiles. This is critical for loading
6907 * rows with multiple column families atomically.
6909 * @param familyPaths List of Pair&lt;byte[] column family, String hfilePath&gt;
6910 * @param assignSeqId
6911 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
6912 * file about to be bulk loaded
6913 * @param copyFile always copy hfiles if true
6914 * @param clusterIds ids from clusters that had already handled the given bulkload event.
6915 * @return Map from family to List of store file paths if successful, null if failed recoverably
6916 * @throws IOException if failed unrecoverably.
6918 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
6919 boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile,
6920 List<String> clusterIds, boolean replicate) throws IOException {
6921 long seqId = -1;
6922 Map<byte[], List<Path>> storeFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
6923 Map<String, Long> storeFilesSizes = new HashMap<>();
6924 Preconditions.checkNotNull(familyPaths);
6925 // we need writeLock for multi-family bulk load
6926 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
6927 boolean isSuccessful = false;
6928 try {
6929 this.writeRequestsCount.increment();
6931 // There possibly was a split that happened between when the split keys
6932 // were gathered and before the HRegion's write lock was taken. We need
6933 // to validate the HFile region before attempting to bulk load all of them
6934 IOException ioException = null;
6935 List<Pair<byte[], String>> failures = new ArrayList<>();
6936 for (Pair<byte[], String> p : familyPaths) {
6937 byte[] familyName = p.getFirst();
6938 String path = p.getSecond();
6940 HStore store = getStore(familyName);
6941 if (store == null) {
6942 ioException = new org.apache.hadoop.hbase.DoNotRetryIOException(
6943 "No such column family " + Bytes.toStringBinary(familyName));
6944 } else {
6945 try {
6946 store.assertBulkLoadHFileOk(new Path(path));
6947 } catch (WrongRegionException wre) {
6948 // recoverable (file doesn't fit in region)
6949 failures.add(p);
6950 } catch (IOException ioe) {
6951 // unrecoverable (hdfs problem)
6952 ioException = ioe;
6956 // validation failed because of some sort of IO problem.
6957 if (ioException != null) {
6958 LOG.error("There was IO error when checking if the bulk load is ok in region {}.", this,
6959 ioException);
6960 throw ioException;
6963 // validation failed, bail out before doing anything permanent.
6964 if (failures.size() != 0) {
6965 StringBuilder list = new StringBuilder();
6966 for (Pair<byte[], String> p : failures) {
6967 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
6968 .append(p.getSecond());
6970 // problem when validating
6971 LOG.warn("There was a recoverable bulk load failure likely due to a split. These (family,"
6972 + " HFile) pairs were not loaded: {}, in region {}", list.toString(), this);
6973 return null;
6976 // We need to assign a sequential ID that's in between two memstores in order to preserve
6977 // the guarantee that all the edits lower than the highest sequential ID from all the
6978 // HFiles are flushed on disk. See HBASE-10958. The sequence id returned when we flush is
6979 // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is
6980 // a sequence id that we can be sure is beyond the last hfile written).
6981 if (assignSeqId) {
6982 FlushResult fs = flushcache(true, false, FlushLifeCycleTracker.DUMMY);
6983 if (fs.isFlushSucceeded()) {
6984 seqId = ((FlushResultImpl)fs).flushSequenceId;
6985 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
6986 seqId = ((FlushResultImpl)fs).flushSequenceId;
6987 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) {
6988 // CANNOT_FLUSH may mean that a flush is already on-going
6989 // we need to wait for that flush to complete
6990 waitForFlushes();
6991 } else {
6992 throw new IOException("Could not bulk load with an assigned sequential ID because the "+
6993 "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason);
6997 Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath =
6998 new TreeMap<>(Bytes.BYTES_COMPARATOR);
6999 for (Pair<byte[], String> p : familyPaths) {
7000 byte[] familyName = p.getFirst();
7001 String path = p.getSecond();
7002 HStore store = getStore(familyName);
7003 if (!familyWithFinalPath.containsKey(familyName)) {
7004 familyWithFinalPath.put(familyName, new ArrayList<>());
7006 List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName);
7007 try {
7008 String finalPath = path;
7009 if (bulkLoadListener != null) {
7010 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile);
7012 Pair<Path, Path> pair = store.preBulkLoadHFile(finalPath, seqId);
7013 lst.add(pair);
7014 } catch (IOException ioe) {
7015 // A failure here can cause an atomicity violation that we currently
7016 // cannot recover from since it is likely a failed HDFS operation.
7018 LOG.error("There was a partial failure due to IO when attempting to" +
7019 " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
7020 if (bulkLoadListener != null) {
7021 try {
7022 bulkLoadListener.failedBulkLoad(familyName, path);
7023 } catch (Exception ex) {
7024 LOG.error("Error while calling failedBulkLoad for family " +
7025 Bytes.toString(familyName) + " with path " + path, ex);
7028 throw ioe;
7032 if (this.getCoprocessorHost() != null) {
7033 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7034 this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue());
7037 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7038 byte[] familyName = entry.getKey();
7039 for (Pair<Path, Path> p : entry.getValue()) {
7040 String path = p.getFirst().toString();
7041 Path commitedStoreFile = p.getSecond();
7042 HStore store = getStore(familyName);
7043 try {
7044 store.bulkLoadHFile(familyName, path, commitedStoreFile);
7045 // Note the size of the store file
7046 try {
7047 FileSystem fs = commitedStoreFile.getFileSystem(baseConf);
7048 storeFilesSizes.put(commitedStoreFile.getName(), fs.getFileStatus(commitedStoreFile)
7049 .getLen());
7050 } catch (IOException e) {
7051 LOG.warn("Failed to find the size of hfile " + commitedStoreFile, e);
7052 storeFilesSizes.put(commitedStoreFile.getName(), 0L);
7055 if(storeFiles.containsKey(familyName)) {
7056 storeFiles.get(familyName).add(commitedStoreFile);
7057 } else {
7058 List<Path> storeFileNames = new ArrayList<>();
7059 storeFileNames.add(commitedStoreFile);
7060 storeFiles.put(familyName, storeFileNames);
7062 if (bulkLoadListener != null) {
7063 bulkLoadListener.doneBulkLoad(familyName, path);
7065 } catch (IOException ioe) {
7066 // A failure here can cause an atomicity violation that we currently
7067 // cannot recover from since it is likely a failed HDFS operation.
7069 // TODO Need a better story for reverting partial failures due to HDFS.
7070 LOG.error("There was a partial failure due to IO when attempting to" +
7071 " load " + Bytes.toString(familyName) + " : " + p.getSecond(), ioe);
7072 if (bulkLoadListener != null) {
7073 try {
7074 bulkLoadListener.failedBulkLoad(familyName, path);
7075 } catch (Exception ex) {
7076 LOG.error("Error while calling failedBulkLoad for family " +
7077 Bytes.toString(familyName) + " with path " + path, ex);
7080 throw ioe;
7085 isSuccessful = true;
7086 if (conf.getBoolean(COMPACTION_AFTER_BULKLOAD_ENABLE, false)) {
7087 // request compaction
7088 familyWithFinalPath.keySet().forEach(family -> {
7089 HStore store = getStore(family);
7090 try {
7091 if (this.rsServices != null && store.needsCompaction()) {
7092 this.rsServices.getCompactionRequestor().requestSystemCompaction(this, store,
7093 "bulkload hfiles request compaction", true);
7094 LOG.info("Request compaction for region {} family {} after bulk load",
7095 this.getRegionInfo().getEncodedName(), store.getColumnFamilyName());
7097 } catch (IOException e) {
7098 LOG.error("bulkload hfiles request compaction error ", e);
7102 } finally {
7103 if (wal != null && !storeFiles.isEmpty()) {
7104 // Write a bulk load event for hfiles that are loaded
7105 try {
7106 WALProtos.BulkLoadDescriptor loadDescriptor =
7107 ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
7108 UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()),
7109 storeFiles, storeFilesSizes, seqId, clusterIds, replicate);
7110 WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(),
7111 loadDescriptor, mvcc, regionReplicationSink.orElse(null));
7112 } catch (IOException ioe) {
7113 if (this.rsServices != null) {
7114 // Have to abort region server because some hfiles has been loaded but we can't write
7115 // the event into WAL
7116 isSuccessful = false;
7117 this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
7122 closeBulkRegionOperation();
7124 return isSuccessful ? storeFiles : null;
7127 @Override
7128 public boolean equals(Object o) {
7129 return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
7130 ((HRegion) o).getRegionInfo().getRegionName());
7133 @Override
7134 public int hashCode() {
7135 return Bytes.hashCode(getRegionInfo().getRegionName());
7138 @Override
7139 public String toString() {
7140 return getRegionInfo().getRegionNameAsString();
7143 // Utility methods
7145 * A utility method to create new instances of HRegion based on the {@link HConstants#REGION_IMPL}
7146 * configuration property.
7147 * @param tableDir qualified path of directory where region should be located, usually the table
7148 * directory.
7149 * @param wal The WAL is the outbound log for any updates to the HRegion The wal file is a logfile
7150 * from the previous execution that's custom-computed for this HRegion. The HRegionServer
7151 * computes and sorts the appropriate wal info for this HRegion. If there is a previous
7152 * file (implying that the HRegion has been written-to before), then read it from the
7153 * supplied path.
7154 * @param fs is the filesystem.
7155 * @param conf is global configuration settings.
7156 * @param regionInfo - RegionInfo that describes the region is new), then read them from the
7157 * supplied path.
7158 * @param htd the table descriptor
7159 * @return the new instance
7161 public static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
7162 Configuration conf, RegionInfo regionInfo, final TableDescriptor htd,
7163 RegionServerServices rsServices) {
7164 try {
7165 @SuppressWarnings("unchecked")
7166 Class<? extends HRegion> regionClass =
7167 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
7169 Constructor<? extends HRegion> c =
7170 regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
7171 Configuration.class, RegionInfo.class, TableDescriptor.class,
7172 RegionServerServices.class);
7174 return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
7175 } catch (Throwable e) {
7176 // todo: what should I throw here?
7177 throw new IllegalStateException("Could not instantiate a region instance.", e);
7182 * Convenience method creating new HRegions. Used by createTable.
7183 * @param info Info for region to create.
7184 * @param rootDir Root directory for HBase instance
7185 * @param wal shared WAL
7186 * @param initialize - true to initialize the region
7187 * @return new HRegion
7189 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7190 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7191 final boolean initialize) throws IOException {
7192 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, initialize, null);
7196 * Convenience method creating new HRegions. Used by createTable.
7197 * @param info Info for region to create.
7198 * @param rootDir Root directory for HBase instance
7199 * @param wal shared WAL
7200 * @param initialize - true to initialize the region
7201 * @param rsRpcServices An interface we can request flushes against.
7202 * @return new HRegion
7204 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7205 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7206 final boolean initialize, RegionServerServices rsRpcServices) throws IOException {
7207 LOG.info("creating " + info + ", tableDescriptor="
7208 + (hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir);
7209 createRegionDir(conf, info, rootDir);
7210 FileSystem fs = rootDir.getFileSystem(conf);
7211 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7212 HRegion region =
7213 HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, rsRpcServices);
7214 if (initialize) {
7215 region.initialize(null);
7217 return region;
7221 * Create a region under the given table directory.
7223 public static HRegion createHRegion(Configuration conf, RegionInfo regionInfo, FileSystem fs,
7224 Path tableDir, TableDescriptor tableDesc) throws IOException {
7225 LOG.info("Creating {}, tableDescriptor={}, under table dir {}", regionInfo, tableDesc,
7226 tableDir);
7227 HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo);
7228 HRegion region = HRegion.newHRegion(tableDir, null, fs, conf, regionInfo, tableDesc, null);
7229 return region;
7233 * Create the region directory in the filesystem.
7235 public static HRegionFileSystem createRegionDir(Configuration configuration, RegionInfo ri,
7236 Path rootDir)
7237 throws IOException {
7238 FileSystem fs = rootDir.getFileSystem(configuration);
7239 Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable());
7240 // If directory already exists, will log warning and keep going. Will try to create
7241 // .regioninfo. If one exists, will overwrite.
7242 return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri);
7245 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7246 final Configuration conf,
7247 final TableDescriptor hTableDescriptor,
7248 final WAL wal)
7249 throws IOException {
7250 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
7255 * Open a Region.
7256 * @param info Info for region to be opened.
7257 * @param wal WAL for region to use. This method will call
7258 * WAL#setSequenceNumber(long) passing the result of the call to
7259 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7260 * up. HRegionStore does this every time it opens a new region.
7261 * @return new HRegion
7263 * @throws IOException
7265 public static HRegion openHRegion(final RegionInfo info,
7266 final TableDescriptor htd, final WAL wal,
7267 final Configuration conf)
7268 throws IOException {
7269 return openHRegion(info, htd, wal, conf, null, null);
7273 * Open a Region.
7274 * @param info Info for region to be opened
7275 * @param htd the table descriptor
7276 * @param wal WAL for region to use. This method will call
7277 * WAL#setSequenceNumber(long) passing the result of the call to
7278 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7279 * up. HRegionStore does this every time it opens a new region.
7280 * @param conf The Configuration object to use.
7281 * @param rsServices An interface we can request flushes against.
7282 * @param reporter An interface we can report progress against.
7283 * @return new HRegion
7285 * @throws IOException
7287 public static HRegion openHRegion(final RegionInfo info,
7288 final TableDescriptor htd, final WAL wal, final Configuration conf,
7289 final RegionServerServices rsServices,
7290 final CancelableProgressable reporter)
7291 throws IOException {
7292 return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
7296 * Open a Region.
7297 * @param rootDir Root directory for HBase instance
7298 * @param info Info for region to be opened.
7299 * @param htd the table descriptor
7300 * @param wal WAL for region to use. This method will call
7301 * WAL#setSequenceNumber(long) passing the result of the call to
7302 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7303 * up. HRegionStore does this every time it opens a new region.
7304 * @param conf The Configuration object to use.
7305 * @return new HRegion
7306 * @throws IOException
7308 public static HRegion openHRegion(Path rootDir, final RegionInfo info,
7309 final TableDescriptor htd, final WAL wal, final Configuration conf)
7310 throws IOException {
7311 return openHRegion(rootDir, info, htd, wal, conf, null, null);
7315 * Open a Region.
7316 * @param rootDir Root directory for HBase instance
7317 * @param info Info for region to be opened.
7318 * @param htd the table descriptor
7319 * @param wal WAL for region to use. This method will call
7320 * WAL#setSequenceNumber(long) passing the result of the call to
7321 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7322 * up. HRegionStore does this every time it opens a new region.
7323 * @param conf The Configuration object to use.
7324 * @param rsServices An interface we can request flushes against.
7325 * @param reporter An interface we can report progress against.
7326 * @return new HRegion
7327 * @throws IOException
7329 public static HRegion openHRegion(final Path rootDir, final RegionInfo info,
7330 final TableDescriptor htd, final WAL wal, final Configuration conf,
7331 final RegionServerServices rsServices,
7332 final CancelableProgressable reporter)
7333 throws IOException {
7334 FileSystem fs = null;
7335 if (rsServices != null) {
7336 fs = rsServices.getFileSystem();
7338 if (fs == null) {
7339 fs = rootDir.getFileSystem(conf);
7341 return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
7345 * Open a Region.
7346 * @param conf The Configuration object to use.
7347 * @param fs Filesystem to use
7348 * @param rootDir Root directory for HBase instance
7349 * @param info Info for region to be opened.
7350 * @param htd the table descriptor
7351 * @param wal WAL for region to use. This method will call
7352 * WAL#setSequenceNumber(long) passing the result of the call to
7353 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7354 * up. HRegionStore does this every time it opens a new region.
7355 * @return new HRegion
7357 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7358 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal)
7359 throws IOException {
7360 return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
7364 * Open a Region.
7365 * @param conf The Configuration object to use.
7366 * @param fs Filesystem to use
7367 * @param rootDir Root directory for HBase instance
7368 * @param info Info for region to be opened.
7369 * @param htd the table descriptor
7370 * @param wal WAL for region to use. This method will call
7371 * WAL#setSequenceNumber(long) passing the result of the call to
7372 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7373 * up. HRegionStore does this every time it opens a new region.
7374 * @param rsServices An interface we can request flushes against.
7375 * @param reporter An interface we can report progress against.
7376 * @return new HRegion
7378 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7379 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7380 final RegionServerServices rsServices, final CancelableProgressable reporter)
7381 throws IOException {
7382 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7383 return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter);
7387 * Open a Region.
7388 * @param conf The Configuration object to use.
7389 * @param fs Filesystem to use
7390 * @param info Info for region to be opened.
7391 * @param htd the table descriptor
7392 * @param wal WAL for region to use. This method will call
7393 * WAL#setSequenceNumber(long) passing the result of the call to
7394 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7395 * up. HRegionStore does this every time it opens a new region.
7396 * @param rsServices An interface we can request flushes against.
7397 * @param reporter An interface we can report progress against.
7398 * @return new HRegion
7399 * @throws NullPointerException if {@code info} is {@code null}
7401 public static HRegion openHRegionFromTableDir(final Configuration conf, final FileSystem fs,
7402 final Path tableDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7403 final RegionServerServices rsServices, final CancelableProgressable reporter)
7404 throws IOException {
7405 Objects.requireNonNull(info, "RegionInfo cannot be null");
7406 LOG.debug("Opening region: {}", info);
7407 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
7408 return r.openHRegion(reporter);
7411 public NavigableMap<byte[], Integer> getReplicationScope() {
7412 return this.replicationScope;
7416 * Useful when reopening a closed region (normally for unit tests)
7417 * @param other original object
7418 * @param reporter An interface we can report progress against.
7419 * @return new HRegion
7421 public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
7422 throws IOException {
7423 HRegionFileSystem regionFs = other.getRegionFileSystem();
7424 HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
7425 other.baseConf, other.getRegionInfo(), other.getTableDescriptor(), null);
7426 return r.openHRegion(reporter);
7429 public static Region openHRegion(final Region other, final CancelableProgressable reporter)
7430 throws IOException {
7431 return openHRegion((HRegion)other, reporter);
7435 * Open HRegion.
7436 * <p/>
7437 * Calls initialize and sets sequenceId.
7438 * @return Returns <code>this</code>
7440 private HRegion openHRegion(final CancelableProgressable reporter) throws IOException {
7441 try {
7442 // Refuse to open the region if we are missing local compression support
7443 TableDescriptorChecker.checkCompression(htableDescriptor);
7444 // Refuse to open the region if encryption configuration is incorrect or
7445 // codec support is missing
7446 LOG.debug("checking encryption for " + this.getRegionInfo().getEncodedName());
7447 TableDescriptorChecker.checkEncryption(conf, htableDescriptor);
7448 // Refuse to open the region if a required class cannot be loaded
7449 LOG.debug("checking classloading for " + this.getRegionInfo().getEncodedName());
7450 TableDescriptorChecker.checkClassLoading(conf, htableDescriptor);
7451 this.openSeqNum = initialize(reporter);
7452 this.mvcc.advanceTo(openSeqNum);
7453 // The openSeqNum must be increased every time when a region is assigned, as we rely on it to
7454 // determine whether a region has been successfully reopened. So here we always write open
7455 // marker, even if the table is read only.
7456 if (wal != null && getRegionServerServices() != null &&
7457 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
7458 writeRegionOpenMarker(wal, openSeqNum);
7460 } catch (Throwable t) {
7461 // By coprocessor path wrong region will open failed,
7462 // MetricsRegionWrapperImpl is already init and not close,
7463 // add region close when open failed
7464 try {
7465 // It is not required to write sequence id file when region open is failed.
7466 // Passing true to skip the sequence id file write.
7467 this.close(true);
7468 } catch (Throwable e) {
7469 LOG.warn("Open region: {} failed. Try close region but got exception ", this.getRegionInfo(),
7472 throw t;
7474 return this;
7478 * Open a Region on a read-only file-system (like hdfs snapshots)
7479 * @param conf The Configuration object to use.
7480 * @param fs Filesystem to use
7481 * @param info Info for region to be opened.
7482 * @param htd the table descriptor
7483 * @return new HRegion
7484 * @throws NullPointerException if {@code info} is {@code null}
7486 public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs,
7487 final Path tableDir, RegionInfo info, final TableDescriptor htd) throws IOException {
7488 Objects.requireNonNull(info, "RegionInfo cannot be null");
7489 if (LOG.isDebugEnabled()) {
7490 LOG.debug("Opening region (readOnly filesystem): " + info);
7492 if (info.getReplicaId() <= 0) {
7493 info = RegionReplicaUtil.getRegionInfoForReplica(info, 1);
7495 HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null);
7496 r.writestate.setReadOnly(true);
7497 return r.openHRegion(null);
7500 public static void warmupHRegion(final RegionInfo info,
7501 final TableDescriptor htd, final WAL wal, final Configuration conf,
7502 final RegionServerServices rsServices,
7503 final CancelableProgressable reporter)
7504 throws IOException {
7506 Objects.requireNonNull(info, "RegionInfo cannot be null");
7507 LOG.debug("Warmup {}", info);
7508 Path rootDir = CommonFSUtils.getRootDir(conf);
7509 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7510 FileSystem fs = null;
7511 if (rsServices != null) {
7512 fs = rsServices.getFileSystem();
7514 if (fs == null) {
7515 fs = rootDir.getFileSystem(conf);
7517 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
7518 r.initializeWarmup(reporter);
7522 * Computes the Path of the HRegion
7524 * @param tabledir qualified path for table
7525 * @param name ENCODED region name
7526 * @return Path of HRegion directory
7527 * @deprecated For tests only; to be removed.
7529 @Deprecated
7530 public static Path getRegionDir(final Path tabledir, final String name) {
7531 return new Path(tabledir, name);
7535 * Determines if the specified row is within the row range specified by the
7536 * specified RegionInfo
7538 * @param info RegionInfo that specifies the row range
7539 * @param row row to be checked
7540 * @return true if the row is within the range specified by the RegionInfo
7542 public static boolean rowIsInRange(RegionInfo info, final byte [] row) {
7543 return ((info.getStartKey().length == 0) ||
7544 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
7545 ((info.getEndKey().length == 0) ||
7546 (Bytes.compareTo(info.getEndKey(), row) > 0));
7549 public static boolean rowIsInRange(RegionInfo info, final byte [] row, final int offset,
7550 final short length) {
7551 return ((info.getStartKey().length == 0) ||
7552 (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
7553 row, offset, length) <= 0)) &&
7554 ((info.getEndKey().length == 0) ||
7555 (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
7558 @Override
7559 public Result get(final Get get) throws IOException {
7560 prepareGet(get);
7561 List<Cell> results = get(get, true);
7562 boolean stale = this.getRegionInfo().getReplicaId() != 0;
7563 return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
7566 void prepareGet(final Get get) throws IOException {
7567 checkRow(get.getRow(), "Get");
7568 // Verify families are all valid
7569 if (get.hasFamilies()) {
7570 for (byte[] family : get.familySet()) {
7571 checkFamily(family);
7573 } else { // Adding all families to scanner
7574 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
7575 get.addFamily(family);
7580 @Override
7581 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
7582 return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
7585 private List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7586 throws IOException {
7587 return TraceUtil.trace(() -> getInternal(get, withCoprocessor, nonceGroup, nonce),
7588 () -> createRegionSpan("Region.get"));
7591 private List<Cell> getInternal(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7592 throws IOException {
7593 List<Cell> results = new ArrayList<>();
7594 long before = EnvironmentEdgeManager.currentTime();
7596 // pre-get CP hook
7597 if (withCoprocessor && (coprocessorHost != null)) {
7598 if (coprocessorHost.preGet(get, results)) {
7599 metricsUpdateForGet(results, before);
7600 return results;
7603 Scan scan = new Scan(get);
7604 if (scan.getLoadColumnFamiliesOnDemandValue() == null) {
7605 scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault());
7607 try (RegionScanner scanner = getScanner(scan, null, nonceGroup, nonce)) {
7608 List<Cell> tmp = new ArrayList<>();
7609 scanner.next(tmp);
7610 // Copy EC to heap, then close the scanner.
7611 // This can be an EXPENSIVE call. It may make an extra copy from offheap to onheap buffers.
7612 // See more details in HBASE-26036.
7613 for (Cell cell : tmp) {
7614 results.add(cell instanceof ByteBufferExtendedCell ?
7615 ((ByteBufferExtendedCell) cell).deepClone(): cell);
7619 // post-get CP hook
7620 if (withCoprocessor && (coprocessorHost != null)) {
7621 coprocessorHost.postGet(get, results);
7624 metricsUpdateForGet(results, before);
7626 return results;
7629 void metricsUpdateForGet(List<Cell> results, long before) {
7630 if (this.metricsRegion != null) {
7631 this.metricsRegion.updateGet(EnvironmentEdgeManager.currentTime() - before);
7633 if (this.rsServices != null && this.rsServices.getMetrics() != null) {
7634 rsServices.getMetrics().updateReadQueryMeter(getRegionInfo().getTable(), 1);
7639 @Override
7640 public Result mutateRow(RowMutations rm) throws IOException {
7641 return mutateRow(rm, HConstants.NO_NONCE, HConstants.NO_NONCE);
7644 public Result mutateRow(RowMutations rm, long nonceGroup, long nonce) throws IOException {
7645 final List<Mutation> m = rm.getMutations();
7646 OperationStatus[] statuses = batchMutate(m.toArray(new Mutation[0]), true, nonceGroup, nonce);
7648 List<Result> results = new ArrayList<>();
7649 for (OperationStatus status : statuses) {
7650 if (status.getResult() != null) {
7651 results.add(status.getResult());
7655 if (results.isEmpty()) {
7656 return null;
7659 // Merge the results of the Increment/Append operations
7660 List<Cell> cells = new ArrayList<>();
7661 for (Result result : results) {
7662 if (result.rawCells() != null) {
7663 cells.addAll(Arrays.asList(result.rawCells()));
7666 return Result.create(cells);
7670 * Perform atomic (all or none) mutations within the region.
7671 * @param mutations The list of mutations to perform.
7672 * <code>mutations</code> can contain operations for multiple rows.
7673 * Caller has to ensure that all rows are contained in this region.
7674 * @param rowsToLock Rows to lock
7675 * @param nonceGroup Optional nonce group of the operation (client Id)
7676 * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
7677 * If multiple rows are locked care should be taken that
7678 * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
7679 * @throws IOException
7681 @Override
7682 public void mutateRowsWithLocks(Collection<Mutation> mutations,
7683 Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
7684 batchMutate(new MutationBatchOperation(this, mutations.toArray(new Mutation[mutations.size()]),
7685 true, nonceGroup, nonce) {
7686 @Override
7687 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch(
7688 List<RowLock> acquiredRowLocks) throws IOException {
7689 RowLock prevRowLock = null;
7690 for (byte[] row : rowsToLock) {
7691 try {
7692 RowLock rowLock = region.getRowLock(row, false, prevRowLock); // write lock
7693 if (rowLock != prevRowLock) {
7694 acquiredRowLocks.add(rowLock);
7695 prevRowLock = rowLock;
7697 } catch (IOException ioe) {
7698 LOG.warn("Failed getting lock, row={}, in region {}", Bytes.toStringBinary(row), this,
7699 ioe);
7700 throw ioe;
7703 return createMiniBatch(size(), size());
7709 * @return statistics about the current load of the region
7711 public ClientProtos.RegionLoadStats getLoadStatistics() {
7712 if (!regionStatsEnabled) {
7713 return null;
7715 ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
7716 stats.setMemStoreLoad((int) (Math.min(100,
7717 (this.memStoreSizing.getMemStoreSize().getHeapSize() * 100) / this.memstoreFlushSize)));
7718 if (rsServices.getHeapMemoryManager() != null) {
7719 // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM,
7720 // so we could just do the calculation below and we'll get a 0.
7721 // treating it as a special case analogous to no HMM instead so that it can be
7722 // programatically treated different from using <1% of heap.
7723 final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent();
7724 if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) {
7725 stats.setHeapOccupancy((int)(occupancy * 100));
7728 stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 ? 100
7729 : rsServices.getCompactionPressure() * 100));
7730 return stats.build();
7733 @Override
7734 public Result append(Append append) throws IOException {
7735 return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
7738 public Result append(Append append, long nonceGroup, long nonce) throws IOException {
7739 return TraceUtil.trace(() -> {
7740 checkReadOnly();
7741 checkResources();
7742 startRegionOperation(Operation.APPEND);
7743 try {
7744 // All edits for the given row (across all column families) must happen atomically.
7745 return mutate(append, true, nonceGroup, nonce).getResult();
7746 } finally {
7747 closeRegionOperation(Operation.APPEND);
7749 }, () -> createRegionSpan("Region.append"));
7752 @Override
7753 public Result increment(Increment increment) throws IOException {
7754 return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
7757 public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException {
7758 return TraceUtil.trace(() -> {
7759 checkReadOnly();
7760 checkResources();
7761 startRegionOperation(Operation.INCREMENT);
7762 try {
7763 // All edits for the given row (across all column families) must happen atomically.
7764 return mutate(increment, true, nonceGroup, nonce).getResult();
7765 } finally {
7766 closeRegionOperation(Operation.INCREMENT);
7768 }, () -> createRegionSpan("Region.increment"));
7772 * @return writeEntry associated with this append
7774 private WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds,
7775 long now, long nonceGroup, long nonce, long origLogSeqNum) throws IOException {
7776 Preconditions.checkArgument(walEdit != null && !walEdit.isEmpty(),
7777 "WALEdit is null or empty!");
7778 Preconditions.checkArgument(!walEdit.isReplay() || origLogSeqNum != SequenceId.NO_SEQUENCE_ID,
7779 "Invalid replay sequence Id for replay WALEdit!");
7780 // Using default cluster id, as this can only happen in the originating cluster.
7781 // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey
7782 // here instead of WALKeyImpl directly to support legacy coprocessors.
7783 WALKeyImpl walKey = walEdit.isReplay()?
7784 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
7785 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
7786 nonceGroup, nonce, mvcc) :
7787 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
7788 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
7789 nonceGroup, nonce, mvcc, this.getReplicationScope());
7790 if (walEdit.isReplay()) {
7791 walKey.setOrigLogSeqNum(origLogSeqNum);
7793 //don't call the coproc hook for writes to the WAL caused by
7794 //system lifecycle events like flushes or compactions
7795 if (this.coprocessorHost != null && !walEdit.isMetaEdit()) {
7796 this.coprocessorHost.preWALAppend(walKey, walEdit);
7798 ServerCall<?> rpcCall = RpcServer.getCurrentServerCallWithCellScanner().orElse(null);
7799 try {
7800 long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit);
7801 WriteEntry writeEntry = walKey.getWriteEntry();
7802 regionReplicationSink.ifPresent(sink -> writeEntry.attachCompletionAction(() -> {
7803 sink.add(walKey, walEdit, rpcCall);
7804 }));
7805 // Call sync on our edit.
7806 if (txid != 0) {
7807 sync(txid, durability);
7809 return writeEntry;
7810 } catch (IOException ioe) {
7811 if (walKey.getWriteEntry() != null) {
7812 mvcc.complete(walKey.getWriteEntry());
7814 throw ioe;
7819 public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HRegion.class, false);
7821 // woefully out of date - currently missing:
7822 // 1 x HashMap - coprocessorServiceHandlers
7823 // 6 x LongAdder - numMutationsWithoutWAL, dataInMemoryWithoutWAL,
7824 // checkAndMutateChecksPassed, checkAndMutateChecksFailed, readRequestsCount,
7825 // writeRequestsCount, cpRequestsCount
7826 // 1 x HRegion$WriteState - writestate
7827 // 1 x RegionCoprocessorHost - coprocessorHost
7828 // 1 x RegionSplitPolicy - splitPolicy
7829 // 1 x MetricsRegion - metricsRegion
7830 // 1 x MetricsRegionWrapperImpl - metricsRegionWrapper
7831 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
7832 ClassSize.OBJECT + // closeLock
7833 (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
7834 (3 * ClassSize.ATOMIC_LONG) + // numPutsWithoutWAL, dataInMemoryWithoutWAL,
7835 // compactionsFailed
7836 (3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, scannerReadPoints, regionLockHolders
7837 WriteState.HEAP_SIZE + // writestate
7838 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
7839 (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
7840 MultiVersionConcurrencyControl.FIXED_SIZE // mvcc
7841 + 2 * ClassSize.TREEMAP // maxSeqIdInStores, replicationScopes
7842 + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress
7843 + ClassSize.STORE_SERVICES // store services
7844 + StoreHotnessProtector.FIXED_SIZE
7847 @Override
7848 public long heapSize() {
7849 // this does not take into account row locks, recent flushes, mvcc entries, and more
7850 return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum();
7854 * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to be
7855 * available for handling {@link #execService(RpcController, CoprocessorServiceCall)} calls.
7856 * <p/>
7857 * Only a single instance may be registered per region for a given {@link Service} subclass (the
7858 * instances are keyed on {@link ServiceDescriptor#getFullName()}.. After the first registration,
7859 * subsequent calls with the same service name will fail with a return value of {@code false}.
7860 * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint
7861 * @return {@code true} if the registration was successful, {@code false} otherwise
7863 public boolean registerService(Service instance) {
7864 // No stacking of instances is allowed for a single service name
7865 ServiceDescriptor serviceDesc = instance.getDescriptorForType();
7866 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
7867 if (coprocessorServiceHandlers.containsKey(serviceName)) {
7868 LOG.error("Coprocessor service {} already registered, rejecting request from {} in region {}",
7869 serviceName, instance, this);
7870 return false;
7873 coprocessorServiceHandlers.put(serviceName, instance);
7874 if (LOG.isDebugEnabled()) {
7875 LOG.debug("Registered coprocessor service: region=" +
7876 Bytes.toStringBinary(getRegionInfo().getRegionName()) + " service=" + serviceName);
7878 return true;
7882 * Executes a single protocol buffer coprocessor endpoint {@link Service} method using
7883 * the registered protocol handlers. {@link Service} implementations must be registered via the
7884 * {@link #registerService(Service)}
7885 * method before they are available.
7887 * @param controller an {@code RpcContoller} implementation to pass to the invoked service
7888 * @param call a {@code CoprocessorServiceCall} instance identifying the service, method,
7889 * and parameters for the method invocation
7890 * @return a protocol buffer {@code Message} instance containing the method's result
7891 * @throws IOException if no registered service handler is found or an error
7892 * occurs during the invocation
7893 * @see #registerService(Service)
7895 public Message execService(RpcController controller, CoprocessorServiceCall call)
7896 throws IOException {
7897 String serviceName = call.getServiceName();
7898 Service service = coprocessorServiceHandlers.get(serviceName);
7899 if (service == null) {
7900 throw new UnknownProtocolException(null, "No registered coprocessor service found for " +
7901 serviceName + " in region " + Bytes.toStringBinary(getRegionInfo().getRegionName()));
7903 ServiceDescriptor serviceDesc = service.getDescriptorForType();
7905 cpRequestsCount.increment();
7906 String methodName = call.getMethodName();
7907 MethodDescriptor methodDesc =
7908 CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
7910 Message.Builder builder =
7911 service.getRequestPrototype(methodDesc).newBuilderForType();
7913 ProtobufUtil.mergeFrom(builder,
7914 call.getRequest().toByteArray());
7915 Message request =
7916 CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
7918 if (coprocessorHost != null) {
7919 request = coprocessorHost.preEndpointInvocation(service, methodName, request);
7922 final Message.Builder responseBuilder =
7923 service.getResponsePrototype(methodDesc).newBuilderForType();
7924 service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
7925 @Override
7926 public void run(Message message) {
7927 if (message != null) {
7928 responseBuilder.mergeFrom(message);
7933 if (coprocessorHost != null) {
7934 coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder);
7936 IOException exception =
7937 org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.getControllerException(controller);
7938 if (exception != null) {
7939 throw exception;
7942 return responseBuilder.build();
7945 public Optional<byte[]> checkSplit() {
7946 return checkSplit(false);
7950 * Return the split point. An empty result indicates the region isn't splittable.
7952 public Optional<byte[]> checkSplit(boolean force) {
7953 // Can't split META
7954 if (this.getRegionInfo().isMetaRegion()) {
7955 return Optional.empty();
7958 // Can't split a region that is closing.
7959 if (this.isClosing()) {
7960 return Optional.empty();
7963 if (!force && !splitPolicy.shouldSplit()) {
7964 return Optional.empty();
7967 byte[] ret = splitPolicy.getSplitPoint();
7968 if (ret != null && ret.length > 0) {
7969 ret = splitRestriction.getRestrictedSplitPoint(ret);
7972 if (ret != null) {
7973 try {
7974 checkRow(ret, "calculated split");
7975 } catch (IOException e) {
7976 LOG.error("Ignoring invalid split for region {}", this, e);
7977 return Optional.empty();
7979 return Optional.of(ret);
7980 } else {
7981 return Optional.empty();
7986 * @return The priority that this region should have in the compaction queue
7988 public int getCompactPriority() {
7989 return stores.values().stream().mapToInt(HStore::getCompactPriority).min()
7990 .orElse(Store.NO_PRIORITY);
7993 /** @return the coprocessor host */
7994 public RegionCoprocessorHost getCoprocessorHost() {
7995 return coprocessorHost;
7998 /** @param coprocessorHost the new coprocessor host */
7999 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
8000 this.coprocessorHost = coprocessorHost;
8003 @Override
8004 public void startRegionOperation() throws IOException {
8005 startRegionOperation(Operation.ANY);
8008 @Override
8009 public void startRegionOperation(Operation op) throws IOException {
8010 boolean isInterruptableOp = false;
8011 switch (op) {
8012 case GET: // interruptible read operations
8013 case SCAN:
8014 isInterruptableOp = true;
8015 checkReadsEnabled();
8016 break;
8017 case INCREMENT: // interruptible write operations
8018 case APPEND:
8019 case PUT:
8020 case DELETE:
8021 case BATCH_MUTATE:
8022 case CHECK_AND_MUTATE:
8023 isInterruptableOp = true;
8024 break;
8025 default: // all others
8026 break;
8028 if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION
8029 || op == Operation.COMPACT_REGION || op == Operation.COMPACT_SWITCH) {
8030 // split, merge or compact region doesn't need to check the closing/closed state or lock the
8031 // region
8032 return;
8034 if (this.closing.get()) {
8035 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8037 lock(lock.readLock());
8038 // Update regionLockHolders ONLY for any startRegionOperation call that is invoked from
8039 // an RPC handler
8040 Thread thisThread = Thread.currentThread();
8041 if (isInterruptableOp) {
8042 regionLockHolders.put(thisThread, true);
8044 if (this.closed.get()) {
8045 lock.readLock().unlock();
8046 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8048 // The unit for snapshot is a region. So, all stores for this region must be
8049 // prepared for snapshot operation before proceeding.
8050 if (op == Operation.SNAPSHOT) {
8051 stores.values().forEach(HStore::preSnapshotOperation);
8053 try {
8054 if (coprocessorHost != null) {
8055 coprocessorHost.postStartRegionOperation(op);
8057 } catch (Exception e) {
8058 if (isInterruptableOp) {
8059 // would be harmless to remove what we didn't add but we know by 'isInterruptableOp'
8060 // if we added this thread to regionLockHolders
8061 regionLockHolders.remove(thisThread);
8063 lock.readLock().unlock();
8064 throw new IOException(e);
8068 @Override
8069 public void closeRegionOperation() throws IOException {
8070 closeRegionOperation(Operation.ANY);
8073 @Override
8074 public void closeRegionOperation(Operation operation) throws IOException {
8075 if (operation == Operation.SNAPSHOT) {
8076 stores.values().forEach(HStore::postSnapshotOperation);
8078 Thread thisThread = Thread.currentThread();
8079 regionLockHolders.remove(thisThread);
8080 lock.readLock().unlock();
8081 if (coprocessorHost != null) {
8082 coprocessorHost.postCloseRegionOperation(operation);
8087 * This method needs to be called before any public call that reads or
8088 * modifies stores in bulk. It has to be called just before a try.
8089 * #closeBulkRegionOperation needs to be called in the try's finally block
8090 * Acquires a writelock and checks if the region is closing or closed.
8091 * @throws NotServingRegionException when the region is closing or closed
8092 * @throws RegionTooBusyException if failed to get the lock in time
8093 * @throws InterruptedIOException if interrupted while waiting for a lock
8095 private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException {
8096 if (this.closing.get()) {
8097 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8099 if (writeLockNeeded) lock(lock.writeLock());
8100 else lock(lock.readLock());
8101 if (this.closed.get()) {
8102 if (writeLockNeeded) lock.writeLock().unlock();
8103 else lock.readLock().unlock();
8104 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8106 regionLockHolders.put(Thread.currentThread(), true);
8110 * Closes the lock. This needs to be called in the finally block corresponding
8111 * to the try block of #startRegionOperation
8113 private void closeBulkRegionOperation(){
8114 regionLockHolders.remove(Thread.currentThread());
8115 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
8116 else lock.readLock().unlock();
8120 * Update LongAdders for number of puts without wal and the size of possible data loss.
8121 * These information are exposed by the region server metrics.
8123 private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) {
8124 numMutationsWithoutWAL.increment();
8125 if (numMutationsWithoutWAL.sum() <= 1) {
8126 LOG.info("writing data to region " + this +
8127 " with WAL disabled. Data may be lost in the event of a crash.");
8130 long mutationSize = 0;
8131 for (List<Cell> cells: familyMap.values()) {
8132 // Optimization: 'foreach' loop is not used. See:
8133 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
8134 assert cells instanceof RandomAccess;
8135 int listSize = cells.size();
8136 for (int i=0; i < listSize; i++) {
8137 Cell cell = cells.get(i);
8138 mutationSize += cell.getSerializedSize();
8142 dataInMemoryWithoutWAL.add(mutationSize);
8145 private void lock(final Lock lock) throws IOException {
8146 lock(lock, 1);
8150 * Try to acquire a lock. Throw RegionTooBusyException
8151 * if failed to get the lock in time. Throw InterruptedIOException
8152 * if interrupted while waiting for the lock.
8154 private void lock(final Lock lock, final int multiplier) throws IOException {
8155 try {
8156 final long waitTime = Math.min(maxBusyWaitDuration,
8157 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
8158 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
8159 // Don't print millis. Message is used as a key over in
8160 // RetriesExhaustedWithDetailsException processing.
8161 final String regionName =
8162 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
8163 final String serverName = this.getRegionServerServices() == null ?
8164 "unknown" : (this.getRegionServerServices().getServerName() == null ?
8165 "unknown" : this.getRegionServerServices().getServerName().toString());
8166 RegionTooBusyException rtbe = new RegionTooBusyException(
8167 "Failed to obtain lock; regionName=" + regionName + ", server=" + serverName);
8168 LOG.warn("Region is too busy to allow lock acquisition.", rtbe);
8169 throw rtbe;
8171 } catch (InterruptedException ie) {
8172 if (LOG.isDebugEnabled()) {
8173 LOG.debug("Interrupted while waiting for a lock in region {}", this);
8175 throw throwOnInterrupt(ie);
8180 * Calls sync with the given transaction ID
8181 * @param txid should sync up to which transaction
8182 * @throws IOException If anything goes wrong with DFS
8184 private void sync(long txid, Durability durability) throws IOException {
8185 if (this.getRegionInfo().isMetaRegion()) {
8186 this.wal.sync(txid);
8187 } else {
8188 switch(durability) {
8189 case USE_DEFAULT:
8190 // do what table defaults to
8191 if (shouldSyncWAL()) {
8192 this.wal.sync(txid);
8194 break;
8195 case SKIP_WAL:
8196 // nothing do to
8197 break;
8198 case ASYNC_WAL:
8199 // nothing do to
8200 break;
8201 case SYNC_WAL:
8202 this.wal.sync(txid, false);
8203 break;
8204 case FSYNC_WAL:
8205 this.wal.sync(txid, true);
8206 break;
8207 default:
8208 throw new RuntimeException("Unknown durability " + durability);
8214 * Check whether we should sync the wal from the table's durability settings
8216 private boolean shouldSyncWAL() {
8217 return regionDurability.ordinal() > Durability.ASYNC_WAL.ordinal();
8220 /** @return the latest sequence number that was read from storage when this region was opened */
8221 public long getOpenSeqNum() {
8222 return this.openSeqNum;
8225 @Override
8226 public Map<byte[], Long> getMaxStoreSeqId() {
8227 return this.maxSeqIdInStores;
8230 public long getOldestSeqIdOfStore(byte[] familyName) {
8231 return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
8234 @Override
8235 public CompactionState getCompactionState() {
8236 boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0;
8237 return (hasMajor ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR)
8238 : (hasMinor ? CompactionState.MINOR : CompactionState.NONE));
8241 public void reportCompactionRequestStart(boolean isMajor){
8242 (isMajor ? majorInProgress : minorInProgress).incrementAndGet();
8245 public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) {
8246 int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet();
8248 // metrics
8249 compactionsFinished.increment();
8250 compactionNumFilesCompacted.add(numFiles);
8251 compactionNumBytesCompacted.add(filesSizeCompacted);
8253 assert newValue >= 0;
8256 public void reportCompactionRequestFailure() {
8257 compactionsFailed.increment();
8260 public void incrementCompactionsQueuedCount() {
8261 compactionsQueued.increment();
8264 public void decrementCompactionsQueuedCount() {
8265 compactionsQueued.decrement();
8268 public void incrementFlushesQueuedCount() {
8269 flushesQueued.increment();
8272 protected void decrementFlushesQueuedCount() {
8273 flushesQueued.decrement();
8277 * If a handler thread is eligible for interrupt, make it ineligible. Should be paired
8278 * with {{@link #enableInterrupts()}.
8280 void disableInterrupts() {
8281 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> false);
8285 * If a handler thread was made ineligible for interrupt via {{@link #disableInterrupts()},
8286 * make it eligible again. No-op if interrupts are already enabled.
8288 void enableInterrupts() {
8289 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> true);
8293 * Interrupt any region options that have acquired the region lock via
8294 * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)},
8295 * or {@link #startBulkRegionOperation(boolean)}.
8297 private void interruptRegionOperations() {
8298 for (Map.Entry<Thread, Boolean> entry: regionLockHolders.entrySet()) {
8299 // An entry in this map will have a boolean value indicating if it is currently
8300 // eligible for interrupt; if so, we should interrupt it.
8301 if (entry.getValue().booleanValue()) {
8302 entry.getKey().interrupt();
8308 * Check thread interrupt status and throw an exception if interrupted.
8309 * @throws NotServingRegionException if region is closing
8310 * @throws InterruptedIOException if interrupted but region is not closing
8312 // Package scope for tests
8313 void checkInterrupt() throws NotServingRegionException, InterruptedIOException {
8314 if (Thread.interrupted()) {
8315 if (this.closing.get()) {
8316 throw new NotServingRegionException(
8317 getRegionInfo().getRegionNameAsString() + " is closing");
8319 throw new InterruptedIOException();
8324 * Throw the correct exception upon interrupt
8325 * @param t cause
8327 // Package scope for tests
8328 IOException throwOnInterrupt(Throwable t) {
8329 if (this.closing.get()) {
8330 return (NotServingRegionException) new NotServingRegionException(
8331 getRegionInfo().getRegionNameAsString() + " is closing")
8332 .initCause(t);
8334 return (InterruptedIOException) new InterruptedIOException().initCause(t);
8338 * {@inheritDoc}
8340 @Override
8341 public void onConfigurationChange(Configuration conf) {
8342 this.storeHotnessProtector.update(conf);
8346 * {@inheritDoc}
8348 @Override
8349 public void registerChildren(ConfigurationManager manager) {
8350 configurationManager = manager;
8351 stores.values().forEach(manager::registerObserver);
8355 * {@inheritDoc}
8357 @Override
8358 public void deregisterChildren(ConfigurationManager manager) {
8359 stores.values().forEach(configurationManager::deregisterObserver);
8362 @Override
8363 public CellComparator getCellComparator() {
8364 return cellComparator;
8367 public long getMemStoreFlushSize() {
8368 return this.memstoreFlushSize;
8372 //// method for debugging tests
8373 void throwException(String title, String regionName) {
8374 StringBuilder buf = new StringBuilder();
8375 buf.append(title + ", ");
8376 buf.append(getRegionInfo().toString());
8377 buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " ");
8378 buf.append("stores: ");
8379 for (HStore s : stores.values()) {
8380 buf.append(s.getColumnFamilyDescriptor().getNameAsString());
8381 buf.append(" size: ");
8382 buf.append(s.getMemStoreSize().getDataSize());
8383 buf.append(" ");
8385 buf.append("end-of-stores");
8386 buf.append(", memstore size ");
8387 buf.append(getMemStoreDataSize());
8388 if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) {
8389 throw new RuntimeException(buf.toString());
8393 @Override
8394 public void requestCompaction(String why, int priority, boolean major,
8395 CompactionLifeCycleTracker tracker) throws IOException {
8396 if (major) {
8397 stores.values().forEach(HStore::triggerMajorCompaction);
8399 rsServices.getCompactionRequestor().requestCompaction(this, why, priority, tracker,
8400 RpcServer.getRequestUser().orElse(null));
8403 @Override
8404 public void requestCompaction(byte[] family, String why, int priority, boolean major,
8405 CompactionLifeCycleTracker tracker) throws IOException {
8406 HStore store = stores.get(family);
8407 if (store == null) {
8408 throw new NoSuchColumnFamilyException("column family " + Bytes.toString(family) +
8409 " does not exist in region " + getRegionInfo().getRegionNameAsString());
8411 if (major) {
8412 store.triggerMajorCompaction();
8414 rsServices.getCompactionRequestor().requestCompaction(this, store, why, priority, tracker,
8415 RpcServer.getRequestUser().orElse(null));
8418 private void requestFlushIfNeeded() throws RegionTooBusyException {
8419 if(isFlushSize(this.memStoreSizing.getMemStoreSize())) {
8420 requestFlush();
8424 private void requestFlush() {
8425 if (this.rsServices == null) {
8426 return;
8428 requestFlush0(FlushLifeCycleTracker.DUMMY);
8431 private void requestFlush0(FlushLifeCycleTracker tracker) {
8432 boolean shouldFlush = false;
8433 synchronized (writestate) {
8434 if (!this.writestate.isFlushRequested()) {
8435 shouldFlush = true;
8436 writestate.flushRequested = true;
8439 if (shouldFlush) {
8440 // Make request outside of synchronize block; HBASE-818.
8441 this.rsServices.getFlushRequester().requestFlush(this, tracker);
8442 if (LOG.isDebugEnabled()) {
8443 LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
8445 } else {
8446 tracker.notExecuted("Flush already requested on " + this);
8450 @Override
8451 public void requestFlush(FlushLifeCycleTracker tracker) throws IOException {
8452 requestFlush0(tracker);
8456 * This method modifies the region's configuration in order to inject replication-related
8457 * features
8458 * @param conf region configurations
8460 private static void decorateRegionConfiguration(Configuration conf) {
8461 if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
8462 String plugins = conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,"");
8463 String replicationCoprocessorClass = ReplicationObserver.class.getCanonicalName();
8464 if (!plugins.contains(replicationCoprocessorClass)) {
8465 conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
8466 (plugins.equals("") ? "" : (plugins + ",")) + replicationCoprocessorClass);
8471 public Optional<RegionReplicationSink> getRegionReplicationSink() {
8472 return regionReplicationSink;
8475 public void addReadRequestsCount(long readRequestsCount) {
8476 this.readRequestsCount.add(readRequestsCount);
8479 public void addWriteRequestsCount(long writeRequestsCount) {
8480 this.writeRequestsCount.add(writeRequestsCount);