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