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