View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.EOFException;
22  import java.io.FileNotFoundException;
23  import java.io.IOException;
24  import java.io.InterruptedIOException;
25  import java.lang.reflect.Constructor;
26  import java.text.ParseException;
27  import java.util.AbstractList;
28  import java.util.ArrayList;
29  import java.util.Arrays;
30  import java.util.Collection;
31  import java.util.Collections;
32  import java.util.Comparator;
33  import java.util.HashMap;
34  import java.util.HashSet;
35  import java.util.Iterator;
36  import java.util.List;
37  import java.util.Map;
38  import java.util.Map.Entry;
39  import java.util.NavigableMap;
40  import java.util.NavigableSet;
41  import java.util.RandomAccess;
42  import java.util.Set;
43  import java.util.TreeMap;
44  import java.util.concurrent.Callable;
45  import java.util.concurrent.CompletionService;
46  import java.util.concurrent.ConcurrentHashMap;
47  import java.util.concurrent.ConcurrentMap;
48  import java.util.concurrent.ConcurrentSkipListMap;
49  import java.util.concurrent.ExecutionException;
50  import java.util.concurrent.ExecutorCompletionService;
51  import java.util.concurrent.ExecutorService;
52  import java.util.concurrent.Executors;
53  import java.util.concurrent.Future;
54  import java.util.concurrent.FutureTask;
55  import java.util.concurrent.ThreadFactory;
56  import java.util.concurrent.ThreadPoolExecutor;
57  import java.util.concurrent.TimeUnit;
58  import java.util.concurrent.TimeoutException;
59  import java.util.concurrent.atomic.AtomicBoolean;
60  import java.util.concurrent.atomic.AtomicInteger;
61  import java.util.concurrent.atomic.AtomicLong;
62  import java.util.concurrent.locks.Lock;
63  import java.util.concurrent.locks.ReadWriteLock;
64  import java.util.concurrent.locks.ReentrantReadWriteLock;
65  
66  import org.apache.commons.lang.RandomStringUtils;
67  import org.apache.commons.logging.Log;
68  import org.apache.commons.logging.LogFactory;
69  import org.apache.hadoop.conf.Configuration;
70  import org.apache.hadoop.fs.FileStatus;
71  import org.apache.hadoop.fs.FileSystem;
72  import org.apache.hadoop.fs.Path;
73  import org.apache.hadoop.hbase.Cell;
74  import org.apache.hadoop.hbase.CellScanner;
75  import org.apache.hadoop.hbase.CellUtil;
76  import org.apache.hadoop.hbase.CompoundConfiguration;
77  import org.apache.hadoop.hbase.DoNotRetryIOException;
78  import org.apache.hadoop.hbase.DroppedSnapshotException;
79  import org.apache.hadoop.hbase.HBaseConfiguration;
80  import org.apache.hadoop.hbase.HColumnDescriptor;
81  import org.apache.hadoop.hbase.HConstants;
82  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
83  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
84  import org.apache.hadoop.hbase.HRegionInfo;
85  import org.apache.hadoop.hbase.HTableDescriptor;
86  import org.apache.hadoop.hbase.KeyValue;
87  import org.apache.hadoop.hbase.KeyValueUtil;
88  import org.apache.hadoop.hbase.NamespaceDescriptor;
89  import org.apache.hadoop.hbase.NotServingRegionException;
90  import org.apache.hadoop.hbase.RegionTooBusyException;
91  import org.apache.hadoop.hbase.TableName;
92  import org.apache.hadoop.hbase.Tag;
93  import org.apache.hadoop.hbase.TagType;
94  import org.apache.hadoop.hbase.UnknownScannerException;
95  import org.apache.hadoop.hbase.backup.HFileArchiver;
96  import org.apache.hadoop.hbase.classification.InterfaceAudience;
97  import org.apache.hadoop.hbase.client.Append;
98  import org.apache.hadoop.hbase.client.Delete;
99  import org.apache.hadoop.hbase.client.Durability;
100 import org.apache.hadoop.hbase.client.Get;
101 import org.apache.hadoop.hbase.client.Increment;
102 import org.apache.hadoop.hbase.client.IsolationLevel;
103 import org.apache.hadoop.hbase.client.Mutation;
104 import org.apache.hadoop.hbase.client.Put;
105 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
106 import org.apache.hadoop.hbase.client.Result;
107 import org.apache.hadoop.hbase.client.RowMutations;
108 import org.apache.hadoop.hbase.client.Scan;
109 import org.apache.hadoop.hbase.conf.ConfigurationManager;
110 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
111 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
112 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
113 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
114 import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
115 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
116 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
117 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
118 import org.apache.hadoop.hbase.filter.FilterWrapper;
119 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
120 import org.apache.hadoop.hbase.io.HeapSize;
121 import org.apache.hadoop.hbase.io.TimeRange;
122 import org.apache.hadoop.hbase.io.hfile.BlockCache;
123 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
124 import org.apache.hadoop.hbase.io.hfile.HFile;
125 import org.apache.hadoop.hbase.ipc.CallerDisconnectedException;
126 import org.apache.hadoop.hbase.ipc.RpcCallContext;
127 import org.apache.hadoop.hbase.ipc.RpcServer;
128 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
129 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
130 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
131 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
132 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
133 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
134 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
135 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
136 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
137 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
138 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
139 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
140 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
141 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
142 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
143 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
144 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
145 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
146 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
147 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
148 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
149 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
150 import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
151 import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory;
152 import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
153 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
154 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
155 import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey;
156 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
157 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
158 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
159 import org.apache.hadoop.hbase.security.User;
160 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
161 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
162 import org.apache.hadoop.hbase.util.ByteStringer;
163 import org.apache.hadoop.hbase.util.Bytes;
164 import org.apache.hadoop.hbase.util.CancelableProgressable;
165 import org.apache.hadoop.hbase.util.ClassSize;
166 import org.apache.hadoop.hbase.util.CompressionTest;
167 import org.apache.hadoop.hbase.util.Counter;
168 import org.apache.hadoop.hbase.util.EncryptionTest;
169 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
170 import org.apache.hadoop.hbase.util.FSTableDescriptors;
171 import org.apache.hadoop.hbase.util.FSUtils;
172 import org.apache.hadoop.hbase.util.HashedBytes;
173 import org.apache.hadoop.hbase.util.Pair;
174 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
175 import org.apache.hadoop.hbase.util.Threads;
176 import org.apache.hadoop.hbase.wal.WAL;
177 import org.apache.hadoop.hbase.wal.WALFactory;
178 import org.apache.hadoop.hbase.wal.WALKey;
179 import org.apache.hadoop.hbase.wal.WALSplitter;
180 import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
181 import org.apache.hadoop.io.MultipleIOException;
182 import org.apache.hadoop.util.StringUtils;
183 import org.apache.htrace.Trace;
184 import org.apache.htrace.TraceScope;
185 
186 import com.google.common.annotations.VisibleForTesting;
187 import com.google.common.base.Optional;
188 import com.google.common.base.Preconditions;
189 import com.google.common.collect.Lists;
190 import com.google.common.collect.Maps;
191 import com.google.common.io.Closeables;
192 import com.google.protobuf.ByteString;
193 import com.google.protobuf.Descriptors;
194 import com.google.protobuf.Message;
195 import com.google.protobuf.RpcCallback;
196 import com.google.protobuf.RpcController;
197 import com.google.protobuf.Service;
198 import com.google.protobuf.TextFormat;
199 
200 @InterfaceAudience.Private
201 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
202   private static final Log LOG = LogFactory.getLog(HRegion.class);
203 
204   public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
205     "hbase.hregion.scan.loadColumnFamiliesOnDemand";
206 
207   /**
208    * Longest time we'll wait on a sequenceid.
209    * Sequenceid comes up out of the WAL subsystem. WAL subsystem can go bad or a test might use
210    * it without cleaning up previous usage properly; generally, a WAL roll is needed. The timeout
211    * is for a latch in WALKey. There is no global accounting of outstanding WALKeys; intentionally
212    * to avoid contention, but it makes it so if an abort or problem, we could be stuck waiting
213    * on the WALKey latch. Revisit.
214    */
215   private final int maxWaitForSeqId;
216   private static final String MAX_WAIT_FOR_SEQ_ID_KEY = "hbase.hregion.max.wait.for.sequenceid.ms";
217   private static final int DEFAULT_MAX_WAIT_FOR_SEQ_ID = 30000;
218 
219   /**
220    * This is the global default value for durability. All tables/mutations not
221    * defining a durability or using USE_DEFAULT will default to this value.
222    */
223   private static final Durability DEFAULT_DURABILITY = Durability.SYNC_WAL;
224 
225   final AtomicBoolean closed = new AtomicBoolean(false);
226 
227   /* Closing can take some time; use the closing flag if there is stuff we don't
228    * want to do while in closing state; e.g. like offer this region up to the
229    * master as a region to close if the carrying regionserver is overloaded.
230    * Once set, it is never cleared.
231    */
232   final AtomicBoolean closing = new AtomicBoolean(false);
233 
234   /**
235    * The max sequence id of flushed data on this region. There is no edit in memory that is
236    * less that this sequence id.
237    */
238   private volatile long maxFlushedSeqId = HConstants.NO_SEQNUM;
239 
240   /**
241    * Record the sequence id of last flush operation. Can be in advance of
242    * {@link #maxFlushedSeqId} when flushing a single column family. In this case,
243    * {@link #maxFlushedSeqId} will be older than the oldest edit in memory.
244    */
245   private volatile long lastFlushOpSeqId = HConstants.NO_SEQNUM;
246 
247   /**
248    * The sequence id of the last replayed open region event from the primary region. This is used
249    * to skip entries before this due to the possibility of replay edits coming out of order from
250    * replication.
251    */
252   protected volatile long lastReplayedOpenRegionSeqId = -1L;
253   protected volatile long lastReplayedCompactionSeqId = -1L;
254   
255   // collects Map(s) of Store to sequence Id when handleFileNotFound() is involved
256   protected List<Map> storeSeqIds = new ArrayList<>();
257 
258   //////////////////////////////////////////////////////////////////////////////
259   // Members
260   //////////////////////////////////////////////////////////////////////////////
261 
262   // map from a locked row to the context for that lock including:
263   // - CountDownLatch for threads waiting on that row
264   // - the thread that owns the lock (allow reentrancy)
265   // - reference count of (reentrant) locks held by the thread
266   // - the row itself
267   private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows =
268       new ConcurrentHashMap<HashedBytes, RowLockContext>();
269 
270   protected final Map<byte[], Store> stores = new ConcurrentSkipListMap<byte[], Store>(
271       Bytes.BYTES_RAWCOMPARATOR);
272 
273   // TODO: account for each registered handler in HeapSize computation
274   private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
275 
276   private final AtomicLong memstoreSize = new AtomicLong(0);
277 
278   // Debug possible data loss due to WAL off
279   final Counter numMutationsWithoutWAL = new Counter();
280   final Counter dataInMemoryWithoutWAL = new Counter();
281 
282   // Debug why CAS operations are taking a while.
283   final Counter checkAndMutateChecksPassed = new Counter();
284   final Counter checkAndMutateChecksFailed = new Counter();
285 
286   //Number of requests
287   final Counter readRequestsCount = new Counter();
288   final Counter writeRequestsCount = new Counter();
289 
290   // Number of requests blocked by memstore size.
291   private final Counter blockedRequestsCount = new Counter();
292 
293   // Compaction counters
294   final AtomicLong compactionsFinished = new AtomicLong(0L);
295   final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L);
296   final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L);
297 
298   private final WAL wal;
299   private final HRegionFileSystem fs;
300   protected final Configuration conf;
301   private final Configuration baseConf;
302   private final KeyValue.KVComparator comparator;
303   private final int rowLockWaitDuration;
304   static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
305 
306   // The internal wait duration to acquire a lock before read/update
307   // from the region. It is not per row. The purpose of this wait time
308   // is to avoid waiting a long time while the region is busy, so that
309   // we can release the IPC handler soon enough to improve the
310   // availability of the region server. It can be adjusted by
311   // tuning configuration "hbase.busy.wait.duration".
312   final long busyWaitDuration;
313   static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
314 
315   // If updating multiple rows in one call, wait longer,
316   // i.e. waiting for busyWaitDuration * # of rows. However,
317   // we can limit the max multiplier.
318   final int maxBusyWaitMultiplier;
319 
320   // Max busy wait duration. There is no point to wait longer than the RPC
321   // purge timeout, when a RPC call will be terminated by the RPC engine.
322   final long maxBusyWaitDuration;
323 
324   // negative number indicates infinite timeout
325   static final long DEFAULT_ROW_PROCESSOR_TIMEOUT = 60 * 1000L;
326   final ExecutorService rowProcessorExecutor = Executors.newCachedThreadPool();
327 
328   private final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
329 
330   /**
331    * The sequence ID that was encountered when this region was opened.
332    */
333   private long openSeqNum = HConstants.NO_SEQNUM;
334 
335   /**
336    * The default setting for whether to enable on-demand CF loading for
337    * scan requests to this region. Requests can override it.
338    */
339   private boolean isLoadingCfsOnDemandDefault = false;
340 
341   private final AtomicInteger majorInProgress = new AtomicInteger(0);
342   private final AtomicInteger minorInProgress = new AtomicInteger(0);
343 
344   //
345   // Context: During replay we want to ensure that we do not lose any data. So, we
346   // have to be conservative in how we replay wals. For each store, we calculate
347   // the maxSeqId up to which the store was flushed. And, skip the edits which
348   // are equal to or lower than maxSeqId for each store.
349   // The following map is populated when opening the region
350   Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
351 
352   /** Saved state from replaying prepare flush cache */
353   private PrepareFlushResult prepareFlushResult = null;
354 
355   /**
356    * Config setting for whether to allow writes when a region is in recovering or not.
357    */
358   private boolean disallowWritesInRecovering = false;
359 
360   // when a region is in recovering state, it can only accept writes not reads
361   private volatile boolean recovering = false;
362 
363   private volatile Optional<ConfigurationManager> configurationManager;
364 
365   /**
366    * @return The smallest mvcc readPoint across all the scanners in this
367    * region. Writes older than this readPoint, are included in every
368    * read operation.
369    */
370   public long getSmallestReadPoint() {
371     long minimumReadPoint;
372     // We need to ensure that while we are calculating the smallestReadPoint
373     // no new RegionScanners can grab a readPoint that we are unaware of.
374     // We achieve this by synchronizing on the scannerReadPoints object.
375     synchronized(scannerReadPoints) {
376       minimumReadPoint = mvcc.getReadPoint();
377 
378       for (Long readPoint: this.scannerReadPoints.values()) {
379         if (readPoint < minimumReadPoint) {
380           minimumReadPoint = readPoint;
381         }
382       }
383     }
384     return minimumReadPoint;
385   }
386 
387   /*
388    * Data structure of write state flags used coordinating flushes,
389    * compactions and closes.
390    */
391   static class WriteState {
392     // Set while a memstore flush is happening.
393     volatile boolean flushing = false;
394     // Set when a flush has been requested.
395     volatile boolean flushRequested = false;
396     // Number of compactions running.
397     AtomicInteger compacting = new AtomicInteger(0);
398     // Gets set in close. If set, cannot compact or flush again.
399     volatile boolean writesEnabled = true;
400     // Set if region is read-only
401     volatile boolean readOnly = false;
402     // whether the reads are enabled. This is different than readOnly, because readOnly is
403     // static in the lifetime of the region, while readsEnabled is dynamic
404     volatile boolean readsEnabled = true;
405 
406     /**
407      * Set flags that make this region read-only.
408      *
409      * @param onOff flip value for region r/o setting
410      */
411     synchronized void setReadOnly(final boolean onOff) {
412       this.writesEnabled = !onOff;
413       this.readOnly = onOff;
414     }
415 
416     boolean isReadOnly() {
417       return this.readOnly;
418     }
419 
420     boolean isFlushRequested() {
421       return this.flushRequested;
422     }
423 
424     void setReadsEnabled(boolean readsEnabled) {
425       this.readsEnabled = readsEnabled;
426     }
427 
428     static final long HEAP_SIZE = ClassSize.align(
429         ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
430   }
431 
432   /**
433    * Objects from this class are created when flushing to describe all the different states that
434    * that method ends up in. The Result enum describes those states. The sequence id should only
435    * be specified if the flush was successful, and the failure message should only be specified
436    * if it didn't flush.
437    */
438   public static class FlushResultImpl implements FlushResult {
439     final Result result;
440     final String failureReason;
441     final long flushSequenceId;
442     final boolean wroteFlushWalMarker;
443 
444     /**
445      * Convenience constructor to use when the flush is successful, the failure message is set to
446      * null.
447      * @param result Expecting FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_COMPACTION_NEEDED.
448      * @param flushSequenceId Generated sequence id that comes right after the edits in the
449      *                        memstores.
450      */
451     FlushResultImpl(Result result, long flushSequenceId) {
452       this(result, flushSequenceId, null, false);
453       assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
454           .FLUSHED_COMPACTION_NEEDED;
455     }
456 
457     /**
458      * Convenience constructor to use when we cannot flush.
459      * @param result Expecting CANNOT_FLUSH_MEMSTORE_EMPTY or CANNOT_FLUSH.
460      * @param failureReason Reason why we couldn't flush.
461      */
462     FlushResultImpl(Result result, String failureReason, boolean wroteFlushMarker) {
463       this(result, -1, failureReason, wroteFlushMarker);
464       assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH;
465     }
466 
467     /**
468      * Constructor with all the parameters.
469      * @param result Any of the Result.
470      * @param flushSequenceId Generated sequence id if the memstores were flushed else -1.
471      * @param failureReason Reason why we couldn't flush, or null.
472      */
473     FlushResultImpl(Result result, long flushSequenceId, String failureReason,
474       boolean wroteFlushMarker) {
475       this.result = result;
476       this.flushSequenceId = flushSequenceId;
477       this.failureReason = failureReason;
478       this.wroteFlushWalMarker = wroteFlushMarker;
479     }
480 
481     /**
482      * Convenience method, the equivalent of checking if result is
483      * FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_NO_COMPACTION_NEEDED.
484      * @return true if the memstores were flushed, else false.
485      */
486     @Override
487     public boolean isFlushSucceeded() {
488       return result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
489           .FLUSHED_COMPACTION_NEEDED;
490     }
491 
492     /**
493      * Convenience method, the equivalent of checking if result is FLUSHED_COMPACTION_NEEDED.
494      * @return True if the flush requested a compaction, else false (doesn't even mean it flushed).
495      */
496     @Override
497     public boolean isCompactionNeeded() {
498       return result == Result.FLUSHED_COMPACTION_NEEDED;
499     }
500 
501     @Override
502     public String toString() {
503       return new StringBuilder()
504         .append("flush result:").append(result).append(", ")
505         .append("failureReason:").append(failureReason).append(",")
506         .append("flush seq id").append(flushSequenceId).toString();
507     }
508 
509     @Override
510     public Result getResult() {
511       return result;
512     }
513   }
514 
515   /** A result object from prepare flush cache stage */
516   @VisibleForTesting
517   static class PrepareFlushResult {
518     final FlushResult result; // indicating a failure result from prepare
519     final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
520     final TreeMap<byte[], List<Path>> committedFiles;
521     final TreeMap<byte[], Long> storeFlushableSize;
522     final long startTime;
523     final long flushOpSeqId;
524     final long flushedSeqId;
525     final long totalFlushableSize;
526 
527     /** Constructs an early exit case */
528     PrepareFlushResult(FlushResult result, long flushSeqId) {
529       this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, 0);
530     }
531 
532     /** Constructs a successful prepare flush result */
533     PrepareFlushResult(
534       TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
535       TreeMap<byte[], List<Path>> committedFiles,
536       TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId,
537       long flushedSeqId, long totalFlushableSize) {
538       this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
539         flushSeqId, flushedSeqId, totalFlushableSize);
540     }
541 
542     private PrepareFlushResult(
543       FlushResult result,
544       TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
545       TreeMap<byte[], List<Path>> committedFiles,
546       TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId,
547       long flushedSeqId, long totalFlushableSize) {
548       this.result = result;
549       this.storeFlushCtxs = storeFlushCtxs;
550       this.committedFiles = committedFiles;
551       this.storeFlushableSize = storeFlushableSize;
552       this.startTime = startTime;
553       this.flushOpSeqId = flushSeqId;
554       this.flushedSeqId = flushedSeqId;
555       this.totalFlushableSize = totalFlushableSize;
556     }
557 
558     public FlushResult getResult() {
559       return this.result;
560     }
561   }
562 
563   final WriteState writestate = new WriteState();
564 
565   long memstoreFlushSize;
566   final long timestampSlop;
567   final long rowProcessorTimeout;
568 
569   // Last flush time for each Store. Useful when we are flushing for each column
570   private final ConcurrentMap<Store, Long> lastStoreFlushTimeMap =
571       new ConcurrentHashMap<Store, Long>();
572 
573   final RegionServerServices rsServices;
574   private RegionServerAccounting rsAccounting;
575   private long flushCheckInterval;
576   // flushPerChanges is to prevent too many changes in memstore
577   private long flushPerChanges;
578   private long blockingMemStoreSize;
579   final long threadWakeFrequency;
580   // Used to guard closes
581   final ReentrantReadWriteLock lock =
582     new ReentrantReadWriteLock();
583 
584   // Stop updates lock
585   private final ReentrantReadWriteLock updatesLock =
586     new ReentrantReadWriteLock();
587   private boolean splitRequest;
588   private byte[] explicitSplitPoint = null;
589 
590   private final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
591 
592   // Coprocessor host
593   private RegionCoprocessorHost coprocessorHost;
594 
595   private HTableDescriptor htableDescriptor = null;
596   private RegionSplitPolicy splitPolicy;
597   private FlushPolicy flushPolicy;
598 
599   private final MetricsRegion metricsRegion;
600   private final MetricsRegionWrapperImpl metricsRegionWrapper;
601   private final Durability durability;
602   private final boolean regionStatsEnabled;
603 
604   /**
605    * HRegion constructor. This constructor should only be used for testing and
606    * extensions.  Instances of HRegion should be instantiated with the
607    * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
608    *
609    * @param tableDir qualified path of directory where region should be located,
610    * usually the table directory.
611    * @param wal The WAL is the outbound log for any updates to the HRegion
612    * The wal file is a logfile from the previous execution that's
613    * custom-computed for this HRegion. The HRegionServer computes and sorts the
614    * appropriate wal info for this HRegion. If there is a previous wal file
615    * (implying that the HRegion has been written-to before), then read it from
616    * the supplied path.
617    * @param fs is the filesystem.
618    * @param confParam is global configuration settings.
619    * @param regionInfo - HRegionInfo that describes the region
620    * is new), then read them from the supplied path.
621    * @param htd the table descriptor
622    * @param rsServices reference to {@link RegionServerServices} or null
623    * @deprecated Use other constructors.
624    */
625   @Deprecated
626   @VisibleForTesting
627   public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
628       final Configuration confParam, final HRegionInfo regionInfo,
629       final HTableDescriptor htd, final RegionServerServices rsServices) {
630     this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
631       wal, confParam, htd, rsServices);
632   }
633 
634   /**
635    * HRegion constructor. This constructor should only be used for testing and
636    * extensions.  Instances of HRegion should be instantiated with the
637    * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
638    *
639    * @param fs is the filesystem.
640    * @param wal The WAL is the outbound log for any updates to the HRegion
641    * The wal file is a logfile from the previous execution that's
642    * custom-computed for this HRegion. The HRegionServer computes and sorts the
643    * appropriate wal info for this HRegion. If there is a previous wal file
644    * (implying that the HRegion has been written-to before), then read it from
645    * the supplied path.
646    * @param confParam is global configuration settings.
647    * @param htd the table descriptor
648    * @param rsServices reference to {@link RegionServerServices} or null
649    */
650   public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam,
651       final HTableDescriptor htd, final RegionServerServices rsServices) {
652     if (htd == null) {
653       throw new IllegalArgumentException("Need table descriptor");
654     }
655 
656     if (confParam instanceof CompoundConfiguration) {
657       throw new IllegalArgumentException("Need original base configuration");
658     }
659 
660     this.comparator = fs.getRegionInfo().getComparator();
661     this.wal = wal;
662     this.fs = fs;
663 
664     // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
665     this.baseConf = confParam;
666     this.conf = new CompoundConfiguration()
667       .add(confParam)
668       .addStringMap(htd.getConfiguration())
669       .addWritableMap(htd.getValues());
670     this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL,
671         DEFAULT_CACHE_FLUSH_INTERVAL);
672     this.flushPerChanges = conf.getLong(MEMSTORE_FLUSH_PER_CHANGES, DEFAULT_FLUSH_PER_CHANGES);
673     if (this.flushPerChanges > MAX_FLUSH_PER_CHANGES) {
674       throw new IllegalArgumentException(MEMSTORE_FLUSH_PER_CHANGES + " can not exceed "
675           + MAX_FLUSH_PER_CHANGES);
676     }
677     this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
678                     DEFAULT_ROWLOCK_WAIT_DURATION);
679 
680     this.maxWaitForSeqId = conf.getInt(MAX_WAIT_FOR_SEQ_ID_KEY, DEFAULT_MAX_WAIT_FOR_SEQ_ID);
681     this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
682     this.htableDescriptor = htd;
683     this.rsServices = rsServices;
684     this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
685     setHTableSpecificConf();
686     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
687 
688     this.busyWaitDuration = conf.getLong(
689       "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
690     this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
691     if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
692       throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
693         + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
694         + maxBusyWaitMultiplier + "). Their product should be positive");
695     }
696     this.maxBusyWaitDuration = conf.getLong("hbase.ipc.client.call.purge.timeout",
697       2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
698 
699     /*
700      * timestamp.slop provides a server-side constraint on the timestamp. This
701      * assumes that you base your TS around currentTimeMillis(). In this case,
702      * throw an error to the user if the user-specified TS is newer than now +
703      * slop. LATEST_TIMESTAMP == don't use this functionality
704      */
705     this.timestampSlop = conf.getLong(
706         "hbase.hregion.keyvalue.timestamp.slop.millisecs",
707         HConstants.LATEST_TIMESTAMP);
708 
709     /**
710      * Timeout for the process time in processRowsWithLocks().
711      * Use -1 to switch off time bound.
712      */
713     this.rowProcessorTimeout = conf.getLong(
714         "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
715     this.durability = htd.getDurability() == Durability.USE_DEFAULT
716         ? DEFAULT_DURABILITY
717         : htd.getDurability();
718     if (rsServices != null) {
719       this.rsAccounting = this.rsServices.getRegionServerAccounting();
720       // don't initialize coprocessors if not running within a regionserver
721       // TODO: revisit if coprocessors should load in other cases
722       this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
723       this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
724       this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper);
725 
726       Map<String, Region> recoveringRegions = rsServices.getRecoveringRegions();
727       String encodedName = getRegionInfo().getEncodedName();
728       if (recoveringRegions != null && recoveringRegions.containsKey(encodedName)) {
729         this.recovering = true;
730         recoveringRegions.put(encodedName, this);
731       }
732     } else {
733       this.metricsRegionWrapper = null;
734       this.metricsRegion = null;
735     }
736     if (LOG.isDebugEnabled()) {
737       // Write out region name as string and its encoded name.
738       LOG.debug("Instantiated " + this);
739     }
740 
741     // by default, we allow writes against a region when it's in recovering
742     this.disallowWritesInRecovering =
743         conf.getBoolean(HConstants.DISALLOW_WRITES_IN_RECOVERING,
744           HConstants.DEFAULT_DISALLOW_WRITES_IN_RECOVERING_CONFIG);
745     configurationManager = Optional.absent();
746 
747     // disable stats tracking system tables, but check the config for everything else
748     this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals(
749         NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) ?
750           false :
751           conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
752               HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
753   }
754 
755   void setHTableSpecificConf() {
756     if (this.htableDescriptor == null) return;
757     long flushSize = this.htableDescriptor.getMemStoreFlushSize();
758 
759     if (flushSize <= 0) {
760       flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
761         HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
762     }
763     this.memstoreFlushSize = flushSize;
764     this.blockingMemStoreSize = this.memstoreFlushSize *
765         conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,
766                 HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER);
767   }
768 
769   /**
770    * Initialize this region.
771    * Used only by tests and SplitTransaction to reopen the region.
772    * You should use createHRegion() or openHRegion()
773    * @return What the next sequence (edit) id should be.
774    * @throws IOException e
775    * @deprecated use HRegion.createHRegion() or HRegion.openHRegion()
776    */
777   @Deprecated
778   public long initialize() throws IOException {
779     return initialize(null);
780   }
781 
782   /**
783    * Initialize this region.
784    *
785    * @param reporter Tickle every so often if initialize is taking a while.
786    * @return What the next sequence (edit) id should be.
787    * @throws IOException e
788    */
789   private long initialize(final CancelableProgressable reporter) throws IOException {
790     MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
791     long nextSeqId = -1;
792     try {
793       nextSeqId = initializeRegionInternals(reporter, status);
794       return nextSeqId;
795     } finally {
796       // nextSeqid will be -1 if the initialization fails.
797       // At least it will be 0 otherwise.
798       if (nextSeqId == -1) {
799         status.abort("Exception during region " + getRegionInfo().getRegionNameAsString() +
800           " initialization.");
801       }
802     }
803   }
804 
805   private long initializeRegionInternals(final CancelableProgressable reporter,
806       final MonitoredTask status) throws IOException {
807     if (coprocessorHost != null) {
808       status.setStatus("Running coprocessor pre-open hook");
809       coprocessorHost.preOpen();
810     }
811 
812     // Write HRI to a file in case we need to recover hbase:meta
813     status.setStatus("Writing region info on filesystem");
814     fs.checkRegionInfoOnFilesystem();
815 
816     // Initialize all the HStores
817     status.setStatus("Initializing all the Stores");
818     long maxSeqId = initializeStores(reporter, status);
819     this.mvcc.advanceTo(maxSeqId);
820     if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
821       // Recover any edits if available.
822       maxSeqId = Math.max(maxSeqId,
823         replayRecoveredEditsIfAny(this.fs.getRegionDir(), maxSeqIdInStores, reporter, status));
824       // Make sure mvcc is up to max.
825       this.mvcc.advanceTo(maxSeqId);
826     }
827     this.lastReplayedOpenRegionSeqId = maxSeqId;
828 
829     this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this));
830     this.writestate.flushRequested = false;
831     this.writestate.compacting.set(0);
832 
833     if (this.writestate.writesEnabled) {
834       // Remove temporary data left over from old regions
835       status.setStatus("Cleaning up temporary data from old regions");
836       fs.cleanupTempDir();
837     }
838 
839     if (this.writestate.writesEnabled) {
840       status.setStatus("Cleaning up detritus from prior splits");
841       // Get rid of any splits or merges that were lost in-progress.  Clean out
842       // these directories here on open.  We may be opening a region that was
843       // being split but we crashed in the middle of it all.
844       fs.cleanupAnySplitDetritus();
845       fs.cleanupMergesDir();
846     }
847 
848     // Initialize split policy
849     this.splitPolicy = RegionSplitPolicy.create(this, conf);
850 
851     // Initialize flush policy
852     this.flushPolicy = FlushPolicyFactory.create(this, conf);
853 
854     long lastFlushTime = EnvironmentEdgeManager.currentTime();
855     for (Store store: stores.values()) {
856       this.lastStoreFlushTimeMap.put(store, lastFlushTime);
857     }
858 
859     // Use maximum of log sequenceid or that which was found in stores
860     // (particularly if no recovered edits, seqid will be -1).
861     long nextSeqid = maxSeqId;
862 
863     // In distributedLogReplay mode, we don't know the last change sequence number because region
864     // is opened before recovery completes. So we add a safety bumper to avoid new sequence number
865     // overlaps used sequence numbers
866     if (this.writestate.writesEnabled) {
867       nextSeqid = WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(), this.fs
868           .getRegionDir(), nextSeqid, (this.recovering ? (this.flushPerChanges + 10000000) : 1));
869     } else {
870       nextSeqid++;
871     }
872 
873     LOG.info("Onlined " + this.getRegionInfo().getShortNameToLog() +
874       "; next sequenceid=" + nextSeqid);
875 
876     // A region can be reopened if failed a split; reset flags
877     this.closing.set(false);
878     this.closed.set(false);
879 
880     if (coprocessorHost != null) {
881       status.setStatus("Running coprocessor post-open hooks");
882       coprocessorHost.postOpen();
883     }
884 
885     status.markComplete("Region opened successfully");
886     return nextSeqid;
887   }
888 
889   /**
890    * Open all Stores.
891    * @param reporter
892    * @param status
893    * @return Highest sequenceId found out in a Store.
894    * @throws IOException
895    */
896   private long initializeStores(final CancelableProgressable reporter, MonitoredTask status)
897   throws IOException {
898     // Load in all the HStores.
899 
900     long maxSeqId = -1;
901     // initialized to -1 so that we pick up MemstoreTS from column families
902     long maxMemstoreTS = -1;
903 
904     if (!htableDescriptor.getFamilies().isEmpty()) {
905       // initialize the thread pool for opening stores in parallel.
906       ThreadPoolExecutor storeOpenerThreadPool =
907         getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog());
908       CompletionService<HStore> completionService =
909         new ExecutorCompletionService<HStore>(storeOpenerThreadPool);
910 
911       // initialize each store in parallel
912       for (final HColumnDescriptor family : htableDescriptor.getFamilies()) {
913         status.setStatus("Instantiating store for column family " + family);
914         completionService.submit(new Callable<HStore>() {
915           @Override
916           public HStore call() throws IOException {
917             return instantiateHStore(family);
918           }
919         });
920       }
921       boolean allStoresOpened = false;
922       try {
923         for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) {
924           Future<HStore> future = completionService.take();
925           HStore store = future.get();
926           this.stores.put(store.getFamily().getName(), store);
927 
928           long storeMaxSequenceId = store.getMaxSequenceId();
929           maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
930               storeMaxSequenceId);
931           if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) {
932             maxSeqId = storeMaxSequenceId;
933           }
934           long maxStoreMemstoreTS = store.getMaxMemstoreTS();
935           if (maxStoreMemstoreTS > maxMemstoreTS) {
936             maxMemstoreTS = maxStoreMemstoreTS;
937           }
938         }
939         allStoresOpened = true;
940       } catch (InterruptedException e) {
941         throw (InterruptedIOException)new InterruptedIOException().initCause(e);
942       } catch (ExecutionException e) {
943         throw new IOException(e.getCause());
944       } finally {
945         storeOpenerThreadPool.shutdownNow();
946         if (!allStoresOpened) {
947           // something went wrong, close all opened stores
948           LOG.error("Could not initialize all stores for the region=" + this);
949           for (Store store : this.stores.values()) {
950             try {
951               store.close();
952             } catch (IOException e) {
953               LOG.warn(e.getMessage());
954             }
955           }
956         }
957       }
958     }
959     return Math.max(maxSeqId, maxMemstoreTS + 1);
960   }
961 
962   private void initializeWarmup(final CancelableProgressable reporter) throws IOException {
963     MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
964     // Initialize all the HStores
965     status.setStatus("Warming up all the Stores");
966     try {
967       initializeStores(reporter, status);
968     } finally {
969       status.markComplete("Done warming up.");
970     }
971   }
972 
973   /**
974    * @return Map of StoreFiles by column family
975    */
976   private NavigableMap<byte[], List<Path>> getStoreFiles() {
977     NavigableMap<byte[], List<Path>> allStoreFiles =
978       new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
979     for (Store store: getStores()) {
980       Collection<StoreFile> storeFiles = store.getStorefiles();
981       if (storeFiles == null) continue;
982       List<Path> storeFileNames = new ArrayList<Path>();
983       for (StoreFile storeFile: storeFiles) {
984         storeFileNames.add(storeFile.getPath());
985       }
986       allStoreFiles.put(store.getFamily().getName(), storeFileNames);
987     }
988     return allStoreFiles;
989   }
990 
991   private void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
992     Map<byte[], List<Path>> storeFiles = getStoreFiles();
993     RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
994       RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId,
995       getRegionServerServices().getServerName(), storeFiles);
996     WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionOpenDesc, mvcc);
997   }
998 
999   private void writeRegionCloseMarker(WAL wal) throws IOException {
1000     Map<byte[], List<Path>> storeFiles = getStoreFiles();
1001     RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor(
1002       RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(),
1003       getRegionServerServices().getServerName(), storeFiles);
1004     WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionEventDesc, mvcc);
1005 
1006     // Store SeqId in HDFS when a region closes
1007     // checking region folder exists is due to many tests which delete the table folder while a
1008     // table is still online
1009     if (this.fs.getFileSystem().exists(this.fs.getRegionDir())) {
1010       WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(), this.fs.getRegionDir(),
1011         mvcc.getReadPoint(), 0);
1012     }
1013   }
1014 
1015   /**
1016    * @return True if this region has references.
1017    */
1018   public boolean hasReferences() {
1019     for (Store store : this.stores.values()) {
1020       if (store.hasReferences()) return true;
1021     }
1022     return false;
1023   }
1024 
1025   @Override
1026   public HDFSBlocksDistribution getHDFSBlocksDistribution() {
1027     HDFSBlocksDistribution hdfsBlocksDistribution =
1028       new HDFSBlocksDistribution();
1029     synchronized (this.stores) {
1030       for (Store store : this.stores.values()) {
1031         Collection<StoreFile> storeFiles = store.getStorefiles();
1032         if (storeFiles == null) continue;
1033         for (StoreFile sf : storeFiles) {
1034           HDFSBlocksDistribution storeFileBlocksDistribution =
1035             sf.getHDFSBlockDistribution();
1036           hdfsBlocksDistribution.add(storeFileBlocksDistribution);
1037         }
1038       }
1039     }
1040     return hdfsBlocksDistribution;
1041   }
1042 
1043   /**
1044    * This is a helper function to compute HDFS block distribution on demand
1045    * @param conf configuration
1046    * @param tableDescriptor HTableDescriptor of the table
1047    * @param regionInfo encoded name of the region
1048    * @return The HDFS blocks distribution for the given region.
1049    * @throws IOException
1050    */
1051   public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
1052       final HTableDescriptor tableDescriptor, final HRegionInfo regionInfo) throws IOException {
1053     Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
1054     return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
1055   }
1056 
1057   /**
1058    * This is a helper function to compute HDFS block distribution on demand
1059    * @param conf configuration
1060    * @param tableDescriptor HTableDescriptor of the table
1061    * @param regionInfo encoded name of the region
1062    * @param tablePath the table directory
1063    * @return The HDFS blocks distribution for the given region.
1064    * @throws IOException
1065    */
1066   public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
1067       final HTableDescriptor tableDescriptor, final HRegionInfo regionInfo,  Path tablePath)
1068       throws IOException {
1069     HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1070     FileSystem fs = tablePath.getFileSystem(conf);
1071 
1072     HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
1073     for (HColumnDescriptor family: tableDescriptor.getFamilies()) {
1074       Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family.getNameAsString());
1075       if (storeFiles == null) continue;
1076       for (StoreFileInfo storeFileInfo : storeFiles) {
1077         try {
1078           hdfsBlocksDistribution.add(storeFileInfo.computeHDFSBlocksDistribution(fs));
1079         } catch (IOException ioe) {
1080           LOG.warn("Error getting hdfs block distribution for " + storeFileInfo);
1081         }
1082       }
1083     }
1084     return hdfsBlocksDistribution;
1085   }
1086 
1087   /**
1088    * Increase the size of mem store in this region and the size of global mem
1089    * store
1090    * @return the size of memstore in this region
1091    */
1092   public long addAndGetGlobalMemstoreSize(long memStoreSize) {
1093     if (this.rsAccounting != null) {
1094       rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
1095     }
1096     long size = this.memstoreSize.addAndGet(memStoreSize);
1097     // This is extremely bad if we make memstoreSize negative. Log as much info on the offending
1098     // caller as possible. (memStoreSize might be a negative value already -- freeing memory)
1099     if (size < 0) {
1100       LOG.error("Asked to modify this region's (" + this.toString()
1101       + ") memstoreSize to a negative value which is incorrect. Current memstoreSize="
1102       + (size-memStoreSize) + ", delta=" + memStoreSize, new Exception());
1103     }
1104     return size;
1105   }
1106 
1107   @Override
1108   public HRegionInfo getRegionInfo() {
1109     return this.fs.getRegionInfo();
1110   }
1111 
1112   /**
1113    * @return Instance of {@link RegionServerServices} used by this HRegion.
1114    * Can be null.
1115    */
1116   RegionServerServices getRegionServerServices() {
1117     return this.rsServices;
1118   }
1119 
1120   @Override
1121   public long getReadRequestsCount() {
1122     return readRequestsCount.get();
1123   }
1124 
1125   @Override
1126   public void updateReadRequestsCount(long i) {
1127     readRequestsCount.add(i);
1128   }
1129 
1130   @Override
1131   public long getWriteRequestsCount() {
1132     return writeRequestsCount.get();
1133   }
1134 
1135   @Override
1136   public void updateWriteRequestsCount(long i) {
1137     writeRequestsCount.add(i);
1138   }
1139 
1140   @Override
1141   public long getMemstoreSize() {
1142     return memstoreSize.get();
1143   }
1144 
1145   @Override
1146   public long getNumMutationsWithoutWAL() {
1147     return numMutationsWithoutWAL.get();
1148   }
1149 
1150   @Override
1151   public long getDataInMemoryWithoutWAL() {
1152     return dataInMemoryWithoutWAL.get();
1153   }
1154 
1155   @Override
1156   public long getBlockedRequestsCount() {
1157     return blockedRequestsCount.get();
1158   }
1159 
1160   @Override
1161   public long getCheckAndMutateChecksPassed() {
1162     return checkAndMutateChecksPassed.get();
1163   }
1164 
1165   @Override
1166   public long getCheckAndMutateChecksFailed() {
1167     return checkAndMutateChecksFailed.get();
1168   }
1169 
1170   @Override
1171   public MetricsRegion getMetrics() {
1172     return metricsRegion;
1173   }
1174 
1175   @Override
1176   public boolean isClosed() {
1177     return this.closed.get();
1178   }
1179 
1180   @Override
1181   public boolean isClosing() {
1182     return this.closing.get();
1183   }
1184 
1185   @Override
1186   public boolean isReadOnly() {
1187     return this.writestate.isReadOnly();
1188   }
1189 
1190   /**
1191    * Reset recovering state of current region
1192    */
1193   public void setRecovering(boolean newState) {
1194     boolean wasRecovering = this.recovering;
1195     // before we flip the recovering switch (enabling reads) we should write the region open
1196     // event to WAL if needed
1197     if (wal != null && getRegionServerServices() != null && !writestate.readOnly
1198         && wasRecovering && !newState) {
1199 
1200       // force a flush only if region replication is set up for this region. Otherwise no need.
1201       boolean forceFlush = getTableDesc().getRegionReplication() > 1;
1202 
1203       MonitoredTask status = TaskMonitor.get().createStatus("Recovering region " + this);
1204 
1205       try {
1206         // force a flush first
1207         if (forceFlush) {
1208           status.setStatus("Flushing region " + this + " because recovery is finished");
1209           internalFlushcache(status);
1210         }
1211 
1212         status.setStatus("Writing region open event marker to WAL because recovery is finished");
1213         try {
1214           long seqId = openSeqNum;
1215           // obtain a new seqId because we possibly have writes and flushes on top of openSeqNum
1216           if (wal != null) {
1217             seqId = getNextSequenceId(wal);
1218           }
1219           writeRegionOpenMarker(wal, seqId);
1220         } catch (IOException e) {
1221           // We cannot rethrow this exception since we are being called from the zk thread. The
1222           // region has already opened. In this case we log the error, but continue
1223           LOG.warn(getRegionInfo().getEncodedName() + " : was not able to write region opening "
1224               + "event to WAL, continuing", e);
1225         }
1226       } catch (IOException ioe) {
1227         // Distributed log replay semantics does not necessarily require a flush, since the replayed
1228         // data is already written again in the WAL. So failed flush should be fine.
1229         LOG.warn(getRegionInfo().getEncodedName() + " : was not able to flush "
1230             + "event to WAL, continuing", ioe);
1231       } finally {
1232         status.cleanup();
1233       }
1234     }
1235 
1236     this.recovering = newState;
1237     if (wasRecovering && !recovering) {
1238       // Call only when wal replay is over.
1239       coprocessorHost.postLogReplay();
1240     }
1241   }
1242 
1243   @Override
1244   public boolean isRecovering() {
1245     return this.recovering;
1246   }
1247 
1248   @Override
1249   public boolean isAvailable() {
1250     return !isClosed() && !isClosing();
1251   }
1252 
1253   /** @return true if region is splittable */
1254   public boolean isSplittable() {
1255     return isAvailable() && !hasReferences();
1256   }
1257 
1258   /**
1259    * @return true if region is mergeable
1260    */
1261   public boolean isMergeable() {
1262     if (!isAvailable()) {
1263       LOG.debug("Region " + getRegionInfo().getRegionNameAsString()
1264           + " is not mergeable because it is closing or closed");
1265       return false;
1266     }
1267     if (hasReferences()) {
1268       LOG.debug("Region " + getRegionInfo().getRegionNameAsString()
1269           + " is not mergeable because it has references");
1270       return false;
1271     }
1272 
1273     return true;
1274   }
1275 
1276   public boolean areWritesEnabled() {
1277     synchronized(this.writestate) {
1278       return this.writestate.writesEnabled;
1279     }
1280   }
1281 
1282    public MultiVersionConcurrencyControl getMVCC() {
1283      return mvcc;
1284    }
1285 
1286    @Override
1287    public long getMaxFlushedSeqId() {
1288      return maxFlushedSeqId;
1289    }
1290 
1291    @Override
1292    public long getReadpoint(IsolationLevel isolationLevel) {
1293      if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
1294        // This scan can read even uncommitted transactions
1295        return Long.MAX_VALUE;
1296      }
1297      return mvcc.getReadPoint();
1298    }
1299 
1300    @Override
1301    public boolean isLoadingCfsOnDemandDefault() {
1302      return this.isLoadingCfsOnDemandDefault;
1303    }
1304 
1305   /**
1306    * Close down this HRegion.  Flush the cache, shut down each HStore, don't
1307    * service any more calls.
1308    *
1309    * <p>This method could take some time to execute, so don't call it from a
1310    * time-sensitive thread.
1311    *
1312    * @return Vector of all the storage files that the HRegion's component
1313    * HStores make use of.  It's a list of all HStoreFile objects. Returns empty
1314    * vector if already closed and null if judged that it should not close.
1315    *
1316    * @throws IOException e
1317    * @throws DroppedSnapshotException Thrown when replay of wal is required
1318    * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1319    * caller MUST abort after this.
1320    */
1321   public Map<byte[], List<StoreFile>> close() throws IOException {
1322     return close(false);
1323   }
1324 
1325   private final Object closeLock = new Object();
1326 
1327   /** Conf key for the periodic flush interval */
1328   public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL =
1329       "hbase.regionserver.optionalcacheflushinterval";
1330   /** Default interval for the memstore flush */
1331   public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
1332   /** Default interval for System tables memstore flush */
1333   public static final int SYSTEM_CACHE_FLUSH_INTERVAL = 300000; // 5 minutes
1334 
1335   /** Conf key to force a flush if there are already enough changes for one region in memstore */
1336   public static final String MEMSTORE_FLUSH_PER_CHANGES =
1337       "hbase.regionserver.flush.per.changes";
1338   public static final long DEFAULT_FLUSH_PER_CHANGES = 30000000; // 30 millions
1339   /**
1340    * The following MAX_FLUSH_PER_CHANGES is large enough because each KeyValue has 20+ bytes
1341    * overhead. Therefore, even 1G empty KVs occupy at least 20GB memstore size for a single region
1342    */
1343   public static final long MAX_FLUSH_PER_CHANGES = 1000000000; // 1G
1344 
1345   /**
1346    * Close down this HRegion.  Flush the cache unless abort parameter is true,
1347    * Shut down each HStore, don't service any more calls.
1348    *
1349    * This method could take some time to execute, so don't call it from a
1350    * time-sensitive thread.
1351    *
1352    * @param abort true if server is aborting (only during testing)
1353    * @return Vector of all the storage files that the HRegion's component
1354    * HStores make use of.  It's a list of HStoreFile objects.  Can be null if
1355    * we are not to close at this time or we are already closed.
1356    *
1357    * @throws IOException e
1358    * @throws DroppedSnapshotException Thrown when replay of wal is required
1359    * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1360    * caller MUST abort after this.
1361    */
1362   public Map<byte[], List<StoreFile>> close(final boolean abort) throws IOException {
1363     // Only allow one thread to close at a time. Serialize them so dual
1364     // threads attempting to close will run up against each other.
1365     MonitoredTask status = TaskMonitor.get().createStatus(
1366         "Closing region " + this +
1367         (abort ? " due to abort" : ""));
1368 
1369     status.setStatus("Waiting for close lock");
1370     try {
1371       synchronized (closeLock) {
1372         return doClose(abort, status);
1373       }
1374     } finally {
1375       status.cleanup();
1376     }
1377   }
1378 
1379   /**
1380    * Exposed for some very specific unit tests.
1381    */
1382   @VisibleForTesting
1383   public void setClosing(boolean closing) {
1384     this.closing.set(closing);
1385   }
1386 
1387   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
1388       justification="I think FindBugs is confused")
1389   private Map<byte[], List<StoreFile>> doClose(final boolean abort, MonitoredTask status)
1390       throws IOException {
1391     if (isClosed()) {
1392       LOG.warn("Region " + this + " already closed");
1393       return null;
1394     }
1395 
1396     if (coprocessorHost != null) {
1397       status.setStatus("Running coprocessor pre-close hooks");
1398       this.coprocessorHost.preClose(abort);
1399     }
1400 
1401     status.setStatus("Disabling compacts and flushes for region");
1402     boolean canFlush = true;
1403     synchronized (writestate) {
1404       // Disable compacting and flushing by background threads for this
1405       // region.
1406       canFlush = !writestate.readOnly;
1407       writestate.writesEnabled = false;
1408       LOG.debug("Closing " + this + ": disabling compactions & flushes");
1409       waitForFlushesAndCompactions();
1410     }
1411     // If we were not just flushing, is it worth doing a preflush...one
1412     // that will clear out of the bulk of the memstore before we put up
1413     // the close flag?
1414     if (!abort && worthPreFlushing() && canFlush) {
1415       status.setStatus("Pre-flushing region before close");
1416       LOG.info("Running close preflush of " + getRegionInfo().getRegionNameAsString());
1417       try {
1418         internalFlushcache(status);
1419       } catch (IOException ioe) {
1420         // Failed to flush the region. Keep going.
1421         status.setStatus("Failed pre-flush " + this + "; " + ioe.getMessage());
1422       }
1423     }
1424 
1425     // block waiting for the lock for closing
1426     lock.writeLock().lock(); // FindBugs: Complains UL_UNRELEASED_LOCK_EXCEPTION_PATH but seems fine
1427     this.closing.set(true);
1428     status.setStatus("Disabling writes for close");
1429     try {
1430       if (this.isClosed()) {
1431         status.abort("Already got closed by another process");
1432         // SplitTransaction handles the null
1433         return null;
1434       }
1435       LOG.debug("Updates disabled for region " + this);
1436       // Don't flush the cache if we are aborting
1437       if (!abort && canFlush) {
1438         int flushCount = 0;
1439         while (this.memstoreSize.get() > 0) {
1440           try {
1441             if (flushCount++ > 0) {
1442               int actualFlushes = flushCount - 1;
1443               if (actualFlushes > 5) {
1444                 // If we tried 5 times and are unable to clear memory, abort
1445                 // so we do not lose data
1446                 throw new DroppedSnapshotException("Failed clearing memory after " +
1447                   actualFlushes + " attempts on region: " +
1448                     Bytes.toStringBinary(getRegionInfo().getRegionName()));
1449               }
1450               LOG.info("Running extra flush, " + actualFlushes +
1451                 " (carrying snapshot?) " + this);
1452             }
1453             internalFlushcache(status);
1454           } catch (IOException ioe) {
1455             status.setStatus("Failed flush " + this + ", putting online again");
1456             synchronized (writestate) {
1457               writestate.writesEnabled = true;
1458             }
1459             // Have to throw to upper layers.  I can't abort server from here.
1460             throw ioe;
1461           }
1462         }
1463       }
1464 
1465       Map<byte[], List<StoreFile>> result =
1466         new TreeMap<byte[], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
1467       if (!stores.isEmpty()) {
1468         // initialize the thread pool for closing stores in parallel.
1469         ThreadPoolExecutor storeCloserThreadPool =
1470           getStoreOpenAndCloseThreadPool("StoreCloserThread-" +
1471             getRegionInfo().getRegionNameAsString());
1472         CompletionService<Pair<byte[], Collection<StoreFile>>> completionService =
1473           new ExecutorCompletionService<Pair<byte[], Collection<StoreFile>>>(storeCloserThreadPool);
1474 
1475         // close each store in parallel
1476         for (final Store store : stores.values()) {
1477           long flushableSize = store.getFlushableSize();
1478           if (!(abort || flushableSize == 0 || writestate.readOnly)) {
1479             if (getRegionServerServices() != null) {
1480               getRegionServerServices().abort("Assertion failed while closing store "
1481                 + getRegionInfo().getRegionNameAsString() + " " + store
1482                 + ". flushableSize expected=0, actual= " + flushableSize
1483                 + ". Current memstoreSize=" + getMemstoreSize() + ". Maybe a coprocessor "
1484                 + "operation failed and left the memstore in a partially updated state.", null);
1485             }
1486           }
1487           completionService
1488               .submit(new Callable<Pair<byte[], Collection<StoreFile>>>() {
1489                 @Override
1490                 public Pair<byte[], Collection<StoreFile>> call() throws IOException {
1491                   return new Pair<byte[], Collection<StoreFile>>(
1492                     store.getFamily().getName(), store.close());
1493                 }
1494               });
1495         }
1496         try {
1497           for (int i = 0; i < stores.size(); i++) {
1498             Future<Pair<byte[], Collection<StoreFile>>> future = completionService.take();
1499             Pair<byte[], Collection<StoreFile>> storeFiles = future.get();
1500             List<StoreFile> familyFiles = result.get(storeFiles.getFirst());
1501             if (familyFiles == null) {
1502               familyFiles = new ArrayList<StoreFile>();
1503               result.put(storeFiles.getFirst(), familyFiles);
1504             }
1505             familyFiles.addAll(storeFiles.getSecond());
1506           }
1507         } catch (InterruptedException e) {
1508           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
1509         } catch (ExecutionException e) {
1510           throw new IOException(e.getCause());
1511         } finally {
1512           storeCloserThreadPool.shutdownNow();
1513         }
1514       }
1515 
1516       status.setStatus("Writing region close event to WAL");
1517       if (!abort && wal != null && getRegionServerServices() != null && !writestate.readOnly) {
1518         writeRegionCloseMarker(wal);
1519       }
1520 
1521       this.closed.set(true);
1522       if (!canFlush) {
1523         addAndGetGlobalMemstoreSize(-memstoreSize.get());
1524       } else if (memstoreSize.get() != 0) {
1525         LOG.error("Memstore size is " + memstoreSize.get());
1526       }
1527       if (coprocessorHost != null) {
1528         status.setStatus("Running coprocessor post-close hooks");
1529         this.coprocessorHost.postClose(abort);
1530       }
1531       if (this.metricsRegion != null) {
1532         this.metricsRegion.close();
1533       }
1534       if (this.metricsRegionWrapper != null) {
1535         Closeables.closeQuietly(this.metricsRegionWrapper);
1536       }
1537       status.markComplete("Closed");
1538       LOG.info("Closed " + this);
1539       return result;
1540     } finally {
1541       lock.writeLock().unlock();
1542     }
1543   }
1544 
1545   @Override
1546   public void waitForFlushesAndCompactions() {
1547     synchronized (writestate) {
1548       if (this.writestate.readOnly) {
1549         // we should not wait for replayed flushed if we are read only (for example in case the
1550         // region is a secondary replica).
1551         return;
1552       }
1553       boolean interrupted = false;
1554       try {
1555         while (writestate.compacting.get() > 0 || writestate.flushing) {
1556           LOG.debug("waiting for " + writestate.compacting + " compactions"
1557             + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
1558           try {
1559             writestate.wait();
1560           } catch (InterruptedException iex) {
1561             // essentially ignore and propagate the interrupt back up
1562             LOG.warn("Interrupted while waiting");
1563             interrupted = true;
1564           }
1565         }
1566       } finally {
1567         if (interrupted) {
1568           Thread.currentThread().interrupt();
1569         }
1570       }
1571     }
1572   }
1573 
1574   protected ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
1575       final String threadNamePrefix) {
1576     int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1577     int maxThreads = Math.min(numStores,
1578         conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1579             HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
1580     return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1581   }
1582 
1583   protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
1584       final String threadNamePrefix) {
1585     int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1586     int maxThreads = Math.max(1,
1587         conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1588             HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
1589             / numStores);
1590     return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1591   }
1592 
1593   static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
1594       final String threadNamePrefix) {
1595     return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
1596       new ThreadFactory() {
1597         private int count = 1;
1598 
1599         @Override
1600         public Thread newThread(Runnable r) {
1601           return new Thread(r, threadNamePrefix + "-" + count++);
1602         }
1603       });
1604   }
1605 
1606    /**
1607     * @return True if its worth doing a flush before we put up the close flag.
1608     */
1609   private boolean worthPreFlushing() {
1610     return this.memstoreSize.get() >
1611       this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
1612   }
1613 
1614   //////////////////////////////////////////////////////////////////////////////
1615   // HRegion accessors
1616   //////////////////////////////////////////////////////////////////////////////
1617 
1618   @Override
1619   public HTableDescriptor getTableDesc() {
1620     return this.htableDescriptor;
1621   }
1622 
1623   /** @return WAL in use for this region */
1624   public WAL getWAL() {
1625     return this.wal;
1626   }
1627 
1628   /**
1629    * A split takes the config from the parent region & passes it to the daughter
1630    * region's constructor. If 'conf' was passed, you would end up using the HTD
1631    * of the parent region in addition to the new daughter HTD. Pass 'baseConf'
1632    * to the daughter regions to avoid this tricky dedupe problem.
1633    * @return Configuration object
1634    */
1635   Configuration getBaseConf() {
1636     return this.baseConf;
1637   }
1638 
1639   /** @return {@link FileSystem} being used by this region */
1640   public FileSystem getFilesystem() {
1641     return fs.getFileSystem();
1642   }
1643 
1644   /** @return the {@link HRegionFileSystem} used by this region */
1645   public HRegionFileSystem getRegionFileSystem() {
1646     return this.fs;
1647   }
1648 
1649   @Override
1650   public long getEarliestFlushTimeForAllStores() {
1651     return lastStoreFlushTimeMap.isEmpty() ? Long.MAX_VALUE : Collections.min(lastStoreFlushTimeMap
1652         .values());
1653   }
1654 
1655   @Override
1656   public long getOldestHfileTs(boolean majorCompactioOnly) throws IOException {
1657     long result = Long.MAX_VALUE;
1658     for (Store store : getStores()) {
1659       Collection<StoreFile> storeFiles = store.getStorefiles();
1660       if (storeFiles == null) continue;
1661       for (StoreFile file : storeFiles) {
1662         StoreFile.Reader sfReader = file.getReader();
1663         if (sfReader == null) continue;
1664         HFile.Reader reader = sfReader.getHFileReader();
1665         if (reader == null) continue;
1666         if (majorCompactioOnly) {
1667           byte[] val = reader.loadFileInfo().get(StoreFile.MAJOR_COMPACTION_KEY);
1668           if (val == null) continue;
1669           if (val == null || !Bytes.toBoolean(val)) {
1670             continue;
1671           }
1672         }
1673         result = Math.min(result, reader.getFileContext().getFileCreateTime());
1674       }
1675     }
1676     return result == Long.MAX_VALUE ? 0 : result;
1677   }
1678 
1679   RegionLoad.Builder setCompleteSequenceId(RegionLoad.Builder regionLoadBldr) {
1680     long lastFlushOpSeqIdLocal = this.lastFlushOpSeqId;
1681     byte[] encodedRegionName = this.getRegionInfo().getEncodedNameAsBytes();
1682     regionLoadBldr.clearStoreCompleteSequenceId();
1683     for (byte[] familyName : this.stores.keySet()) {
1684       long earliest = this.wal.getEarliestMemstoreSeqNum(encodedRegionName, familyName);
1685       // Subtract - 1 to go earlier than the current oldest, unflushed edit in memstore; this will
1686       // give us a sequence id that is for sure flushed. We want edit replay to start after this
1687       // sequence id in this region. If NO_SEQNUM, use the regions maximum flush id.
1688       long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1;
1689       regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId.
1690         newBuilder().setFamilyName(ByteString.copyFrom(familyName)).setSequenceId(csid).build());
1691     }
1692     return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId());
1693   }
1694 
1695   //////////////////////////////////////////////////////////////////////////////
1696   // HRegion maintenance.
1697   //
1698   // These methods are meant to be called periodically by the HRegionServer for
1699   // upkeep.
1700   //////////////////////////////////////////////////////////////////////////////
1701 
1702   /** @return returns size of largest HStore. */
1703   public long getLargestHStoreSize() {
1704     long size = 0;
1705     for (Store h : stores.values()) {
1706       long storeSize = h.getSize();
1707       if (storeSize > size) {
1708         size = storeSize;
1709       }
1710     }
1711     return size;
1712   }
1713 
1714   /**
1715    * @return KeyValue Comparator
1716    */
1717   public KeyValue.KVComparator getComparator() {
1718     return this.comparator;
1719   }
1720 
1721   /*
1722    * Do preparation for pending compaction.
1723    * @throws IOException
1724    */
1725   protected void doRegionCompactionPrep() throws IOException {
1726   }
1727 
1728   @Override
1729   public void triggerMajorCompaction() throws IOException {
1730     for (Store s : getStores()) {
1731       s.triggerMajorCompaction();
1732     }
1733   }
1734 
1735   @Override
1736   public void compact(final boolean majorCompaction) throws IOException {
1737     if (majorCompaction) {
1738       triggerMajorCompaction();
1739     }
1740     for (Store s : getStores()) {
1741       CompactionContext compaction = s.requestCompaction();
1742       if (compaction != null) {
1743         CompactionThroughputController controller = null;
1744         if (rsServices != null) {
1745           controller = CompactionThroughputControllerFactory.create(rsServices, conf);
1746         }
1747         if (controller == null) {
1748           controller = NoLimitCompactionThroughputController.INSTANCE;
1749         }
1750         compact(compaction, s, controller, null);
1751       }
1752     }
1753   }
1754 
1755   /**
1756    * This is a helper function that compact all the stores synchronously
1757    * It is used by utilities and testing
1758    *
1759    * @throws IOException e
1760    */
1761   public void compactStores() throws IOException {
1762     for (Store s : getStores()) {
1763       CompactionContext compaction = s.requestCompaction();
1764       if (compaction != null) {
1765         compact(compaction, s, NoLimitCompactionThroughputController.INSTANCE, null);
1766       }
1767     }
1768   }
1769 
1770   /**
1771    * This is a helper function that compact the given store
1772    * It is used by utilities and testing
1773    *
1774    * @throws IOException e
1775    */
1776   @VisibleForTesting
1777   void compactStore(byte[] family, CompactionThroughputController throughputController)
1778       throws IOException {
1779     Store s = getStore(family);
1780     CompactionContext compaction = s.requestCompaction();
1781     if (compaction != null) {
1782       compact(compaction, s, throughputController, null);
1783     }
1784   }
1785 
1786   /*
1787    * Called by compaction thread and after region is opened to compact the
1788    * HStores if necessary.
1789    *
1790    * <p>This operation could block for a long time, so don't call it from a
1791    * time-sensitive thread.
1792    *
1793    * Note that no locking is necessary at this level because compaction only
1794    * conflicts with a region split, and that cannot happen because the region
1795    * server does them sequentially and not in parallel.
1796    *
1797    * @param compaction Compaction details, obtained by requestCompaction()
1798    * @param throughputController
1799    * @return whether the compaction completed
1800    */
1801   public boolean compact(CompactionContext compaction, Store store,
1802       CompactionThroughputController throughputController) throws IOException {
1803     return compact(compaction, store, throughputController, null);
1804   }
1805 
1806   public boolean compact(CompactionContext compaction, Store store,
1807       CompactionThroughputController throughputController, User user) throws IOException {
1808     assert compaction != null && compaction.hasSelection();
1809     assert !compaction.getRequest().getFiles().isEmpty();
1810     if (this.closing.get() || this.closed.get()) {
1811       LOG.debug("Skipping compaction on " + this + " because closing/closed");
1812       store.cancelRequestedCompaction(compaction);
1813       return false;
1814     }
1815     MonitoredTask status = null;
1816     boolean requestNeedsCancellation = true;
1817     // block waiting for the lock for compaction
1818     lock.readLock().lock();
1819     try {
1820       byte[] cf = Bytes.toBytes(store.getColumnFamilyName());
1821       if (stores.get(cf) != store) {
1822         LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this
1823             + " has been re-instantiated, cancel this compaction request. "
1824             + " It may be caused by the roll back of split transaction");
1825         return false;
1826       }
1827 
1828       status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this);
1829       if (this.closed.get()) {
1830         String msg = "Skipping compaction on " + this + " because closed";
1831         LOG.debug(msg);
1832         status.abort(msg);
1833         return false;
1834       }
1835       boolean wasStateSet = false;
1836       try {
1837         synchronized (writestate) {
1838           if (writestate.writesEnabled) {
1839             wasStateSet = true;
1840             writestate.compacting.incrementAndGet();
1841           } else {
1842             String msg = "NOT compacting region " + this + ". Writes disabled.";
1843             LOG.info(msg);
1844             status.abort(msg);
1845             return false;
1846           }
1847         }
1848         LOG.info("Starting compaction on " + store + " in region " + this
1849             + (compaction.getRequest().isOffPeak()?" as an off-peak compaction":""));
1850         doRegionCompactionPrep();
1851         try {
1852           status.setStatus("Compacting store " + store);
1853           // We no longer need to cancel the request on the way out of this
1854           // method because Store#compact will clean up unconditionally
1855           requestNeedsCancellation = false;
1856           store.compact(compaction, throughputController, user);
1857         } catch (InterruptedIOException iioe) {
1858           String msg = "compaction interrupted";
1859           LOG.info(msg, iioe);
1860           status.abort(msg);
1861           return false;
1862         }
1863       } finally {
1864         if (wasStateSet) {
1865           synchronized (writestate) {
1866             writestate.compacting.decrementAndGet();
1867             if (writestate.compacting.get() <= 0) {
1868               writestate.notifyAll();
1869             }
1870           }
1871         }
1872       }
1873       status.markComplete("Compaction complete");
1874       return true;
1875     } finally {
1876       try {
1877         if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction);
1878         if (status != null) status.cleanup();
1879       } finally {
1880         lock.readLock().unlock();
1881       }
1882     }
1883   }
1884 
1885   @Override
1886   public FlushResult flush(boolean force) throws IOException {
1887     return flushcache(force, false);
1888   }
1889 
1890   /**
1891    * Flush the cache.
1892    *
1893    * When this method is called the cache will be flushed unless:
1894    * <ol>
1895    *   <li>the cache is empty</li>
1896    *   <li>the region is closed.</li>
1897    *   <li>a flush is already in progress</li>
1898    *   <li>writes are disabled</li>
1899    * </ol>
1900    *
1901    * <p>This method may block for some time, so it should not be called from a
1902    * time-sensitive thread.
1903    * @param forceFlushAllStores whether we want to flush all stores
1904    * @param writeFlushRequestWalMarker whether to write the flush request marker to WAL
1905    * @return whether the flush is success and whether the region needs compacting
1906    *
1907    * @throws IOException general io exceptions
1908    * @throws DroppedSnapshotException Thrown when replay of wal is required
1909    * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1910    * caller MUST abort after this.
1911    */
1912   public FlushResult flushcache(boolean forceFlushAllStores, boolean writeFlushRequestWalMarker)
1913       throws IOException {
1914     // fail-fast instead of waiting on the lock
1915     if (this.closing.get()) {
1916       String msg = "Skipping flush on " + this + " because closing";
1917       LOG.debug(msg);
1918       return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
1919     }
1920     MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
1921     status.setStatus("Acquiring readlock on region");
1922     // block waiting for the lock for flushing cache
1923     lock.readLock().lock();
1924     try {
1925       if (this.closed.get()) {
1926         String msg = "Skipping flush on " + this + " because closed";
1927         LOG.debug(msg);
1928         status.abort(msg);
1929         return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
1930       }
1931       if (coprocessorHost != null) {
1932         status.setStatus("Running coprocessor pre-flush hooks");
1933         coprocessorHost.preFlush();
1934       }
1935       // TODO: this should be managed within memstore with the snapshot, updated only after flush
1936       // successful
1937       if (numMutationsWithoutWAL.get() > 0) {
1938         numMutationsWithoutWAL.set(0);
1939         dataInMemoryWithoutWAL.set(0);
1940       }
1941       synchronized (writestate) {
1942         if (!writestate.flushing && writestate.writesEnabled) {
1943           this.writestate.flushing = true;
1944         } else {
1945           if (LOG.isDebugEnabled()) {
1946             LOG.debug("NOT flushing memstore for region " + this
1947                 + ", flushing=" + writestate.flushing + ", writesEnabled="
1948                 + writestate.writesEnabled);
1949           }
1950           String msg = "Not flushing since "
1951               + (writestate.flushing ? "already flushing"
1952               : "writes not enabled");
1953           status.abort(msg);
1954           return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
1955         }
1956       }
1957 
1958       try {
1959         Collection<Store> specificStoresToFlush =
1960             forceFlushAllStores ? stores.values() : flushPolicy.selectStoresToFlush();
1961         FlushResult fs = internalFlushcache(specificStoresToFlush,
1962           status, writeFlushRequestWalMarker);
1963 
1964         if (coprocessorHost != null) {
1965           status.setStatus("Running post-flush coprocessor hooks");
1966           coprocessorHost.postFlush();
1967         }
1968 
1969         status.markComplete("Flush successful");
1970         return fs;
1971       } finally {
1972         synchronized (writestate) {
1973           writestate.flushing = false;
1974           this.writestate.flushRequested = false;
1975           writestate.notifyAll();
1976         }
1977       }
1978     } finally {
1979       lock.readLock().unlock();
1980       status.cleanup();
1981     }
1982   }
1983 
1984   /**
1985    * Should the store be flushed because it is old enough.
1986    * <p>
1987    * Every FlushPolicy should call this to determine whether a store is old enough to flush(except
1988    * that you always flush all stores). Otherwise the {@link #shouldFlush()} method will always
1989    * returns true which will make a lot of flush requests.
1990    */
1991   boolean shouldFlushStore(Store store) {
1992     long earliest = this.wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
1993       store.getFamily().getName()) - 1;
1994     if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
1995       if (LOG.isDebugEnabled()) {
1996         LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " +
1997           getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest +
1998           " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint());
1999       }
2000       return true;
2001     }
2002     if (this.flushCheckInterval <= 0) {
2003       return false;
2004     }
2005     long now = EnvironmentEdgeManager.currentTime();
2006     if (store.timeOfOldestEdit() < now - this.flushCheckInterval) {
2007       if (LOG.isDebugEnabled()) {
2008         LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of " +
2009           getRegionInfo().getEncodedName() + " because time of oldest edit=" +
2010             store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now);
2011       }
2012       return true;
2013     }
2014     return false;
2015   }
2016 
2017   /**
2018    * Should the memstore be flushed now
2019    */
2020   boolean shouldFlush(final StringBuffer whyFlush) {
2021     whyFlush.setLength(0);
2022     // This is a rough measure.
2023     if (this.maxFlushedSeqId > 0
2024           && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())) {
2025       whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush");
2026       return true;
2027     }
2028     long modifiedFlushCheckInterval = flushCheckInterval;
2029     if (getRegionInfo().isSystemTable() &&
2030         getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
2031       modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
2032     }
2033     if (modifiedFlushCheckInterval <= 0) { //disabled
2034       return false;
2035     }
2036     long now = EnvironmentEdgeManager.currentTime();
2037     //if we flushed in the recent past, we don't need to do again now
2038     if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) {
2039       return false;
2040     }
2041     //since we didn't flush in the recent past, flush now if certain conditions
2042     //are met. Return true on first such memstore hit.
2043     for (Store s : getStores()) {
2044       if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
2045         // we have an old enough edit in the memstore, flush
2046         whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
2047         return true;
2048       }
2049     }
2050     return false;
2051   }
2052 
2053   /**
2054    * Flushing all stores.
2055    *
2056    * @see #internalFlushcache(Collection, MonitoredTask, boolean)
2057    */
2058   private FlushResult internalFlushcache(MonitoredTask status)
2059       throws IOException {
2060     return internalFlushcache(stores.values(), status, false);
2061   }
2062 
2063   /**
2064    * Flushing given stores.
2065    *
2066    * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean)
2067    */
2068   private FlushResult internalFlushcache(final Collection<Store> storesToFlush,
2069       MonitoredTask status, boolean writeFlushWalMarker) throws IOException {
2070     return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush,
2071         status, writeFlushWalMarker);
2072   }
2073 
2074   /**
2075    * Flush the memstore. Flushing the memstore is a little tricky. We have a lot
2076    * of updates in the memstore, all of which have also been written to the wal.
2077    * We need to write those updates in the memstore out to disk, while being
2078    * able to process reads/writes as much as possible during the flush
2079    * operation.
2080    * <p>
2081    * This method may block for some time. Every time you call it, we up the
2082    * regions sequence id even if we don't flush; i.e. the returned region id
2083    * will be at least one larger than the last edit applied to this region. The
2084    * returned id does not refer to an actual edit. The returned id can be used
2085    * for say installing a bulk loaded file just ahead of the last hfile that was
2086    * the result of this flush, etc.
2087    *
2088    * @param wal
2089    *          Null if we're NOT to go via wal.
2090    * @param myseqid
2091    *          The seqid to use if <code>wal</code> is null writing out flush
2092    *          file.
2093    * @param storesToFlush
2094    *          The list of stores to flush.
2095    * @return object describing the flush's state
2096    * @throws IOException
2097    *           general io exceptions
2098    * @throws DroppedSnapshotException
2099    *           Thrown when replay of wal is required because a Snapshot was not
2100    *           properly persisted.
2101    */
2102   protected FlushResult internalFlushcache(final WAL wal, final long myseqid,
2103       final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
2104           throws IOException {
2105     PrepareFlushResult result
2106       = internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker);
2107     if (result.result == null) {
2108       return internalFlushCacheAndCommit(wal, status, result, storesToFlush);
2109     } else {
2110       return result.result; // early exit due to failure from prepare stage
2111     }
2112   }
2113 
2114   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
2115       justification="FindBugs seems confused about trxId")
2116   protected PrepareFlushResult internalPrepareFlushCache(final WAL wal, final long myseqid,
2117       final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
2118   throws IOException {
2119     if (this.rsServices != null && this.rsServices.isAborted()) {
2120       // Don't flush when server aborting, it's unsafe
2121       throw new IOException("Aborting flush because server is aborted...");
2122     }
2123     final long startTime = EnvironmentEdgeManager.currentTime();
2124     // If nothing to flush, return, but we need to safely update the region sequence id
2125     if (this.memstoreSize.get() <= 0) {
2126       // Take an update lock because am about to change the sequence id and we want the sequence id
2127       // to be at the border of the empty memstore.
2128       MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
2129       this.updatesLock.writeLock().lock();
2130       try {
2131         if (this.memstoreSize.get() <= 0) {
2132           // Presume that if there are still no edits in the memstore, then there are no edits for
2133           // this region out in the WAL subsystem so no need to do any trickery clearing out
2134           // edits in the WAL system. Up the sequence number so the resulting flush id is for
2135           // sure just beyond the last appended region edit (useful as a marker when bulk loading,
2136           // etc.). NOTE: The writeEntry write number is NOT in the WAL.. there is no WAL writing
2137           // here.
2138           if (wal != null) {
2139             writeEntry = mvcc.begin();
2140             long flushOpSeqId = writeEntry.getWriteNumber();
2141             FlushResult flushResult = new FlushResultImpl(
2142                 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
2143                 flushOpSeqId,
2144                 "Nothing to flush",
2145                 writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
2146             // TODO: Lets see if we hang here, if there is a scenario where an outstanding reader
2147             // with a read point is in advance of this write point.
2148             mvcc.completeAndWait(writeEntry);
2149             writeEntry = null;
2150             return new PrepareFlushResult(flushResult, myseqid);
2151           } else {
2152             return new PrepareFlushResult(
2153               new FlushResultImpl(
2154                   FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
2155                   "Nothing to flush",
2156                   false),
2157               myseqid);
2158           }
2159         }
2160       } finally {
2161         this.updatesLock.writeLock().unlock();
2162         if (writeEntry != null) {
2163           mvcc.complete(writeEntry);
2164         }
2165       }
2166     }
2167 
2168     if (LOG.isInfoEnabled()) {
2169       // Log a fat line detailing what is being flushed.
2170       StringBuilder perCfExtras = null;
2171       if (!isAllFamilies(storesToFlush)) {
2172         perCfExtras = new StringBuilder();
2173         for (Store store: storesToFlush) {
2174           perCfExtras.append("; ").append(store.getColumnFamilyName());
2175           perCfExtras.append("=").append(StringUtils.byteDesc(store.getMemStoreSize()));
2176         }
2177       }
2178       LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() +
2179         " column families, memstore=" + StringUtils.byteDesc(this.memstoreSize.get()) +
2180         ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
2181         ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + myseqid));
2182     }
2183     // Stop updates while we snapshot the memstore of all of these regions' stores. We only have
2184     // to do this for a moment.  It is quick. We also set the memstore size to zero here before we
2185     // allow updates again so its value will represent the size of the updates received
2186     // during flush
2187 
2188     // We have to take an update lock during snapshot, or else a write could end up in both snapshot
2189     // and memstore (makes it difficult to do atomic rows then)
2190     status.setStatus("Obtaining lock to block concurrent updates");
2191     // block waiting for the lock for internal flush
2192     this.updatesLock.writeLock().lock();
2193     status.setStatus("Preparing to flush by snapshotting stores in " +
2194       getRegionInfo().getEncodedName());
2195     long totalFlushableSizeOfFlushableStores = 0;
2196 
2197     Set<byte[]> flushedFamilyNames = new HashSet<byte[]>();
2198     for (Store store: storesToFlush) {
2199       flushedFamilyNames.add(store.getFamily().getName());
2200     }
2201 
2202     TreeMap<byte[], StoreFlushContext> storeFlushCtxs
2203       = new TreeMap<byte[], StoreFlushContext>(Bytes.BYTES_COMPARATOR);
2204     TreeMap<byte[], List<Path>> committedFiles = new TreeMap<byte[], List<Path>>(
2205         Bytes.BYTES_COMPARATOR);
2206     TreeMap<byte[], Long> storeFlushableSize
2207         = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
2208     // The sequence id of this flush operation which is used to log FlushMarker and pass to
2209     // createFlushContext to use as the store file's sequence id. It can be in advance of edits
2210     // still in the memstore, edits that are in other column families yet to be flushed.
2211     long flushOpSeqId = HConstants.NO_SEQNUM;
2212     // The max flushed sequence id after this flush operation completes. All edits in memstore
2213     // will be in advance of this sequence id.
2214     long flushedSeqId = HConstants.NO_SEQNUM;
2215     byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes();
2216 
2217     long trxId = 0;
2218     MultiVersionConcurrencyControl.WriteEntry writeEntry = mvcc.begin();
2219     // wait for all in-progress transactions to commit to WAL before
2220     // we can start the flush. This prevents
2221     // uncommitted transactions from being written into HFiles.
2222     // We have to block before we start the flush, otherwise keys that
2223     // were removed via a rollbackMemstore could be written to Hfiles.
2224     mvcc.completeAndWait(writeEntry);
2225     // set writeEntry to null to prevent mvcc.complete from being called again inside finally
2226     // block
2227     writeEntry = null;
2228     try {
2229       try {
2230         if (wal != null) {
2231           Long earliestUnflushedSequenceIdForTheRegion =
2232             wal.startCacheFlush(encodedRegionName, flushedFamilyNames);
2233           if (earliestUnflushedSequenceIdForTheRegion == null) {
2234             // This should never happen. This is how startCacheFlush signals flush cannot proceed.
2235             String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
2236             status.setStatus(msg);
2237             return new PrepareFlushResult(
2238               new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false),
2239               myseqid);
2240           }
2241           flushOpSeqId = getNextSequenceId(wal);
2242           // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
2243           flushedSeqId =
2244             earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM?
2245               flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
2246         } else {
2247           // use the provided sequence Id as WAL is not being used for this flush.
2248           flushedSeqId = flushOpSeqId = myseqid;
2249         }
2250 
2251         for (Store s : storesToFlush) {
2252           totalFlushableSizeOfFlushableStores += s.getFlushableSize();
2253           storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId));
2254           committedFiles.put(s.getFamily().getName(), null); // for writing stores to WAL
2255           storeFlushableSize.put(s.getFamily().getName(), s.getFlushableSize());
2256         }
2257 
2258         // write the snapshot start to WAL
2259         if (wal != null && !writestate.readOnly) {
2260           FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
2261             getRegionInfo(), flushOpSeqId, committedFiles);
2262           // no sync. Sync is below where we do not hold the updates lock
2263           trxId = WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2264             desc, false, mvcc);
2265         }
2266 
2267         // Prepare flush (take a snapshot)
2268         for (StoreFlushContext flush : storeFlushCtxs.values()) {
2269           flush.prepare();
2270         }
2271       } catch (IOException ex) {
2272         if (wal != null) {
2273           if (trxId > 0) { // check whether we have already written START_FLUSH to WAL
2274             try {
2275               FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2276                 getRegionInfo(), flushOpSeqId, committedFiles);
2277               WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2278                 desc, false, mvcc);
2279             } catch (Throwable t) {
2280               LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL:" +
2281                   StringUtils.stringifyException(t));
2282               // ignore this since we will be aborting the RS with DSE.
2283             }
2284           }
2285           // we have called wal.startCacheFlush(), now we have to abort it
2286           wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2287           throw ex; // let upper layers deal with it.
2288         }
2289       } finally {
2290         this.updatesLock.writeLock().unlock();
2291       }
2292       String s = "Finished memstore snapshotting " + this +
2293         ", syncing WAL and waiting on mvcc, flushsize=" + totalFlushableSizeOfFlushableStores;
2294       status.setStatus(s);
2295       if (LOG.isTraceEnabled()) LOG.trace(s);
2296       // sync unflushed WAL changes
2297       // see HBASE-8208 for details
2298       if (wal != null) {
2299         try {
2300           wal.sync(); // ensure that flush marker is sync'ed
2301         } catch (IOException ioe) {
2302           wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2303           throw ioe;
2304         }
2305       }
2306     } finally {
2307       if (writeEntry != null) {
2308         // In case of failure just mark current writeEntry as complete.
2309         mvcc.complete(writeEntry);
2310       }
2311     }
2312     return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
2313         flushOpSeqId, flushedSeqId, totalFlushableSizeOfFlushableStores);
2314   }
2315 
2316   /**
2317    * @param families
2318    * @return True if passed Set is all families in the region.
2319    */
2320   private boolean isAllFamilies(final Collection<Store> families) {
2321     return families == null || this.stores.size() == families.size();
2322   }
2323 
2324   /**
2325    * Writes a marker to WAL indicating a flush is requested but cannot be complete due to various
2326    * reasons. Ignores exceptions from WAL. Returns whether the write succeeded.
2327    * @param wal
2328    * @return whether WAL write was successful
2329    */
2330   private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker) {
2331     if (writeFlushWalMarker && wal != null && !writestate.readOnly) {
2332       FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH,
2333         getRegionInfo(), -1, new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR));
2334       try {
2335         WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2336           desc, true, mvcc);
2337         return true;
2338       } catch (IOException e) {
2339         LOG.warn(getRegionInfo().getEncodedName() + " : "
2340             + "Received exception while trying to write the flush request to wal", e);
2341       }
2342     }
2343     return false;
2344   }
2345 
2346   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
2347       justification="Intentional; notify is about completed flush")
2348   protected FlushResult internalFlushCacheAndCommit(
2349         final WAL wal, MonitoredTask status, final PrepareFlushResult prepareResult,
2350         final Collection<Store> storesToFlush)
2351     throws IOException {
2352 
2353     // prepare flush context is carried via PrepareFlushResult
2354     TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs;
2355     TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles;
2356     long startTime = prepareResult.startTime;
2357     long flushOpSeqId = prepareResult.flushOpSeqId;
2358     long flushedSeqId = prepareResult.flushedSeqId;
2359     long totalFlushableSizeOfFlushableStores = prepareResult.totalFlushableSize;
2360 
2361     String s = "Flushing stores of " + this;
2362     status.setStatus(s);
2363     if (LOG.isTraceEnabled()) LOG.trace(s);
2364 
2365     // Any failure from here on out will be catastrophic requiring server
2366     // restart so wal content can be replayed and put back into the memstore.
2367     // Otherwise, the snapshot content while backed up in the wal, it will not
2368     // be part of the current running servers state.
2369     boolean compactionRequested = false;
2370     try {
2371       // A.  Flush memstore to all the HStores.
2372       // Keep running vector of all store files that includes both old and the
2373       // just-made new flush store file. The new flushed file is still in the
2374       // tmp directory.
2375 
2376       for (StoreFlushContext flush : storeFlushCtxs.values()) {
2377         flush.flushCache(status);
2378       }
2379 
2380       // Switch snapshot (in memstore) -> new hfile (thus causing
2381       // all the store scanners to reset/reseek).
2382       Iterator<Store> it = storesToFlush.iterator();
2383       // stores.values() and storeFlushCtxs have same order
2384       for (StoreFlushContext flush : storeFlushCtxs.values()) {
2385         boolean needsCompaction = flush.commit(status);
2386         if (needsCompaction) {
2387           compactionRequested = true;
2388         }
2389         byte[] storeName = it.next().getFamily().getName();
2390         List<Path> storeCommittedFiles = flush.getCommittedFiles();
2391         committedFiles.put(storeName, storeCommittedFiles);
2392         // Flush committed no files, indicating flush is empty or flush was canceled
2393         if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
2394           totalFlushableSizeOfFlushableStores -= prepareResult.storeFlushableSize.get(storeName);
2395         }
2396       }
2397       storeFlushCtxs.clear();
2398 
2399       // Set down the memstore size by amount of flush.
2400       this.addAndGetGlobalMemstoreSize(-totalFlushableSizeOfFlushableStores);
2401 
2402       if (wal != null) {
2403         // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
2404         FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
2405           getRegionInfo(), flushOpSeqId, committedFiles);
2406         WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2407           desc, true, mvcc);
2408       }
2409     } catch (Throwable t) {
2410       // An exception here means that the snapshot was not persisted.
2411       // The wal needs to be replayed so its content is restored to memstore.
2412       // Currently, only a server restart will do this.
2413       // We used to only catch IOEs but its possible that we'd get other
2414       // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
2415       // all and sundry.
2416       if (wal != null) {
2417         try {
2418           FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2419             getRegionInfo(), flushOpSeqId, committedFiles);
2420           WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2421             desc, false, mvcc);
2422         } catch (Throwable ex) {
2423           LOG.warn(getRegionInfo().getEncodedName() + " : "
2424               + "failed writing ABORT_FLUSH marker to WAL", ex);
2425           // ignore this since we will be aborting the RS with DSE.
2426         }
2427         wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2428       }
2429       DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
2430           Bytes.toStringBinary(getRegionInfo().getRegionName()));
2431       dse.initCause(t);
2432       status.abort("Flush failed: " + StringUtils.stringifyException(t));
2433 
2434       // Callers for flushcache() should catch DroppedSnapshotException and abort the region server.
2435       // However, since we may have the region read lock, we cannot call close(true) here since
2436       // we cannot promote to a write lock. Instead we are setting closing so that all other region
2437       // operations except for close will be rejected.
2438       this.closing.set(true);
2439 
2440       if (rsServices != null) {
2441         // This is a safeguard against the case where the caller fails to explicitly handle aborting
2442         rsServices.abort("Replay of WAL required. Forcing server shutdown", dse);
2443       }
2444 
2445       throw dse;
2446     }
2447 
2448     // If we get to here, the HStores have been written.
2449     if (wal != null) {
2450       wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2451     }
2452 
2453     // Record latest flush time
2454     for (Store store: storesToFlush) {
2455       this.lastStoreFlushTimeMap.put(store, startTime);
2456     }
2457 
2458     this.maxFlushedSeqId = flushedSeqId;
2459     this.lastFlushOpSeqId = flushOpSeqId;
2460 
2461     // C. Finally notify anyone waiting on memstore to clear:
2462     // e.g. checkResources().
2463     synchronized (this) {
2464       notifyAll(); // FindBugs NN_NAKED_NOTIFY
2465     }
2466 
2467     long time = EnvironmentEdgeManager.currentTime() - startTime;
2468     long memstoresize = this.memstoreSize.get();
2469     String msg = "Finished memstore flush of ~"
2470         + StringUtils.byteDesc(totalFlushableSizeOfFlushableStores) + "/"
2471         + totalFlushableSizeOfFlushableStores + ", currentsize="
2472         + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
2473         + " for region " + this + " in " + time + "ms, sequenceid="
2474         + flushOpSeqId +  ", compaction requested=" + compactionRequested
2475         + ((wal == null) ? "; wal=null" : "");
2476     LOG.info(msg);
2477     status.setStatus(msg);
2478 
2479     return new FlushResultImpl(compactionRequested ?
2480         FlushResult.Result.FLUSHED_COMPACTION_NEEDED :
2481           FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED,
2482         flushOpSeqId);
2483   }
2484 
2485   /**
2486    * Method to safely get the next sequence number.
2487    * @return Next sequence number unassociated with any actual edit.
2488    * @throws IOException
2489    */
2490   @VisibleForTesting
2491   protected long getNextSequenceId(final WAL wal) throws IOException {
2492     // TODO: For review. Putting an empty edit in to get a sequenceid out will not work if the
2493     // WAL is banjaxed... if it has gotten an exception and the WAL has not yet been rolled or
2494     // aborted. In this case, we'll just get stuck here. For now, until HBASE-12751, just have
2495     // a timeout. May happen in tests after we tightened the semantic via HBASE-14317.
2496     // Also, the getSequenceId blocks on a latch. There is no global list of outstanding latches
2497     // so if an abort or stop, there is no way to call them in.
2498     WALKey key = this.appendEmptyEdit(wal);
2499     mvcc.complete(key.getWriteEntry());
2500     return key.getSequenceId(this.maxWaitForSeqId);
2501   }
2502 
2503   //////////////////////////////////////////////////////////////////////////////
2504   // get() methods for client use.
2505   //////////////////////////////////////////////////////////////////////////////
2506 
2507   @Override
2508   public Result getClosestRowBefore(final byte [] row, final byte [] family) throws IOException {
2509     if (coprocessorHost != null) {
2510       Result result = new Result();
2511       if (coprocessorHost.preGetClosestRowBefore(row, family, result)) {
2512         return result;
2513       }
2514     }
2515     // look across all the HStores for this region and determine what the
2516     // closest key is across all column families, since the data may be sparse
2517     checkRow(row, "getClosestRowBefore");
2518     startRegionOperation(Operation.GET);
2519     this.readRequestsCount.increment();
2520     try {
2521       Store store = getStore(family);
2522       // get the closest key. (HStore.getRowKeyAtOrBefore can return null)
2523       Cell key = store.getRowKeyAtOrBefore(row);
2524       Result result = null;
2525       if (key != null) {
2526         Get get = new Get(CellUtil.cloneRow(key));
2527         get.addFamily(family);
2528         result = get(get);
2529       }
2530       if (coprocessorHost != null) {
2531         coprocessorHost.postGetClosestRowBefore(row, family, result);
2532       }
2533       return result;
2534     } finally {
2535       closeRegionOperation(Operation.GET);
2536     }
2537   }
2538 
2539   @Override
2540   public RegionScanner getScanner(Scan scan) throws IOException {
2541    return getScanner(scan, null);
2542   }
2543 
2544   @Override
2545   public RegionScanner getScanner(Scan scan,
2546       List<KeyValueScanner> additionalScanners) throws IOException {
2547     startRegionOperation(Operation.SCAN);
2548     try {
2549       // Verify families are all valid
2550       if (!scan.hasFamilies()) {
2551         // Adding all families to scanner
2552         for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
2553           scan.addFamily(family);
2554         }
2555       } else {
2556         for (byte [] family : scan.getFamilyMap().keySet()) {
2557           checkFamily(family);
2558         }
2559       }
2560       return instantiateRegionScanner(scan, additionalScanners);
2561     } finally {
2562       closeRegionOperation(Operation.SCAN);
2563     }
2564   }
2565 
2566   protected RegionScanner instantiateRegionScanner(Scan scan,
2567       List<KeyValueScanner> additionalScanners) throws IOException {
2568     if (scan.isReversed()) {
2569       if (scan.getFilter() != null) {
2570         scan.getFilter().setReversed(true);
2571       }
2572       return new ReversedRegionScannerImpl(scan, additionalScanners, this);
2573     }
2574     return new RegionScannerImpl(scan, additionalScanners, this);
2575   }
2576 
2577   @Override
2578   public void prepareDelete(Delete delete) throws IOException {
2579     // Check to see if this is a deleteRow insert
2580     if(delete.getFamilyCellMap().isEmpty()){
2581       for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
2582         // Don't eat the timestamp
2583         delete.addFamily(family, delete.getTimeStamp());
2584       }
2585     } else {
2586       for(byte [] family : delete.getFamilyCellMap().keySet()) {
2587         if(family == null) {
2588           throw new NoSuchColumnFamilyException("Empty family is invalid");
2589         }
2590         checkFamily(family);
2591       }
2592     }
2593   }
2594 
2595   @Override
2596   public void delete(Delete delete) throws IOException {
2597     checkReadOnly();
2598     checkResources();
2599     startRegionOperation(Operation.DELETE);
2600     try {
2601       delete.getRow();
2602       // All edits for the given row (across all column families) must happen atomically.
2603       doBatchMutate(delete);
2604     } finally {
2605       closeRegionOperation(Operation.DELETE);
2606     }
2607   }
2608 
2609   /**
2610    * Row needed by below method.
2611    */
2612   private static final byte [] FOR_UNIT_TESTS_ONLY = Bytes.toBytes("ForUnitTestsOnly");
2613 
2614   /**
2615    * This is used only by unit tests. Not required to be a public API.
2616    * @param familyMap map of family to edits for the given family.
2617    * @throws IOException
2618    */
2619   void delete(NavigableMap<byte[], List<Cell>> familyMap,
2620       Durability durability) throws IOException {
2621     Delete delete = new Delete(FOR_UNIT_TESTS_ONLY);
2622     delete.setFamilyCellMap(familyMap);
2623     delete.setDurability(durability);
2624     doBatchMutate(delete);
2625   }
2626 
2627   @Override
2628   public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap,
2629       byte[] byteNow) throws IOException {
2630     for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
2631 
2632       byte[] family = e.getKey();
2633       List<Cell> cells = e.getValue();
2634       assert cells instanceof RandomAccess;
2635 
2636       Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
2637       int listSize = cells.size();
2638       for (int i=0; i < listSize; i++) {
2639         Cell cell = cells.get(i);
2640         //  Check if time is LATEST, change to time of most recent addition if so
2641         //  This is expensive.
2642         if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP && CellUtil.isDeleteType(cell)) {
2643           byte[] qual = CellUtil.cloneQualifier(cell);
2644           if (qual == null) qual = HConstants.EMPTY_BYTE_ARRAY;
2645 
2646           Integer count = kvCount.get(qual);
2647           if (count == null) {
2648             kvCount.put(qual, 1);
2649           } else {
2650             kvCount.put(qual, count + 1);
2651           }
2652           count = kvCount.get(qual);
2653 
2654           Get get = new Get(CellUtil.cloneRow(cell));
2655           get.setMaxVersions(count);
2656           get.addColumn(family, qual);
2657           if (coprocessorHost != null) {
2658             if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
2659                 byteNow, get)) {
2660               updateDeleteLatestVersionTimeStamp(cell, get, count, byteNow);
2661             }
2662           } else {
2663             updateDeleteLatestVersionTimeStamp(cell, get, count, byteNow);
2664           }
2665         } else {
2666           CellUtil.updateLatestStamp(cell, byteNow, 0);
2667         }
2668       }
2669     }
2670   }
2671 
2672   void updateDeleteLatestVersionTimeStamp(Cell cell, Get get, int count, byte[] byteNow)
2673       throws IOException {
2674     List<Cell> result = get(get, false);
2675 
2676     if (result.size() < count) {
2677       // Nothing to delete
2678       CellUtil.updateLatestStamp(cell, byteNow, 0);
2679       return;
2680     }
2681     if (result.size() > count) {
2682       throw new RuntimeException("Unexpected size: " + result.size());
2683     }
2684     Cell getCell = result.get(count - 1);
2685     CellUtil.setTimestamp(cell, getCell.getTimestamp());
2686   }
2687 
2688   @Override
2689   public void put(Put put) throws IOException {
2690     checkReadOnly();
2691 
2692     // Do a rough check that we have resources to accept a write.  The check is
2693     // 'rough' in that between the resource check and the call to obtain a
2694     // read lock, resources may run out.  For now, the thought is that this
2695     // will be extremely rare; we'll deal with it when it happens.
2696     checkResources();
2697     startRegionOperation(Operation.PUT);
2698     try {
2699       // All edits for the given row (across all column families) must happen atomically.
2700       doBatchMutate(put);
2701     } finally {
2702       closeRegionOperation(Operation.PUT);
2703     }
2704   }
2705 
2706   /**
2707    * Struct-like class that tracks the progress of a batch operation,
2708    * accumulating status codes and tracking the index at which processing
2709    * is proceeding.
2710    */
2711   private abstract static class BatchOperationInProgress<T> {
2712     T[] operations;
2713     int nextIndexToProcess = 0;
2714     OperationStatus[] retCodeDetails;
2715     WALEdit[] walEditsFromCoprocessors;
2716 
2717     public BatchOperationInProgress(T[] operations) {
2718       this.operations = operations;
2719       this.retCodeDetails = new OperationStatus[operations.length];
2720       this.walEditsFromCoprocessors = new WALEdit[operations.length];
2721       Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
2722     }
2723 
2724     public abstract Mutation getMutation(int index);
2725     public abstract long getNonceGroup(int index);
2726     public abstract long getNonce(int index);
2727     /** This method is potentially expensive and should only be used for non-replay CP path. */
2728     public abstract Mutation[] getMutationsForCoprocs();
2729     public abstract boolean isInReplay();
2730     public abstract long getReplaySequenceId();
2731 
2732     public boolean isDone() {
2733       return nextIndexToProcess == operations.length;
2734     }
2735   }
2736 
2737   private static class MutationBatch extends BatchOperationInProgress<Mutation> {
2738     private long nonceGroup;
2739     private long nonce;
2740     public MutationBatch(Mutation[] operations, long nonceGroup, long nonce) {
2741       super(operations);
2742       this.nonceGroup = nonceGroup;
2743       this.nonce = nonce;
2744     }
2745 
2746     @Override
2747     public Mutation getMutation(int index) {
2748       return this.operations[index];
2749     }
2750 
2751     @Override
2752     public long getNonceGroup(int index) {
2753       return nonceGroup;
2754     }
2755 
2756     @Override
2757     public long getNonce(int index) {
2758       return nonce;
2759     }
2760 
2761     @Override
2762     public Mutation[] getMutationsForCoprocs() {
2763       return this.operations;
2764     }
2765 
2766     @Override
2767     public boolean isInReplay() {
2768       return false;
2769     }
2770 
2771     @Override
2772     public long getReplaySequenceId() {
2773       return 0;
2774     }
2775   }
2776 
2777   private static class ReplayBatch extends BatchOperationInProgress<MutationReplay> {
2778     private long replaySeqId = 0;
2779     public ReplayBatch(MutationReplay[] operations, long seqId) {
2780       super(operations);
2781       this.replaySeqId = seqId;
2782     }
2783 
2784     @Override
2785     public Mutation getMutation(int index) {
2786       return this.operations[index].mutation;
2787     }
2788 
2789     @Override
2790     public long getNonceGroup(int index) {
2791       return this.operations[index].nonceGroup;
2792     }
2793 
2794     @Override
2795     public long getNonce(int index) {
2796       return this.operations[index].nonce;
2797     }
2798 
2799     @Override
2800     public Mutation[] getMutationsForCoprocs() {
2801       assert false;
2802       throw new RuntimeException("Should not be called for replay batch");
2803     }
2804 
2805     @Override
2806     public boolean isInReplay() {
2807       return true;
2808     }
2809 
2810     @Override
2811     public long getReplaySequenceId() {
2812       return this.replaySeqId;
2813     }
2814   }
2815 
2816   @Override
2817   public OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce)
2818       throws IOException {
2819     // As it stands, this is used for 3 things
2820     //  * batchMutate with single mutation - put/delete, separate or from checkAndMutate.
2821     //  * coprocessor calls (see ex. BulkDeleteEndpoint).
2822     // So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd...
2823     return batchMutate(new MutationBatch(mutations, nonceGroup, nonce));
2824   }
2825 
2826   public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
2827     return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
2828   }
2829 
2830   @Override
2831   public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
2832       throws IOException {
2833     if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo())
2834         && replaySeqId < lastReplayedOpenRegionSeqId) {
2835       // if it is a secondary replica we should ignore these entries silently
2836       // since they are coming out of order
2837       if (LOG.isTraceEnabled()) {
2838         LOG.trace(getRegionInfo().getEncodedName() + " : "
2839           + "Skipping " + mutations.length + " mutations with replaySeqId=" + replaySeqId
2840           + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId);
2841         for (MutationReplay mut : mutations) {
2842           LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation);
2843         }
2844       }
2845 
2846       OperationStatus[] statuses = new OperationStatus[mutations.length];
2847       for (int i = 0; i < statuses.length; i++) {
2848         statuses[i] = OperationStatus.SUCCESS;
2849       }
2850       return statuses;
2851     }
2852     return batchMutate(new ReplayBatch(mutations, replaySeqId));
2853   }
2854 
2855   /**
2856    * Perform a batch of mutations.
2857    * It supports only Put and Delete mutations and will ignore other types passed.
2858    * @param batchOp contains the list of mutations
2859    * @return an array of OperationStatus which internally contains the
2860    *         OperationStatusCode and the exceptionMessage if any.
2861    * @throws IOException
2862    */
2863   OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
2864     boolean initialized = false;
2865     Operation op = batchOp.isInReplay() ? Operation.REPLAY_BATCH_MUTATE : Operation.BATCH_MUTATE;
2866     startRegionOperation(op);
2867     try {
2868       while (!batchOp.isDone()) {
2869         if (!batchOp.isInReplay()) {
2870           checkReadOnly();
2871         }
2872         checkResources();
2873 
2874         if (!initialized) {
2875           this.writeRequestsCount.add(batchOp.operations.length);
2876           if (!batchOp.isInReplay()) {
2877             doPreMutationHook(batchOp);
2878           }
2879           initialized = true;
2880         }
2881         doMiniBatchMutation(batchOp);
2882         long newSize = this.getMemstoreSize();
2883         if (isFlushSize(newSize)) {
2884           requestFlush();
2885         }
2886       }
2887     } finally {
2888       closeRegionOperation(op);
2889     }
2890     return batchOp.retCodeDetails;
2891   }
2892 
2893 
2894   private void doPreMutationHook(BatchOperationInProgress<?> batchOp)
2895       throws IOException {
2896     /* Run coprocessor pre hook outside of locks to avoid deadlock */
2897     WALEdit walEdit = new WALEdit();
2898     if (coprocessorHost != null) {
2899       for (int i = 0 ; i < batchOp.operations.length; i++) {
2900         Mutation m = batchOp.getMutation(i);
2901         if (m instanceof Put) {
2902           if (coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) {
2903             // pre hook says skip this Put
2904             // mark as success and skip in doMiniBatchMutation
2905             batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2906           }
2907         } else if (m instanceof Delete) {
2908           Delete curDel = (Delete) m;
2909           if (curDel.getFamilyCellMap().isEmpty()) {
2910             // handle deleting a row case
2911             prepareDelete(curDel);
2912           }
2913           if (coprocessorHost.preDelete(curDel, walEdit, m.getDurability())) {
2914             // pre hook says skip this Delete
2915             // mark as success and skip in doMiniBatchMutation
2916             batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2917           }
2918         } else {
2919           // In case of passing Append mutations along with the Puts and Deletes in batchMutate
2920           // mark the operation return code as failure so that it will not be considered in
2921           // the doMiniBatchMutation
2922           batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.FAILURE,
2923               "Put/Delete mutations only supported in batchMutate() now");
2924         }
2925         if (!walEdit.isEmpty()) {
2926           batchOp.walEditsFromCoprocessors[i] = walEdit;
2927           walEdit = new WALEdit();
2928         }
2929       }
2930     }
2931   }
2932 
2933   @SuppressWarnings("unchecked")
2934   private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp) throws IOException {
2935     boolean isInReplay = batchOp.isInReplay();
2936     // variable to note if all Put items are for the same CF -- metrics related
2937     boolean putsCfSetConsistent = true;
2938     //The set of columnFamilies first seen for Put.
2939     Set<byte[]> putsCfSet = null;
2940     // variable to note if all Delete items are for the same CF -- metrics related
2941     boolean deletesCfSetConsistent = true;
2942     //The set of columnFamilies first seen for Delete.
2943     Set<byte[]> deletesCfSet = null;
2944 
2945     long currentNonceGroup = HConstants.NO_NONCE, currentNonce = HConstants.NO_NONCE;
2946     WALEdit walEdit = new WALEdit(isInReplay);
2947     MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
2948     long txid = 0;
2949     boolean doRollBackMemstore = false;
2950     boolean locked = false;
2951 
2952     /** Keep track of the locks we hold so we can release them in finally clause */
2953     List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
2954     // reference family maps directly so coprocessors can mutate them if desired
2955     Map<byte[], List<Cell>>[] familyMaps = new Map[batchOp.operations.length];
2956     // We try to set up a batch in the range [firstIndex,lastIndexExclusive)
2957     int firstIndex = batchOp.nextIndexToProcess;
2958     int lastIndexExclusive = firstIndex;
2959     boolean success = false;
2960     int noOfPuts = 0, noOfDeletes = 0;
2961     WALKey walKey = null;
2962     long mvccNum = 0;
2963     long addedSize = 0;
2964     try {
2965       // ------------------------------------
2966       // STEP 1. Try to acquire as many locks as we can, and ensure
2967       // we acquire at least one.
2968       // ----------------------------------
2969       int numReadyToWrite = 0;
2970       long now = EnvironmentEdgeManager.currentTime();
2971       while (lastIndexExclusive < batchOp.operations.length) {
2972         Mutation mutation = batchOp.getMutation(lastIndexExclusive);
2973         boolean isPutMutation = mutation instanceof Put;
2974 
2975         Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
2976         // store the family map reference to allow for mutations
2977         familyMaps[lastIndexExclusive] = familyMap;
2978 
2979         // skip anything that "ran" already
2980         if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
2981             != OperationStatusCode.NOT_RUN) {
2982           lastIndexExclusive++;
2983           continue;
2984         }
2985 
2986         try {
2987           if (isPutMutation) {
2988             // Check the families in the put. If bad, skip this one.
2989             if (isInReplay) {
2990               removeNonExistentColumnFamilyForReplay(familyMap);
2991             } else {
2992               checkFamilies(familyMap.keySet());
2993             }
2994             checkTimestamps(mutation.getFamilyCellMap(), now);
2995           } else {
2996             prepareDelete((Delete) mutation);
2997           }
2998           checkRow(mutation.getRow(), "doMiniBatchMutation");
2999         } catch (NoSuchColumnFamilyException nscf) {
3000           LOG.warn("No such column family in batch mutation", nscf);
3001           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
3002               OperationStatusCode.BAD_FAMILY, nscf.getMessage());
3003           lastIndexExclusive++;
3004           continue;
3005         } catch (FailedSanityCheckException fsce) {
3006           LOG.warn("Batch Mutation did not pass sanity check", fsce);
3007           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
3008               OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
3009           lastIndexExclusive++;
3010           continue;
3011         } catch (WrongRegionException we) {
3012           LOG.warn("Batch mutation had a row that does not belong to this region", we);
3013           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
3014               OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
3015           lastIndexExclusive++;
3016           continue;
3017         }
3018 
3019         // If we haven't got any rows in our batch, we should block to
3020         // get the next one.
3021         RowLock rowLock = null;
3022         try {
3023           rowLock = getRowLock(mutation.getRow(), true);
3024         } catch (IOException ioe) {
3025           LOG.warn("Failed getting lock in batch put, row="
3026             + Bytes.toStringBinary(mutation.getRow()), ioe);
3027         }
3028         if (rowLock == null) {
3029           // We failed to grab another lock
3030           break; // stop acquiring more rows for this batch
3031         } else {
3032           acquiredRowLocks.add(rowLock);
3033         }
3034 
3035         lastIndexExclusive++;
3036         numReadyToWrite++;
3037 
3038         if (isPutMutation) {
3039           // If Column Families stay consistent through out all of the
3040           // individual puts then metrics can be reported as a mutliput across
3041           // column families in the first put.
3042           if (putsCfSet == null) {
3043             putsCfSet = mutation.getFamilyCellMap().keySet();
3044           } else {
3045             putsCfSetConsistent = putsCfSetConsistent
3046                 && mutation.getFamilyCellMap().keySet().equals(putsCfSet);
3047           }
3048         } else {
3049           if (deletesCfSet == null) {
3050             deletesCfSet = mutation.getFamilyCellMap().keySet();
3051           } else {
3052             deletesCfSetConsistent = deletesCfSetConsistent
3053                 && mutation.getFamilyCellMap().keySet().equals(deletesCfSet);
3054           }
3055         }
3056       }
3057 
3058       // we should record the timestamp only after we have acquired the rowLock,
3059       // otherwise, newer puts/deletes are not guaranteed to have a newer timestamp
3060       now = EnvironmentEdgeManager.currentTime();
3061       byte[] byteNow = Bytes.toBytes(now);
3062 
3063       // Nothing to put/delete -- an exception in the above such as NoSuchColumnFamily?
3064       if (numReadyToWrite <= 0) return 0L;
3065 
3066       // We've now grabbed as many mutations off the list as we can
3067 
3068       // ------------------------------------
3069       // STEP 2. Update any LATEST_TIMESTAMP timestamps
3070       // ----------------------------------
3071       for (int i = firstIndex; !isInReplay && i < lastIndexExclusive; i++) {
3072         // skip invalid
3073         if (batchOp.retCodeDetails[i].getOperationStatusCode()
3074             != OperationStatusCode.NOT_RUN) continue;
3075 
3076         Mutation mutation = batchOp.getMutation(i);
3077         if (mutation instanceof Put) {
3078           updateCellTimestamps(familyMaps[i].values(), byteNow);
3079           noOfPuts++;
3080         } else {
3081           prepareDeleteTimestamps(mutation, familyMaps[i], byteNow);
3082           noOfDeletes++;
3083         }
3084         rewriteCellTags(familyMaps[i], mutation);
3085       }
3086 
3087       lock(this.updatesLock.readLock(), numReadyToWrite);
3088       locked = true;
3089 
3090       // calling the pre CP hook for batch mutation
3091       if (!isInReplay && coprocessorHost != null) {
3092         MiniBatchOperationInProgress<Mutation> miniBatchOp =
3093           new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3094           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
3095         if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
3096       }
3097 
3098       // ------------------------------------
3099       // STEP 3. Build WAL edit
3100       // ----------------------------------
3101       Durability durability = Durability.USE_DEFAULT;
3102       for (int i = firstIndex; i < lastIndexExclusive; i++) {
3103         // Skip puts that were determined to be invalid during preprocessing
3104         if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
3105           continue;
3106         }
3107 
3108         Mutation m = batchOp.getMutation(i);
3109         Durability tmpDur = getEffectiveDurability(m.getDurability());
3110         if (tmpDur.ordinal() > durability.ordinal()) {
3111           durability = tmpDur;
3112         }
3113         if (tmpDur == Durability.SKIP_WAL) {
3114           recordMutationWithoutWal(m.getFamilyCellMap());
3115           continue;
3116         }
3117 
3118         long nonceGroup = batchOp.getNonceGroup(i), nonce = batchOp.getNonce(i);
3119         // In replay, the batch may contain multiple nonces. If so, write WALEdit for each.
3120         // Given how nonces are originally written, these should be contiguous.
3121         // They don't have to be, it will still work, just write more WALEdits than needed.
3122         if (nonceGroup != currentNonceGroup || nonce != currentNonce) {
3123           if (walEdit.size() > 0) {
3124             assert isInReplay;
3125             if (!isInReplay) {
3126               throw new IOException("Multiple nonces per batch and not in replay");
3127             }
3128             // txid should always increase, so having the one from the last call is ok.
3129             // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
3130             walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
3131               this.htableDescriptor.getTableName(), now, m.getClusterIds(),
3132               currentNonceGroup, currentNonce, mvcc);
3133             txid = this.wal.append(this.htableDescriptor,  this.getRegionInfo(),  walKey,
3134               walEdit, true);
3135             walEdit = new WALEdit(isInReplay);
3136             walKey = null;
3137           }
3138           currentNonceGroup = nonceGroup;
3139           currentNonce = nonce;
3140         }
3141 
3142         // Add WAL edits by CP
3143         WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
3144         if (fromCP != null) {
3145           for (Cell cell : fromCP.getCells()) {
3146             walEdit.add(cell);
3147           }
3148         }
3149         addFamilyMapToWALEdit(familyMaps[i], walEdit);
3150       }
3151 
3152       // -------------------------
3153       // STEP 4. Append the final edit to WAL. Do not sync wal.
3154       // -------------------------
3155       Mutation mutation = batchOp.getMutation(firstIndex);
3156       if (isInReplay) {
3157         // use wal key from the original
3158         walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
3159           this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
3160           mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
3161         long replaySeqId = batchOp.getReplaySequenceId();
3162         walKey.setOrigLogSeqNum(replaySeqId);
3163       }
3164       if (walEdit.size() > 0) {
3165         if (!isInReplay) {
3166         // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
3167         walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
3168             this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
3169             mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
3170         }
3171         txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
3172       }
3173       // ------------------------------------
3174       // Acquire the latest mvcc number
3175       // ----------------------------------
3176       if (walKey == null) {
3177         // If this is a skip wal operation just get the read point from mvcc
3178         walKey = this.appendEmptyEdit(this.wal);
3179       }
3180       if (!isInReplay) {
3181         writeEntry = walKey.getWriteEntry();
3182         mvccNum = writeEntry.getWriteNumber();
3183       } else {
3184         mvccNum = batchOp.getReplaySequenceId();
3185       }
3186 
3187       // ------------------------------------
3188       // STEP 5. Write back to memstore
3189       // Write to memstore. It is ok to write to memstore
3190       // first without syncing the WAL because we do not roll
3191       // forward the memstore MVCC. The MVCC will be moved up when
3192       // the complete operation is done. These changes are not yet
3193       // visible to scanners till we update the MVCC. The MVCC is
3194       // moved only when the sync is complete.
3195       // ----------------------------------
3196       for (int i = firstIndex; i < lastIndexExclusive; i++) {
3197         if (batchOp.retCodeDetails[i].getOperationStatusCode()
3198             != OperationStatusCode.NOT_RUN) {
3199           continue;
3200         }
3201         doRollBackMemstore = true; // If we have a failure, we need to clean what we wrote
3202         addedSize += applyFamilyMapToMemstore(familyMaps[i], mvccNum, isInReplay);
3203       }
3204 
3205       // -------------------------------
3206       // STEP 6. Release row locks, etc.
3207       // -------------------------------
3208       if (locked) {
3209         this.updatesLock.readLock().unlock();
3210         locked = false;
3211       }
3212       releaseRowLocks(acquiredRowLocks);
3213 
3214       // -------------------------
3215       // STEP 7. Sync wal.
3216       // -------------------------
3217       if (txid != 0) {
3218         syncOrDefer(txid, durability);
3219       }
3220 
3221       doRollBackMemstore = false;
3222       // update memstore size
3223       this.addAndGetGlobalMemstoreSize(addedSize);
3224 
3225       // calling the post CP hook for batch mutation
3226       if (!isInReplay && coprocessorHost != null) {
3227         MiniBatchOperationInProgress<Mutation> miniBatchOp =
3228           new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3229           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
3230         coprocessorHost.postBatchMutate(miniBatchOp);
3231       }
3232 
3233       // ------------------------------------------------------------------
3234       // STEP 8. Advance mvcc. This will make this put visible to scanners and getters.
3235       // ------------------------------------------------------------------
3236       if (writeEntry != null) {
3237         mvcc.completeAndWait(writeEntry);
3238         writeEntry = null;
3239       } else if (isInReplay) {
3240         // ensure that the sequence id of the region is at least as big as orig log seq id
3241         mvcc.advanceTo(mvccNum);
3242       }
3243 
3244       for (int i = firstIndex; i < lastIndexExclusive; i ++) {
3245         if (batchOp.retCodeDetails[i] == OperationStatus.NOT_RUN) {
3246           batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
3247         }
3248       }
3249 
3250       // ------------------------------------
3251       // STEP 9. Run coprocessor post hooks. This should be done after the wal is
3252       // synced so that the coprocessor contract is adhered to.
3253       // ------------------------------------
3254       if (!isInReplay && coprocessorHost != null) {
3255         for (int i = firstIndex; i < lastIndexExclusive; i++) {
3256           // only for successful puts
3257           if (batchOp.retCodeDetails[i].getOperationStatusCode()
3258               != OperationStatusCode.SUCCESS) {
3259             continue;
3260           }
3261           Mutation m = batchOp.getMutation(i);
3262           if (m instanceof Put) {
3263             coprocessorHost.postPut((Put) m, walEdit, m.getDurability());
3264           } else {
3265             coprocessorHost.postDelete((Delete) m, walEdit, m.getDurability());
3266           }
3267         }
3268       }
3269 
3270       success = true;
3271       return addedSize;
3272     } finally {
3273       // if the wal sync was unsuccessful, remove keys from memstore
3274       if (doRollBackMemstore) {
3275         for (int j = 0; j < familyMaps.length; j++) {
3276           for(List<Cell> cells:familyMaps[j].values()) {
3277             rollbackMemstore(cells);
3278           }
3279         }
3280         if (writeEntry != null) mvcc.complete(writeEntry);
3281       } else {
3282         if (writeEntry != null) {
3283           mvcc.completeAndWait(writeEntry);
3284         }
3285       }
3286 
3287       if (locked) {
3288         this.updatesLock.readLock().unlock();
3289       }
3290       releaseRowLocks(acquiredRowLocks);
3291 
3292       // See if the column families were consistent through the whole thing.
3293       // if they were then keep them. If they were not then pass a null.
3294       // null will be treated as unknown.
3295       // Total time taken might be involving Puts and Deletes.
3296       // Split the time for puts and deletes based on the total number of Puts and Deletes.
3297 
3298       if (noOfPuts > 0) {
3299         // There were some Puts in the batch.
3300         if (this.metricsRegion != null) {
3301           this.metricsRegion.updatePut();
3302         }
3303       }
3304       if (noOfDeletes > 0) {
3305         // There were some Deletes in the batch.
3306         if (this.metricsRegion != null) {
3307           this.metricsRegion.updateDelete();
3308         }
3309       }
3310       if (!success) {
3311         for (int i = firstIndex; i < lastIndexExclusive; i++) {
3312           if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
3313             batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
3314           }
3315         }
3316       }
3317       if (coprocessorHost != null && !batchOp.isInReplay()) {
3318         // call the coprocessor hook to do any finalization steps
3319         // after the put is done
3320         MiniBatchOperationInProgress<Mutation> miniBatchOp =
3321             new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3322                 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex,
3323                 lastIndexExclusive);
3324         coprocessorHost.postBatchMutateIndispensably(miniBatchOp, success);
3325       }
3326 
3327       batchOp.nextIndexToProcess = lastIndexExclusive;
3328     }
3329   }
3330 
3331   /**
3332    * Returns effective durability from the passed durability and
3333    * the table descriptor.
3334    */
3335   protected Durability getEffectiveDurability(Durability d) {
3336     return d == Durability.USE_DEFAULT ? this.durability : d;
3337   }
3338 
3339   //TODO, Think that gets/puts and deletes should be refactored a bit so that
3340   //the getting of the lock happens before, so that you would just pass it into
3341   //the methods. So in the case of checkAndMutate you could just do lockRow,
3342   //get, put, unlockRow or something
3343 
3344   @Override
3345   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
3346       CompareOp compareOp, ByteArrayComparable comparator, Mutation w,
3347       boolean writeToWAL)
3348   throws IOException{
3349     checkReadOnly();
3350     //TODO, add check for value length or maybe even better move this to the
3351     //client if this becomes a global setting
3352     checkResources();
3353     boolean isPut = w instanceof Put;
3354     if (!isPut && !(w instanceof Delete))
3355       throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " +
3356           "be Put or Delete");
3357     if (!Bytes.equals(row, w.getRow())) {
3358       throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " +
3359           "getRow must match the passed row");
3360     }
3361 
3362     startRegionOperation();
3363     try {
3364       Get get = new Get(row);
3365       checkFamily(family);
3366       get.addColumn(family, qualifier);
3367 
3368       // Lock row - note that doBatchMutate will relock this row if called
3369       RowLock rowLock = getRowLock(get.getRow());
3370       // wait for all previous transactions to complete (with lock held)
3371       mvcc.await();
3372       try {
3373         if (this.getCoprocessorHost() != null) {
3374           Boolean processed = null;
3375           if (w instanceof Put) {
3376             processed = this.getCoprocessorHost().preCheckAndPutAfterRowLock(row, family,
3377                 qualifier, compareOp, comparator, (Put) w);
3378           } else if (w instanceof Delete) {
3379             processed = this.getCoprocessorHost().preCheckAndDeleteAfterRowLock(row, family,
3380                 qualifier, compareOp, comparator, (Delete) w);
3381           }
3382           if (processed != null) {
3383             return processed;
3384           }
3385         }
3386         List<Cell> result = get(get, false);
3387 
3388         boolean valueIsNull = comparator.getValue() == null ||
3389           comparator.getValue().length == 0;
3390         boolean matches = false;
3391         long cellTs = 0;
3392         if (result.size() == 0 && valueIsNull) {
3393           matches = true;
3394         } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
3395             valueIsNull) {
3396           matches = true;
3397           cellTs = result.get(0).getTimestamp();
3398         } else if (result.size() == 1 && !valueIsNull) {
3399           Cell kv = result.get(0);
3400           cellTs = kv.getTimestamp();
3401           int compareResult = comparator.compareTo(kv.getValueArray(),
3402               kv.getValueOffset(), kv.getValueLength());
3403           switch (compareOp) {
3404           case LESS:
3405             matches = compareResult < 0;
3406             break;
3407           case LESS_OR_EQUAL:
3408             matches = compareResult <= 0;
3409             break;
3410           case EQUAL:
3411             matches = compareResult == 0;
3412             break;
3413           case NOT_EQUAL:
3414             matches = compareResult != 0;
3415             break;
3416           case GREATER_OR_EQUAL:
3417             matches = compareResult >= 0;
3418             break;
3419           case GREATER:
3420             matches = compareResult > 0;
3421             break;
3422           default:
3423             throw new RuntimeException("Unknown Compare op " + compareOp.name());
3424           }
3425         }
3426         //If matches put the new put or delete the new delete
3427         if (matches) {
3428           // We have acquired the row lock already. If the system clock is NOT monotonically
3429           // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
3430           // larger timestamp than what was observed via Get. doBatchMutate already does this, but
3431           // there is no way to pass the cellTs. See HBASE-14054.
3432           long now = EnvironmentEdgeManager.currentTime();
3433           long ts = Math.max(now, cellTs); // ensure write is not eclipsed
3434           byte[] byteTs = Bytes.toBytes(ts);
3435 
3436           if (w instanceof Put) {
3437             updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
3438           }
3439           // else delete is not needed since it already does a second get, and sets the timestamp
3440           // from get (see prepareDeleteTimestamps).
3441 
3442           // All edits for the given row (across all column families) must
3443           // happen atomically.
3444           doBatchMutate(w);
3445           this.checkAndMutateChecksPassed.increment();
3446           return true;
3447         }
3448         this.checkAndMutateChecksFailed.increment();
3449         return false;
3450       } finally {
3451         rowLock.release();
3452       }
3453     } finally {
3454       closeRegionOperation();
3455     }
3456   }
3457 
3458   //TODO, Think that gets/puts and deletes should be refactored a bit so that
3459   //the getting of the lock happens before, so that you would just pass it into
3460   //the methods. So in the case of checkAndMutate you could just do lockRow,
3461   //get, put, unlockRow or something
3462 
3463   @Override
3464   public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
3465       CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm,
3466       boolean writeToWAL) throws IOException {
3467     checkReadOnly();
3468     //TODO, add check for value length or maybe even better move this to the
3469     //client if this becomes a global setting
3470     checkResources();
3471 
3472     startRegionOperation();
3473     try {
3474       Get get = new Get(row);
3475       checkFamily(family);
3476       get.addColumn(family, qualifier);
3477 
3478       // Lock row - note that doBatchMutate will relock this row if called
3479       RowLock rowLock = getRowLock(get.getRow());
3480       // wait for all previous transactions to complete (with lock held)
3481       mvcc.await();
3482       try {
3483         List<Cell> result = get(get, false);
3484 
3485         boolean valueIsNull = comparator.getValue() == null ||
3486             comparator.getValue().length == 0;
3487         boolean matches = false;
3488         long cellTs = 0;
3489         if (result.size() == 0 && valueIsNull) {
3490           matches = true;
3491         } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
3492             valueIsNull) {
3493           matches = true;
3494           cellTs = result.get(0).getTimestamp();
3495         } else if (result.size() == 1 && !valueIsNull) {
3496           Cell kv = result.get(0);
3497           cellTs = kv.getTimestamp();
3498           int compareResult = comparator.compareTo(kv.getValueArray(),
3499               kv.getValueOffset(), kv.getValueLength());
3500           switch (compareOp) {
3501           case LESS:
3502             matches = compareResult < 0;
3503             break;
3504           case LESS_OR_EQUAL:
3505             matches = compareResult <= 0;
3506             break;
3507           case EQUAL:
3508             matches = compareResult == 0;
3509             break;
3510           case NOT_EQUAL:
3511             matches = compareResult != 0;
3512             break;
3513           case GREATER_OR_EQUAL:
3514             matches = compareResult >= 0;
3515             break;
3516           case GREATER:
3517             matches = compareResult > 0;
3518             break;
3519           default:
3520             throw new RuntimeException("Unknown Compare op " + compareOp.name());
3521           }
3522         }
3523         //If matches put the new put or delete the new delete
3524         if (matches) {
3525           // We have acquired the row lock already. If the system clock is NOT monotonically
3526           // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
3527           // larger timestamp than what was observed via Get. doBatchMutate already does this, but
3528           // there is no way to pass the cellTs. See HBASE-14054.
3529           long now = EnvironmentEdgeManager.currentTime();
3530           long ts = Math.max(now, cellTs); // ensure write is not eclipsed
3531           byte[] byteTs = Bytes.toBytes(ts);
3532 
3533           for (Mutation w : rm.getMutations()) {
3534             if (w instanceof Put) {
3535               updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
3536             }
3537             // else delete is not needed since it already does a second get, and sets the timestamp
3538             // from get (see prepareDeleteTimestamps).
3539           }
3540 
3541           // All edits for the given row (across all column families) must
3542           // happen atomically.
3543           mutateRow(rm);
3544           this.checkAndMutateChecksPassed.increment();
3545           return true;
3546         }
3547         this.checkAndMutateChecksFailed.increment();
3548         return false;
3549       } finally {
3550         rowLock.release();
3551       }
3552     } finally {
3553       closeRegionOperation();
3554     }
3555   }
3556 
3557   private void doBatchMutate(Mutation mutation) throws IOException {
3558     // Currently this is only called for puts and deletes, so no nonces.
3559     OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation});
3560     if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
3561       throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
3562     } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
3563       throw new NoSuchColumnFamilyException(batchMutate[0].getExceptionMsg());
3564     }
3565   }
3566 
3567   /**
3568    * Complete taking the snapshot on the region. Writes the region info and adds references to the
3569    * working snapshot directory.
3570    *
3571    * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
3572    * arg.  (In the future other cancellable HRegion methods could eventually add a
3573    * {@link ForeignExceptionSnare}, or we could do something fancier).
3574    *
3575    * @param desc snapshot description object
3576    * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to
3577    *   bail out.  This is allowed to be null and will just be ignored in that case.
3578    * @throws IOException if there is an external or internal error causing the snapshot to fail
3579    */
3580   public void addRegionToSnapshot(SnapshotDescription desc,
3581       ForeignExceptionSnare exnSnare) throws IOException {
3582     Path rootDir = FSUtils.getRootDir(conf);
3583     Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir);
3584 
3585     SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
3586                                                         snapshotDir, desc, exnSnare);
3587     manifest.addRegion(this);
3588   }
3589 
3590   @Override
3591   public void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
3592       throws IOException {
3593     for (List<Cell> cells: cellItr) {
3594       if (cells == null) continue;
3595       assert cells instanceof RandomAccess;
3596       int listSize = cells.size();
3597       for (int i = 0; i < listSize; i++) {
3598         CellUtil.updateLatestStamp(cells.get(i), now, 0);
3599       }
3600     }
3601   }
3602 
3603   /**
3604    * Possibly rewrite incoming cell tags.
3605    */
3606   void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
3607     // Check if we have any work to do and early out otherwise
3608     // Update these checks as more logic is added here
3609 
3610     if (m.getTTL() == Long.MAX_VALUE) {
3611       return;
3612     }
3613 
3614     // From this point we know we have some work to do
3615 
3616     for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
3617       List<Cell> cells = e.getValue();
3618       assert cells instanceof RandomAccess;
3619       int listSize = cells.size();
3620       for (int i = 0; i < listSize; i++) {
3621         Cell cell = cells.get(i);
3622         List<Tag> newTags = Tag.carryForwardTags(null, cell);
3623         newTags = carryForwardTTLTag(newTags, m);
3624 
3625         // Rewrite the cell with the updated set of tags
3626         cells.set(i, new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
3627           cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
3628           cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
3629           cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
3630           cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
3631           newTags));
3632       }
3633     }
3634   }
3635 
3636   /*
3637    * Check if resources to support an update.
3638    *
3639    * We throw RegionTooBusyException if above memstore limit
3640    * and expect client to retry using some kind of backoff
3641   */
3642   private void checkResources() throws RegionTooBusyException {
3643     // If catalog region, do not impose resource constraints or block updates.
3644     if (this.getRegionInfo().isMetaRegion()) return;
3645 
3646     if (this.memstoreSize.get() > this.blockingMemStoreSize) {
3647       blockedRequestsCount.increment();
3648       requestFlush();
3649       throw new RegionTooBusyException("Above memstore limit, " +
3650           "regionName=" + (this.getRegionInfo() == null ? "unknown" :
3651           this.getRegionInfo().getRegionNameAsString()) +
3652           ", server=" + (this.getRegionServerServices() == null ? "unknown" :
3653           this.getRegionServerServices().getServerName()) +
3654           ", memstoreSize=" + memstoreSize.get() +
3655           ", blockingMemStoreSize=" + blockingMemStoreSize);
3656     }
3657   }
3658 
3659   /**
3660    * @throws IOException Throws exception if region is in read-only mode.
3661    */
3662   protected void checkReadOnly() throws IOException {
3663     if (isReadOnly()) {
3664       throw new DoNotRetryIOException("region is read only");
3665     }
3666   }
3667 
3668   protected void checkReadsEnabled() throws IOException {
3669     if (!this.writestate.readsEnabled) {
3670       throw new IOException(getRegionInfo().getEncodedName()
3671         + ": The region's reads are disabled. Cannot serve the request");
3672     }
3673   }
3674 
3675   public void setReadsEnabled(boolean readsEnabled) {
3676    if (readsEnabled && !this.writestate.readsEnabled) {
3677      LOG.info(getRegionInfo().getEncodedName() + " : Enabling reads for region.");
3678     }
3679     this.writestate.setReadsEnabled(readsEnabled);
3680   }
3681 
3682   /**
3683    * Add updates first to the wal and then add values to memstore.
3684    * Warning: Assumption is caller has lock on passed in row.
3685    * @param edits Cell updates by column
3686    * @throws IOException
3687    */
3688   private void put(final byte [] row, byte [] family, List<Cell> edits)
3689   throws IOException {
3690     NavigableMap<byte[], List<Cell>> familyMap;
3691     familyMap = new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
3692 
3693     familyMap.put(family, edits);
3694     Put p = new Put(row);
3695     p.setFamilyCellMap(familyMap);
3696     doBatchMutate(p);
3697   }
3698 
3699   /**
3700    * Atomically apply the given map of family->edits to the memstore.
3701    * This handles the consistency control on its own, but the caller
3702    * should already have locked updatesLock.readLock(). This also does
3703    * <b>not</b> check the families for validity.
3704    *
3705    * @param familyMap Map of kvs per family
3706    * @param localizedWriteEntry The WriteEntry of the MVCC for this transaction.
3707    *        If null, then this method internally creates a mvcc transaction.
3708    * @param output newly added KVs into memstore
3709    * @param isInReplay true when adding replayed KVs into memstore
3710    * @return the additional memory usage of the memstore caused by the
3711    * new entries.
3712    * @throws IOException
3713    */
3714   private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap,
3715     long mvccNum, boolean isInReplay) throws IOException {
3716     long size = 0;
3717 
3718     for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3719       byte[] family = e.getKey();
3720       List<Cell> cells = e.getValue();
3721       assert cells instanceof RandomAccess;
3722       Store store = getStore(family);
3723       int listSize = cells.size();
3724       for (int i=0; i < listSize; i++) {
3725         Cell cell = cells.get(i);
3726         if (cell.getSequenceId() == 0 || isInReplay) {
3727           CellUtil.setSequenceId(cell, mvccNum);
3728         }
3729         size += store.add(cell);
3730       }
3731     }
3732 
3733      return size;
3734    }
3735 
3736   /**
3737    * Remove all the keys listed in the map from the memstore. This method is
3738    * called when a Put/Delete has updated memstore but subsequently fails to update
3739    * the wal. This method is then invoked to rollback the memstore.
3740    */
3741   private void rollbackMemstore(List<Cell> memstoreCells) {
3742     int kvsRolledback = 0;
3743 
3744     for (Cell cell : memstoreCells) {
3745       byte[] family = CellUtil.cloneFamily(cell);
3746       Store store = getStore(family);
3747       store.rollback(cell);
3748       kvsRolledback++;
3749     }
3750     LOG.debug("rollbackMemstore rolled back " + kvsRolledback);
3751   }
3752 
3753   @Override
3754   public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException {
3755     for (byte[] family : families) {
3756       checkFamily(family);
3757     }
3758   }
3759 
3760   /**
3761    * During replay, there could exist column families which are removed between region server
3762    * failure and replay
3763    */
3764   private void removeNonExistentColumnFamilyForReplay(
3765       final Map<byte[], List<Cell>> familyMap) {
3766     List<byte[]> nonExistentList = null;
3767     for (byte[] family : familyMap.keySet()) {
3768       if (!this.htableDescriptor.hasFamily(family)) {
3769         if (nonExistentList == null) {
3770           nonExistentList = new ArrayList<byte[]>();
3771         }
3772         nonExistentList.add(family);
3773       }
3774     }
3775     if (nonExistentList != null) {
3776       for (byte[] family : nonExistentList) {
3777         // Perhaps schema was changed between crash and replay
3778         LOG.info("No family for " + Bytes.toString(family) + " omit from reply.");
3779         familyMap.remove(family);
3780       }
3781     }
3782   }
3783 
3784   @Override
3785   public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
3786       throws FailedSanityCheckException {
3787     if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
3788       return;
3789     }
3790     long maxTs = now + timestampSlop;
3791     for (List<Cell> kvs : familyMap.values()) {
3792       assert kvs instanceof RandomAccess;
3793       int listSize  = kvs.size();
3794       for (int i=0; i < listSize; i++) {
3795         Cell cell = kvs.get(i);
3796         // see if the user-side TS is out of range. latest = server-side
3797         long ts = cell.getTimestamp();
3798         if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
3799           throw new FailedSanityCheckException("Timestamp for KV out of range "
3800               + cell + " (too.new=" + timestampSlop + ")");
3801         }
3802       }
3803     }
3804   }
3805 
3806   /**
3807    * Append the given map of family->edits to a WALEdit data structure.
3808    * This does not write to the WAL itself.
3809    * @param familyMap map of family->edits
3810    * @param walEdit the destination entry to append into
3811    */
3812   private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
3813       WALEdit walEdit) {
3814     for (List<Cell> edits : familyMap.values()) {
3815       assert edits instanceof RandomAccess;
3816       int listSize = edits.size();
3817       for (int i=0; i < listSize; i++) {
3818         Cell cell = edits.get(i);
3819         walEdit.add(cell);
3820       }
3821     }
3822   }
3823 
3824   private void requestFlush() {
3825     if (this.rsServices == null) {
3826       return;
3827     }
3828     synchronized (writestate) {
3829       if (this.writestate.isFlushRequested()) {
3830         return;
3831       }
3832       writestate.flushRequested = true;
3833     }
3834     // Make request outside of synchronize block; HBASE-818.
3835     this.rsServices.getFlushRequester().requestFlush(this, false);
3836     if (LOG.isDebugEnabled()) {
3837       LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
3838     }
3839   }
3840 
3841   /*
3842    * @param size
3843    * @return True if size is over the flush threshold
3844    */
3845   private boolean isFlushSize(final long size) {
3846     return size > this.memstoreFlushSize;
3847   }
3848 
3849   /**
3850    * Read the edits put under this region by wal splitting process.  Put
3851    * the recovered edits back up into this region.
3852    *
3853    * <p>We can ignore any wal message that has a sequence ID that's equal to or
3854    * lower than minSeqId.  (Because we know such messages are already
3855    * reflected in the HFiles.)
3856    *
3857    * <p>While this is running we are putting pressure on memory yet we are
3858    * outside of our usual accounting because we are not yet an onlined region
3859    * (this stuff is being run as part of Region initialization).  This means
3860    * that if we're up against global memory limits, we'll not be flagged to flush
3861    * because we are not online. We can't be flushed by usual mechanisms anyways;
3862    * we're not yet online so our relative sequenceids are not yet aligned with
3863    * WAL sequenceids -- not till we come up online, post processing of split
3864    * edits.
3865    *
3866    * <p>But to help relieve memory pressure, at least manage our own heap size
3867    * flushing if are in excess of per-region limits.  Flushing, though, we have
3868    * to be careful and avoid using the regionserver/wal sequenceid.  Its running
3869    * on a different line to whats going on in here in this region context so if we
3870    * crashed replaying these edits, but in the midst had a flush that used the
3871    * regionserver wal with a sequenceid in excess of whats going on in here
3872    * in this region and with its split editlogs, then we could miss edits the
3873    * next time we go to recover. So, we have to flush inline, using seqids that
3874    * make sense in a this single region context only -- until we online.
3875    *
3876    * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of
3877    * the maxSeqId for the store to be applied, else its skipped.
3878    * @return the sequence id of the last edit added to this region out of the
3879    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3880    * @throws IOException
3881    */
3882   protected long replayRecoveredEditsIfAny(final Path regiondir,
3883       Map<byte[], Long> maxSeqIdInStores,
3884       final CancelableProgressable reporter, final MonitoredTask status)
3885       throws IOException {
3886     long minSeqIdForTheRegion = -1;
3887     for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
3888       if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
3889         minSeqIdForTheRegion = maxSeqIdInStore;
3890       }
3891     }
3892     long seqid = minSeqIdForTheRegion;
3893 
3894     FileSystem fs = this.fs.getFileSystem();
3895     NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
3896     if (LOG.isDebugEnabled()) {
3897       LOG.debug("Found " + (files == null ? 0 : files.size())
3898         + " recovered edits file(s) under " + regiondir);
3899     }
3900 
3901     if (files == null || files.isEmpty()) return seqid;
3902 
3903     for (Path edits: files) {
3904       if (edits == null || !fs.exists(edits)) {
3905         LOG.warn("Null or non-existent edits file: " + edits);
3906         continue;
3907       }
3908       if (isZeroLengthThenDelete(fs, edits)) continue;
3909 
3910       long maxSeqId;
3911       String fileName = edits.getName();
3912       maxSeqId = Math.abs(Long.parseLong(fileName));
3913       if (maxSeqId <= minSeqIdForTheRegion) {
3914         if (LOG.isDebugEnabled()) {
3915           String msg = "Maximum sequenceid for this wal is " + maxSeqId
3916             + " and minimum sequenceid for the region is " + minSeqIdForTheRegion
3917             + ", skipped the whole file, path=" + edits;
3918           LOG.debug(msg);
3919         }
3920         continue;
3921       }
3922 
3923       try {
3924         // replay the edits. Replay can return -1 if everything is skipped, only update
3925         // if seqId is greater
3926         seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter));
3927       } catch (IOException e) {
3928         boolean skipErrors = conf.getBoolean(
3929             HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
3930             conf.getBoolean(
3931                 "hbase.skip.errors",
3932                 HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
3933         if (conf.get("hbase.skip.errors") != null) {
3934           LOG.warn(
3935               "The property 'hbase.skip.errors' has been deprecated. Please use " +
3936               HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
3937         }
3938         if (skipErrors) {
3939           Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
3940           LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
3941               + "=true so continuing. Renamed " + edits +
3942               " as " + p, e);
3943         } else {
3944           throw e;
3945         }
3946       }
3947     }
3948     // The edits size added into rsAccounting during this replaying will not
3949     // be required any more. So just clear it.
3950     if (this.rsAccounting != null) {
3951       this.rsAccounting.clearRegionReplayEditsSize(getRegionInfo().getRegionName());
3952     }
3953     if (seqid > minSeqIdForTheRegion) {
3954       // Then we added some edits to memory. Flush and cleanup split edit files.
3955       internalFlushcache(null, seqid, stores.values(), status, false);
3956     }
3957     // Now delete the content of recovered edits.  We're done w/ them.
3958     if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
3959       // For debugging data loss issues!
3960       // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
3961       // column family. Have to fake out file type too by casting our recovered.edits as storefiles
3962       String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regiondir).getName();
3963       Set<StoreFile> fakeStoreFiles = new HashSet<StoreFile>(files.size());
3964       for (Path file: files) {
3965         fakeStoreFiles.add(new StoreFile(getRegionFileSystem().getFileSystem(), file, this.conf,
3966           null, null));
3967       }
3968       getRegionFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
3969     } else {
3970       for (Path file: files) {
3971         if (!fs.delete(file, false)) {
3972           LOG.error("Failed delete of " + file);
3973         } else {
3974           LOG.debug("Deleted recovered.edits file=" + file);
3975         }
3976       }
3977     }
3978     return seqid;
3979   }
3980 
3981   /*
3982    * @param edits File of recovered edits.
3983    * @param maxSeqIdInStores Maximum sequenceid found in each store.  Edits in wal
3984    * must be larger than this to be replayed for each store.
3985    * @param reporter
3986    * @return the sequence id of the last edit added to this region out of the
3987    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3988    * @throws IOException
3989    */
3990   private long replayRecoveredEdits(final Path edits,
3991       Map<byte[], Long> maxSeqIdInStores, final CancelableProgressable reporter)
3992     throws IOException {
3993     String msg = "Replaying edits from " + edits;
3994     LOG.info(msg);
3995     MonitoredTask status = TaskMonitor.get().createStatus(msg);
3996     FileSystem fs = this.fs.getFileSystem();
3997 
3998     status.setStatus("Opening recovered edits");
3999     WAL.Reader reader = null;
4000     try {
4001       reader = WALFactory.createReader(fs, edits, conf);
4002       long currentEditSeqId = -1;
4003       long currentReplaySeqId = -1;
4004       long firstSeqIdInLog = -1;
4005       long skippedEdits = 0;
4006       long editsCount = 0;
4007       long intervalEdits = 0;
4008       WAL.Entry entry;
4009       Store store = null;
4010       boolean reported_once = false;
4011       ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
4012 
4013       try {
4014         // How many edits seen before we check elapsed time
4015         int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
4016         // How often to send a progress report (default 1/2 master timeout)
4017         int period = this.conf.getInt("hbase.hstore.report.period", 300000);
4018         long lastReport = EnvironmentEdgeManager.currentTime();
4019 
4020         while ((entry = reader.next()) != null) {
4021           WALKey key = entry.getKey();
4022           WALEdit val = entry.getEdit();
4023 
4024           if (ng != null) { // some test, or nonces disabled
4025             ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
4026           }
4027 
4028           if (reporter != null) {
4029             intervalEdits += val.size();
4030             if (intervalEdits >= interval) {
4031               // Number of edits interval reached
4032               intervalEdits = 0;
4033               long cur = EnvironmentEdgeManager.currentTime();
4034               if (lastReport + period <= cur) {
4035                 status.setStatus("Replaying edits..." +
4036                     " skipped=" + skippedEdits +
4037                     " edits=" + editsCount);
4038                 // Timeout reached
4039                 if(!reporter.progress()) {
4040                   msg = "Progressable reporter failed, stopping replay";
4041                   LOG.warn(msg);
4042                   status.abort(msg);
4043                   throw new IOException(msg);
4044                 }
4045                 reported_once = true;
4046                 lastReport = cur;
4047               }
4048             }
4049           }
4050 
4051           if (firstSeqIdInLog == -1) {
4052             firstSeqIdInLog = key.getLogSeqNum();
4053           }
4054           if (currentEditSeqId > key.getLogSeqNum()) {
4055             // when this condition is true, it means we have a serious defect because we need to
4056             // maintain increasing SeqId for WAL edits per region
4057             LOG.error(getRegionInfo().getEncodedName() + " : "
4058                  + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
4059                 + "; edit=" + val);
4060           } else {
4061             currentEditSeqId = key.getLogSeqNum();
4062           }
4063           currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
4064             key.getOrigLogSeqNum() : currentEditSeqId;
4065 
4066           // Start coprocessor replay here. The coprocessor is for each WALEdit
4067           // instead of a KeyValue.
4068           if (coprocessorHost != null) {
4069             status.setStatus("Running pre-WAL-restore hook in coprocessors");
4070             if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
4071               // if bypass this wal entry, ignore it ...
4072               continue;
4073             }
4074           }
4075           boolean checkRowWithinBoundary = false;
4076           // Check this edit is for this region.
4077           if (!Bytes.equals(key.getEncodedRegionName(),
4078               this.getRegionInfo().getEncodedNameAsBytes())) {
4079             checkRowWithinBoundary = true;
4080           }
4081 
4082           boolean flush = false;
4083           for (Cell cell: val.getCells()) {
4084             // Check this edit is for me. Also, guard against writing the special
4085             // METACOLUMN info such as HBASE::CACHEFLUSH entries
4086             if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
4087               // if region names don't match, skipp replaying compaction marker
4088               if (!checkRowWithinBoundary) {
4089                 //this is a special edit, we should handle it
4090                 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
4091                 if (compaction != null) {
4092                   //replay the compaction
4093                   replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
4094                 }
4095               }
4096               skippedEdits++;
4097               continue;
4098             }
4099             // Figure which store the edit is meant for.
4100             if (store == null || !CellUtil.matchingFamily(cell, store.getFamily().getName())) {
4101               store = getStore(cell);
4102             }
4103             if (store == null) {
4104               // This should never happen.  Perhaps schema was changed between
4105               // crash and redeploy?
4106               LOG.warn("No family for " + cell);
4107               skippedEdits++;
4108               continue;
4109             }
4110             if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
4111               cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
4112               LOG.warn("Row of " + cell + " is not within region boundary");
4113               skippedEdits++;
4114               continue;
4115             }
4116             // Now, figure if we should skip this edit.
4117             if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
4118                 .getName())) {
4119               skippedEdits++;
4120               continue;
4121             }
4122             CellUtil.setSequenceId(cell, currentReplaySeqId);
4123 
4124             // Once we are over the limit, restoreEdit will keep returning true to
4125             // flush -- but don't flush until we've played all the kvs that make up
4126             // the WALEdit.
4127             flush |= restoreEdit(store, cell);
4128             editsCount++;
4129           }
4130           if (flush) {
4131             internalFlushcache(null, currentEditSeqId, stores.values(), status, false);
4132           }
4133 
4134           if (coprocessorHost != null) {
4135             coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
4136           }
4137         }
4138       } catch (EOFException eof) {
4139         Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4140         msg = "Encountered EOF. Most likely due to Master failure during " +
4141             "wal splitting, so we have this data in another edit.  " +
4142             "Continuing, but renaming " + edits + " as " + p;
4143         LOG.warn(msg, eof);
4144         status.abort(msg);
4145       } catch (IOException ioe) {
4146         // If the IOE resulted from bad file format,
4147         // then this problem is idempotent and retrying won't help
4148         if (ioe.getCause() instanceof ParseException) {
4149           Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4150           msg = "File corruption encountered!  " +
4151               "Continuing, but renaming " + edits + " as " + p;
4152           LOG.warn(msg, ioe);
4153           status.setStatus(msg);
4154         } else {
4155           status.abort(StringUtils.stringifyException(ioe));
4156           // other IO errors may be transient (bad network connection,
4157           // checksum exception on one datanode, etc).  throw & retry
4158           throw ioe;
4159         }
4160       }
4161       if (reporter != null && !reported_once) {
4162         reporter.progress();
4163       }
4164       msg = "Applied " + editsCount + ", skipped " + skippedEdits +
4165         ", firstSequenceIdInLog=" + firstSeqIdInLog +
4166         ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
4167       status.markComplete(msg);
4168       LOG.debug(msg);
4169       return currentEditSeqId;
4170     } finally {
4171       status.cleanup();
4172       if (reader != null) {
4173          reader.close();
4174       }
4175     }
4176   }
4177 
4178   /**
4179    * Call to complete a compaction. Its for the case where we find in the WAL a compaction
4180    * that was not finished.  We could find one recovering a WAL after a regionserver crash.
4181    * See HBASE-2331.
4182    */
4183   void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
4184       boolean removeFiles, long replaySeqId)
4185       throws IOException {
4186     try {
4187       checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
4188         "Compaction marker from WAL ", compaction);
4189     } catch (WrongRegionException wre) {
4190       if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4191         // skip the compaction marker since it is not for this region
4192         return;
4193       }
4194       throw wre;
4195     }
4196 
4197     synchronized (writestate) {
4198       if (replaySeqId < lastReplayedOpenRegionSeqId) {
4199         LOG.warn(getRegionInfo().getEncodedName() + " : "
4200             + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4201             + " because its sequence id " + replaySeqId + " is smaller than this regions "
4202             + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
4203         return;
4204       }
4205       if (replaySeqId < lastReplayedCompactionSeqId) {
4206         LOG.warn(getRegionInfo().getEncodedName() + " : "
4207             + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4208             + " because its sequence id " + replaySeqId + " is smaller than this regions "
4209             + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId);
4210         return;
4211       } else {
4212         lastReplayedCompactionSeqId = replaySeqId;
4213       }
4214 
4215       if (LOG.isDebugEnabled()) {
4216         LOG.debug(getRegionInfo().getEncodedName() + " : "
4217             + "Replaying compaction marker " + TextFormat.shortDebugString(compaction)
4218             + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId="
4219             + lastReplayedOpenRegionSeqId);
4220       }
4221 
4222       startRegionOperation(Operation.REPLAY_EVENT);
4223       try {
4224         Store store = this.getStore(compaction.getFamilyName().toByteArray());
4225         if (store == null) {
4226           LOG.warn(getRegionInfo().getEncodedName() + " : "
4227               + "Found Compaction WAL edit for deleted family:"
4228               + Bytes.toString(compaction.getFamilyName().toByteArray()));
4229           return;
4230         }
4231         store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
4232         logRegionFiles();
4233       } catch (FileNotFoundException ex) {
4234         LOG.warn(getRegionInfo().getEncodedName() + " : "
4235             + "At least one of the store files in compaction: "
4236             + TextFormat.shortDebugString(compaction)
4237             + " doesn't exist any more. Skip loading the file(s)", ex);
4238       } finally {
4239         closeRegionOperation(Operation.REPLAY_EVENT);
4240       }
4241     }
4242   }
4243 
4244   void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
4245     checkTargetRegion(flush.getEncodedRegionName().toByteArray(),
4246       "Flush marker from WAL ", flush);
4247 
4248     if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4249       return; // if primary nothing to do
4250     }
4251 
4252     if (LOG.isDebugEnabled()) {
4253       LOG.debug(getRegionInfo().getEncodedName() + " : "
4254           + "Replaying flush marker " + TextFormat.shortDebugString(flush));
4255     }
4256 
4257     startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close
4258     try {
4259       FlushAction action = flush.getAction();
4260       switch (action) {
4261       case START_FLUSH:
4262         replayWALFlushStartMarker(flush);
4263         break;
4264       case COMMIT_FLUSH:
4265         replayWALFlushCommitMarker(flush);
4266         break;
4267       case ABORT_FLUSH:
4268         replayWALFlushAbortMarker(flush);
4269         break;
4270       case CANNOT_FLUSH:
4271         replayWALFlushCannotFlushMarker(flush, replaySeqId);
4272         break;
4273       default:
4274         LOG.warn(getRegionInfo().getEncodedName() + " : " +
4275           "Received a flush event with unknown action, ignoring. " +
4276           TextFormat.shortDebugString(flush));
4277         break;
4278       }
4279 
4280       logRegionFiles();
4281     } finally {
4282       closeRegionOperation(Operation.REPLAY_EVENT);
4283     }
4284   }
4285 
4286   /** Replay the flush marker from primary region by creating a corresponding snapshot of
4287    * the store memstores, only if the memstores do not have a higher seqId from an earlier wal
4288    * edit (because the events may be coming out of order).
4289    */
4290   @VisibleForTesting
4291   PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
4292     long flushSeqId = flush.getFlushSequenceNumber();
4293 
4294     HashSet<Store> storesToFlush = new HashSet<Store>();
4295     for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
4296       byte[] family = storeFlush.getFamilyName().toByteArray();
4297       Store store = getStore(family);
4298       if (store == null) {
4299         LOG.warn(getRegionInfo().getEncodedName() + " : "
4300           + "Received a flush start marker from primary, but the family is not found. Ignoring"
4301           + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
4302         continue;
4303       }
4304       storesToFlush.add(store);
4305     }
4306 
4307     MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
4308 
4309     // we will use writestate as a coarse-grain lock for all the replay events
4310     // (flush, compaction, region open etc)
4311     synchronized (writestate) {
4312       try {
4313         if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
4314           LOG.warn(getRegionInfo().getEncodedName() + " : "
4315               + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4316               + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4317               + " of " + lastReplayedOpenRegionSeqId);
4318           return null;
4319         }
4320         if (numMutationsWithoutWAL.get() > 0) {
4321           numMutationsWithoutWAL.set(0);
4322           dataInMemoryWithoutWAL.set(0);
4323         }
4324 
4325         if (!writestate.flushing) {
4326           // we do not have an active snapshot and corresponding this.prepareResult. This means
4327           // we can just snapshot our memstores and continue as normal.
4328 
4329           // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal
4330           PrepareFlushResult prepareResult = internalPrepareFlushCache(null,
4331             flushSeqId, storesToFlush, status, false);
4332           if (prepareResult.result == null) {
4333             // save the PrepareFlushResult so that we can use it later from commit flush
4334             this.writestate.flushing = true;
4335             this.prepareFlushResult = prepareResult;
4336             status.markComplete("Flush prepare successful");
4337             if (LOG.isDebugEnabled()) {
4338               LOG.debug(getRegionInfo().getEncodedName() + " : "
4339                   + " Prepared flush with seqId:" + flush.getFlushSequenceNumber());
4340             }
4341           } else {
4342             // special case empty memstore. We will still save the flush result in this case, since
4343             // our memstore ie empty, but the primary is still flushing
4344             if (prepareResult.getResult().getResult() ==
4345                   FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
4346               this.writestate.flushing = true;
4347               this.prepareFlushResult = prepareResult;
4348               if (LOG.isDebugEnabled()) {
4349                 LOG.debug(getRegionInfo().getEncodedName() + " : "
4350                   + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
4351               }
4352             }
4353             status.abort("Flush prepare failed with " + prepareResult.result);
4354             // nothing much to do. prepare flush failed because of some reason.
4355           }
4356           return prepareResult;
4357         } else {
4358           // we already have an active snapshot.
4359           if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
4360             // They define the same flush. Log and continue.
4361             LOG.warn(getRegionInfo().getEncodedName() + " : "
4362                 + "Received a flush prepare marker with the same seqId: " +
4363                 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4364                 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4365             // ignore
4366           } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
4367             // We received a flush with a smaller seqNum than what we have prepared. We can only
4368             // ignore this prepare flush request.
4369             LOG.warn(getRegionInfo().getEncodedName() + " : "
4370                 + "Received a flush prepare marker with a smaller seqId: " +
4371                 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4372                 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4373             // ignore
4374           } else {
4375             // We received a flush with a larger seqNum than what we have prepared
4376             LOG.warn(getRegionInfo().getEncodedName() + " : "
4377                 + "Received a flush prepare marker with a larger seqId: " +
4378                 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4379                 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4380             // We do not have multiple active snapshots in the memstore or a way to merge current
4381             // memstore snapshot with the contents and resnapshot for now. We cannot take
4382             // another snapshot and drop the previous one because that will cause temporary
4383             // data loss in the secondary. So we ignore this for now, deferring the resolution
4384             // to happen when we see the corresponding flush commit marker. If we have a memstore
4385             // snapshot with x, and later received another prepare snapshot with y (where x < y),
4386             // when we see flush commit for y, we will drop snapshot for x, and can also drop all
4387             // the memstore edits if everything in memstore is < y. This is the usual case for
4388             // RS crash + recovery where we might see consequtive prepare flush wal markers.
4389             // Otherwise, this will cause more memory to be used in secondary replica until a
4390             // further prapare + commit flush is seen and replayed.
4391           }
4392         }
4393       } finally {
4394         status.cleanup();
4395         writestate.notifyAll();
4396       }
4397     }
4398     return null;
4399   }
4400 
4401   @VisibleForTesting
4402   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4403     justification="Intentional; post memstore flush")
4404   void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
4405     MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
4406 
4407     // check whether we have the memstore snapshot with the corresponding seqId. Replay to
4408     // secondary region replicas are in order, except for when the region moves or then the
4409     // region server crashes. In those cases, we may receive replay requests out of order from
4410     // the original seqIds.
4411     synchronized (writestate) {
4412       try {
4413         if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
4414           LOG.warn(getRegionInfo().getEncodedName() + " : "
4415             + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4416             + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4417             + " of " + lastReplayedOpenRegionSeqId);
4418           return;
4419         }
4420 
4421         if (writestate.flushing) {
4422           PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
4423           if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
4424             if (LOG.isDebugEnabled()) {
4425               LOG.debug(getRegionInfo().getEncodedName() + " : "
4426                   + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
4427                   + " and a previous prepared snapshot was found");
4428             }
4429             // This is the regular case where we received commit flush after prepare flush
4430             // corresponding to the same seqId.
4431             replayFlushInStores(flush, prepareFlushResult, true);
4432 
4433             // Set down the memstore size by amount of flush.
4434             this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
4435 
4436             this.prepareFlushResult = null;
4437             writestate.flushing = false;
4438           } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
4439             // This should not happen normally. However, lets be safe and guard against these cases
4440             // we received a flush commit with a smaller seqId than what we have prepared
4441             // we will pick the flush file up from this commit (if we have not seen it), but we
4442             // will not drop the memstore
4443             LOG.warn(getRegionInfo().getEncodedName() + " : "
4444                 + "Received a flush commit marker with smaller seqId: "
4445                 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
4446                 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
4447                 +"  prepared memstore snapshot");
4448             replayFlushInStores(flush, prepareFlushResult, false);
4449 
4450             // snapshot is not dropped, so memstore sizes should not be decremented
4451             // we still have the prepared snapshot, flushing should still be true
4452           } else {
4453             // This should not happen normally. However, lets be safe and guard against these cases
4454             // we received a flush commit with a larger seqId than what we have prepared
4455             // we will pick the flush file for this. We will also obtain the updates lock and
4456             // look for contents of the memstore to see whether we have edits after this seqId.
4457             // If not, we will drop all the memstore edits and the snapshot as well.
4458             LOG.warn(getRegionInfo().getEncodedName() + " : "
4459                 + "Received a flush commit marker with larger seqId: "
4460                 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " +
4461                 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
4462                 +" memstore snapshot");
4463 
4464             replayFlushInStores(flush, prepareFlushResult, true);
4465 
4466             // Set down the memstore size by amount of flush.
4467             this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
4468 
4469             // Inspect the memstore contents to see whether the memstore contains only edits
4470             // with seqId smaller than the flush seqId. If so, we can discard those edits.
4471             dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
4472 
4473             this.prepareFlushResult = null;
4474             writestate.flushing = false;
4475           }
4476           // If we were waiting for observing a flush or region opening event for not showing
4477           // partial data after a secondary region crash, we can allow reads now. We can only make
4478           // sure that we are not showing partial data (for example skipping some previous edits)
4479           // until we observe a full flush start and flush commit. So if we were not able to find
4480           // a previous flush we will not enable reads now.
4481           this.setReadsEnabled(true);
4482         } else {
4483           LOG.warn(getRegionInfo().getEncodedName() + " : "
4484               + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
4485               + ", but no previous prepared snapshot was found");
4486           // There is no corresponding prepare snapshot from before.
4487           // We will pick up the new flushed file
4488           replayFlushInStores(flush, null, false);
4489 
4490           // Inspect the memstore contents to see whether the memstore contains only edits
4491           // with seqId smaller than the flush seqId. If so, we can discard those edits.
4492           dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
4493         }
4494 
4495         status.markComplete("Flush commit successful");
4496 
4497         // Update the last flushed sequence id for region.
4498         this.maxFlushedSeqId = flush.getFlushSequenceNumber();
4499 
4500         // advance the mvcc read point so that the new flushed file is visible.
4501         mvcc.advanceTo(flush.getFlushSequenceNumber());
4502 
4503       } catch (FileNotFoundException ex) {
4504         LOG.warn(getRegionInfo().getEncodedName() + " : "
4505             + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
4506             + " doesn't exist any more. Skip loading the file(s)", ex);
4507       }
4508       finally {
4509         status.cleanup();
4510         writestate.notifyAll();
4511       }
4512     }
4513 
4514     // C. Finally notify anyone waiting on memstore to clear:
4515     // e.g. checkResources().
4516     synchronized (this) {
4517       notifyAll(); // FindBugs NN_NAKED_NOTIFY
4518     }
4519   }
4520 
4521   /**
4522    * Replays the given flush descriptor by opening the flush files in stores and dropping the
4523    * memstore snapshots if requested.
4524    * @param flush
4525    * @param prepareFlushResult
4526    * @param dropMemstoreSnapshot
4527    * @throws IOException
4528    */
4529   private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
4530       boolean dropMemstoreSnapshot)
4531       throws IOException {
4532     for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
4533       byte[] family = storeFlush.getFamilyName().toByteArray();
4534       Store store = getStore(family);
4535       if (store == null) {
4536         LOG.warn(getRegionInfo().getEncodedName() + " : "
4537             + "Received a flush commit marker from primary, but the family is not found."
4538             + "Ignoring StoreFlushDescriptor:" + storeFlush);
4539         continue;
4540       }
4541       List<String> flushFiles = storeFlush.getFlushOutputList();
4542       StoreFlushContext ctx = null;
4543       long startTime = EnvironmentEdgeManager.currentTime();
4544       if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
4545         ctx = store.createFlushContext(flush.getFlushSequenceNumber());
4546       } else {
4547         ctx = prepareFlushResult.storeFlushCtxs.get(family);
4548         startTime = prepareFlushResult.startTime;
4549       }
4550 
4551       if (ctx == null) {
4552         LOG.warn(getRegionInfo().getEncodedName() + " : "
4553             + "Unexpected: flush commit marker received from store "
4554             + Bytes.toString(family) + " but no associated flush context. Ignoring");
4555         continue;
4556       }
4557 
4558       ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush
4559 
4560       // Record latest flush time
4561       this.lastStoreFlushTimeMap.put(store, startTime);
4562     }
4563   }
4564 
4565   /**
4566    * Drops the memstore contents after replaying a flush descriptor or region open event replay
4567    * if the memstore edits have seqNums smaller than the given seq id
4568    * @throws IOException
4569    */
4570   private long dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException {
4571     long totalFreedSize = 0;
4572     this.updatesLock.writeLock().lock();
4573     try {
4574 
4575       long currentSeqId = mvcc.getReadPoint();
4576       if (seqId >= currentSeqId) {
4577         // then we can drop the memstore contents since everything is below this seqId
4578         LOG.info(getRegionInfo().getEncodedName() + " : "
4579             + "Dropping memstore contents as well since replayed flush seqId: "
4580             + seqId + " is greater than current seqId:" + currentSeqId);
4581 
4582         // Prepare flush (take a snapshot) and then abort (drop the snapshot)
4583         if (store == null) {
4584           for (Store s : stores.values()) {
4585             totalFreedSize += doDropStoreMemstoreContentsForSeqId(s, currentSeqId);
4586           }
4587         } else {
4588           totalFreedSize += doDropStoreMemstoreContentsForSeqId(store, currentSeqId);
4589         }
4590       } else {
4591         LOG.info(getRegionInfo().getEncodedName() + " : "
4592             + "Not dropping memstore contents since replayed flush seqId: "
4593             + seqId + " is smaller than current seqId:" + currentSeqId);
4594       }
4595     } finally {
4596       this.updatesLock.writeLock().unlock();
4597     }
4598     return totalFreedSize;
4599   }
4600 
4601   private long doDropStoreMemstoreContentsForSeqId(Store s, long currentSeqId) throws IOException {
4602     long snapshotSize = s.getFlushableSize();
4603     this.addAndGetGlobalMemstoreSize(-snapshotSize);
4604     StoreFlushContext ctx = s.createFlushContext(currentSeqId);
4605     ctx.prepare();
4606     ctx.abort();
4607     return snapshotSize;
4608   }
4609 
4610   private void replayWALFlushAbortMarker(FlushDescriptor flush) {
4611     // nothing to do for now. A flush abort will cause a RS abort which means that the region
4612     // will be opened somewhere else later. We will see the region open event soon, and replaying
4613     // that will drop the snapshot
4614   }
4615 
4616   private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
4617     synchronized (writestate) {
4618       if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
4619         LOG.warn(getRegionInfo().getEncodedName() + " : "
4620           + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4621           + " because its sequence id " + replaySeqId + " is smaller than this regions "
4622           + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
4623         return;
4624       }
4625 
4626       // If we were waiting for observing a flush or region opening event for not showing partial
4627       // data after a secondary region crash, we can allow reads now. This event means that the
4628       // primary was not able to flush because memstore is empty when we requested flush. By the
4629       // time we observe this, we are guaranteed to have up to date seqId with our previous
4630       // assignment.
4631       this.setReadsEnabled(true);
4632     }
4633   }
4634 
4635   @VisibleForTesting
4636   PrepareFlushResult getPrepareFlushResult() {
4637     return prepareFlushResult;
4638   }
4639 
4640   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4641       justification="Intentional; cleared the memstore")
4642   void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
4643     checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
4644       "RegionEvent marker from WAL ", regionEvent);
4645 
4646     startRegionOperation(Operation.REPLAY_EVENT);
4647     try {
4648       if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4649         return; // if primary nothing to do
4650       }
4651 
4652       if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
4653         // nothing to do on REGION_CLOSE for now.
4654         return;
4655       }
4656       if (regionEvent.getEventType() != EventType.REGION_OPEN) {
4657         LOG.warn(getRegionInfo().getEncodedName() + " : "
4658             + "Unknown region event received, ignoring :"
4659             + TextFormat.shortDebugString(regionEvent));
4660         return;
4661       }
4662 
4663       if (LOG.isDebugEnabled()) {
4664         LOG.debug(getRegionInfo().getEncodedName() + " : "
4665           + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent));
4666       }
4667 
4668       // we will use writestate as a coarse-grain lock for all the replay events
4669       synchronized (writestate) {
4670         // Replication can deliver events out of order when primary region moves or the region
4671         // server crashes, since there is no coordination between replication of different wal files
4672         // belonging to different region servers. We have to safe guard against this case by using
4673         // region open event's seqid. Since this is the first event that the region puts (after
4674         // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
4675         // smaller than this seqId
4676         if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
4677           this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
4678         } else {
4679           LOG.warn(getRegionInfo().getEncodedName() + " : "
4680             + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent)
4681             + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4682             + " of " + lastReplayedOpenRegionSeqId);
4683           return;
4684         }
4685 
4686         // region open lists all the files that the region has at the time of the opening. Just pick
4687         // all the files and drop prepared flushes and empty memstores
4688         for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
4689           // stores of primary may be different now
4690           byte[] family = storeDescriptor.getFamilyName().toByteArray();
4691           Store store = getStore(family);
4692           if (store == null) {
4693             LOG.warn(getRegionInfo().getEncodedName() + " : "
4694                 + "Received a region open marker from primary, but the family is not found. "
4695                 + "Ignoring. StoreDescriptor:" + storeDescriptor);
4696             continue;
4697           }
4698 
4699           long storeSeqId = store.getMaxSequenceId();
4700           List<String> storeFiles = storeDescriptor.getStoreFileList();
4701           try {
4702             store.refreshStoreFiles(storeFiles); // replace the files with the new ones
4703           } catch (FileNotFoundException ex) {
4704             LOG.warn(getRegionInfo().getEncodedName() + " : "
4705                     + "At least one of the store files: " + storeFiles
4706                     + " doesn't exist any more. Skip loading the file(s)", ex);
4707             continue;
4708           }
4709           if (store.getMaxSequenceId() != storeSeqId) {
4710             // Record latest flush time if we picked up new files
4711             lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
4712           }
4713 
4714           if (writestate.flushing) {
4715             // only drop memstore snapshots if they are smaller than last flush for the store
4716             if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
4717               StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
4718                   null : this.prepareFlushResult.storeFlushCtxs.get(family);
4719               if (ctx != null) {
4720                 long snapshotSize = store.getFlushableSize();
4721                 ctx.abort();
4722                 this.addAndGetGlobalMemstoreSize(-snapshotSize);
4723                 this.prepareFlushResult.storeFlushCtxs.remove(family);
4724               }
4725             }
4726           }
4727 
4728           // Drop the memstore contents if they are now smaller than the latest seen flushed file
4729           dropMemstoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
4730           if (storeSeqId > this.maxFlushedSeqId) {
4731             this.maxFlushedSeqId = storeSeqId;
4732           }
4733         }
4734 
4735         // if all stores ended up dropping their snapshots, we can safely drop the
4736         // prepareFlushResult
4737         dropPrepareFlushIfPossible();
4738 
4739         // advance the mvcc read point so that the new flushed file is visible.
4740         mvcc.await();
4741 
4742         // If we were waiting for observing a flush or region opening event for not showing partial
4743         // data after a secondary region crash, we can allow reads now.
4744         this.setReadsEnabled(true);
4745 
4746         // C. Finally notify anyone waiting on memstore to clear:
4747         // e.g. checkResources().
4748         synchronized (this) {
4749           notifyAll(); // FindBugs NN_NAKED_NOTIFY
4750         }
4751       }
4752       logRegionFiles();
4753     } finally {
4754       closeRegionOperation(Operation.REPLAY_EVENT);
4755     }
4756   }
4757 
4758   void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
4759     checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
4760       "BulkLoad marker from WAL ", bulkLoadEvent);
4761 
4762     if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4763       return; // if primary nothing to do
4764     }
4765 
4766     if (LOG.isDebugEnabled()) {
4767       LOG.debug(getRegionInfo().getEncodedName() + " : "
4768               +  "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent));
4769     }
4770     // check if multiple families involved
4771     boolean multipleFamilies = false;
4772     byte[] family = null;
4773     for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
4774       byte[] fam = storeDescriptor.getFamilyName().toByteArray();
4775       if (family == null) {
4776         family = fam;
4777       } else if (!Bytes.equals(family, fam)) {
4778         multipleFamilies = true;
4779         break;
4780       }
4781     }
4782 
4783     startBulkRegionOperation(multipleFamilies);
4784     try {
4785       // we will use writestate as a coarse-grain lock for all the replay events
4786       synchronized (writestate) {
4787         // Replication can deliver events out of order when primary region moves or the region
4788         // server crashes, since there is no coordination between replication of different wal files
4789         // belonging to different region servers. We have to safe guard against this case by using
4790         // region open event's seqid. Since this is the first event that the region puts (after
4791         // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
4792         // smaller than this seqId
4793         if (bulkLoadEvent.getBulkloadSeqNum() >= 0
4794             && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) {
4795           LOG.warn(getRegionInfo().getEncodedName() + " : "
4796               + "Skipping replaying bulkload event :"
4797               + TextFormat.shortDebugString(bulkLoadEvent)
4798               + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
4799               + " =" + lastReplayedOpenRegionSeqId);
4800 
4801           return;
4802         }
4803 
4804         for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
4805           // stores of primary may be different now
4806           family = storeDescriptor.getFamilyName().toByteArray();
4807           Store store = getStore(family);
4808           if (store == null) {
4809             LOG.warn(getRegionInfo().getEncodedName() + " : "
4810                     + "Received a bulk load marker from primary, but the family is not found. "
4811                     + "Ignoring. StoreDescriptor:" + storeDescriptor);
4812             continue;
4813           }
4814 
4815           List<String> storeFiles = storeDescriptor.getStoreFileList();
4816           for (String storeFile : storeFiles) {
4817             StoreFileInfo storeFileInfo = null;
4818             try {
4819               storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile);
4820               store.bulkLoadHFile(storeFileInfo);
4821             } catch(FileNotFoundException ex) {
4822               LOG.warn(getRegionInfo().getEncodedName() + " : "
4823                       + ((storeFileInfo != null) ? storeFileInfo.toString() :
4824                             (new Path(Bytes.toString(family), storeFile)).toString())
4825                       + " doesn't exist any more. Skip loading the file");
4826             }
4827           }
4828         }
4829       }
4830       if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
4831         mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
4832       }
4833     } finally {
4834       closeBulkRegionOperation();
4835     }
4836   }
4837 
4838   /**
4839    * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult
4840    */
4841   private void dropPrepareFlushIfPossible() {
4842     if (writestate.flushing) {
4843       boolean canDrop = true;
4844       if (prepareFlushResult.storeFlushCtxs != null) {
4845         for (Entry<byte[], StoreFlushContext> entry
4846             : prepareFlushResult.storeFlushCtxs.entrySet()) {
4847           Store store = getStore(entry.getKey());
4848           if (store == null) {
4849             continue;
4850           }
4851           if (store.getSnapshotSize() > 0) {
4852             canDrop = false;
4853             break;
4854           }
4855         }
4856       }
4857 
4858       // this means that all the stores in the region has finished flushing, but the WAL marker
4859       // may not have been written or we did not receive it yet.
4860       if (canDrop) {
4861         writestate.flushing = false;
4862         this.prepareFlushResult = null;
4863       }
4864     }
4865   }
4866 
4867   @Override
4868   public boolean refreshStoreFiles() throws IOException {
4869     return refreshStoreFiles(false);
4870   }
4871 
4872   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4873       justification="Notify is about post replay. Intentional")
4874   protected boolean refreshStoreFiles(boolean force) throws IOException {
4875     if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4876       return false; // if primary nothing to do
4877     }
4878 
4879     if (LOG.isDebugEnabled()) {
4880       LOG.debug(getRegionInfo().getEncodedName() + " : "
4881           + "Refreshing store files to see whether we can free up memstore");
4882     }
4883 
4884     long totalFreedSize = 0;
4885 
4886     long smallestSeqIdInStores = Long.MAX_VALUE;
4887 
4888     startRegionOperation(); // obtain region close lock
4889     try {
4890       Map<Store, Long> map = new HashMap<Store, Long>();
4891       synchronized (writestate) {
4892         for (Store store : getStores()) {
4893           // TODO: some stores might see new data from flush, while others do not which
4894           // MIGHT break atomic edits across column families.
4895           long maxSeqIdBefore = store.getMaxSequenceId();
4896 
4897           // refresh the store files. This is similar to observing a region open wal marker.
4898           store.refreshStoreFiles();
4899 
4900           long storeSeqId = store.getMaxSequenceId();
4901           if (storeSeqId < smallestSeqIdInStores) {
4902             smallestSeqIdInStores = storeSeqId;
4903           }
4904 
4905           // see whether we can drop the memstore or the snapshot
4906           if (storeSeqId > maxSeqIdBefore) {
4907 
4908             if (writestate.flushing) {
4909               // only drop memstore snapshots if they are smaller than last flush for the store
4910               if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
4911                 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
4912                     null : this.prepareFlushResult.storeFlushCtxs.get(store.getFamily().getName());
4913                 if (ctx != null) {
4914                   long snapshotSize = store.getFlushableSize();
4915                   ctx.abort();
4916                   this.addAndGetGlobalMemstoreSize(-snapshotSize);
4917                   this.prepareFlushResult.storeFlushCtxs.remove(store.getFamily().getName());
4918                   totalFreedSize += snapshotSize;
4919                 }
4920               }
4921             }
4922 
4923             map.put(store, storeSeqId);
4924           }
4925         }
4926 
4927         // if all stores ended up dropping their snapshots, we can safely drop the
4928         // prepareFlushResult
4929         dropPrepareFlushIfPossible();
4930 
4931         // advance the mvcc read point so that the new flushed files are visible.
4932           // either greater than flush seq number or they were already picked up via flush.
4933           for (Store s : getStores()) {
4934             mvcc.advanceTo(s.getMaxMemstoreTS());
4935           }
4936 
4937 
4938         // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely
4939         // skip all edits that are to be replayed in the future with that has a smaller seqId
4940         // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits
4941         // that we have picked the flush files for
4942         if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
4943           this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
4944         }
4945       }
4946       if (!map.isEmpty()) {
4947         if (!force) {
4948           for (Map.Entry<Store, Long> entry : map.entrySet()) {
4949             // Drop the memstore contents if they are now smaller than the latest seen flushed file
4950             totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey());
4951           }
4952         } else {
4953           synchronized (storeSeqIds) {
4954             // don't try to acquire write lock of updatesLock now
4955             storeSeqIds.add(map);
4956           }
4957         }
4958       }
4959       // C. Finally notify anyone waiting on memstore to clear:
4960       // e.g. checkResources().
4961       synchronized (this) {
4962         notifyAll(); // FindBugs NN_NAKED_NOTIFY
4963       }
4964       return totalFreedSize > 0;
4965     } finally {
4966       closeRegionOperation();
4967     }
4968   }
4969 
4970   private void logRegionFiles() {
4971     if (LOG.isTraceEnabled()) {
4972       LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
4973       for (Store s : stores.values()) {
4974         Collection<StoreFile> storeFiles = s.getStorefiles();
4975         if (storeFiles == null) continue;
4976         for (StoreFile sf : storeFiles) {
4977           LOG.trace(getRegionInfo().getEncodedName() + " : " + sf);
4978         }
4979       }
4980     }
4981   }
4982 
4983   /** Checks whether the given regionName is either equal to our region, or that
4984    * the regionName is the primary region to our corresponding range for the secondary replica.
4985    */
4986   private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
4987       throws WrongRegionException {
4988     if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
4989       return;
4990     }
4991 
4992     if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) &&
4993         Bytes.equals(encodedRegionName,
4994           this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) {
4995       return;
4996     }
4997 
4998     throw new WrongRegionException(exceptionMsg + payload
4999       + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
5000       + " does not match this region: " + this.getRegionInfo());
5001   }
5002 
5003   /**
5004    * Used by tests
5005    * @param s Store to add edit too.
5006    * @param cell Cell to add.
5007    * @return True if we should flush.
5008    */
5009   protected boolean restoreEdit(final Store s, final Cell cell) {
5010     long kvSize = s.add(cell);
5011     if (this.rsAccounting != null) {
5012       rsAccounting.addAndGetRegionReplayEditsSize(getRegionInfo().getRegionName(), kvSize);
5013     }
5014     return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
5015   }
5016 
5017   /*
5018    * @param fs
5019    * @param p File to check.
5020    * @return True if file was zero-length (and if so, we'll delete it in here).
5021    * @throws IOException
5022    */
5023   private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
5024       throws IOException {
5025     FileStatus stat = fs.getFileStatus(p);
5026     if (stat.getLen() > 0) return false;
5027     LOG.warn("File " + p + " is zero-length, deleting.");
5028     fs.delete(p, false);
5029     return true;
5030   }
5031 
5032   protected HStore instantiateHStore(final HColumnDescriptor family) throws IOException {
5033     return new HStore(this, family, this.conf);
5034   }
5035 
5036   @Override
5037   public Store getStore(final byte[] column) {
5038     return this.stores.get(column);
5039   }
5040 
5041   /**
5042    * Return HStore instance. Does not do any copy: as the number of store is limited, we
5043    *  iterate on the list.
5044    */
5045   private Store getStore(Cell cell) {
5046     for (Map.Entry<byte[], Store> famStore : stores.entrySet()) {
5047       if (Bytes.equals(
5048           cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
5049           famStore.getKey(), 0, famStore.getKey().length)) {
5050         return famStore.getValue();
5051       }
5052     }
5053 
5054     return null;
5055   }
5056 
5057   @Override
5058   public List<Store> getStores() {
5059     List<Store> list = new ArrayList<Store>(stores.size());
5060     list.addAll(stores.values());
5061     return list;
5062   }
5063 
5064   @Override
5065   public List<String> getStoreFileList(final byte [][] columns)
5066     throws IllegalArgumentException {
5067     List<String> storeFileNames = new ArrayList<String>();
5068     synchronized(closeLock) {
5069       for(byte[] column : columns) {
5070         Store store = this.stores.get(column);
5071         if (store == null) {
5072           throw new IllegalArgumentException("No column family : " +
5073               new String(column) + " available");
5074         }
5075         Collection<StoreFile> storeFiles = store.getStorefiles();
5076         if (storeFiles == null) continue;
5077         for (StoreFile storeFile: storeFiles) {
5078           storeFileNames.add(storeFile.getPath().toString());
5079         }
5080 
5081         logRegionFiles();
5082       }
5083     }
5084     return storeFileNames;
5085   }
5086 
5087   //////////////////////////////////////////////////////////////////////////////
5088   // Support code
5089   //////////////////////////////////////////////////////////////////////////////
5090 
5091   /** Make sure this is a valid row for the HRegion */
5092   void checkRow(final byte [] row, String op) throws IOException {
5093     if (!rowIsInRange(getRegionInfo(), row)) {
5094       throw new WrongRegionException("Requested row out of range for " +
5095           op + " on HRegion " + this + ", startKey='" +
5096           Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" +
5097           Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" +
5098           Bytes.toStringBinary(row) + "'");
5099     }
5100   }
5101 
5102 
5103   /**
5104    * Get an exclusive ( write lock ) lock on a given row.
5105    * @param row Which row to lock.
5106    * @return A locked RowLock. The lock is exclusive and already aqquired.
5107    * @throws IOException
5108    */
5109   public RowLock getRowLock(byte[] row) throws IOException {
5110     return getRowLock(row, false);
5111   }
5112 
5113   /**
5114    *
5115    * Get a row lock for the specified row. All locks are reentrant.
5116    *
5117    * Before calling this function make sure that a region operation has already been
5118    * started (the calling thread has already acquired the region-close-guard lock).
5119    * @param row The row actions will be performed against
5120    * @param readLock is the lock reader or writer. True indicates that a non-exlcusive
5121    *                 lock is requested
5122    */
5123   public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
5124     // Make sure the row is inside of this region before getting the lock for it.
5125     checkRow(row, "row lock");
5126     // create an object to use a a key in the row lock map
5127     HashedBytes rowKey = new HashedBytes(row);
5128 
5129     RowLockContext rowLockContext = null;
5130     RowLockImpl result = null;
5131     TraceScope traceScope = null;
5132 
5133     // If we're tracing start a span to show how long this took.
5134     if (Trace.isTracing()) {
5135       traceScope = Trace.startSpan("HRegion.getRowLock");
5136       traceScope.getSpan().addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock"));
5137     }
5138 
5139     try {
5140       // Keep trying until we have a lock or error out.
5141       // TODO: do we need to add a time component here?
5142       while (result == null) {
5143 
5144         // Try adding a RowLockContext to the lockedRows.
5145         // If we can add it then there's no other transactions currently running.
5146         rowLockContext = new RowLockContext(rowKey);
5147         RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext);
5148 
5149         // if there was a running transaction then there's already a context.
5150         if (existingContext != null) {
5151           rowLockContext = existingContext;
5152         }
5153 
5154         // Now try an get the lock.
5155         //
5156         // This can fail as
5157         if (readLock) {
5158           result = rowLockContext.newReadLock();
5159         } else {
5160           result = rowLockContext.newWriteLock();
5161         }
5162       }
5163       if (!result.getLock().tryLock(this.rowLockWaitDuration, TimeUnit.MILLISECONDS)) {
5164         if (traceScope != null) {
5165           traceScope.getSpan().addTimelineAnnotation("Failed to get row lock");
5166         }
5167         result = null;
5168         // Clean up the counts just in case this was the thing keeping the context alive.
5169         rowLockContext.cleanUp();
5170         throw new IOException("Timed out waiting for lock for row: " + rowKey);
5171       }
5172       rowLockContext.setThreadName(Thread.currentThread().getName());
5173       return result;
5174     } catch (InterruptedException ie) {
5175       LOG.warn("Thread interrupted waiting for lock on row: " + rowKey);
5176       InterruptedIOException iie = new InterruptedIOException();
5177       iie.initCause(ie);
5178       if (traceScope != null) {
5179         traceScope.getSpan().addTimelineAnnotation("Interrupted exception getting row lock");
5180       }
5181       Thread.currentThread().interrupt();
5182       throw iie;
5183     } finally {
5184       if (traceScope != null) {
5185         traceScope.close();
5186       }
5187     }
5188   }
5189 
5190   @Override
5191   public void releaseRowLocks(List<RowLock> rowLocks) {
5192     if (rowLocks != null) {
5193       for (RowLock rowLock : rowLocks) {
5194         rowLock.release();
5195       }
5196       rowLocks.clear();
5197     }
5198   }
5199 
5200   public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
5201     return lockedRows;
5202   }
5203 
5204   @VisibleForTesting
5205   class RowLockContext {
5206     private final HashedBytes row;
5207     final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
5208     final AtomicBoolean usable = new AtomicBoolean(true);
5209     final AtomicInteger count = new AtomicInteger(0);
5210     final Object lock = new Object();
5211     private String threadName;
5212 
5213     RowLockContext(HashedBytes row) {
5214       this.row = row;
5215     }
5216 
5217     RowLockImpl newWriteLock() {
5218       Lock l = readWriteLock.writeLock();
5219       return getRowLock(l);
5220     }
5221     RowLockImpl newReadLock() {
5222       Lock l = readWriteLock.readLock();
5223       return getRowLock(l);
5224     }
5225 
5226     private RowLockImpl getRowLock(Lock l) {
5227       count.incrementAndGet();
5228       synchronized (lock) {
5229         if (usable.get()) {
5230           return new RowLockImpl(this, l);
5231         } else {
5232           return null;
5233         }
5234       }
5235     }
5236 
5237     void cleanUp() {
5238       long c = count.decrementAndGet();
5239       if (c <= 0) {
5240         synchronized (lock) {
5241           if (count.get() <= 0 ){
5242             usable.set(false);
5243             RowLockContext removed = lockedRows.remove(row);
5244             assert removed == this: "we should never remove a different context";
5245           }
5246         }
5247       }
5248     }
5249 
5250     public void setThreadName(String threadName) {
5251       this.threadName = threadName;
5252     }
5253 
5254     @Override
5255     public String toString() {
5256       return "RowLockContext{" +
5257           "row=" + row +
5258           ", readWriteLock=" + readWriteLock +
5259           ", count=" + count +
5260           ", threadName=" + threadName +
5261           '}';
5262     }
5263   }
5264 
5265   /**
5266    * Class used to represent a lock on a row.
5267    */
5268   public static class RowLockImpl implements RowLock {
5269     private final RowLockContext context;
5270     private final Lock lock;
5271 
5272     public RowLockImpl(RowLockContext context, Lock lock) {
5273       this.context = context;
5274       this.lock = lock;
5275     }
5276 
5277     public Lock getLock() {
5278       return lock;
5279     }
5280 
5281     @VisibleForTesting
5282     public RowLockContext getContext() {
5283       return context;
5284     }
5285 
5286     @Override
5287     public void release() {
5288       lock.unlock();
5289       context.cleanUp();
5290     }
5291 
5292     @Override
5293     public String toString() {
5294       return "RowLockImpl{" +
5295           "context=" + context +
5296           ", lock=" + lock +
5297           '}';
5298     }
5299   }
5300 
5301   /**
5302    * Determines whether multiple column families are present
5303    * Precondition: familyPaths is not null
5304    *
5305    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
5306    */
5307   private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
5308     boolean multipleFamilies = false;
5309     byte[] family = null;
5310     for (Pair<byte[], String> pair : familyPaths) {
5311       byte[] fam = pair.getFirst();
5312       if (family == null) {
5313         family = fam;
5314       } else if (!Bytes.equals(family, fam)) {
5315         multipleFamilies = true;
5316         break;
5317       }
5318     }
5319     return multipleFamilies;
5320   }
5321 
5322   @Override
5323   public boolean bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
5324       BulkLoadListener bulkLoadListener) throws IOException {
5325     long seqId = -1;
5326     Map<byte[], List<Path>> storeFiles = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
5327     Preconditions.checkNotNull(familyPaths);
5328     // we need writeLock for multi-family bulk load
5329     startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
5330     try {
5331       this.writeRequestsCount.increment();
5332 
5333       // There possibly was a split that happened between when the split keys
5334       // were gathered and before the HRegion's write lock was taken.  We need
5335       // to validate the HFile region before attempting to bulk load all of them
5336       List<IOException> ioes = new ArrayList<IOException>();
5337       List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
5338       for (Pair<byte[], String> p : familyPaths) {
5339         byte[] familyName = p.getFirst();
5340         String path = p.getSecond();
5341 
5342         Store store = getStore(familyName);
5343         if (store == null) {
5344           IOException ioe = new org.apache.hadoop.hbase.DoNotRetryIOException(
5345               "No such column family " + Bytes.toStringBinary(familyName));
5346           ioes.add(ioe);
5347         } else {
5348           try {
5349             store.assertBulkLoadHFileOk(new Path(path));
5350           } catch (WrongRegionException wre) {
5351             // recoverable (file doesn't fit in region)
5352             failures.add(p);
5353           } catch (IOException ioe) {
5354             // unrecoverable (hdfs problem)
5355             ioes.add(ioe);
5356           }
5357         }
5358       }
5359 
5360       // validation failed because of some sort of IO problem.
5361       if (ioes.size() != 0) {
5362         IOException e = MultipleIOException.createIOException(ioes);
5363         LOG.error("There were one or more IO errors when checking if the bulk load is ok.", e);
5364         throw e;
5365       }
5366 
5367       // validation failed, bail out before doing anything permanent.
5368       if (failures.size() != 0) {
5369         StringBuilder list = new StringBuilder();
5370         for (Pair<byte[], String> p : failures) {
5371           list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
5372               .append(p.getSecond());
5373         }
5374         // problem when validating
5375         LOG.warn("There was a recoverable bulk load failure likely due to a" +
5376             " split.  These (family, HFile) pairs were not loaded: " + list);
5377         return false;
5378       }
5379 
5380       // We need to assign a sequential ID that's in between two memstores in order to preserve
5381       // the guarantee that all the edits lower than the highest sequential ID from all the
5382       // HFiles are flushed on disk. See HBASE-10958.  The sequence id returned when we flush is
5383       // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is
5384       // a sequence id that we can be sure is beyond the last hfile written).
5385       if (assignSeqId) {
5386         FlushResult fs = flushcache(true, false);
5387         if (fs.isFlushSucceeded()) {
5388           seqId = ((FlushResultImpl)fs).flushSequenceId;
5389         } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
5390           seqId = ((FlushResultImpl)fs).flushSequenceId;
5391         } else {
5392           throw new IOException("Could not bulk load with an assigned sequential ID because the "+
5393             "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason);
5394         }
5395       }
5396 
5397       for (Pair<byte[], String> p : familyPaths) {
5398         byte[] familyName = p.getFirst();
5399         String path = p.getSecond();
5400         Store store = getStore(familyName);
5401         try {
5402           String finalPath = path;
5403           if (bulkLoadListener != null) {
5404             finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
5405           }
5406           Path commitedStoreFile = store.bulkLoadHFile(finalPath, seqId);
5407 
5408           if(storeFiles.containsKey(familyName)) {
5409             storeFiles.get(familyName).add(commitedStoreFile);
5410           } else {
5411             List<Path> storeFileNames = new ArrayList<Path>();
5412             storeFileNames.add(commitedStoreFile);
5413             storeFiles.put(familyName, storeFileNames);
5414           }
5415           if (bulkLoadListener != null) {
5416             bulkLoadListener.doneBulkLoad(familyName, path);
5417           }
5418         } catch (IOException ioe) {
5419           // A failure here can cause an atomicity violation that we currently
5420           // cannot recover from since it is likely a failed HDFS operation.
5421 
5422           // TODO Need a better story for reverting partial failures due to HDFS.
5423           LOG.error("There was a partial failure due to IO when attempting to" +
5424               " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
5425           if (bulkLoadListener != null) {
5426             try {
5427               bulkLoadListener.failedBulkLoad(familyName, path);
5428             } catch (Exception ex) {
5429               LOG.error("Error while calling failedBulkLoad for family " +
5430                   Bytes.toString(familyName) + " with path " + path, ex);
5431             }
5432           }
5433           throw ioe;
5434         }
5435       }
5436 
5437       return true;
5438     } finally {
5439       if (wal != null && !storeFiles.isEmpty()) {
5440         // write a bulk load event when not all hfiles are loaded
5441         try {
5442           WALProtos.BulkLoadDescriptor loadDescriptor = ProtobufUtil.toBulkLoadDescriptor(
5443               this.getRegionInfo().getTable(),
5444               ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles, seqId);
5445           WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(),
5446               loadDescriptor, mvcc);
5447         } catch (IOException ioe) {
5448           if (this.rsServices != null) {
5449             // Have to abort region server because some hfiles has been loaded but we can't write
5450             // the event into WAL
5451             this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
5452           }
5453         }
5454       }
5455 
5456       closeBulkRegionOperation();
5457     }
5458   }
5459 
5460   @Override
5461   public boolean equals(Object o) {
5462     return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
5463                                                 ((HRegion) o).getRegionInfo().getRegionName());
5464   }
5465 
5466   @Override
5467   public int hashCode() {
5468     return Bytes.hashCode(getRegionInfo().getRegionName());
5469   }
5470 
5471   @Override
5472   public String toString() {
5473     return getRegionInfo().getRegionNameAsString();
5474   }
5475 
5476   /**
5477    * RegionScannerImpl is used to combine scanners from multiple Stores (aka column families).
5478    */
5479   class RegionScannerImpl implements RegionScanner {
5480     // Package local for testability
5481     KeyValueHeap storeHeap = null;
5482     /** Heap of key-values that are not essential for the provided filters and are thus read
5483      * on demand, if on-demand column family loading is enabled.*/
5484     KeyValueHeap joinedHeap = null;
5485     /**
5486      * If the joined heap data gathering is interrupted due to scan limits, this will
5487      * contain the row for which we are populating the values.*/
5488     protected Cell joinedContinuationRow = null;
5489     private boolean filterClosed = false;
5490 
5491     protected final int isScan;
5492     protected final byte[] stopRow;
5493     protected final HRegion region;
5494 
5495     private final long readPt;
5496     private final long maxResultSize;
5497     private final ScannerContext defaultScannerContext;
5498     private final FilterWrapper filter;
5499 
5500     @Override
5501     public HRegionInfo getRegionInfo() {
5502       return region.getRegionInfo();
5503     }
5504 
5505     RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
5506         throws IOException {
5507       this.region = region;
5508       this.maxResultSize = scan.getMaxResultSize();
5509       if (scan.hasFilter()) {
5510         this.filter = new FilterWrapper(scan.getFilter());
5511       } else {
5512         this.filter = null;
5513       }
5514 
5515       /**
5516        * By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default
5517        * scanner context that can be used to enforce the batch limit in the event that a
5518        * ScannerContext is not specified during an invocation of next/nextRaw
5519        */
5520       defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build();
5521 
5522       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) && !scan.isGetScan()) {
5523         this.stopRow = null;
5524       } else {
5525         this.stopRow = scan.getStopRow();
5526       }
5527       // If we are doing a get, we want to be [startRow,endRow] normally
5528       // it is [startRow,endRow) and if startRow=endRow we get nothing.
5529       this.isScan = scan.isGetScan() ? -1 : 0;
5530 
5531       // synchronize on scannerReadPoints so that nobody calculates
5532       // getSmallestReadPoint, before scannerReadPoints is updated.
5533       IsolationLevel isolationLevel = scan.getIsolationLevel();
5534       synchronized(scannerReadPoints) {
5535         this.readPt = getReadpoint(isolationLevel);
5536         scannerReadPoints.put(this, this.readPt);
5537       }
5538 
5539       // Here we separate all scanners into two lists - scanner that provide data required
5540       // by the filter to operate (scanners list) and all others (joinedScanners list).
5541       List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
5542       List<KeyValueScanner> joinedScanners = new ArrayList<KeyValueScanner>();
5543       // Store all already instantiated scanners for exception handling
5544       List<KeyValueScanner> instantiatedScanners = new ArrayList<KeyValueScanner>();
5545       // handle additionalScanners
5546       if (additionalScanners != null && !additionalScanners.isEmpty()) {
5547         scanners.addAll(additionalScanners);
5548         instantiatedScanners.addAll(additionalScanners);
5549       }
5550 
5551       try {
5552         for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
5553           Store store = stores.get(entry.getKey());
5554           KeyValueScanner scanner;
5555           try {
5556             scanner = store.getScanner(scan, entry.getValue(), this.readPt);
5557           } catch (FileNotFoundException e) {
5558             throw handleFileNotFound(e);
5559           }
5560           instantiatedScanners.add(scanner);
5561           if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
5562               || this.filter.isFamilyEssential(entry.getKey())) {
5563             scanners.add(scanner);
5564           } else {
5565             joinedScanners.add(scanner);
5566           }
5567         }
5568         initializeKVHeap(scanners, joinedScanners, region);
5569       } catch (Throwable t) {
5570         throw handleException(instantiatedScanners, t);
5571       }
5572     }
5573 
5574     protected void initializeKVHeap(List<KeyValueScanner> scanners,
5575         List<KeyValueScanner> joinedScanners, HRegion region)
5576         throws IOException {
5577       this.storeHeap = new KeyValueHeap(scanners, region.comparator);
5578       if (!joinedScanners.isEmpty()) {
5579         this.joinedHeap = new KeyValueHeap(joinedScanners, region.comparator);
5580       }
5581     }
5582 
5583     private IOException handleException(List<KeyValueScanner> instantiatedScanners,
5584         Throwable t) {
5585       // remove scaner read point before throw the exception
5586       scannerReadPoints.remove(this);
5587       if (storeHeap != null) {
5588         storeHeap.close();
5589         storeHeap = null;
5590         if (joinedHeap != null) {
5591           joinedHeap.close();
5592           joinedHeap = null;
5593         }
5594       } else {
5595         // close all already instantiated scanners before throwing the exception
5596         for (KeyValueScanner scanner : instantiatedScanners) {
5597           scanner.close();
5598         }
5599       }
5600       return t instanceof IOException ? (IOException) t : new IOException(t);
5601     }
5602 
5603     @Override
5604     public long getMaxResultSize() {
5605       return maxResultSize;
5606     }
5607 
5608     @Override
5609     public long getMvccReadPoint() {
5610       return this.readPt;
5611     }
5612 
5613     @Override
5614     public int getBatch() {
5615       return this.defaultScannerContext.getBatchLimit();
5616     }
5617 
5618     /**
5619      * Reset both the filter and the old filter.
5620      *
5621      * @throws IOException in case a filter raises an I/O exception.
5622      */
5623     protected void resetFilters() throws IOException {
5624       if (filter != null) {
5625         filter.reset();
5626       }
5627     }
5628 
5629     @Override
5630     public boolean next(List<Cell> outResults)
5631         throws IOException {
5632       // apply the batching limit by default
5633       return next(outResults, defaultScannerContext);
5634     }
5635 
5636     @Override
5637     public synchronized boolean next(List<Cell> outResults, ScannerContext scannerContext)
5638     throws IOException {
5639       if (this.filterClosed) {
5640         throw new UnknownScannerException("Scanner was closed (timed out?) " +
5641             "after we renewed it. Could be caused by a very slow scanner " +
5642             "or a lengthy garbage collection");
5643       }
5644       startRegionOperation(Operation.SCAN);
5645       readRequestsCount.increment();
5646       try {
5647         return nextRaw(outResults, scannerContext);
5648       } finally {
5649         closeRegionOperation(Operation.SCAN);
5650       }
5651     }
5652 
5653     @Override
5654     public boolean nextRaw(List<Cell> outResults) throws IOException {
5655       // Use the RegionScanner's context by default
5656       return nextRaw(outResults, defaultScannerContext);
5657     }
5658 
5659     @Override
5660     public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext)
5661         throws IOException {
5662       if (storeHeap == null) {
5663         // scanner is closed
5664         throw new UnknownScannerException("Scanner was closed");
5665       }
5666       boolean moreValues;
5667       if (outResults.isEmpty()) {
5668         // Usually outResults is empty. This is true when next is called
5669         // to handle scan or get operation.
5670         moreValues = nextInternal(outResults, scannerContext);
5671       } else {
5672         List<Cell> tmpList = new ArrayList<Cell>();
5673         moreValues = nextInternal(tmpList, scannerContext);
5674         outResults.addAll(tmpList);
5675       }
5676 
5677       // If the size limit was reached it means a partial Result is being returned. Returning a
5678       // partial Result means that we should not reset the filters; filters should only be reset in
5679       // between rows
5680       if (!scannerContext.midRowResultFormed()) resetFilters();
5681 
5682       if (isFilterDoneInternal()) {
5683         moreValues = false;
5684       }
5685       return moreValues;
5686     }
5687 
5688     /**
5689      * @return true if more cells exist after this batch, false if scanner is done
5690      */
5691     private boolean populateFromJoinedHeap(List<Cell> results, ScannerContext scannerContext)
5692             throws IOException {
5693       assert joinedContinuationRow != null;
5694       boolean moreValues =
5695           populateResult(results, this.joinedHeap, scannerContext,
5696           joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
5697           joinedContinuationRow.getRowLength());
5698 
5699       if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5700         // We are done with this row, reset the continuation.
5701         joinedContinuationRow = null;
5702       }
5703       // As the data is obtained from two independent heaps, we need to
5704       // ensure that result list is sorted, because Result relies on that.
5705       Collections.sort(results, comparator);
5706       return moreValues;
5707     }
5708 
5709     /**
5710      * Fetches records with currentRow into results list, until next row, batchLimit (if not -1) is
5711      * reached, or remainingResultSize (if not -1) is reaced
5712      * @param heap KeyValueHeap to fetch data from.It must be positioned on correct row before call.
5713      * @param scannerContext
5714      * @param currentRow Byte array with key we are fetching.
5715      * @param offset offset for currentRow
5716      * @param length length for currentRow
5717      * @return state of last call to {@link KeyValueHeap#next()}
5718      */
5719     private boolean populateResult(List<Cell> results, KeyValueHeap heap,
5720         ScannerContext scannerContext, byte[] currentRow, int offset, short length)
5721         throws IOException {
5722       Cell nextKv;
5723       boolean moreCellsInRow = false;
5724       boolean tmpKeepProgress = scannerContext.getKeepProgress();
5725       // Scanning between column families and thus the scope is between cells
5726       LimitScope limitScope = LimitScope.BETWEEN_CELLS;
5727       try {
5728         do {
5729           // We want to maintain any progress that is made towards the limits while scanning across
5730           // different column families. To do this, we toggle the keep progress flag on during calls
5731           // to the StoreScanner to ensure that any progress made thus far is not wiped away.
5732           scannerContext.setKeepProgress(true);
5733           heap.next(results, scannerContext);
5734           scannerContext.setKeepProgress(tmpKeepProgress);
5735 
5736           nextKv = heap.peek();
5737           moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length);
5738           if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
5739 
5740           if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
5741             return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
5742           } else if (scannerContext.checkSizeLimit(limitScope)) {
5743             ScannerContext.NextState state =
5744                 moreCellsInRow? NextState.SIZE_LIMIT_REACHED_MID_ROW: NextState.SIZE_LIMIT_REACHED;
5745             return scannerContext.setScannerState(state).hasMoreValues();
5746           } else if (scannerContext.checkTimeLimit(limitScope)) {
5747             ScannerContext.NextState state =
5748                 moreCellsInRow? NextState.TIME_LIMIT_REACHED_MID_ROW: NextState.TIME_LIMIT_REACHED;
5749             return scannerContext.setScannerState(state).hasMoreValues();
5750           }
5751         } while (moreCellsInRow);
5752       } catch (FileNotFoundException e) {
5753         throw handleFileNotFound(e);
5754       }
5755       return nextKv != null;
5756     }
5757 
5758     /**
5759      * Based on the nextKv in the heap, and the current row, decide whether or not there are more
5760      * cells to be read in the heap. If the row of the nextKv in the heap matches the current row
5761      * then there are more cells to be read in the row.
5762      * @param nextKv
5763      * @param currentRow
5764      * @param offset
5765      * @param length
5766      * @return true When there are more cells in the row to be read
5767      */
5768     private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset,
5769         short length) {
5770       return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length);
5771     }
5772 
5773     /*
5774      * @return True if a filter rules the scanner is over, done.
5775      */
5776     @Override
5777     public synchronized boolean isFilterDone() throws IOException {
5778       return isFilterDoneInternal();
5779     }
5780 
5781     private boolean isFilterDoneInternal() throws IOException {
5782       return this.filter != null && this.filter.filterAllRemaining();
5783     }
5784 
5785     private boolean nextInternal(List<Cell> results, ScannerContext scannerContext)
5786         throws IOException {
5787       if (!results.isEmpty()) {
5788         throw new IllegalArgumentException("First parameter should be an empty list");
5789       }
5790       if (scannerContext == null) {
5791         throw new IllegalArgumentException("Scanner context cannot be null");
5792       }
5793       RpcCallContext rpcCall = RpcServer.getCurrentCall();
5794 
5795       // Save the initial progress from the Scanner context in these local variables. The progress
5796       // may need to be reset a few times if rows are being filtered out so we save the initial
5797       // progress.
5798       int initialBatchProgress = scannerContext.getBatchProgress();
5799       long initialSizeProgress = scannerContext.getSizeProgress();
5800       long initialTimeProgress = scannerContext.getTimeProgress();
5801 
5802       // The loop here is used only when at some point during the next we determine
5803       // that due to effects of filters or otherwise, we have an empty row in the result.
5804       // Then we loop and try again. Otherwise, we must get out on the first iteration via return,
5805       // "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row,
5806       // and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow).
5807       while (true) {
5808         // Starting to scan a new row. Reset the scanner progress according to whether or not
5809         // progress should be kept.
5810         if (scannerContext.getKeepProgress()) {
5811           // Progress should be kept. Reset to initial values seen at start of method invocation.
5812           scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
5813             initialTimeProgress);
5814         } else {
5815           scannerContext.clearProgress();
5816         }
5817 
5818         if (rpcCall != null) {
5819           // If a user specifies a too-restrictive or too-slow scanner, the
5820           // client might time out and disconnect while the server side
5821           // is still processing the request. We should abort aggressively
5822           // in that case.
5823           long afterTime = rpcCall.disconnectSince();
5824           if (afterTime >= 0) {
5825             throw new CallerDisconnectedException(
5826                 "Aborting on region " + getRegionInfo().getRegionNameAsString() + ", call " +
5827                     this + " after " + afterTime + " ms, since " +
5828                     "caller disconnected");
5829           }
5830         }
5831 
5832         // Let's see what we have in the storeHeap.
5833         Cell current = this.storeHeap.peek();
5834 
5835         byte[] currentRow = null;
5836         int offset = 0;
5837         short length = 0;
5838         if (current != null) {
5839           currentRow = current.getRowArray();
5840           offset = current.getRowOffset();
5841           length = current.getRowLength();
5842         }
5843 
5844         boolean stopRow = isStopRow(currentRow, offset, length);
5845         // When has filter row is true it means that the all the cells for a particular row must be
5846         // read before a filtering decision can be made. This means that filters where hasFilterRow
5847         // run the risk of encountering out of memory errors in the case that they are applied to a
5848         // table that has very large rows.
5849         boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow();
5850 
5851         // If filter#hasFilterRow is true, partial results are not allowed since allowing them
5852         // would prevent the filters from being evaluated. Thus, if it is true, change the
5853         // scope of any limits that could potentially create partial results to
5854         // LimitScope.BETWEEN_ROWS so that those limits are not reached mid-row
5855         if (hasFilterRow) {
5856           if (LOG.isTraceEnabled()) {
5857             LOG.trace("filter#hasFilterRow is true which prevents partial results from being "
5858                 + " formed. Changing scope of limits that may create partials");
5859           }
5860           scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS);
5861           scannerContext.setTimeLimitScope(LimitScope.BETWEEN_ROWS);
5862         }
5863 
5864         // Check if we were getting data from the joinedHeap and hit the limit.
5865         // If not, then it's main path - getting results from storeHeap.
5866         if (joinedContinuationRow == null) {
5867           // First, check if we are at a stop row. If so, there are no more results.
5868           if (stopRow) {
5869             if (hasFilterRow) {
5870               filter.filterRowCells(results);
5871             }
5872             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5873           }
5874 
5875           // Check if rowkey filter wants to exclude this row. If so, loop to next.
5876           // Technically, if we hit limits before on this row, we don't need this call.
5877           if (filterRowKey(currentRow, offset, length)) {
5878             incrementCountOfRowsFilteredMetric(scannerContext);
5879             // early check, see HBASE-16296
5880             if (isFilterDoneInternal()) {
5881               return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5882             }
5883             // Typically the count of rows scanned is incremented inside #populateResult. However,
5884             // here we are filtering a row based purely on its row key, preventing us from calling
5885             // #populateResult. Thus, perform the necessary increment here to rows scanned metric
5886             incrementCountOfRowsScannedMetric(scannerContext);
5887             boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
5888             if (!moreRows) {
5889               return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5890             }
5891             results.clear();
5892             continue;
5893           }
5894 
5895           // Ok, we are good, let's try to get some results from the main heap.
5896           populateResult(results, this.storeHeap, scannerContext, currentRow, offset, length);
5897 
5898           if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5899             if (hasFilterRow) {
5900               throw new IncompatibleFilterException(
5901                   "Filter whose hasFilterRow() returns true is incompatible with scans that must "
5902                       + " stop mid-row because of a limit. ScannerContext:" + scannerContext);
5903             }
5904             return true;
5905           }
5906 
5907           Cell nextKv = this.storeHeap.peek();
5908           stopRow = nextKv == null ||
5909               isStopRow(nextKv.getRowArray(), nextKv.getRowOffset(), nextKv.getRowLength());
5910           // save that the row was empty before filters applied to it.
5911           final boolean isEmptyRow = results.isEmpty();
5912 
5913           // We have the part of the row necessary for filtering (all of it, usually).
5914           // First filter with the filterRow(List).
5915           FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;
5916           if (hasFilterRow) {
5917             ret = filter.filterRowCellsWithRet(results);
5918 
5919             // We don't know how the results have changed after being filtered. Must set progress
5920             // according to contents of results now. However, a change in the results should not
5921             // affect the time progress. Thus preserve whatever time progress has been made
5922             long timeProgress = scannerContext.getTimeProgress();
5923             if (scannerContext.getKeepProgress()) {
5924               scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
5925                 initialTimeProgress);
5926             } else {
5927               scannerContext.clearProgress();
5928             }
5929             scannerContext.setTimeProgress(timeProgress);
5930             scannerContext.incrementBatchProgress(results.size());
5931             for (Cell cell : results) {
5932               scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell));
5933             }
5934           }
5935 
5936           if (isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE || filterRow()) {
5937             incrementCountOfRowsFilteredMetric(scannerContext);
5938             results.clear();
5939             boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
5940             if (!moreRows) {
5941               return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5942             }
5943 
5944             // This row was totally filtered out, if this is NOT the last row,
5945             // we should continue on. Otherwise, nothing else to do.
5946             if (!stopRow) continue;
5947             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5948           }
5949 
5950           // Ok, we are done with storeHeap for this row.
5951           // Now we may need to fetch additional, non-essential data into row.
5952           // These values are not needed for filter to work, so we postpone their
5953           // fetch to (possibly) reduce amount of data loads from disk.
5954           if (this.joinedHeap != null) {
5955             boolean mayHaveData = joinedHeapMayHaveData(currentRow, offset, length);
5956             if (mayHaveData) {
5957               joinedContinuationRow = current;
5958               populateFromJoinedHeap(results, scannerContext);
5959 
5960               if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5961                 return true;
5962               }
5963             }
5964           }
5965         } else {
5966           // Populating from the joined heap was stopped by limits, populate some more.
5967           populateFromJoinedHeap(results, scannerContext);
5968           if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5969             return true;
5970           }
5971         }
5972         // We may have just called populateFromJoinedMap and hit the limits. If that is
5973         // the case, we need to call it again on the next next() invocation.
5974         if (joinedContinuationRow != null) {
5975           return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
5976         }
5977 
5978         // Finally, we are done with both joinedHeap and storeHeap.
5979         // Double check to prevent empty rows from appearing in result. It could be
5980         // the case when SingleColumnValueExcludeFilter is used.
5981         if (results.isEmpty()) {
5982           incrementCountOfRowsFilteredMetric(scannerContext);
5983           boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
5984           if (!moreRows) {
5985             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5986           }
5987           if (!stopRow) continue;
5988         }
5989 
5990         if (stopRow) {
5991           return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5992         } else {
5993           return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
5994         }
5995       }
5996     }
5997 
5998     protected void incrementCountOfRowsFilteredMetric(ScannerContext scannerContext) {
5999       if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
6000 
6001       scannerContext.getMetrics().countOfRowsFiltered.incrementAndGet();
6002     }
6003 
6004     protected void incrementCountOfRowsScannedMetric(ScannerContext scannerContext) {
6005       if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
6006 
6007       scannerContext.getMetrics().countOfRowsScanned.incrementAndGet();
6008     }
6009 
6010     /**
6011      * @param currentRow
6012      * @param offset
6013      * @param length
6014      * @return true when the joined heap may have data for the current row
6015      * @throws IOException
6016      */
6017     private boolean joinedHeapMayHaveData(byte[] currentRow, int offset, short length)
6018         throws IOException {
6019       Cell nextJoinedKv = joinedHeap.peek();
6020       boolean matchCurrentRow =
6021           nextJoinedKv != null && CellUtil.matchingRow(nextJoinedKv, currentRow, offset, length);
6022       boolean matchAfterSeek = false;
6023 
6024       // If the next value in the joined heap does not match the current row, try to seek to the
6025       // correct row
6026       if (!matchCurrentRow) {
6027         Cell firstOnCurrentRow = KeyValueUtil.createFirstOnRow(currentRow, offset, length);
6028         boolean seekSuccessful = this.joinedHeap.requestSeek(firstOnCurrentRow, true, true);
6029         matchAfterSeek =
6030             seekSuccessful && joinedHeap.peek() != null
6031                 && CellUtil.matchingRow(joinedHeap.peek(), currentRow, offset, length);
6032       }
6033 
6034       return matchCurrentRow || matchAfterSeek;
6035     }
6036 
6037     /**
6038      * This function is to maintain backward compatibility for 0.94 filters. HBASE-6429 combines
6039      * both filterRow & filterRow(List<KeyValue> kvs) functions. While 0.94 code or older, it may
6040      * not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only returns
6041      * true when filterRow(List<KeyValue> kvs) is overridden not the filterRow(). Therefore, the
6042      * filterRow() will be skipped.
6043      */
6044     private boolean filterRow() throws IOException {
6045       // when hasFilterRow returns true, filter.filterRow() will be called automatically inside
6046       // filterRowCells(List<Cell> kvs) so we skip that scenario here.
6047       return filter != null && (!filter.hasFilterRow())
6048           && filter.filterRow();
6049     }
6050 
6051     private boolean filterRowKey(byte[] row, int offset, short length) throws IOException {
6052       return filter != null
6053           && filter.filterRowKey(row, offset, length);
6054     }
6055 
6056     protected boolean nextRow(ScannerContext scannerContext, byte[] currentRow, int offset,
6057         short length) throws IOException {
6058       assert this.joinedContinuationRow == null:
6059         "Trying to go to next row during joinedHeap read.";
6060       Cell next;
6061       while ((next = this.storeHeap.peek()) != null &&
6062              CellUtil.matchingRow(next, currentRow, offset, length)) {
6063         this.storeHeap.next(MOCKED_LIST);
6064       }
6065       resetFilters();
6066 
6067       // Calling the hook in CP which allows it to do a fast forward
6068       return this.region.getCoprocessorHost() == null
6069           || this.region.getCoprocessorHost()
6070               .postScannerFilterRow(this, currentRow, offset, length);
6071     }
6072 
6073     protected boolean isStopRow(byte[] currentRow, int offset, short length) {
6074       return currentRow == null ||
6075           (stopRow != null &&
6076           comparator.compareRows(stopRow, 0, stopRow.length,
6077             currentRow, offset, length) <= isScan);
6078     }
6079 
6080     @Override
6081     public synchronized void close() {
6082       if (storeHeap != null) {
6083         storeHeap.close();
6084         storeHeap = null;
6085       }
6086       if (joinedHeap != null) {
6087         joinedHeap.close();
6088         joinedHeap = null;
6089       }
6090       // no need to synchronize here.
6091       scannerReadPoints.remove(this);
6092       this.filterClosed = true;
6093     }
6094 
6095     KeyValueHeap getStoreHeapForTesting() {
6096       return storeHeap;
6097     }
6098 
6099     @Override
6100     public synchronized boolean reseek(byte[] row) throws IOException {
6101       if (row == null) {
6102         throw new IllegalArgumentException("Row cannot be null.");
6103       }
6104       boolean result = false;
6105       startRegionOperation();
6106       KeyValue kv = KeyValueUtil.createFirstOnRow(row);
6107       try {
6108         // use request seek to make use of the lazy seek option. See HBASE-5520
6109         result = this.storeHeap.requestSeek(kv, true, true);
6110         if (this.joinedHeap != null) {
6111           result = this.joinedHeap.requestSeek(kv, true, true) || result;
6112         }
6113       } catch (FileNotFoundException e) {
6114         throw handleFileNotFound(e);
6115       } finally {
6116         closeRegionOperation();
6117       }
6118       return result;
6119     }
6120 
6121     private IOException handleFileNotFound(FileNotFoundException fnfe) throws IOException {
6122       // tries to refresh the store files, otherwise shutdown the RS.
6123       // TODO: add support for abort() of a single region and trigger reassignment.
6124       try {
6125         region.refreshStoreFiles(true);
6126         return new IOException("unable to read store file");
6127       } catch (IOException e) {
6128         String msg = "a store file got lost: " + fnfe.getMessage();
6129         LOG.error(msg);
6130         LOG.error("unable to refresh store files", e);
6131         abortRegionServer(msg);
6132         return new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " closing");
6133       }
6134     }
6135 
6136     private void abortRegionServer(String msg) throws IOException {
6137       if (rsServices instanceof HRegionServer) {
6138         ((HRegionServer)rsServices).abort(msg);
6139       }
6140       throw new UnsupportedOperationException("not able to abort RS after: " + msg);
6141     }
6142   }
6143 
6144   // Utility methods
6145   /**
6146    * A utility method to create new instances of HRegion based on the
6147    * {@link HConstants#REGION_IMPL} configuration property.
6148    * @param tableDir qualified path of directory where region should be located,
6149    * usually the table directory.
6150    * @param wal The WAL is the outbound log for any updates to the HRegion
6151    * The wal file is a logfile from the previous execution that's
6152    * custom-computed for this HRegion. The HRegionServer computes and sorts the
6153    * appropriate wal info for this HRegion. If there is a previous file
6154    * (implying that the HRegion has been written-to before), then read it from
6155    * the supplied path.
6156    * @param fs is the filesystem.
6157    * @param conf is global configuration settings.
6158    * @param regionInfo - HRegionInfo that describes the region
6159    * is new), then read them from the supplied path.
6160    * @param htd the table descriptor
6161    * @return the new instance
6162    */
6163   static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
6164       Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
6165       RegionServerServices rsServices) {
6166     try {
6167       @SuppressWarnings("unchecked")
6168       Class<? extends HRegion> regionClass =
6169           (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
6170 
6171       Constructor<? extends HRegion> c =
6172           regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
6173               Configuration.class, HRegionInfo.class, HTableDescriptor.class,
6174               RegionServerServices.class);
6175 
6176       return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
6177     } catch (Throwable e) {
6178       // todo: what should I throw here?
6179       throw new IllegalStateException("Could not instantiate a region instance.", e);
6180     }
6181   }
6182 
6183   /**
6184    * Convenience method creating new HRegions. Used by createTable and by the
6185    * bootstrap code in the HMaster constructor.
6186    * Note, this method creates an {@link WAL} for the created region. It
6187    * needs to be closed explicitly.  Use {@link HRegion#getWAL()} to get
6188    * access.  <b>When done with a region created using this method, you will
6189    * need to explicitly close the {@link WAL} it created too; it will not be
6190    * done for you.  Not closing the wal will leave at least a daemon thread
6191    * running.</b>  Call {@link #closeHRegion(HRegion)} and it will do
6192    * necessary cleanup for you.
6193    * @param info Info for region to create.
6194    * @param rootDir Root directory for HBase instance
6195    * @return new HRegion
6196    *
6197    * @throws IOException
6198    */
6199   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6200       final Configuration conf, final HTableDescriptor hTableDescriptor)
6201   throws IOException {
6202     return createHRegion(info, rootDir, conf, hTableDescriptor, null);
6203   }
6204 
6205   /**
6206    * This will do the necessary cleanup a call to
6207    * {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)}
6208    * requires.  This method will close the region and then close its
6209    * associated {@link WAL} file.  You can still use it if you call the other createHRegion,
6210    * the one that takes an {@link WAL} instance but don't be surprised by the
6211    * call to the {@link WAL#close()} on the {@link WAL} the
6212    * HRegion was carrying.
6213    * @throws IOException
6214    */
6215   public static void closeHRegion(final HRegion r) throws IOException {
6216     if (r == null) return;
6217     r.close();
6218     if (r.getWAL() == null) return;
6219     r.getWAL().close();
6220   }
6221 
6222   /**
6223    * Convenience method creating new HRegions. Used by createTable.
6224    * The {@link WAL} for the created region needs to be closed explicitly.
6225    * Use {@link HRegion#getWAL()} to get access.
6226    *
6227    * @param info Info for region to create.
6228    * @param rootDir Root directory for HBase instance
6229    * @param wal shared WAL
6230    * @param initialize - true to initialize the region
6231    * @return new HRegion
6232    *
6233    * @throws IOException
6234    */
6235   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6236                                       final Configuration conf,
6237                                       final HTableDescriptor hTableDescriptor,
6238                                       final WAL wal,
6239                                       final boolean initialize)
6240       throws IOException {
6241     return createHRegion(info, rootDir, conf, hTableDescriptor,
6242         wal, initialize, false);
6243   }
6244 
6245   /**
6246    * Convenience method creating new HRegions. Used by createTable.
6247    * The {@link WAL} for the created region needs to be closed
6248    * explicitly, if it is not null.
6249    * Use {@link HRegion#getWAL()} to get access.
6250    *
6251    * @param info Info for region to create.
6252    * @param rootDir Root directory for HBase instance
6253    * @param wal shared WAL
6254    * @param initialize - true to initialize the region
6255    * @param ignoreWAL - true to skip generate new wal if it is null, mostly for createTable
6256    * @return new HRegion
6257    * @throws IOException
6258    */
6259   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6260                                       final Configuration conf,
6261                                       final HTableDescriptor hTableDescriptor,
6262                                       final WAL wal,
6263                                       final boolean initialize, final boolean ignoreWAL)
6264       throws IOException {
6265       Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6266       return createHRegion(info, rootDir, tableDir, conf, hTableDescriptor, wal, initialize,
6267           ignoreWAL);
6268   }
6269 
6270   /**
6271    * Convenience method creating new HRegions. Used by createTable.
6272    * The {@link WAL} for the created region needs to be closed
6273    * explicitly, if it is not null.
6274    * Use {@link HRegion#getWAL()} to get access.
6275    *
6276    * @param info Info for region to create.
6277    * @param rootDir Root directory for HBase instance
6278    * @param tableDir table directory
6279    * @param wal shared WAL
6280    * @param initialize - true to initialize the region
6281    * @param ignoreWAL - true to skip generate new wal if it is null, mostly for createTable
6282    * @return new HRegion
6283    * @throws IOException
6284    */
6285   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6286       final Path tableDir, final Configuration conf, final HTableDescriptor hTableDescriptor,
6287       final WAL wal, final boolean initialize, final boolean ignoreWAL)
6288   throws IOException {
6289     LOG.info("creating HRegion " + info.getTable().getNameAsString()
6290         + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
6291         " Table name == " + info.getTable().getNameAsString());
6292     FileSystem fs = FileSystem.get(conf);
6293     HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
6294     WAL effectiveWAL = wal;
6295     if (wal == null && !ignoreWAL) {
6296       // TODO HBASE-11983 There'll be no roller for this wal?
6297       // The WAL subsystem will use the default rootDir rather than the passed in rootDir
6298       // unless I pass along via the conf.
6299       Configuration confForWAL = new Configuration(conf);
6300       confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
6301       effectiveWAL = (new WALFactory(confForWAL,
6302           Collections.<WALActionsListener>singletonList(new MetricsWAL()),
6303           "hregion-" + RandomStringUtils.randomNumeric(8))).
6304             getWAL(info.getEncodedNameAsBytes());
6305     }
6306     HRegion region = HRegion.newHRegion(tableDir,
6307         effectiveWAL, fs, conf, info, hTableDescriptor, null);
6308     if (initialize) region.initialize(null);
6309     return region;
6310   }
6311 
6312   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6313                                       final Configuration conf,
6314                                       final HTableDescriptor hTableDescriptor,
6315                                       final WAL wal)
6316     throws IOException {
6317     return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
6318   }
6319 
6320 
6321   /**
6322    * Open a Region.
6323    * @param info Info for region to be opened.
6324    * @param wal WAL for region to use. This method will call
6325    * WAL#setSequenceNumber(long) passing the result of the call to
6326    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6327    * up.  HRegionStore does this every time it opens a new region.
6328    * @return new HRegion
6329    *
6330    * @throws IOException
6331    */
6332   public static HRegion openHRegion(final HRegionInfo info,
6333       final HTableDescriptor htd, final WAL wal,
6334       final Configuration conf)
6335   throws IOException {
6336     return openHRegion(info, htd, wal, conf, null, null);
6337   }
6338 
6339   /**
6340    * Open a Region.
6341    * @param info Info for region to be opened
6342    * @param htd the table descriptor
6343    * @param wal WAL for region to use. This method will call
6344    * WAL#setSequenceNumber(long) passing the result of the call to
6345    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6346    * up.  HRegionStore does this every time it opens a new region.
6347    * @param conf The Configuration object to use.
6348    * @param rsServices An interface we can request flushes against.
6349    * @param reporter An interface we can report progress against.
6350    * @return new HRegion
6351    *
6352    * @throws IOException
6353    */
6354   public static HRegion openHRegion(final HRegionInfo info,
6355     final HTableDescriptor htd, final WAL wal, final Configuration conf,
6356     final RegionServerServices rsServices,
6357     final CancelableProgressable reporter)
6358   throws IOException {
6359     return openHRegion(FSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
6360   }
6361 
6362   /**
6363    * Open a Region.
6364    * @param rootDir Root directory for HBase instance
6365    * @param info Info for region to be opened.
6366    * @param htd the table descriptor
6367    * @param wal WAL for region to use. This method will call
6368    * WAL#setSequenceNumber(long) passing the result of the call to
6369    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6370    * up.  HRegionStore does this every time it opens a new region.
6371    * @param conf The Configuration object to use.
6372    * @return new HRegion
6373    * @throws IOException
6374    */
6375   public static HRegion openHRegion(Path rootDir, final HRegionInfo info,
6376       final HTableDescriptor htd, final WAL wal, final Configuration conf)
6377   throws IOException {
6378     return openHRegion(rootDir, info, htd, wal, conf, null, null);
6379   }
6380 
6381   /**
6382    * Open a Region.
6383    * @param rootDir Root directory for HBase instance
6384    * @param info Info for region to be opened.
6385    * @param htd the table descriptor
6386    * @param wal WAL for region to use. This method will call
6387    * WAL#setSequenceNumber(long) passing the result of the call to
6388    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6389    * up.  HRegionStore does this every time it opens a new region.
6390    * @param conf The Configuration object to use.
6391    * @param rsServices An interface we can request flushes against.
6392    * @param reporter An interface we can report progress against.
6393    * @return new HRegion
6394    * @throws IOException
6395    */
6396   public static HRegion openHRegion(final Path rootDir, final HRegionInfo info,
6397       final HTableDescriptor htd, final WAL wal, final Configuration conf,
6398       final RegionServerServices rsServices,
6399       final CancelableProgressable reporter)
6400   throws IOException {
6401     FileSystem fs = null;
6402     if (rsServices != null) {
6403       fs = rsServices.getFileSystem();
6404     }
6405     if (fs == null) {
6406       fs = FileSystem.get(conf);
6407     }
6408     return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
6409   }
6410 
6411   /**
6412    * Open a Region.
6413    * @param conf The Configuration object to use.
6414    * @param fs Filesystem to use
6415    * @param rootDir Root directory for HBase instance
6416    * @param info Info for region to be opened.
6417    * @param htd the table descriptor
6418    * @param wal WAL for region to use. This method will call
6419    * WAL#setSequenceNumber(long) passing the result of the call to
6420    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6421    * up.  HRegionStore does this every time it opens a new region.
6422    * @return new HRegion
6423    * @throws IOException
6424    */
6425   public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
6426       final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal)
6427       throws IOException {
6428     return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
6429   }
6430 
6431   /**
6432    * Open a Region.
6433    * @param conf The Configuration object to use.
6434    * @param fs Filesystem to use
6435    * @param rootDir Root directory for HBase instance
6436    * @param info Info for region to be opened.
6437    * @param htd the table descriptor
6438    * @param wal WAL for region to use. This method will call
6439    * WAL#setSequenceNumber(long) passing the result of the call to
6440    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6441    * up.  HRegionStore does this every time it opens a new region.
6442    * @param rsServices An interface we can request flushes against.
6443    * @param reporter An interface we can report progress against.
6444    * @return new HRegion
6445    * @throws IOException
6446    */
6447   public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
6448       final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal,
6449       final RegionServerServices rsServices, final CancelableProgressable reporter)
6450       throws IOException {
6451     Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6452     return openHRegion(conf, fs, rootDir, tableDir, info, htd, wal, rsServices, reporter);
6453   }
6454 
6455   /**
6456    * Open a Region.
6457    * @param conf The Configuration object to use.
6458    * @param fs Filesystem to use
6459    * @param rootDir Root directory for HBase instance
6460    * @param info Info for region to be opened.
6461    * @param htd the table descriptor
6462    * @param wal WAL for region to use. This method will call
6463    * WAL#setSequenceNumber(long) passing the result of the call to
6464    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6465    * up.  HRegionStore does this every time it opens a new region.
6466    * @param rsServices An interface we can request flushes against.
6467    * @param reporter An interface we can report progress against.
6468    * @return new HRegion
6469    * @throws IOException
6470    */
6471   public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
6472       final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd,
6473       final WAL wal, final RegionServerServices rsServices,
6474       final CancelableProgressable reporter)
6475       throws IOException {
6476     if (info == null) throw new NullPointerException("Passed region info is null");
6477     if (LOG.isDebugEnabled()) {
6478       LOG.debug("Opening region: " + info);
6479     }
6480     HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
6481     return r.openHRegion(reporter);
6482   }
6483 
6484 
6485   /**
6486    * Useful when reopening a closed region (normally for unit tests)
6487    * @param other original object
6488    * @param reporter An interface we can report progress against.
6489    * @return new HRegion
6490    * @throws IOException
6491    */
6492   public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
6493       throws IOException {
6494     HRegionFileSystem regionFs = other.getRegionFileSystem();
6495     HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
6496         other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
6497     return r.openHRegion(reporter);
6498   }
6499 
6500   public static Region openHRegion(final Region other, final CancelableProgressable reporter)
6501         throws IOException {
6502     return openHRegion((HRegion)other, reporter);
6503   }
6504 
6505   /**
6506    * Open HRegion.
6507    * Calls initialize and sets sequenceId.
6508    * @return Returns <code>this</code>
6509    * @throws IOException
6510    */
6511   protected HRegion openHRegion(final CancelableProgressable reporter)
6512   throws IOException {
6513     // Refuse to open the region if we are missing local compression support
6514     checkCompressionCodecs();
6515     // Refuse to open the region if encryption configuration is incorrect or
6516     // codec support is missing
6517     checkEncryption();
6518     // Refuse to open the region if a required class cannot be loaded
6519     checkClassLoading();
6520     this.openSeqNum = initialize(reporter);
6521     this.mvcc.advanceTo(openSeqNum);
6522     if (wal != null && getRegionServerServices() != null && !writestate.readOnly
6523         && !recovering) {
6524       // Only write the region open event marker to WAL if (1) we are not read-only
6525       // (2) dist log replay is off or we are not recovering. In case region is
6526       // recovering, the open event will be written at setRecovering(false)
6527       writeRegionOpenMarker(wal, openSeqNum);
6528     }
6529     return this;
6530   }
6531 
6532   public static void warmupHRegion(final HRegionInfo info,
6533       final HTableDescriptor htd, final WAL wal, final Configuration conf,
6534       final RegionServerServices rsServices,
6535       final CancelableProgressable reporter)
6536       throws IOException {
6537 
6538     if (info == null) throw new NullPointerException("Passed region info is null");
6539 
6540     if (LOG.isDebugEnabled()) {
6541       LOG.debug("HRegion.Warming up region: " + info);
6542     }
6543 
6544     Path rootDir = FSUtils.getRootDir(conf);
6545     Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6546 
6547     FileSystem fs = null;
6548     if (rsServices != null) {
6549       fs = rsServices.getFileSystem();
6550     }
6551     if (fs == null) {
6552       fs = FileSystem.get(conf);
6553     }
6554 
6555     HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
6556     r.initializeWarmup(reporter);
6557   }
6558 
6559 
6560   private void checkCompressionCodecs() throws IOException {
6561     for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
6562       CompressionTest.testCompression(fam.getCompression());
6563       CompressionTest.testCompression(fam.getCompactionCompression());
6564     }
6565   }
6566 
6567   private void checkEncryption() throws IOException {
6568     for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
6569       EncryptionTest.testEncryption(conf, fam.getEncryptionType(), fam.getEncryptionKey());
6570     }
6571   }
6572 
6573   private void checkClassLoading() throws IOException {
6574     RegionSplitPolicy.getSplitPolicyClass(this.htableDescriptor, conf);
6575     RegionCoprocessorHost.testTableCoprocessorAttrs(conf, this.htableDescriptor);
6576   }
6577 
6578   /**
6579    * Create a daughter region from given a temp directory with the region data.
6580    * @param hri Spec. for daughter region to open.
6581    * @throws IOException
6582    */
6583   HRegion createDaughterRegionFromSplits(final HRegionInfo hri) throws IOException {
6584     // Move the files from the temporary .splits to the final /table/region directory
6585     fs.commitDaughterRegion(hri);
6586 
6587     // Create the daughter HRegion instance
6588     HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(), fs.getFileSystem(),
6589         this.getBaseConf(), hri, this.getTableDesc(), rsServices);
6590     r.readRequestsCount.set(this.getReadRequestsCount() / 2);
6591     r.writeRequestsCount.set(this.getWriteRequestsCount() / 2);
6592     return r;
6593   }
6594 
6595   /**
6596    * Create a merged region given a temp directory with the region data.
6597    * @param region_b another merging region
6598    * @return merged HRegion
6599    * @throws IOException
6600    */
6601   HRegion createMergedRegionFromMerges(final HRegionInfo mergedRegionInfo,
6602       final HRegion region_b) throws IOException {
6603     HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(),
6604         fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
6605         this.getTableDesc(), this.rsServices);
6606     r.readRequestsCount.set(this.getReadRequestsCount()
6607         + region_b.getReadRequestsCount());
6608     r.writeRequestsCount.set(this.getWriteRequestsCount()
6609 
6610         + region_b.getWriteRequestsCount());
6611     this.fs.commitMergedRegion(mergedRegionInfo);
6612     return r;
6613   }
6614 
6615   /**
6616    * Inserts a new region's meta information into the passed
6617    * <code>meta</code> region. Used by the HMaster bootstrap code adding
6618    * new table to hbase:meta table.
6619    *
6620    * @param meta hbase:meta HRegion to be updated
6621    * @param r HRegion to add to <code>meta</code>
6622    *
6623    * @throws IOException
6624    */
6625   // TODO remove since only test and merge use this
6626   public static void addRegionToMETA(final HRegion meta, final HRegion r) throws IOException {
6627     meta.checkResources();
6628     // The row key is the region name
6629     byte[] row = r.getRegionInfo().getRegionName();
6630     final long now = EnvironmentEdgeManager.currentTime();
6631     final List<Cell> cells = new ArrayList<Cell>(2);
6632     cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
6633       HConstants.REGIONINFO_QUALIFIER, now,
6634       r.getRegionInfo().toByteArray()));
6635     // Set into the root table the version of the meta table.
6636     cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
6637       HConstants.META_VERSION_QUALIFIER, now,
6638       Bytes.toBytes(HConstants.META_VERSION)));
6639     meta.put(row, HConstants.CATALOG_FAMILY, cells);
6640   }
6641 
6642   /**
6643    * Computes the Path of the HRegion
6644    *
6645    * @param tabledir qualified path for table
6646    * @param name ENCODED region name
6647    * @return Path of HRegion directory
6648    * @deprecated For tests only; to be removed.
6649    */
6650   @Deprecated
6651   public static Path getRegionDir(final Path tabledir, final String name) {
6652     return new Path(tabledir, name);
6653   }
6654 
6655   /**
6656    * Computes the Path of the HRegion
6657    *
6658    * @param rootdir qualified path of HBase root directory
6659    * @param info HRegionInfo for the region
6660    * @return qualified path of region directory
6661    * @deprecated For tests only; to be removed.
6662    */
6663   @Deprecated
6664   @VisibleForTesting
6665   public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
6666     return new Path(
6667       FSUtils.getTableDir(rootdir, info.getTable()), info.getEncodedName());
6668   }
6669 
6670   /**
6671    * Determines if the specified row is within the row range specified by the
6672    * specified HRegionInfo
6673    *
6674    * @param info HRegionInfo that specifies the row range
6675    * @param row row to be checked
6676    * @return true if the row is within the range specified by the HRegionInfo
6677    */
6678   public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
6679     return ((info.getStartKey().length == 0) ||
6680         (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
6681         ((info.getEndKey().length == 0) ||
6682             (Bytes.compareTo(info.getEndKey(), row) > 0));
6683   }
6684 
6685   public static boolean rowIsInRange(HRegionInfo info, final byte [] row, final int offset,
6686       final short length) {
6687     return ((info.getStartKey().length == 0) ||
6688         (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
6689           row, offset, length) <= 0)) &&
6690         ((info.getEndKey().length == 0) ||
6691           (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
6692   }
6693 
6694   /**
6695    * Merge two HRegions.  The regions must be adjacent and must not overlap.
6696    *
6697    * @return new merged HRegion
6698    * @throws IOException
6699    */
6700   public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
6701   throws IOException {
6702     HRegion a = srcA;
6703     HRegion b = srcB;
6704 
6705     // Make sure that srcA comes first; important for key-ordering during
6706     // write of the merged file.
6707     if (srcA.getRegionInfo().getStartKey() == null) {
6708       if (srcB.getRegionInfo().getStartKey() == null) {
6709         throw new IOException("Cannot merge two regions with null start key");
6710       }
6711       // A's start key is null but B's isn't. Assume A comes before B
6712     } else if ((srcB.getRegionInfo().getStartKey() == null) ||
6713       (Bytes.compareTo(srcA.getRegionInfo().getStartKey(),
6714         srcB.getRegionInfo().getStartKey()) > 0)) {
6715       a = srcB;
6716       b = srcA;
6717     }
6718 
6719     if (!(Bytes.compareTo(a.getRegionInfo().getEndKey(),
6720         b.getRegionInfo().getStartKey()) == 0)) {
6721       throw new IOException("Cannot merge non-adjacent regions");
6722     }
6723     return merge(a, b);
6724   }
6725 
6726   /**
6727    * Merge two regions whether they are adjacent or not.
6728    *
6729    * @param a region a
6730    * @param b region b
6731    * @return new merged region
6732    * @throws IOException
6733    */
6734   public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
6735     if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
6736       throw new IOException("Regions do not belong to the same table");
6737     }
6738 
6739     FileSystem fs = a.getRegionFileSystem().getFileSystem();
6740     // Make sure each region's cache is empty
6741     a.flush(true);
6742     b.flush(true);
6743 
6744     // Compact each region so we only have one store file per family
6745     a.compact(true);
6746     if (LOG.isDebugEnabled()) {
6747       LOG.debug("Files for region: " + a);
6748       a.getRegionFileSystem().logFileSystemState(LOG);
6749     }
6750     b.compact(true);
6751     if (LOG.isDebugEnabled()) {
6752       LOG.debug("Files for region: " + b);
6753       b.getRegionFileSystem().logFileSystemState(LOG);
6754     }
6755 
6756     RegionMergeTransactionImpl rmt = new RegionMergeTransactionImpl(a, b, true);
6757     if (!rmt.prepare(null)) {
6758       throw new IOException("Unable to merge regions " + a + " and " + b);
6759     }
6760     HRegionInfo mergedRegionInfo = rmt.getMergedRegionInfo();
6761     LOG.info("starting merge of regions: " + a + " and " + b
6762         + " into new region " + mergedRegionInfo.getRegionNameAsString()
6763         + " with start key <"
6764         + Bytes.toStringBinary(mergedRegionInfo.getStartKey())
6765         + "> and end key <"
6766         + Bytes.toStringBinary(mergedRegionInfo.getEndKey()) + ">");
6767     HRegion dstRegion;
6768     try {
6769       dstRegion = (HRegion)rmt.execute(null, null);
6770     } catch (IOException ioe) {
6771       rmt.rollback(null, null);
6772       throw new IOException("Failed merging region " + a + " and " + b
6773           + ", and successfully rolled back");
6774     }
6775     dstRegion.compact(true);
6776 
6777     if (LOG.isDebugEnabled()) {
6778       LOG.debug("Files for new region");
6779       dstRegion.getRegionFileSystem().logFileSystemState(LOG);
6780     }
6781 
6782     if (dstRegion.getRegionFileSystem().hasReferences(dstRegion.getTableDesc())) {
6783       throw new IOException("Merged region " + dstRegion
6784           + " still has references after the compaction, is compaction canceled?");
6785     }
6786 
6787     // Archiving the 'A' region
6788     HFileArchiver.archiveRegion(a.getBaseConf(), fs, a.getRegionInfo());
6789     // Archiving the 'B' region
6790     HFileArchiver.archiveRegion(b.getBaseConf(), fs, b.getRegionInfo());
6791 
6792     LOG.info("merge completed. New region is " + dstRegion);
6793     return dstRegion;
6794   }
6795 
6796   @Override
6797   public Result get(final Get get) throws IOException {
6798     checkRow(get.getRow(), "Get");
6799     // Verify families are all valid
6800     if (get.hasFamilies()) {
6801       for (byte [] family: get.familySet()) {
6802         checkFamily(family);
6803       }
6804     } else { // Adding all families to scanner
6805       for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
6806         get.addFamily(family);
6807       }
6808     }
6809     List<Cell> results = get(get, true);
6810     boolean stale = this.getRegionInfo().getReplicaId() != 0;
6811     return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
6812   }
6813 
6814   @Override
6815   public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
6816 
6817     List<Cell> results = new ArrayList<Cell>();
6818 
6819     // pre-get CP hook
6820     if (withCoprocessor && (coprocessorHost != null)) {
6821        if (coprocessorHost.preGet(get, results)) {
6822          return results;
6823        }
6824     }
6825 
6826     Scan scan = new Scan(get);
6827 
6828     RegionScanner scanner = null;
6829     try {
6830       scanner = getScanner(scan);
6831       scanner.next(results);
6832     } finally {
6833       if (scanner != null)
6834         scanner.close();
6835     }
6836 
6837     // post-get CP hook
6838     if (withCoprocessor && (coprocessorHost != null)) {
6839       coprocessorHost.postGet(get, results);
6840     }
6841 
6842     // do after lock
6843     if (this.metricsRegion != null) {
6844       long totalSize = 0L;
6845       for (Cell cell : results) {
6846         totalSize += CellUtil.estimatedSerializedSizeOf(cell);
6847       }
6848       this.metricsRegion.updateGet(totalSize);
6849     }
6850 
6851     return results;
6852   }
6853 
6854   @Override
6855   public void mutateRow(RowMutations rm) throws IOException {
6856     // Don't need nonces here - RowMutations only supports puts and deletes
6857     mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
6858   }
6859 
6860   /**
6861    * Perform atomic mutations within the region w/o nonces.
6862    * See {@link #mutateRowsWithLocks(Collection, Collection, long, long)}
6863    */
6864   public void mutateRowsWithLocks(Collection<Mutation> mutations,
6865       Collection<byte[]> rowsToLock) throws IOException {
6866     mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
6867   }
6868 
6869   /**
6870    * Perform atomic mutations within the region.
6871    * @param mutations The list of mutations to perform.
6872    * <code>mutations</code> can contain operations for multiple rows.
6873    * Caller has to ensure that all rows are contained in this region.
6874    * @param rowsToLock Rows to lock
6875    * @param nonceGroup Optional nonce group of the operation (client Id)
6876    * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
6877    * If multiple rows are locked care should be taken that
6878    * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
6879    * @throws IOException
6880    */
6881   @Override
6882   public void mutateRowsWithLocks(Collection<Mutation> mutations,
6883       Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
6884     MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
6885     processRowsWithLocks(proc, -1, nonceGroup, nonce);
6886   }
6887 
6888   /**
6889    * @return the current load statistics for the the region
6890    */
6891   public ClientProtos.RegionLoadStats getRegionStats() {
6892     if (!regionStatsEnabled) {
6893       return null;
6894     }
6895     ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
6896     stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreSize.get() * 100) / this
6897         .memstoreFlushSize)));
6898     if (rsServices.getHeapMemoryManager() != null) {
6899       // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM,
6900       // so we could just do the calculation below and we'll get a 0.
6901       // treating it as a special case analogous to no HMM instead so that it can be
6902       // programatically treated different from using <1% of heap.
6903       final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent();
6904       if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) {
6905         stats.setHeapOccupancy((int)(occupancy * 100));
6906       }
6907     }
6908     stats.setCompactionPressure((int)rsServices.getCompactionPressure()*100 > 100 ? 100 :
6909                 (int)rsServices.getCompactionPressure()*100);
6910     return stats.build();
6911   }
6912 
6913   @Override
6914   public void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException {
6915     processRowsWithLocks(processor, rowProcessorTimeout, HConstants.NO_NONCE,
6916       HConstants.NO_NONCE);
6917   }
6918 
6919   @Override
6920   public void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce)
6921       throws IOException {
6922     processRowsWithLocks(processor, rowProcessorTimeout, nonceGroup, nonce);
6923   }
6924 
6925   @Override
6926   public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout,
6927       long nonceGroup, long nonce) throws IOException {
6928 
6929     for (byte[] row : processor.getRowsToLock()) {
6930       checkRow(row, "processRowsWithLocks");
6931     }
6932     if (!processor.readOnly()) {
6933       checkReadOnly();
6934     }
6935     checkResources();
6936 
6937     startRegionOperation();
6938     WALEdit walEdit = new WALEdit();
6939 
6940     // 1. Run pre-process hook
6941     try {
6942       processor.preProcess(this, walEdit);
6943     } catch (IOException e) {
6944       closeRegionOperation();
6945       throw e;
6946     }
6947     // Short circuit the read only case
6948     if (processor.readOnly()) {
6949       try {
6950         long now = EnvironmentEdgeManager.currentTime();
6951         doProcessRowWithTimeout(
6952             processor, now, this, null, null, timeout);
6953         processor.postProcess(this, walEdit, true);
6954       } finally {
6955         closeRegionOperation();
6956       }
6957       return;
6958     }
6959 
6960     MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
6961     boolean locked;
6962     boolean walSyncSuccessful = false;
6963     List<RowLock> acquiredRowLocks;
6964     long addedSize = 0;
6965     List<Mutation> mutations = new ArrayList<Mutation>();
6966     Collection<byte[]> rowsToLock = processor.getRowsToLock();
6967     long mvccNum = 0;
6968     WALKey walKey = null;
6969     try {
6970       // 2. Acquire the row lock(s)
6971       acquiredRowLocks = new ArrayList<RowLock>(rowsToLock.size());
6972       for (byte[] row : rowsToLock) {
6973         // Attempt to lock all involved rows, throw if any lock times out
6974         // use a writer lock for mixed reads and writes
6975         acquiredRowLocks.add(getRowLock(row));
6976       }
6977       // 3. Region lock
6978       lock(this.updatesLock.readLock(), acquiredRowLocks.size() == 0 ? 1 : acquiredRowLocks.size());
6979       locked = true;
6980 
6981       long now = EnvironmentEdgeManager.currentTime();
6982       try {
6983         // 4. Let the processor scan the rows, generate mutations and add
6984         //    waledits
6985         doProcessRowWithTimeout(
6986             processor, now, this, mutations, walEdit, timeout);
6987 
6988         if (!mutations.isEmpty()) {
6989 
6990           // 5. Call the preBatchMutate hook
6991           processor.preBatchMutate(this, walEdit);
6992 
6993           long txid = 0;
6994           // 6. Append no sync
6995           if (!walEdit.isEmpty()) {
6996             // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
6997             walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
6998               this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
6999               processor.getClusterIds(), nonceGroup, nonce, mvcc);
7000             txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(),
7001                 walKey, walEdit, true);
7002           }
7003           if(walKey == null){
7004             // since we use wal sequence Id as mvcc, for SKIP_WAL changes we need a "faked" WALEdit
7005             // to get a sequence id assigned which is done by FSWALEntry#stampRegionSequenceId
7006             walKey = this.appendEmptyEdit(this.wal);
7007           }
7008 
7009           // 7. Start mvcc transaction
7010           writeEntry = walKey.getWriteEntry();
7011           mvccNum = walKey.getSequenceId();
7012 
7013 
7014 
7015           // 8. Apply to memstore
7016           for (Mutation m : mutations) {
7017             // Handle any tag based cell features
7018             rewriteCellTags(m.getFamilyCellMap(), m);
7019 
7020             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7021               Cell cell = cellScanner.current();
7022               CellUtil.setSequenceId(cell, mvccNum);
7023               Store store = getStore(cell);
7024               if (store == null) {
7025                 checkFamily(CellUtil.cloneFamily(cell));
7026                 // unreachable
7027               }
7028               addedSize += store.add(cell);
7029             }
7030           }
7031 
7032           // 9. Release region lock
7033           if (locked) {
7034             this.updatesLock.readLock().unlock();
7035             locked = false;
7036           }
7037 
7038           // 10. Release row lock(s)
7039           releaseRowLocks(acquiredRowLocks);
7040 
7041           // 11. Sync edit log
7042           if (txid != 0) {
7043             syncOrDefer(txid, getEffectiveDurability(processor.useDurability()));
7044           }
7045           walSyncSuccessful = true;
7046           // 12. call postBatchMutate hook
7047           processor.postBatchMutate(this);
7048         }
7049       } finally {
7050         // TODO: Make this method look like all other methods that are doing append/sync and
7051         // memstore rollback such as append and doMiniBatchMutation. Currently it is a little
7052         // different. Make them all share same code!
7053         if (!mutations.isEmpty() && !walSyncSuccessful) {
7054           LOG.warn("Wal sync failed. Roll back " + mutations.size() +
7055               " memstore keyvalues for row(s):" + StringUtils.byteToHexString(
7056               processor.getRowsToLock().iterator().next()) + "...");
7057           for (Mutation m : mutations) {
7058             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7059               Cell cell = cellScanner.current();
7060               getStore(cell).rollback(cell);
7061             }
7062           }
7063           if (writeEntry != null) {
7064             mvcc.complete(writeEntry);
7065             writeEntry = null;
7066           }
7067         }
7068         // 13. Roll mvcc forward
7069         if (writeEntry != null) {
7070           mvcc.completeAndWait(writeEntry);
7071         }
7072         if (locked) {
7073           this.updatesLock.readLock().unlock();
7074         }
7075         // release locks if some were acquired but another timed out
7076         releaseRowLocks(acquiredRowLocks);
7077       }
7078 
7079       // 14. Run post-process hook
7080       processor.postProcess(this, walEdit, walSyncSuccessful);
7081 
7082     } finally {
7083       closeRegionOperation();
7084       if (!mutations.isEmpty() &&
7085           isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize))) {
7086         requestFlush();
7087       }
7088     }
7089   }
7090 
7091   private void doProcessRowWithTimeout(final RowProcessor<?,?> processor,
7092                                        final long now,
7093                                        final HRegion region,
7094                                        final List<Mutation> mutations,
7095                                        final WALEdit walEdit,
7096                                        final long timeout) throws IOException {
7097     // Short circuit the no time bound case.
7098     if (timeout < 0) {
7099       try {
7100         processor.process(now, region, mutations, walEdit);
7101       } catch (IOException e) {
7102         LOG.warn("RowProcessor:" + processor.getClass().getName() +
7103             " throws Exception on row(s):" +
7104             Bytes.toStringBinary(
7105               processor.getRowsToLock().iterator().next()) + "...", e);
7106         throw e;
7107       }
7108       return;
7109     }
7110 
7111     // Case with time bound
7112     FutureTask<Void> task =
7113       new FutureTask<Void>(new Callable<Void>() {
7114         @Override
7115         public Void call() throws IOException {
7116           try {
7117             processor.process(now, region, mutations, walEdit);
7118             return null;
7119           } catch (IOException e) {
7120             LOG.warn("RowProcessor:" + processor.getClass().getName() +
7121                 " throws Exception on row(s):" +
7122                 Bytes.toStringBinary(
7123                     processor.getRowsToLock().iterator().next()) + "...", e);
7124             throw e;
7125           }
7126         }
7127       });
7128     rowProcessorExecutor.execute(task);
7129     try {
7130       task.get(timeout, TimeUnit.MILLISECONDS);
7131     } catch (TimeoutException te) {
7132       LOG.error("RowProcessor timeout:" + timeout + " ms on row(s):" +
7133           Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) +
7134           "...");
7135       throw new IOException(te);
7136     } catch (Exception e) {
7137       throw new IOException(e);
7138     }
7139   }
7140 
7141   /**
7142    * @param cell
7143    * @param tags
7144    * @return The passed-in List<Tag> but with the tags from <code>cell</code> added.
7145    */
7146   private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
7147     if (cell.getTagsLength() <= 0) return tags;
7148     List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags;
7149     Iterator<Tag> i =
7150         CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
7151     while (i.hasNext()) newTags.add(i.next());
7152     return newTags;
7153   }
7154 
7155   /**
7156    * Run a Get against passed in <code>store</code> on passed <code>row</code>, etc.
7157    * @param store
7158    * @param row
7159    * @param family
7160    * @param tr
7161    * @return Get result.
7162    * @throws IOException
7163    */
7164   private List<Cell> doGet(final Store store, final byte [] row,
7165       final Map.Entry<byte[], List<Cell>> family, final TimeRange tr)
7166   throws IOException {
7167     // Sort the cells so that they match the order that they
7168     // appear in the Get results. Otherwise, we won't be able to
7169     // find the existing values if the cells are not specified
7170     // in order by the client since cells are in an array list.
7171     Collections.sort(family.getValue(), store.getComparator());
7172     // Get previous values for all columns in this family
7173     Get get = new Get(row);
7174     for (Cell cell : family.getValue()) {
7175       get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell));
7176     }
7177     if (tr != null) get.setTimeRange(tr.getMin(), tr.getMax());
7178     return get(get, false);
7179   }
7180 
7181   public Result append(Append append) throws IOException {
7182     return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
7183   }
7184 
7185   // TODO: There's a lot of boiler plate code identical to increment.
7186   // We should refactor append and increment as local get-mutate-put
7187   // transactions, so all stores only go through one code path for puts.
7188 
7189   // dropMemstoreContentsForSeqId() would acquire write lock of updatesLock
7190   // We perform this operation outside of the read lock of updatesLock to avoid dead lock
7191   // See HBASE-16304
7192   @SuppressWarnings("unchecked")
7193   private void dropMemstoreContents() throws IOException {
7194     long totalFreedSize = 0;
7195     while (!storeSeqIds.isEmpty()) {
7196       Map<Store, Long> map = null;
7197       synchronized (storeSeqIds) {
7198         if (storeSeqIds.isEmpty()) break;
7199         map = storeSeqIds.remove(storeSeqIds.size()-1);
7200       }
7201       for (Map.Entry<Store, Long> entry : map.entrySet()) {
7202         // Drop the memstore contents if they are now smaller than the latest seen flushed file
7203         totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey());
7204       }
7205     }
7206     if (totalFreedSize > 0) {
7207       LOG.debug("Freed " + totalFreedSize + " bytes from memstore");
7208     }
7209   }
7210 
7211   @Override
7212   public Result append(Append mutate, long nonceGroup, long nonce) throws IOException {
7213     Operation op = Operation.APPEND;
7214     byte[] row = mutate.getRow();
7215     checkRow(row, op.toString());
7216     boolean flush = false;
7217     Durability durability = getEffectiveDurability(mutate.getDurability());
7218     boolean writeToWAL = durability != Durability.SKIP_WAL;
7219     WALEdit walEdits = null;
7220     List<Cell> allKVs = new ArrayList<Cell>(mutate.size());
7221     Map<Store, List<Cell>> tempMemstore = new HashMap<Store, List<Cell>>();
7222     long size = 0;
7223     long txid = 0;
7224     checkReadOnly();
7225     checkResources();
7226     // Lock row
7227     startRegionOperation(op);
7228     this.writeRequestsCount.increment();
7229     RowLock rowLock = null;
7230     WALKey walKey = null;
7231     boolean doRollBackMemstore = false;
7232     try {
7233       rowLock = getRowLock(row);
7234       assert rowLock != null;
7235       try {
7236         lock(this.updatesLock.readLock());
7237         try {
7238           // Wait for all prior MVCC transactions to finish - while we hold the row lock
7239           // (so that we are guaranteed to see the latest state when we do our Get)
7240           mvcc.await();
7241           if (this.coprocessorHost != null) {
7242             Result r = this.coprocessorHost.preAppendAfterRowLock(mutate);
7243             if (r!= null) {
7244               return r;
7245             }
7246           }
7247           long now = EnvironmentEdgeManager.currentTime();
7248           // Process each family
7249           for (Map.Entry<byte[], List<Cell>> family : mutate.getFamilyCellMap().entrySet()) {
7250             Store store = stores.get(family.getKey());
7251             List<Cell> kvs = new ArrayList<Cell>(family.getValue().size());
7252 
7253             List<Cell> results = doGet(store, row, family, null);
7254 
7255             // Iterate the input columns and update existing values if they were
7256             // found, otherwise add new column initialized to the append value
7257 
7258             // Avoid as much copying as possible. We may need to rewrite and
7259             // consolidate tags. Bytes are only copied once.
7260             // Would be nice if KeyValue had scatter/gather logic
7261             int idx = 0;
7262             for (Cell cell : family.getValue()) {
7263               Cell newCell;
7264               Cell oldCell = null;
7265               if (idx < results.size()
7266                   && CellUtil.matchingQualifier(results.get(idx), cell)) {
7267                 oldCell = results.get(idx);
7268                 long ts = Math.max(now, oldCell.getTimestamp() + 1);
7269 
7270                 // Process cell tags
7271                 // Make a union of the set of tags in the old and new KVs
7272                 List<Tag> tags = Tag.carryForwardTags(null, oldCell);
7273                 tags = Tag.carryForwardTags(tags, cell);
7274                 tags = carryForwardTTLTag(tags, mutate);
7275 
7276                 newCell = getNewCell(row, ts, cell, oldCell, Tag.fromList(tags));
7277 
7278                 idx++;
7279               } else {
7280                 // Append's KeyValue.Type==Put and ts==HConstants.LATEST_TIMESTAMP
7281                 CellUtil.updateLatestStamp(cell, now);
7282 
7283                 // Cell TTL handling
7284                 newCell = getNewCell(mutate, cell);
7285               }
7286 
7287               // Give coprocessors a chance to update the new cell
7288               if (coprocessorHost != null) {
7289                 newCell = coprocessorHost.postMutationBeforeWAL(RegionObserver.MutationType.APPEND,
7290                     mutate, oldCell, newCell);
7291               }
7292               kvs.add(newCell);
7293 
7294               // Append update to WAL
7295               if (writeToWAL) {
7296                 if (walEdits == null) {
7297                   walEdits = new WALEdit();
7298                 }
7299                 walEdits.add(newCell);
7300               }
7301             }
7302 
7303             //store the kvs to the temporary memstore before writing WAL
7304             tempMemstore.put(store, kvs);
7305           }
7306 
7307           // Actually write to WAL now
7308           if (walEdits != null && !walEdits.isEmpty()) {
7309             if (writeToWAL) {
7310               // Using default cluster id, as this can only happen in the originating
7311               // cluster. A slave cluster receives the final value (not the delta)
7312               // as a Put.
7313               // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
7314               walKey = new HLogKey(
7315                   getRegionInfo().getEncodedNameAsBytes(),
7316                   this.htableDescriptor.getTableName(),
7317                   WALKey.NO_SEQUENCE_ID,
7318                   nonceGroup,
7319                   nonce,
7320                   mvcc);
7321               txid =
7322                 this.wal.append(this.htableDescriptor, getRegionInfo(), walKey, walEdits, true);
7323             } else {
7324               recordMutationWithoutWal(mutate.getFamilyCellMap());
7325             }
7326           }
7327           if (walKey == null) {
7328             // Append a faked WALEdit in order for SKIP_WAL updates to get mvcc assigned
7329             walKey = this.appendEmptyEdit(this.wal);
7330           }
7331           // Do a get on the write entry... this will block until sequenceid is assigned... w/o it,
7332           // TestAtomicOperation fails.
7333           walKey.getWriteEntry();
7334 
7335           // Actually write to Memstore now
7336           if (!tempMemstore.isEmpty()) {
7337             for (Map.Entry<Store, List<Cell>> entry : tempMemstore.entrySet()) {
7338               Store store = entry.getKey();
7339               if (store.getFamily().getMaxVersions() == 1) {
7340                 // upsert if VERSIONS for this CF == 1
7341                 // Is this right? It immediately becomes visible? St.Ack 20150907
7342                 size += store.upsert(entry.getValue(), getSmallestReadPoint());
7343               } else {
7344                 // otherwise keep older versions around
7345                 for (Cell cell: entry.getValue()) {
7346                   // This stamping of sequenceid seems redundant; it is happening down in
7347                   // FSHLog when we consume edits off the ring buffer.
7348                   CellUtil.setSequenceId(cell, walKey.getWriteEntry().getWriteNumber());
7349                   size += store.add(cell);
7350                   doRollBackMemstore = true;
7351                 }
7352               }
7353               // We add to all KVs here whereas when doing increment, we do it
7354               // earlier... why?
7355               allKVs.addAll(entry.getValue());
7356             }
7357 
7358             size = this.addAndGetGlobalMemstoreSize(size);
7359             flush = isFlushSize(size);
7360           }
7361         } finally {
7362           this.updatesLock.readLock().unlock();
7363           // For increment/append, a region scanner for doing a get operation could throw 
7364           // FileNotFoundException. So we call dropMemstoreContents() in finally block
7365           // after releasing read lock
7366           dropMemstoreContents();
7367         }
7368 
7369       } finally {
7370         rowLock.release();
7371         rowLock = null;
7372       }
7373       // sync the transaction log outside the rowlock
7374       if(txid != 0){
7375         syncOrDefer(txid, durability);
7376       }
7377       doRollBackMemstore = false;
7378     } finally {
7379       if (rowLock != null) {
7380         rowLock.release();
7381       }
7382       // if the wal sync was unsuccessful, remove keys from memstore
7383       WriteEntry we = walKey != null? walKey.getWriteEntry(): null;
7384       if (doRollBackMemstore) {
7385         rollbackMemstore(allKVs);
7386         if (we != null) mvcc.complete(we);
7387       } else if (we != null) {
7388         mvcc.completeAndWait(we);
7389       }
7390 
7391       closeRegionOperation(op);
7392     }
7393 
7394     if (this.metricsRegion != null) {
7395       this.metricsRegion.updateAppend();
7396     }
7397 
7398     if (flush) {
7399       // Request a cache flush. Do it outside update lock.
7400       requestFlush();
7401     }
7402 
7403     return mutate.isReturnResults() ? Result.create(allKVs) : null;
7404   }
7405 
7406   private static Cell getNewCell(final byte [] row, final long ts, final Cell cell,
7407       final Cell oldCell, final byte [] tagBytes) {
7408     // allocate an empty cell once
7409     Cell newCell = new KeyValue(row.length, cell.getFamilyLength(),
7410         cell.getQualifierLength(), ts, KeyValue.Type.Put,
7411         oldCell.getValueLength() + cell.getValueLength(),
7412         tagBytes == null? 0: tagBytes.length);
7413     // copy in row, family, and qualifier
7414     System.arraycopy(cell.getRowArray(), cell.getRowOffset(),
7415       newCell.getRowArray(), newCell.getRowOffset(), cell.getRowLength());
7416     System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(),
7417       newCell.getFamilyArray(), newCell.getFamilyOffset(),
7418       cell.getFamilyLength());
7419     System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(),
7420       newCell.getQualifierArray(), newCell.getQualifierOffset(),
7421       cell.getQualifierLength());
7422     // copy in the value
7423     System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(),
7424       newCell.getValueArray(), newCell.getValueOffset(),
7425       oldCell.getValueLength());
7426     System.arraycopy(cell.getValueArray(), cell.getValueOffset(),
7427       newCell.getValueArray(),
7428       newCell.getValueOffset() + oldCell.getValueLength(),
7429       cell.getValueLength());
7430     // Copy in tag data
7431     if (tagBytes != null) {
7432       System.arraycopy(tagBytes, 0, newCell.getTagsArray(), newCell.getTagsOffset(),
7433         tagBytes.length);
7434     }
7435     return newCell;
7436   }
7437 
7438   private static Cell getNewCell(final Mutation mutate, final Cell cell) {
7439     Cell newCell = null;
7440     if (mutate.getTTL() != Long.MAX_VALUE) {
7441       // Add the new TTL tag
7442       newCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
7443           cell.getRowLength(),
7444         cell.getFamilyArray(), cell.getFamilyOffset(),
7445           cell.getFamilyLength(),
7446         cell.getQualifierArray(), cell.getQualifierOffset(),
7447           cell.getQualifierLength(),
7448         cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
7449         cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
7450         carryForwardTTLTag(mutate));
7451     } else {
7452       newCell = cell;
7453     }
7454     return newCell;
7455   }
7456 
7457   public Result increment(Increment increment) throws IOException {
7458     return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
7459   }
7460 
7461   // TODO: There's a lot of boiler plate code identical to append.
7462   // We should refactor append and increment as local get-mutate-put
7463   // transactions, so all stores only go through one code path for puts.
7464 
7465   // They are subtley different in quiet a few ways. This came out only
7466   // after study. I am not sure that many of the differences are intentional.
7467   // TODO: St.Ack 20150907
7468 
7469   @Override
7470   public Result increment(Increment mutation, long nonceGroup, long nonce)
7471   throws IOException {
7472     Operation op = Operation.INCREMENT;
7473     checkReadOnly();
7474     checkResources();
7475     checkRow(mutation.getRow(), op.toString());
7476     checkFamilies(mutation.getFamilyCellMap().keySet());
7477     startRegionOperation(op);
7478     this.writeRequestsCount.increment();
7479     try {
7480       // Which Increment is it? Narrow increment-only consistency or slow (default) and general
7481       // row-wide consistency.
7482 
7483       // So, difference between fastAndNarrowConsistencyIncrement and slowButConsistentIncrement is
7484       // that the former holds the row lock until the sync completes; this allows us to reason that
7485       // there are no other writers afoot when we read the current increment value. The row lock
7486       // means that we do not need to wait on mvcc reads to catch up to writes before we proceed
7487       // with the read, the root of the slowdown seen in HBASE-14460. The fast-path also does not
7488       // wait on mvcc to complete before returning to the client. We also reorder the write so that
7489       // the update of memstore happens AFTER sync returns; i.e. the write pipeline does less
7490       // zigzagging now.
7491       //
7492       // See the comment on INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY
7493       // for the constraints that apply when you take this code path; it is correct but only if
7494       // Increments are used mutating an Increment Cell; mixing concurrent Put+Delete and Increment
7495       // will yield indeterminate results.
7496       return doIncrement(mutation, nonceGroup, nonce);
7497     } finally {
7498       if (this.metricsRegion != null) this.metricsRegion.updateIncrement();
7499       closeRegionOperation(op);
7500     }
7501   }
7502 
7503   private Result doIncrement(Increment increment, long nonceGroup, long nonce) throws IOException {
7504     RowLock rowLock = null;
7505     WALKey walKey = null;
7506     boolean doRollBackMemstore = false;
7507     long accumulatedResultSize = 0;
7508     List<Cell> allKVs = new ArrayList<Cell>(increment.size());
7509     List<Cell> memstoreCells = new ArrayList<Cell>();
7510     Durability effectiveDurability = getEffectiveDurability(increment.getDurability());
7511     try {
7512       rowLock = getRowLock(increment.getRow());
7513       long txid = 0;
7514       try {
7515         lock(this.updatesLock.readLock());
7516         try {
7517           // Wait for all prior MVCC transactions to finish - while we hold the row lock
7518           // (so that we are guaranteed to see the latest increment)
7519           this.mvcc.await();
7520           if (this.coprocessorHost != null) {
7521             Result r = this.coprocessorHost.preIncrementAfterRowLock(increment);
7522             if (r != null) return r;
7523           }
7524           long now = EnvironmentEdgeManager.currentTime();
7525           final boolean writeToWAL = effectiveDurability != Durability.SKIP_WAL;
7526           WALEdit walEdits = null;
7527           // Process increments a Store/family at a time.
7528           // Accumulate edits for memstore to add later after we've added to WAL.
7529           Map<Store, List<Cell>> forMemStore = new HashMap<Store, List<Cell>>();
7530           for (Map.Entry<byte [], List<Cell>> entry: increment.getFamilyCellMap().entrySet()) {
7531             byte [] columnFamilyName = entry.getKey();
7532             List<Cell> increments = entry.getValue();
7533             Store store = this.stores.get(columnFamilyName);
7534             // Do increment for this store; be sure to 'sort' the increments first so increments
7535             // match order in which we get back current Cells when we get.
7536             List<Cell> results = applyIncrementsToColumnFamily(increment, columnFamilyName,
7537                 sort(increments, store.getComparator()), now,
7538                 MultiVersionConcurrencyControl.NO_WRITE_NUMBER, allKVs, null);
7539             if (!results.isEmpty()) {
7540               forMemStore.put(store, results);
7541               // Prepare WAL updates
7542               if (writeToWAL) {
7543                 if (walEdits == null) walEdits = new WALEdit();
7544                 walEdits.getCells().addAll(results);
7545               }
7546             }
7547           }
7548           // Actually write to WAL now. If walEdits is non-empty, we write the WAL.
7549           if (walEdits != null && !walEdits.isEmpty()) {
7550             // Using default cluster id, as this can only happen in the originating cluster.
7551             // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey
7552             // here instead of WALKey directly to support legacy coprocessors.
7553             walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
7554               this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce,
7555               getMVCC());
7556             txid =
7557               this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdits, true);
7558           } else {
7559             // Append a faked WALEdit in order for SKIP_WAL updates to get mvccNum assigned
7560             walKey = this.appendEmptyEdit(this.wal);
7561           }
7562           // Get WriteEntry. Will wait on assign of the sequence id.
7563           walKey.getWriteEntry();
7564 
7565           // Now write to memstore, a family at a time.
7566           for (Map.Entry<Store, List<Cell>> entry: forMemStore.entrySet()) {
7567             Store store = entry.getKey();
7568             List<Cell> results = entry.getValue();
7569             if (store.getFamily().getMaxVersions() == 1) {
7570               // Upsert if VERSIONS for this CF == 1
7571               accumulatedResultSize += store.upsert(results, getSmallestReadPoint());
7572               // TODO: St.Ack 20151222 Why no rollback in this case?
7573             } else {
7574               // Otherwise keep older versions around
7575               for (Cell cell: results) {
7576                 // Why we need this?
7577                 CellUtil.setSequenceId(cell, walKey.getWriteEntry().getWriteNumber());
7578                 accumulatedResultSize += store.add(cell);
7579                 doRollBackMemstore = true;
7580               }
7581             }
7582           }
7583         } finally {
7584           this.updatesLock.readLock().unlock();
7585           // For increment/append, a region scanner for doing a get operation could throw 
7586           // FileNotFoundException. So we call dropMemstoreContents() in finally block
7587           // after releasing read lock
7588           dropMemstoreContents();
7589         }
7590       } finally {
7591         rowLock.release();
7592         rowLock = null;
7593       }
7594       // sync the transaction log outside the rowlock
7595       if(txid != 0) {
7596         syncOrDefer(txid, durability);
7597       }
7598       doRollBackMemstore = false;
7599     } finally {
7600       if (rowLock != null) {
7601         rowLock.release();
7602       }
7603       // if the wal sync was unsuccessful, remove keys from memstore
7604       if (doRollBackMemstore) {
7605         rollbackMemstore(memstoreCells);
7606         if (walKey != null) mvcc.complete(walKey.getWriteEntry());
7607       } else {
7608         if (walKey != null) mvcc.completeAndWait(walKey.getWriteEntry());
7609       }
7610     }
7611 
7612     // Request a cache flush.  Do it outside update lock.
7613     if (isFlushSize(this.addAndGetGlobalMemstoreSize(accumulatedResultSize))) requestFlush();
7614     return increment.isReturnResults() ? Result.create(allKVs) : null;
7615   }
7616 
7617   /**
7618    * @return Sorted list of <code>cells</code> using <code>comparator</code>
7619    */
7620   private static List<Cell> sort(List<Cell> cells, final Comparator<Cell> comparator) {
7621     Collections.sort(cells, comparator);
7622     return cells;
7623   }
7624 
7625   /**
7626    * Apply increments to a column family.
7627    * @param sortedIncrements The passed in increments to apply MUST be sorted so that they match
7628    * the order that they appear in the Get results (get results will be sorted on return).
7629    * Otherwise, we won't be able to find the existing values if the cells are not specified in
7630    * order by the client since cells are in an array list.
7631    * @islation Isolation level to use when running the 'get'. Pass null for default.
7632    * @return Resulting increments after <code>sortedIncrements</code> have been applied to current
7633    * values (if any -- else passed increment is the final result).
7634    * @throws IOException
7635    */
7636   private List<Cell> applyIncrementsToColumnFamily(Increment increment, byte[] columnFamilyName,
7637       List<Cell> sortedIncrements, long now, long mvccNum, List<Cell> allKVs,
7638       final IsolationLevel isolation)
7639   throws IOException {
7640     List<Cell> results = new ArrayList<Cell>(sortedIncrements.size());
7641     byte [] row = increment.getRow();
7642     // Get previous values for all columns in this family
7643     List<Cell> currentValues =
7644         getIncrementCurrentValue(increment, columnFamilyName, sortedIncrements, isolation);
7645     // Iterate the input columns and update existing values if they were found, otherwise
7646     // add new column initialized to the increment amount
7647     int idx = 0;
7648     for (int i = 0; i < sortedIncrements.size(); i++) {
7649       Cell inc = sortedIncrements.get(i);
7650       long incrementAmount = getLongValue(inc);
7651       // If increment amount == 0, then don't write this Increment to the WAL.
7652       boolean writeBack = (incrementAmount != 0);
7653       // Carry forward any tags that might have been added by a coprocessor.
7654       List<Tag> tags = Tag.carryForwardTags(inc);
7655 
7656       Cell currentValue = null;
7657       long ts = now;
7658       if (idx < currentValues.size() && CellUtil.matchingQualifier(currentValues.get(idx), inc)) {
7659         currentValue = currentValues.get(idx);
7660         ts = Math.max(now, currentValue.getTimestamp() + 1);
7661         incrementAmount += getLongValue(currentValue);
7662         // Carry forward all tags
7663         tags = Tag.carryForwardTags(tags, currentValue);
7664         if (i < (sortedIncrements.size() - 1) &&
7665             !CellUtil.matchingQualifier(inc, sortedIncrements.get(i + 1))) idx++;
7666       }
7667 
7668       // Append new incremented KeyValue to list
7669       byte [] qualifier = CellUtil.cloneQualifier(inc);
7670       byte [] incrementAmountInBytes = Bytes.toBytes(incrementAmount);
7671       tags = carryForwardTTLTag(tags, increment);
7672 
7673       Cell newValue = new KeyValue(row, 0, row.length,
7674         columnFamilyName, 0, columnFamilyName.length,
7675         qualifier, 0, qualifier.length,
7676         ts, KeyValue.Type.Put,
7677         incrementAmountInBytes, 0, incrementAmountInBytes.length,
7678         tags);
7679 
7680       // Don't set an mvcc if none specified. The mvcc may be assigned later in case where we
7681       // write the memstore AFTER we sync our edit to the log.
7682       if (mvccNum != MultiVersionConcurrencyControl.NO_WRITE_NUMBER) {
7683         CellUtil.setSequenceId(newValue, mvccNum);
7684       }
7685 
7686       // Give coprocessors a chance to update the new cell
7687       if (coprocessorHost != null) {
7688         newValue = coprocessorHost.postMutationBeforeWAL(
7689             RegionObserver.MutationType.INCREMENT, increment, currentValue, newValue);
7690       }
7691       allKVs.add(newValue);
7692       if (writeBack) {
7693         results.add(newValue);
7694       }
7695     }
7696     return results;
7697   }
7698 
7699   /**
7700    * @return Get the long out of the passed in Cell
7701    * @throws DoNotRetryIOException
7702    */
7703   private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
7704     int len = cell.getValueLength();
7705     if (len != Bytes.SIZEOF_LONG) {
7706       // throw DoNotRetryIOException instead of IllegalArgumentException
7707       throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
7708     }
7709     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), len);
7710   }
7711 
7712   /**
7713    * Do a specific Get on passed <code>columnFamily</code> and column qualifiers
7714    * from <code>incrementCoordinates</code> only.
7715    * @param increment
7716    * @param columnFamily
7717    * @param incrementCoordinates
7718    * @return Return the Cells to Increment
7719    * @throws IOException
7720    */
7721   private List<Cell> getIncrementCurrentValue(final Increment increment, byte [] columnFamily,
7722       final List<Cell> increments, final IsolationLevel isolation)
7723   throws IOException {
7724     Get get = new Get(increment.getRow());
7725     if (isolation != null) get.setIsolationLevel(isolation);
7726     for (Cell cell: increments) {
7727       get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
7728     }
7729     TimeRange tr = increment.getTimeRange();
7730     if (tr != null) {
7731       get.setTimeRange(tr.getMin(), tr.getMax());
7732     }
7733     return get(get, false);
7734   }
7735 
7736   private