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
;
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
;
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
297 protected volatile long lastReplayedOpenRegionSeqId
= -1L;
298 protected volatile long lastReplayedCompactionSeqId
= -1L;
300 //////////////////////////////////////////////////////////////////////////////
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
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
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
;
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
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
{
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
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
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.
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).
558 public boolean isCompactionNeeded() {
559 return result
== Result
.FLUSHED_COMPACTION_NEEDED
;
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();
571 public Result
getResult() {
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 */
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() {
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() {
639 * Records that a {@link WrongRegionException} has been observed.
641 void sawWrongRegion() {
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() {
667 * Records that a {@link NoSuchColumnFamilyException} has been observed.
669 void sawNoSuchFamily() {
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
;
696 private final ReentrantReadWriteLock updatesLock
= new ReentrantReadWriteLock();
698 private final MultiVersionConcurrencyControl mvcc
;
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
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.
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
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
) {
769 throw new IllegalArgumentException("Need table descriptor");
772 if (confParam
instanceof CompoundConfiguration
) {
773 throw new IllegalArgumentException("Need original base configuration");
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
);
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
) ?
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) {
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()
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);
954 nextSeqId
= initializeRegionInternals(reporter
, status
);
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
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());
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() +
979 if (LOG
.isDebugEnabled()) {
980 LOG
.debug("Region open journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
981 status
.prettyPrintJournal());
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();
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
);
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(),
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");
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()) ||
1107 regionReplicationSink
= Optional
.empty();
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()));
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.
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
>() {
1147 public HStore
call() throws IOException
{
1148 return instantiateHStore(family
, warmup
);
1152 boolean allStoresOpened
= false;
1153 boolean hasSloppyStores
= false;
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())
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());
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()) {
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());
1208 initializeStores(reporter
, status
, true);
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) {
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
,
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
{
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) {
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
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());
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
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());
1385 public RegionInfo
getRegionInfo() {
1386 return this.fs
.getRegionInfo();
1390 * @return Instance of {@link RegionServerServices} used by this HRegion.
1393 RegionServerServices
getRegionServerServices() {
1394 return this.rsServices
;
1398 public long getReadRequestsCount() {
1399 return readRequestsCount
.sum();
1403 public long getCpRequestsCount() {
1404 return cpRequestsCount
.sum();
1408 public long getFilteredReadRequestsCount() {
1409 return filteredReadRequestsCount
.sum();
1413 public long getWriteRequestsCount() {
1414 return writeRequestsCount
.sum();
1418 public long getMemStoreDataSize() {
1419 return memStoreSizing
.getDataSize();
1423 public long getMemStoreHeapSize() {
1424 return memStoreSizing
.getHeapSize();
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
;
1438 public long getNumMutationsWithoutWAL() {
1439 return numMutationsWithoutWAL
.sum();
1443 public long getDataInMemoryWithoutWAL() {
1444 return dataInMemoryWithoutWAL
.sum();
1448 public long getBlockedRequestsCount() {
1449 return blockedRequestsCount
.sum();
1453 public long getCheckAndMutateChecksPassed() {
1454 return checkAndMutateChecksPassed
.sum();
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
1465 public MetricsRegion
getMetrics() {
1466 return metricsRegion
;
1470 public boolean isClosed() {
1471 return this.closed
.get();
1475 public boolean isClosing() {
1476 return this.closing
.get();
1480 public boolean isReadOnly() {
1481 return this.writestate
.isReadOnly();
1485 public boolean isAvailable() {
1486 return !isClosed() && !isClosing();
1490 public boolean isSplittable() {
1491 return splitPolicy
.canSplit();
1495 public boolean isMergeable() {
1496 if (!isAvailable()) {
1497 LOG
.debug("Region " + this
1498 + " is not mergeable because it is closing or closed");
1501 if (hasReferences()) {
1502 LOG
.debug("Region " + this
1503 + " is not mergeable because it has references");
1510 public boolean areWritesEnabled() {
1511 synchronized(this.writestate
) {
1512 return this.writestate
.writesEnabled
;
1516 public MultiVersionConcurrencyControl
getMVCC() {
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");
1623 synchronized (closeLock
) {
1624 return doClose(abort
, status
);
1627 if (LOG
.isDebugEnabled()) {
1628 LOG
.debug("Region close journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
1629 status
.prettyPrintJournal());
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
{
1657 LOG
.warn("Region " + this + " already closed");
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
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
1679 if (!abort
&& worthPreFlushing() && canFlush
) {
1680 status
.setStatus("Pre-flushing region before close");
1681 LOG
.info("Running close preflush of {}", this.getRegionInfo().getEncodedName());
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;
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;
1731 long start
= EnvironmentEdgeManager
.currentTime();
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;
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
+
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.
1764 String msg
= "Failed to acquire close lock on " + this + " after waiting " +
1765 elapsedWaitTime
+ " ms";
1768 // If we failed to acquire the write lock, abort the server
1769 rsServices
.abort(msg
, null);
1771 throw new IOException(msg
);
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");
1789 if (this.isClosed()) {
1790 status
.abort("Already got closed by another process");
1791 // SplitTransaction handles the 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;
1800 long remainingSize
= this.memStoreSizing
.getDataSize();
1801 while (remainingSize
> 0) {
1803 internalFlushcache(status
);
1805 LOG
.info("Running extra flush, " + flushCount
+
1806 " (carrying snapshot?) " + this);
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.
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);
1855 .submit(new Callable
<Pair
<byte[], Collection
<HStoreFile
>>>() {
1857 public Pair
<byte[], Collection
<HStoreFile
>> call() throws IOException
{
1858 return new Pair
<>(store
.getColumnFamilyDescriptor().getName(), store
.close());
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
);
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();
1898 regionReplicationSink
.get().waitUntilStopped();
1899 } catch (InterruptedException e
) {
1900 throw throwOnInterrupt(e
);
1903 this.closed
.set(true);
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);
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
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).
1937 boolean interrupted
= false;
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);
1944 } catch (InterruptedException iex
) {
1945 // essentially ignore and propagate the interrupt back up
1946 LOG
.warn("Interrupted while waiting in region {}", this);
1953 Thread
.currentThread().interrupt();
1960 * Wait for all current flushes of the region to complete
1962 public void waitForFlushes() {
1963 waitForFlushes(0);// Unbound wait
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).
1974 if (!writestate
.flushing
) return true;
1975 long start
= EnvironmentEdgeManager
.currentTime();
1977 boolean interrupted
= false;
1978 LOG
.debug("waiting for cache flush to complete for region " + this);
1980 while (writestate
.flushing
) {
1981 if (timeout
> 0 && duration
>= timeout
) break;
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);
1991 duration
= EnvironmentEdgeManager
.currentTime() - start
;
1996 Thread
.currentThread().interrupt();
1999 LOG
.debug("Waited {} ms for region {} flush to complete", duration
, this);
2000 return !(writestate
.flushing
);
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
)
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;
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 //////////////////////////////////////////////////////////////////////////////
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() {
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() {
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
);
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());
2144 public long getEarliestFlushTimeForAllStores() {
2145 return Collections
.min(lastStoreFlushTimeMap
.values());
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) {
2156 for (HStoreFile file
: storeFiles
) {
2157 StoreFileReader sfReader
= file
.getReader();
2158 if (sfReader
== null) {
2161 HFile
.Reader reader
= sfReader
.getHFileReader();
2162 if (reader
== null) {
2165 if (majorCompactionOnly
) {
2166 byte[] val
= reader
.getHFileInfo().get(MAJOR_COMPACTION_KEY
);
2167 if (val
== null || !Bytes
.toBoolean(val
)) {
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
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
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.
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.
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());
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
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
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()
2343 * synchronized (writestate) {
2344 * if (writestate.writesEnabled) {
2345 * wasStateSet = true;
2346 * ++writestate.compacting;
2348 * String msg = "NOT compacting region " + this + ". Writes disabled.";
2350 * status.abort(msg);
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();
2360 * if (closeChecker != null && closeChecker.isSizeLimit(store, len)) {
2361 * progress.cancel();
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
);
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
);
2383 MonitoredTask status
= null;
2384 boolean requestNeedsCancellation
= true;
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");
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";
2402 boolean wasStateSet
= false;
2404 synchronized (writestate
) {
2405 if (writestate
.writesEnabled
) {
2407 writestate
.compacting
.incrementAndGet();
2409 String msg
= "NOT compacting region " + this + ". Writes disabled.";
2415 LOG
.info("Starting compaction of {} in {}{}", store
, this,
2416 (compaction
.getRequest().isOffPeak()?
" as an off-peak compaction":""));
2417 doRegionCompactionPrep();
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
);
2432 synchronized (writestate
) {
2433 writestate
.compacting
.decrementAndGet();
2434 if (writestate
.compacting
.get() <= 0) {
2435 writestate
.notifyAll();
2440 status
.markComplete("Compaction complete");
2443 if (requestNeedsCancellation
) store
.cancelRequestedCompaction(compaction
);
2444 if (status
!= null) {
2445 LOG
.debug("Compaction status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2446 status
.prettyPrintJournal());
2455 * <p>When this method is called the cache will be flushed unless:
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>
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
{
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
,
2487 /** @return the detailed result code */
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
);
2510 * When this method is called the cache will be flushed unless:
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>
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";
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;
2545 if (this.closed
.get()) {
2546 String msg
= "Skipping flush on " + this + " because closed";
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
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;
2566 String msg
= "NOT flushing " + this + " as " + (writestate
.flushing ?
"already flushing"
2567 : "writes are not enabled");
2571 return new FlushResultImpl(FlushResult
.Result
.CANNOT_FLUSH
, msg
, false);
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
);
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());
2600 synchronized (writestate
) {
2601 writestate
.flushing
= false;
2602 this.writestate
.flushRequested
= false;
2603 writestate
.notifyAll();
2607 lock
.readLock().unlock();
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());
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.
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());
2648 if (this.flushCheckInterval
<= 0) {
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
);
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");
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
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
)) {
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");
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
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
);
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
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;
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.).
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.
2782 return new PrepareFlushResult(flushResult
, myseqid
);
2784 return new PrepareFlushResult(new FlushResultImpl(
2785 FlushResult
.Result
.CANNOT_FLUSH_MEMSTORE_EMPTY
, "Nothing to flush", false), myseqid
);
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
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();
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),
2840 flushOpSeqId
= getNextSequenceId(wal
);
2841 // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
2843 earliestUnflushedSequenceIdForTheRegion
.longValue() == HConstants
.NO_SEQNUM?
2844 flushOpSeqId
: earliestUnflushedSequenceIdForTheRegion
.longValue() - 1;
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,
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
);
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()) {
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;
2922 FlushDescriptor desc
= ProtobufUtil
.toFlushDescriptor(FlushAction
.ABORT_FLUSH
,
2923 getRegionInfo(), flushOpSeqId
, committedFiles
);
2924 WALUtil
.writeFlushMarker(wal
, this.getReplicationScope(), getRegionInfo(), desc
, false, mvcc
,
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
{
2944 wal
.sync(); // ensure that flush marker is sync'ed
2945 } catch (IOException ioe
) {
2946 wal
.abortCacheFlush(hri
.getEncodedNameAsBytes());
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.
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
));
2969 WALUtil
.writeFlushMarker(wal
, this.getReplicationScope(), getRegionInfo(), desc
, true, mvcc
,
2972 } catch (IOException e
) {
2973 LOG
.warn(getRegionInfo().getEncodedName() + " : "
2974 + "Received exception while trying to write the flush request to wal", e
);
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;
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
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
);
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
3061 FlushDescriptor desc
= ProtobufUtil
.toFlushDescriptor(FlushAction
.ABORT_FLUSH
,
3062 getRegionInfo(), flushOpSeqId
, committedFiles
);
3063 WALUtil
.writeFlushMarker(wal
, this.replicationScope
, getRegionInfo(), desc
, false, mvcc
,
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
);
3090 // If we get to here, the HStores have been written.
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" : "");
3120 status
.setStatus(msg
);
3122 if (rsServices
!= null && rsServices
.getMetrics() != null) {
3123 rsServices
.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(),
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 //////////////////////////////////////////////////////////////////////////////
3149 public RegionScannerImpl
getScanner(Scan scan
) throws IOException
{
3150 return getScanner(scan
, null);
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
);
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
);
3171 for (byte[] family
: scan
.getFamilyMap().keySet()) {
3172 checkFamily(family
);
3175 return instantiateRegionScanner(scan
, additionalScanners
, nonceGroup
, nonce
);
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());
3205 for(byte [] family
: delete
.getFamilyCellMap().keySet()) {
3206 if(family
== null) {
3207 throw new NoSuchColumnFamilyException("Empty family is invalid");
3209 checkFamily(family
, delete
.getDurability());
3215 public void delete(Delete delete
) throws IOException
{
3216 TraceUtil
.trace(() -> {
3219 startRegionOperation(Operation
.DELETE
);
3221 // All edits for the given row (across all column families) must happen atomically.
3222 return mutate(delete
);
3224 closeRegionOperation(Operation
.DELETE
);
3226 }, () -> createRegionSpan("Region.delete"));
3230 * Set up correct timestamps in the KVs in Delete object.
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);
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
,
3266 updateDeleteLatestVersionTimestamp(cell
, get
, count
, byteNow
);
3269 updateDeleteLatestVersionTimestamp(cell
, get
, count
, byteNow
);
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
);
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());
3300 public void put(Put put
) throws IOException
{
3301 TraceUtil
.trace(() -> {
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.
3309 startRegionOperation(Operation
.PUT
);
3311 // All edits for the given row (across all column families) must happen atomically.
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
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
)) {
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
)
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
);
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
;
3453 return operations
.length
;
3456 public boolean isOperationPending(int index
) {
3457 return retCodeDetails
[index
].getOperationStatusCode() == OperationStatusCode
.NOT_RUN
;
3460 public List
<UUID
> getClusterIds() {
3462 return getMutation(0).getClusterIds();
3465 boolean isAtomic() {
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
);
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());
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
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());
3523 LOG
.warn(msg
, fsce
);
3524 observedExceptions
.sawFailedSanityCheck();
3526 retCodeDetails
[index
] = new OperationStatus(
3527 OperationStatusCode
.SANITY_CHECK_FAILURE
, fsce
.getMessage());
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());
3537 observedExceptions
.sawWrongRegion();
3539 retCodeDetails
[index
] = new OperationStatus(
3540 OperationStatusCode
.SANITY_CHECK_FAILURE
, we
.getMessage());
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
)) {
3567 if (!isOperationPending(lastIndexExclusive
)) {
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();
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
);
3586 retCodeDetails
[lastIndexExclusive
] =
3587 new OperationStatus(OperationStatusCode
.STORE_TOO_BUSY
, rtbe
.getMessage());
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;
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;
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;
3611 } catch (Throwable throwable
) {
3612 throwException
= true;
3615 if (throwException
) {
3616 region
.storeHotnessProtector
.finish(curFamilyCellMap
);
3619 if (rowLock
== null) {
3620 // We failed to grab another lock
3622 region
.storeHotnessProtector
.finish(curFamilyCellMap
);
3623 throw new IOException("Can't apply all operations atomically!");
3625 break; // Stop acquiring more rows for this batch
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
;
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());
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()) {
3687 walEdit
.add(familyCellMaps
[index
]);
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()) {
3719 // miniBatchOp is null, if and only if lockRowsAndBuildMiniBatch throwing exception.
3720 // This case was handled.
3721 if (miniBatchOp
== null) {
3725 final int finalLastIndexExclusive
= miniBatchOp
.getLastIndexExclusive();
3727 for (int i
= nextIndexToProcess
; i
< finalLastIndexExclusive
; i
++) {
3728 switch (retCodeDetails
[i
].getOperationStatusCode()) {
3731 region
.storeHotnessProtector
.finish(getMutation(i
).getFamilyCellMap());
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
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
;
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
;
3782 public Mutation
getMutation(int index
) {
3783 return this.operations
[index
];
3787 public long getNonceGroup(int index
) {
3792 public long getNonce(int index
) {
3797 public Mutation
[] getMutationsForCoprocs() {
3798 return this.operations
;
3802 public boolean isInReplay() {
3807 public long getOrigLogSeqNum() {
3808 return SequenceId
.NO_SEQUENCE_ID
;
3812 public void startRegionOperation() throws IOException
{
3813 region
.startRegionOperation(Operation
.BATCH_MUTATE
);
3817 public void closeRegionOperation() throws IOException
{
3818 region
.closeRegionOperation(Operation
.BATCH_MUTATE
);
3822 public void checkAndPreparePut(Put p
) throws IOException
{
3823 region
.checkFamilies(p
.getFamilyCellMap().keySet(), p
.getDurability());
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
;
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
;
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
);
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();
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();
3901 returnResults
= ((Append
) mutation
).isReturnResults();
3904 // For nonce operations
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
);
3912 result
= Result
.EMPTY_RESULT
;
3914 retCodeDetails
[index
] = new OperationStatus(OperationStatusCode
.SUCCESS
, result
);
3918 Result result
= null;
3919 if (region
.coprocessorHost
!= null) {
3920 if (mutation
instanceof Increment
) {
3921 result
= region
.coprocessorHost
.preIncrementAfterRowLock((Increment
) mutation
);
3923 result
= region
.coprocessorHost
.preAppendAfterRowLock((Append
) mutation
);
3926 if (result
!= null) {
3927 retCodeDetails
[index
] = new OperationStatus(OperationStatusCode
.SUCCESS
,
3928 returnResults ? result
: Result
.EMPTY_RESULT
);
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();
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());
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
) {
3976 canProceed
= region
.rsServices
.getNonceManager()
3977 .startOperation(nonceGroup
, nonce
, region
.rsServices
);
3978 } catch (InterruptedException ex
) {
3979 throw new InterruptedIOException("Nonce start operation interrupted");
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
) {
4005 Increment increment
= (Increment
) mutation
;
4006 get
.setTimeRange(increment
.getTimeRange().getMin(), increment
.getTimeRange().getMax());
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());
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
));
4035 ret
.computeIfAbsent(store
.getColumnFamilyDescriptor().getName(),
4036 key
-> new ArrayList
<>()).add(cell
);
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
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
));
4075 if (mutation
instanceof Increment
) {
4076 tr
= ((Increment
) mutation
).getTimeRange();
4078 tr
= ((Append
) mutation
).getTimeRange();
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.
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
));
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())
4119 if (region
.maxCellSize
> 0) {
4120 int newCellSize
= PrivateCellUtil
.estimatedSerializedSizeOf(newCell
);
4121 if (newCellSize
> region
.maxCellSize
) {
4123 "Cell with size " + newCellSize
+ " exceeds limit of " + region
.maxCellSize
+
4124 " bytes in region " + this;
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
))
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
);
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!");
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());
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());
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
]) {
4246 new OperationStatus(retCodeDetails
[i
].getOperationStatusCode(), result
);
4248 } else if (m
instanceof Append
) {
4249 Result result
= region
.getCoprocessorHost().postAppend((Append
) m
, results
[i
],
4251 if (result
!= results
[i
]) {
4253 new OperationStatus(retCodeDetails
[i
].getOperationStatusCode(), result
);
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;
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
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
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
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
4355 retCodeDetails
[index
] = new OperationStatus(OperationStatusCode
.SUCCESS
, result
);
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) {
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());
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());
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
;
4431 public Mutation
getMutation(int index
) {
4432 return this.operations
[index
].mutation
;
4436 public long getNonceGroup(int index
) {
4437 return this.operations
[index
].nonceGroup
;
4441 public long getNonce(int index
) {
4442 return this.operations
[index
].nonce
;
4446 public Mutation
[] getMutationsForCoprocs() {
4451 public boolean isInReplay() {
4456 public long getOrigLogSeqNum() {
4457 return this.origLogSeqNum
;
4461 public void startRegionOperation() throws IOException
{
4462 region
.startRegionOperation(Operation
.REPLAY_BATCH_MUTATE
);
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
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
);
4496 public void checkAndPrepare() throws IOException
{
4497 long now
= EnvironmentEdgeManager
.currentTime();
4498 visitBatchOperations(true, this.size(), (int index
) -> {
4499 checkAndPrepareMutation(index
, now
);
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());
4517 public WriteEntry
writeMiniBatchOperationsToMemStore(
4518 final MiniBatchOperationInProgress
<Mutation
> miniBatchOp
, final WriteEntry writeEntry
)
4519 throws IOException
{
4520 super.writeMiniBatchOperationsToMemStore(miniBatchOp
, getOrigLogSeqNum());
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
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
));
4544 public OperationStatus
[] batchMutate(Mutation
[] mutations
) throws IOException
{
4545 // If the mutations has any Increment/Append operations, we need to do batchMutate atomically
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
;
4578 return batchMutate(new ReplayBatchOperation(this, mutations
, replaySeqId
));
4582 * Perform a batch of mutations.
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}.
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();
4603 while (!batchOp
.isDone()) {
4604 if (!batchOp
.isInReplay()) {
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();
4616 doMiniBatchMutate(batchOp
);
4617 requestFlushIfNeeded();
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.
4649 // STEP 1. Try to acquire as many locks as we can and build mini-batch of operations with
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?
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
4666 lock(this.updatesLock
.readLock(), miniBatchOp
.getReadyToWriteCount());
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
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
);
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
);
4713 // Call complete rather than completeAndWait because we probably had error if walKey != null
4714 if (writeEntry
!= null) mvcc
.complete(writeEntry
);
4717 this.updatesLock
.readLock().unlock();
4719 releaseRowLocks(acquiredRowLocks
);
4723 final int finalLastIndexExclusive
=
4724 miniBatchOp
!= null ? miniBatchOp
.getLastIndexExclusive() : batchOp
.size();
4725 final boolean finalSuccess
= success
;
4726 batchOp
.visitBatchOperations(true, finalLastIndexExclusive
,
4728 Mutation mutation
= batchOp
.getMutation(i
);
4729 if (mutation
instanceof Increment
|| mutation
instanceof Append
) {
4731 batchOp
.retCodeDetails
[i
] = new OperationStatus(OperationStatusCode
.SUCCESS
,
4732 batchOp
.results
[i
]);
4734 batchOp
.retCodeDetails
[i
] = OperationStatus
.FAILURE
;
4737 batchOp
.retCodeDetails
[i
] =
4738 finalSuccess ? OperationStatus
.SUCCESS
: OperationStatus
.FAILURE
;
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
;
4759 public boolean checkAndMutate(byte[] row
, byte[] family
, byte[] qualifier
, CompareOperator op
,
4760 ByteArrayComparable comparator
, TimeRange timeRange
, Mutation mutation
) throws IOException
{
4761 CheckAndMutate checkAndMutate
;
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
);
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();
4781 public boolean checkAndMutate(byte[] row
, Filter filter
, TimeRange timeRange
, Mutation mutation
)
4782 throws IOException
{
4783 CheckAndMutate checkAndMutate
;
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
);
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();
4803 public boolean checkAndRowMutate(byte[] row
, byte[] family
, byte[] qualifier
, CompareOperator op
,
4804 ByteArrayComparable comparator
, TimeRange timeRange
, RowMutations rm
) throws IOException
{
4805 CheckAndMutate checkAndMutate
;
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();
4817 public boolean checkAndRowMutate(byte[] row
, Filter filter
, TimeRange timeRange
, RowMutations rm
)
4818 throws IOException
{
4819 CheckAndMutate checkAndMutate
;
4821 checkAndMutate
= CheckAndMutate
.newBuilder(row
).ifMatches(filter
).timeRange(timeRange
)
4823 } catch (IllegalArgumentException e
) {
4824 throw new DoNotRetryIOException(e
.getMessage());
4826 return checkAndMutate(checkAndMutate
).isSuccess();
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();
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();
4863 rowMutations
= (RowMutations
) checkAndMutate
.getAction();
4866 if (mutation
!= null) {
4867 checkMutationType(mutation
);
4868 checkRow(mutation
, row
);
4870 checkRow(rowMutations
, row
);
4873 // TODO, add check for value length also move this check to the client
4875 startRegionOperation();
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);
4892 if (this.getCoprocessorHost() != null) {
4893 CheckAndMutateResult result
=
4894 getCoprocessorHost().preCheckAndMutateAfterRowLock(checkAndMutate
);
4895 if (result
!= null) {
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;
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()) {
4913 cellTs
= result
.get(0).getTimestamp();
4916 boolean valueIsNull
=
4917 comparator
.getValue() == null || comparator
.getValue().length
== 0;
4918 if (result
.isEmpty() && valueIsNull
) {
4920 } else if (result
.size() > 0 && result
.get(0).getValueLength() == 0 && valueIsNull
) {
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
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).
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.
4958 if (mutation
!= null) {
4959 r
= mutate(mutation
, true, nonceGroup
, nonce
).getResult();
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);
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;
4995 matches
= compareResult
< 0;
4998 matches
= compareResult
<= 0;
5001 matches
= compareResult
== 0;
5004 matches
= compareResult
!= 0;
5006 case GREATER_OR_EQUAL
:
5007 matches
= compareResult
>= 0;
5010 matches
= compareResult
> 0;
5013 throw new RuntimeException("Unknown Compare op " + op
.name());
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());
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.
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
) {
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.
5121 * We throw RegionTooBusyException if above memstore limit and expect client to retry using some
5124 private void checkResources() throws RegionTooBusyException
{
5125 // If catalog region, do not impose resource constraints or block updates.
5126 if (this.getRegionInfo().isMetaRegion()) {
5130 MemStoreSize mss
= this.memStoreSizing
.getMemStoreSize();
5131 if (mss
.getHeapSize() + mss
.getOffHeapSize() > this.blockingMemStoreSize
) {
5132 blockedRequestsCount
.increment();
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="
5145 LOG
.warn("Region is too busy due to exceeding memstore size limit.", rtbe
);
5151 * @throws IOException Throws exception if region is in read-only mode.
5153 private void checkReadOnly() throws IOException
{
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;
5183 store
.upsert(cells
, getSmallestReadPoint(), memstoreAccounting
);
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
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
) {
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
+ ")");
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
);
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
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();
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
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
);
5349 deleteRecoveredEdits(walFS
, Iterables
.concat(files
, filesUnderWrongRegionWALDir
));
5350 deleteRecoveredEdits(rootFS
, filesUnderRootDir
);
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
,
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())) {
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()));
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
);
5399 if (isZeroLengthThenDelete(fs
, fs
.getFileStatus(edits
), edits
)) {
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
;
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
);
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.");
5435 Path p
= WALSplitUtil
.moveAsideBadEditsFile(fs
, edits
);
5436 LOG
.error(HConstants
.HREGION_EDITS_REPLAY_SKIP_ERRORS
+ "=true so continuing. Renamed "
5437 + edits
+ " as " + p
,
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
;
5455 MonitoredTask status
= TaskMonitor
.get().createStatus(msg
);
5457 status
.setStatus("Opening recovered edits");
5458 WAL
.Reader reader
= null;
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;
5468 HStore store
= null;
5469 boolean reported_once
= false;
5470 ServerNonceManager ng
= this.rsServices
== null ?
null : this.rsServices
.getNonceManager();
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
5496 long cur
= EnvironmentEdgeManager
.currentTime();
5497 if (lastReport
+ period
<= cur
) {
5498 status
.setStatus("Replaying edits..." +
5499 " skipped=" + skippedEdits
+
5500 " edits=" + editsCount
);
5502 if(!reporter
.progress()) {
5503 msg
= "Progressable reporter failed, stopping replay for region " + this;
5506 throw new IOException(msg
);
5508 reported_once
= true;
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
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 ...
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
);
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);
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);
5581 // Now, figure if we should skip this edit.
5582 if (key
.getSequenceId() <= maxSeqIdInStores
.get(store
.getColumnFamilyDescriptor()
5587 PrivateCellUtil
.setSequenceId(cell
, currentReplaySeqId
);
5589 restoreEdit(store
, cell
, memStoreSizing
);
5592 MemStoreSize mss
= memStoreSizing
.getMemStoreSize();
5593 incMemStoreSize(mss
);
5594 flush
= isFlushSize(this.memStoreSizing
.getMemStoreSize());
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;
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
;
5623 status
.setStatus(msg
);
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
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
);
5639 return currentEditSeqId
;
5642 if (reader
!= null) {
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.
5653 void replayWALCompactionMarker(CompactionDescriptor compaction
, boolean pickCompactionFiles
,
5654 boolean removeFiles
, long replaySeqId
)
5655 throws IOException
{
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
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
);
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
);
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
);
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()));
5701 store
.replayCompactionMarker(compaction
, pickCompactionFiles
, removeFiles
);
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
);
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
5729 FlushAction action
= flush
.getAction();
5732 replayWALFlushStartMarker(flush
);
5735 replayWALFlushCommitMarker(flush
);
5738 replayWALFlushAbortMarker(flush
);
5741 replayWALFlushCannotFlushMarker(flush
, replaySeqId
);
5744 LOG
.warn(getRegionInfo().getEncodedName() + " : " +
5745 "Received a flush event with unknown action, ignoring. " +
5746 TextFormat
.shortDebugString(flush
));
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
));
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
) {
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
);
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());
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
;
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");
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");
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.
5864 writestate
.notifyAll();
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
) {
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
);
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
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);
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
);
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.
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
);
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
);
6014 ctx
= prepareFlushResult
.storeFlushCtxs
.get(family
);
6015 startTime
= prepareFlushResult
.startTime
;
6019 LOG
.warn(getRegionInfo().getEncodedName() + " : "
6020 + "Unexpected: flush commit marker received from store "
6021 + Bytes
.toString(family
) + " but no associated flush context. Ignoring");
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();
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
)) {
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
);
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);
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();
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();
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();
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
));
6109 totalFreedSize
.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store
, currentSeqId
));
6112 LOG
.info(getRegionInfo().getEncodedName() + " : "
6113 + "Not dropping memstore contents since replayed flush seqId: "
6114 + seqId
+ " is smaller than current seqId:" + currentSeqId
);
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
);
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
);
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
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
);
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.
6177 if (regionEvent
.getEventType() != EventType
.REGION_OPEN
) {
6178 LOG
.warn(getRegionInfo().getEncodedName() + " : "
6179 + "Unknown region event received, ignoring :"
6180 + TextFormat
.shortDebugString(regionEvent
));
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();
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
);
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
);
6220 long storeSeqId
= store
.getMaxSequenceId().orElse(0L);
6221 List
<String
> storeFiles
= storeDescriptor
.getStoreFileList();
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
);
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
);
6241 MemStoreSize mss
= store
.getFlushableSize();
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.
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
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) {
6298 } else if (!Bytes
.equals(family
, fam
)) {
6299 multipleFamilies
= true;
6304 startBulkRegionOperation(multipleFamilies
);
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
);
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
);
6336 List
<String
> storeFiles
= storeDescriptor
.getStoreFileList();
6337 for (String storeFile
: storeFiles
) {
6338 StoreFileInfo storeFileInfo
= null;
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());
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
6368 HStore store
= getStore(entry
.getKey());
6369 if (store
== null) {
6372 if (store
.getSnapshotSize().getDataSize() > 0) {
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.
6382 writestate
.flushing
= false;
6383 this.prepareFlushResult
= null;
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
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());
6435 MemStoreSize mss
= store
.getFlushableSize();
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())
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;
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
)) {
6504 if (!RegionReplicaUtil
.isDefaultReplica(this.getRegionInfo()) &&
6505 Bytes
.equals(encodedRegionName
,
6506 this.fs
.getRegionInfoForFS().getEncodedNameAsBytes())) {
6510 throw new WrongRegionException(exceptionMsg
+ payload
6511 + " targetted for region " + Bytes
.toStringBinary(encodedRegionName
)
6512 + " does not match this region: " + this.getRegionInfo());
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
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) {
6536 LOG
.warn("File " + p
+ " is zero-length, deleting.");
6537 fs
.delete(p
, false);
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
+
6549 return new HMobStore(this, family
, this.conf
, warmup
);
6551 return new HStore(this, family
, this.conf
, warmup
);
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
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);
6569 public List
<HStore
> getStores() {
6570 return new ArrayList
<>(stores
.values());
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) {
6587 for (HStoreFile storeFile
: storeFiles
) {
6588 storeFileNames
.add(storeFile
.getPath().toString());
6594 return storeFileNames
;
6597 //////////////////////////////////////////////////////////////////////////////
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);
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;
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.
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())) {
6660 result
= rowLockContext
.newReadLock();
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
);
6686 // If timeToDeadline is larger than rowLockWaitDuration, we can not drop the request.
6687 throw new IOException(message
);
6690 rowLockContext
.setThreadName(Thread
.currentThread().getName());
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
);
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
,
6722 private void releaseRowLocks(List
<RowLock
> rowLocks
) {
6723 if (rowLocks
!= null) {
6724 for (RowLock rowLock
: rowLocks
) {
6731 public int getReadLockCount() {
6732 return lock
.getReadLockCount();
6735 public ConcurrentHashMap
<HashedBytes
, RowLockContext
> getLockedRows() {
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
) {
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
) {
6764 return new RowLockImpl(this, l
);
6772 long c
= count
.decrementAndGet();
6774 synchronized (lock
) {
6775 if (count
.get() <= 0 && usable
.get()){ // Don't attempt to remove row if already removed
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
;
6789 public String
toString() {
6790 return "RowLockContext{" +
6792 ", readWriteLock=" + readWriteLock
+
6793 ", count=" + count
+
6794 ", threadName=" + threadName
+
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
;
6811 public Lock
getLock() {
6815 public RowLockContext
getContext() {
6820 public void release() {
6826 public String
toString() {
6827 return "RowLockImpl{" +
6828 "context=" + context
+
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) {
6847 } else if (!Bytes
.equals(family
, fam
)) {
6848 multipleFamilies
= true;
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<byte[] column family, String hfilePath>
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,
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
)
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<byte[] column family, String hfilePath>
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
{
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;
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
));
6946 store
.assertBulkLoadHFileOk(new Path(path
));
6947 } catch (WrongRegionException wre
) {
6948 // recoverable (file doesn't fit in region)
6950 } catch (IOException ioe
) {
6951 // unrecoverable (hdfs problem)
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,
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);
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).
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
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
);
7008 String finalPath
= path
;
7009 if (bulkLoadListener
!= null) {
7010 finalPath
= bulkLoadListener
.prepareBulkLoad(familyName
, path
, copyFile
);
7012 Pair
<Path
, Path
> pair
= store
.preBulkLoadHFile(finalPath
, seqId
);
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) {
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
);
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
);
7044 store
.bulkLoadHFile(familyName
, path
, commitedStoreFile
);
7045 // Note the size of the store file
7047 FileSystem fs
= commitedStoreFile
.getFileSystem(baseConf
);
7048 storeFilesSizes
.put(commitedStoreFile
.getName(), fs
.getFileStatus(commitedStoreFile
)
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
);
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) {
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
);
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
);
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
);
7103 if (wal
!= null && !storeFiles
.isEmpty()) {
7104 // Write a bulk load event for hfiles that are loaded
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;
7128 public boolean equals(Object o
) {
7129 return o
instanceof HRegion
&& Bytes
.equals(getRegionInfo().getRegionName(),
7130 ((HRegion
) o
).getRegionInfo().getRegionName());
7134 public int hashCode() {
7135 return Bytes
.hashCode(getRegionInfo().getRegionName());
7139 public String
toString() {
7140 return getRegionInfo().getRegionNameAsString();
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
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
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
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
) {
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());
7213 HRegion
.newHRegion(tableDir
, wal
, fs
, conf
, info
, hTableDescriptor
, rsRpcServices
);
7215 region
.initialize(null);
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
,
7227 HRegionFileSystem
.createRegionOnFileSystem(conf
, fs
, tableDir
, regionInfo
);
7228 HRegion region
= HRegion
.newHRegion(tableDir
, null, fs
, conf
, regionInfo
, tableDesc
, null);
7233 * Create the region directory in the filesystem.
7235 public static HRegionFileSystem
createRegionDir(Configuration configuration
, RegionInfo ri
,
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
,
7249 throws IOException
{
7250 return createHRegion(info
, rootDir
, conf
, hTableDescriptor
, wal
, true);
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);
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
);
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);
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();
7339 fs
= rootDir
.getFileSystem(conf
);
7341 return openHRegion(conf
, fs
, rootDir
, info
, htd
, wal
, rsServices
, reporter
);
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);
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
);
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
);
7437 * Calls initialize and sets sequenceId.
7438 * @return Returns <code>this</code>
7440 private HRegion
openHRegion(final CancelableProgressable reporter
) throws IOException
{
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
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.
7468 } catch (Throwable e
) {
7469 LOG
.warn("Open region: {} failed. Try close region but got exception ", this.getRegionInfo(),
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();
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.
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));
7559 public Result
get(final Get get
) throws IOException
{
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
);
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();
7597 if (withCoprocessor
&& (coprocessorHost
!= null)) {
7598 if (coprocessorHost
.preGet(get
, results
)) {
7599 metricsUpdateForGet(results
, before
);
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
<>();
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
);
7620 if (withCoprocessor
&& (coprocessorHost
!= null)) {
7621 coprocessorHost
.postGet(get
, results
);
7624 metricsUpdateForGet(results
, before
);
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);
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()) {
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
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
) {
7687 public MiniBatchOperationInProgress
<Mutation
> lockRowsAndBuildMiniBatch(
7688 List
<RowLock
> acquiredRowLocks
) throws IOException
{
7689 RowLock prevRowLock
= null;
7690 for (byte[] row
: rowsToLock
) {
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,
7703 return createMiniBatch(size(), size());
7709 * @return statistics about the current load of the region
7711 public ClientProtos
.RegionLoadStats
getLoadStatistics() {
7712 if (!regionStatsEnabled
) {
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();
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(() -> {
7742 startRegionOperation(Operation
.APPEND
);
7744 // All edits for the given row (across all column families) must happen atomically.
7745 return mutate(append
, true, nonceGroup
, nonce
).getResult();
7747 closeRegionOperation(Operation
.APPEND
);
7749 }, () -> createRegionSpan("Region.append"));
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(() -> {
7761 startRegionOperation(Operation
.INCREMENT
);
7763 // All edits for the given row (across all column families) must happen atomically.
7764 return mutate(increment
, true, nonceGroup
, nonce
).getResult();
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);
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
);
7805 // Call sync on our edit.
7807 sync(txid
, durability
);
7810 } catch (IOException ioe
) {
7811 if (walKey
.getWriteEntry() != null) {
7812 mvcc
.complete(walKey
.getWriteEntry());
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
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.
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);
7873 coprocessorServiceHandlers
.put(serviceName
, instance
);
7874 if (LOG
.isDebugEnabled()) {
7875 LOG
.debug("Registered coprocessor service: region=" +
7876 Bytes
.toStringBinary(getRegionInfo().getRegionName()) + " service=" + serviceName
);
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());
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
>() {
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) {
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
) {
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
);
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
);
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
;
8004 public void startRegionOperation() throws IOException
{
8005 startRegionOperation(Operation
.ANY
);
8009 public void startRegionOperation(Operation op
) throws IOException
{
8010 boolean isInterruptableOp
= false;
8012 case GET
: // interruptible read operations
8014 isInterruptableOp
= true;
8015 checkReadsEnabled();
8017 case INCREMENT
: // interruptible write operations
8022 case CHECK_AND_MUTATE
:
8023 isInterruptableOp
= true;
8025 default: // all others
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
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
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
);
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
);
8069 public void closeRegionOperation() throws IOException
{
8070 closeRegionOperation(Operation
.ANY
);
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
{
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
{
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
);
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
);
8188 switch(durability
) {
8190 // do what table defaults to
8191 if (shouldSyncWAL()) {
8192 this.wal
.sync(txid
);
8202 this.wal
.sync(txid
, false);
8205 this.wal
.sync(txid
, true);
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
;
8226 public Map
<byte[], Long
> getMaxStoreSeqId() {
8227 return this.maxSeqIdInStores
;
8230 public long getOldestSeqIdOfStore(byte[] familyName
) {
8231 return wal
.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName
);
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();
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
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")
8334 return (InterruptedIOException
) new InterruptedIOException().initCause(t
);
8341 public void onConfigurationChange(Configuration conf
) {
8342 this.storeHotnessProtector
.update(conf
);
8349 public void registerChildren(ConfigurationManager manager
) {
8350 configurationManager
= manager
;
8351 stores
.values().forEach(manager
::registerObserver
);
8358 public void deregisterChildren(ConfigurationManager manager
) {
8359 stores
.values().forEach(configurationManager
::deregisterObserver
);
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());
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());
8394 public void requestCompaction(String why
, int priority
, boolean major
,
8395 CompactionLifeCycleTracker tracker
) throws IOException
{
8397 stores
.values().forEach(HStore
::triggerMajorCompaction
);
8399 rsServices
.getCompactionRequestor().requestCompaction(this, why
, priority
, tracker
,
8400 RpcServer
.getRequestUser().orElse(null));
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());
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())) {
8424 private void requestFlush() {
8425 if (this.rsServices
== null) {
8428 requestFlush0(FlushLifeCycleTracker
.DUMMY
);
8431 private void requestFlush0(FlushLifeCycleTracker tracker
) {
8432 boolean shouldFlush
= false;
8433 synchronized (writestate
) {
8434 if (!this.writestate
.isFlushRequested()) {
8436 writestate
.flushRequested
= true;
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());
8446 tracker
.notExecuted("Flush already requested on " + this);
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
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
);