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       // calling the post CP hook for batch mutation
3223       if (!isInReplay && coprocessorHost != null) {
3224         MiniBatchOperationInProgress<Mutation> miniBatchOp =
3225           new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3226           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
3227         coprocessorHost.postBatchMutate(miniBatchOp);
3228       }
3229 
3230       // ------------------------------------------------------------------
3231       // STEP 8. Advance mvcc. This will make this put visible to scanners and getters.
3232       // ------------------------------------------------------------------
3233       if (writeEntry != null) {
3234         mvcc.completeAndWait(writeEntry);
3235         writeEntry = null;
3236       } else if (isInReplay) {
3237         // ensure that the sequence id of the region is at least as big as orig log seq id
3238         mvcc.advanceTo(mvccNum);
3239       }
3240 
3241       for (int i = firstIndex; i < lastIndexExclusive; i ++) {
3242         if (batchOp.retCodeDetails[i] == OperationStatus.NOT_RUN) {
3243           batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
3244         }
3245       }
3246 
3247       // ------------------------------------
3248       // STEP 9. Run coprocessor post hooks. This should be done after the wal is
3249       // synced so that the coprocessor contract is adhered to.
3250       // ------------------------------------
3251       if (!isInReplay && coprocessorHost != null) {
3252         for (int i = firstIndex; i < lastIndexExclusive; i++) {
3253           // only for successful puts
3254           if (batchOp.retCodeDetails[i].getOperationStatusCode()
3255               != OperationStatusCode.SUCCESS) {
3256             continue;
3257           }
3258           Mutation m = batchOp.getMutation(i);
3259           if (m instanceof Put) {
3260             coprocessorHost.postPut((Put) m, walEdit, m.getDurability());
3261           } else {
3262             coprocessorHost.postDelete((Delete) m, walEdit, m.getDurability());
3263           }
3264         }
3265       }
3266 
3267       success = true;
3268       return addedSize;
3269     } finally {
3270       // if the wal sync was unsuccessful, remove keys from memstore
3271       if (doRollBackMemstore) {
3272         for (int j = 0; j < familyMaps.length; j++) {
3273           for(List<Cell> cells:familyMaps[j].values()) {
3274             rollbackMemstore(cells);
3275           }
3276         }
3277         if (writeEntry != null) mvcc.complete(writeEntry);
3278       } else {
3279         this.addAndGetGlobalMemstoreSize(addedSize);
3280         if (writeEntry != null) {
3281           mvcc.completeAndWait(writeEntry);
3282         }
3283       }
3284 
3285       if (locked) {
3286         this.updatesLock.readLock().unlock();
3287       }
3288       releaseRowLocks(acquiredRowLocks);
3289 
3290       // See if the column families were consistent through the whole thing.
3291       // if they were then keep them. If they were not then pass a null.
3292       // null will be treated as unknown.
3293       // Total time taken might be involving Puts and Deletes.
3294       // Split the time for puts and deletes based on the total number of Puts and Deletes.
3295 
3296       if (noOfPuts > 0) {
3297         // There were some Puts in the batch.
3298         if (this.metricsRegion != null) {
3299           this.metricsRegion.updatePut();
3300         }
3301       }
3302       if (noOfDeletes > 0) {
3303         // There were some Deletes in the batch.
3304         if (this.metricsRegion != null) {
3305           this.metricsRegion.updateDelete();
3306         }
3307       }
3308       if (!success) {
3309         for (int i = firstIndex; i < lastIndexExclusive; i++) {
3310           if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
3311             batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
3312           }
3313         }
3314       }
3315       if (coprocessorHost != null && !batchOp.isInReplay()) {
3316         // call the coprocessor hook to do any finalization steps
3317         // after the put is done
3318         MiniBatchOperationInProgress<Mutation> miniBatchOp =
3319             new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3320                 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex,
3321                 lastIndexExclusive);
3322         coprocessorHost.postBatchMutateIndispensably(miniBatchOp, success);
3323       }
3324 
3325       batchOp.nextIndexToProcess = lastIndexExclusive;
3326     }
3327   }
3328 
3329   /**
3330    * Returns effective durability from the passed durability and
3331    * the table descriptor.
3332    */
3333   protected Durability getEffectiveDurability(Durability d) {
3334     return d == Durability.USE_DEFAULT ? this.durability : d;
3335   }
3336 
3337   //TODO, Think that gets/puts and deletes should be refactored a bit so that
3338   //the getting of the lock happens before, so that you would just pass it into
3339   //the methods. So in the case of checkAndMutate you could just do lockRow,
3340   //get, put, unlockRow or something
3341 
3342   @Override
3343   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
3344       CompareOp compareOp, ByteArrayComparable comparator, Mutation w,
3345       boolean writeToWAL)
3346   throws IOException{
3347     checkReadOnly();
3348     //TODO, add check for value length or maybe even better move this to the
3349     //client if this becomes a global setting
3350     checkResources();
3351     boolean isPut = w instanceof Put;
3352     if (!isPut && !(w instanceof Delete))
3353       throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " +
3354           "be Put or Delete");
3355     if (!Bytes.equals(row, w.getRow())) {
3356       throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " +
3357           "getRow must match the passed row");
3358     }
3359 
3360     startRegionOperation();
3361     try {
3362       Get get = new Get(row);
3363       checkFamily(family);
3364       get.addColumn(family, qualifier);
3365 
3366       // Lock row - note that doBatchMutate will relock this row if called
3367       RowLock rowLock = getRowLock(get.getRow());
3368       // wait for all previous transactions to complete (with lock held)
3369       mvcc.await();
3370       try {
3371         if (this.getCoprocessorHost() != null) {
3372           Boolean processed = null;
3373           if (w instanceof Put) {
3374             processed = this.getCoprocessorHost().preCheckAndPutAfterRowLock(row, family,
3375                 qualifier, compareOp, comparator, (Put) w);
3376           } else if (w instanceof Delete) {
3377             processed = this.getCoprocessorHost().preCheckAndDeleteAfterRowLock(row, family,
3378                 qualifier, compareOp, comparator, (Delete) w);
3379           }
3380           if (processed != null) {
3381             return processed;
3382           }
3383         }
3384         List<Cell> result = get(get, false);
3385 
3386         boolean valueIsNull = comparator.getValue() == null ||
3387           comparator.getValue().length == 0;
3388         boolean matches = false;
3389         long cellTs = 0;
3390         if (result.size() == 0 && valueIsNull) {
3391           matches = true;
3392         } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
3393             valueIsNull) {
3394           matches = true;
3395           cellTs = result.get(0).getTimestamp();
3396         } else if (result.size() == 1 && !valueIsNull) {
3397           Cell kv = result.get(0);
3398           cellTs = kv.getTimestamp();
3399           int compareResult = comparator.compareTo(kv.getValueArray(),
3400               kv.getValueOffset(), kv.getValueLength());
3401           switch (compareOp) {
3402           case LESS:
3403             matches = compareResult < 0;
3404             break;
3405           case LESS_OR_EQUAL:
3406             matches = compareResult <= 0;
3407             break;
3408           case EQUAL:
3409             matches = compareResult == 0;
3410             break;
3411           case NOT_EQUAL:
3412             matches = compareResult != 0;
3413             break;
3414           case GREATER_OR_EQUAL:
3415             matches = compareResult >= 0;
3416             break;
3417           case GREATER:
3418             matches = compareResult > 0;
3419             break;
3420           default:
3421             throw new RuntimeException("Unknown Compare op " + compareOp.name());
3422           }
3423         }
3424         //If matches put the new put or delete the new delete
3425         if (matches) {
3426           // We have acquired the row lock already. If the system clock is NOT monotonically
3427           // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
3428           // larger timestamp than what was observed via Get. doBatchMutate already does this, but
3429           // there is no way to pass the cellTs. See HBASE-14054.
3430           long now = EnvironmentEdgeManager.currentTime();
3431           long ts = Math.max(now, cellTs); // ensure write is not eclipsed
3432           byte[] byteTs = Bytes.toBytes(ts);
3433 
3434           if (w instanceof Put) {
3435             updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
3436           }
3437           // else delete is not needed since it already does a second get, and sets the timestamp
3438           // from get (see prepareDeleteTimestamps).
3439 
3440           // All edits for the given row (across all column families) must
3441           // happen atomically.
3442           doBatchMutate(w);
3443           this.checkAndMutateChecksPassed.increment();
3444           return true;
3445         }
3446         this.checkAndMutateChecksFailed.increment();
3447         return false;
3448       } finally {
3449         rowLock.release();
3450       }
3451     } finally {
3452       closeRegionOperation();
3453     }
3454   }
3455 
3456   //TODO, Think that gets/puts and deletes should be refactored a bit so that
3457   //the getting of the lock happens before, so that you would just pass it into
3458   //the methods. So in the case of checkAndMutate you could just do lockRow,
3459   //get, put, unlockRow or something
3460 
3461   @Override
3462   public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
3463       CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm,
3464       boolean writeToWAL) throws IOException {
3465     checkReadOnly();
3466     //TODO, add check for value length or maybe even better move this to the
3467     //client if this becomes a global setting
3468     checkResources();
3469 
3470     startRegionOperation();
3471     try {
3472       Get get = new Get(row);
3473       checkFamily(family);
3474       get.addColumn(family, qualifier);
3475 
3476       // Lock row - note that doBatchMutate will relock this row if called
3477       RowLock rowLock = getRowLock(get.getRow());
3478       // wait for all previous transactions to complete (with lock held)
3479       mvcc.await();
3480       try {
3481         List<Cell> result = get(get, false);
3482 
3483         boolean valueIsNull = comparator.getValue() == null ||
3484             comparator.getValue().length == 0;
3485         boolean matches = false;
3486         long cellTs = 0;
3487         if (result.size() == 0 && valueIsNull) {
3488           matches = true;
3489         } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
3490             valueIsNull) {
3491           matches = true;
3492           cellTs = result.get(0).getTimestamp();
3493         } else if (result.size() == 1 && !valueIsNull) {
3494           Cell kv = result.get(0);
3495           cellTs = kv.getTimestamp();
3496           int compareResult = comparator.compareTo(kv.getValueArray(),
3497               kv.getValueOffset(), kv.getValueLength());
3498           switch (compareOp) {
3499           case LESS:
3500             matches = compareResult < 0;
3501             break;
3502           case LESS_OR_EQUAL:
3503             matches = compareResult <= 0;
3504             break;
3505           case EQUAL:
3506             matches = compareResult == 0;
3507             break;
3508           case NOT_EQUAL:
3509             matches = compareResult != 0;
3510             break;
3511           case GREATER_OR_EQUAL:
3512             matches = compareResult >= 0;
3513             break;
3514           case GREATER:
3515             matches = compareResult > 0;
3516             break;
3517           default:
3518             throw new RuntimeException("Unknown Compare op " + compareOp.name());
3519           }
3520         }
3521         //If matches put the new put or delete the new delete
3522         if (matches) {
3523           // We have acquired the row lock already. If the system clock is NOT monotonically
3524           // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
3525           // larger timestamp than what was observed via Get. doBatchMutate already does this, but
3526           // there is no way to pass the cellTs. See HBASE-14054.
3527           long now = EnvironmentEdgeManager.currentTime();
3528           long ts = Math.max(now, cellTs); // ensure write is not eclipsed
3529           byte[] byteTs = Bytes.toBytes(ts);
3530 
3531           for (Mutation w : rm.getMutations()) {
3532             if (w instanceof Put) {
3533               updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
3534             }
3535             // else delete is not needed since it already does a second get, and sets the timestamp
3536             // from get (see prepareDeleteTimestamps).
3537           }
3538 
3539           // All edits for the given row (across all column families) must
3540           // happen atomically.
3541           mutateRow(rm);
3542           this.checkAndMutateChecksPassed.increment();
3543           return true;
3544         }
3545         this.checkAndMutateChecksFailed.increment();
3546         return false;
3547       } finally {
3548         rowLock.release();
3549       }
3550     } finally {
3551       closeRegionOperation();
3552     }
3553   }
3554 
3555   private void doBatchMutate(Mutation mutation) throws IOException {
3556     // Currently this is only called for puts and deletes, so no nonces.
3557     OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation});
3558     if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
3559       throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
3560     } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
3561       throw new NoSuchColumnFamilyException(batchMutate[0].getExceptionMsg());
3562     }
3563   }
3564 
3565   /**
3566    * Complete taking the snapshot on the region. Writes the region info and adds references to the
3567    * working snapshot directory.
3568    *
3569    * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
3570    * arg.  (In the future other cancellable HRegion methods could eventually add a
3571    * {@link ForeignExceptionSnare}, or we could do something fancier).
3572    *
3573    * @param desc snapshot description object
3574    * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to
3575    *   bail out.  This is allowed to be null and will just be ignored in that case.
3576    * @throws IOException if there is an external or internal error causing the snapshot to fail
3577    */
3578   public void addRegionToSnapshot(SnapshotDescription desc,
3579       ForeignExceptionSnare exnSnare) throws IOException {
3580     Path rootDir = FSUtils.getRootDir(conf);
3581     Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir);
3582 
3583     SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
3584                                                         snapshotDir, desc, exnSnare);
3585     manifest.addRegion(this);
3586   }
3587 
3588   @Override
3589   public void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
3590       throws IOException {
3591     for (List<Cell> cells: cellItr) {
3592       if (cells == null) continue;
3593       assert cells instanceof RandomAccess;
3594       int listSize = cells.size();
3595       for (int i = 0; i < listSize; i++) {
3596         CellUtil.updateLatestStamp(cells.get(i), now, 0);
3597       }
3598     }
3599   }
3600 
3601   /**
3602    * Possibly rewrite incoming cell tags.
3603    */
3604   void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
3605     // Check if we have any work to do and early out otherwise
3606     // Update these checks as more logic is added here
3607 
3608     if (m.getTTL() == Long.MAX_VALUE) {
3609       return;
3610     }
3611 
3612     // From this point we know we have some work to do
3613 
3614     for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
3615       List<Cell> cells = e.getValue();
3616       assert cells instanceof RandomAccess;
3617       int listSize = cells.size();
3618       for (int i = 0; i < listSize; i++) {
3619         Cell cell = cells.get(i);
3620         List<Tag> newTags = Tag.carryForwardTags(null, cell);
3621         newTags = carryForwardTTLTag(newTags, m);
3622 
3623         // Rewrite the cell with the updated set of tags
3624         cells.set(i, new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
3625           cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
3626           cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
3627           cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
3628           cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
3629           newTags));
3630       }
3631     }
3632   }
3633 
3634   /*
3635    * Check if resources to support an update.
3636    *
3637    * We throw RegionTooBusyException if above memstore limit
3638    * and expect client to retry using some kind of backoff
3639   */
3640   private void checkResources() throws RegionTooBusyException {
3641     // If catalog region, do not impose resource constraints or block updates.
3642     if (this.getRegionInfo().isMetaRegion()) return;
3643 
3644     if (this.memstoreSize.get() > this.blockingMemStoreSize) {
3645       blockedRequestsCount.increment();
3646       requestFlush();
3647       throw new RegionTooBusyException("Above memstore limit, " +
3648           "regionName=" + (this.getRegionInfo() == null ? "unknown" :
3649           this.getRegionInfo().getRegionNameAsString()) +
3650           ", server=" + (this.getRegionServerServices() == null ? "unknown" :
3651           this.getRegionServerServices().getServerName()) +
3652           ", memstoreSize=" + memstoreSize.get() +
3653           ", blockingMemStoreSize=" + blockingMemStoreSize);
3654     }
3655   }
3656 
3657   /**
3658    * @throws IOException Throws exception if region is in read-only mode.
3659    */
3660   protected void checkReadOnly() throws IOException {
3661     if (isReadOnly()) {
3662       throw new DoNotRetryIOException("region is read only");
3663     }
3664   }
3665 
3666   protected void checkReadsEnabled() throws IOException {
3667     if (!this.writestate.readsEnabled) {
3668       throw new IOException(getRegionInfo().getEncodedName()
3669         + ": The region's reads are disabled. Cannot serve the request");
3670     }
3671   }
3672 
3673   public void setReadsEnabled(boolean readsEnabled) {
3674    if (readsEnabled && !this.writestate.readsEnabled) {
3675      LOG.info(getRegionInfo().getEncodedName() + " : Enabling reads for region.");
3676     }
3677     this.writestate.setReadsEnabled(readsEnabled);
3678   }
3679 
3680   /**
3681    * Add updates first to the wal and then add values to memstore.
3682    * Warning: Assumption is caller has lock on passed in row.
3683    * @param edits Cell updates by column
3684    * @throws IOException
3685    */
3686   private void put(final byte [] row, byte [] family, List<Cell> edits)
3687   throws IOException {
3688     NavigableMap<byte[], List<Cell>> familyMap;
3689     familyMap = new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
3690 
3691     familyMap.put(family, edits);
3692     Put p = new Put(row);
3693     p.setFamilyCellMap(familyMap);
3694     doBatchMutate(p);
3695   }
3696 
3697   /**
3698    * Atomically apply the given map of family->edits to the memstore.
3699    * This handles the consistency control on its own, but the caller
3700    * should already have locked updatesLock.readLock(). This also does
3701    * <b>not</b> check the families for validity.
3702    *
3703    * @param familyMap Map of kvs per family
3704    * @param localizedWriteEntry The WriteEntry of the MVCC for this transaction.
3705    *        If null, then this method internally creates a mvcc transaction.
3706    * @param output newly added KVs into memstore
3707    * @param isInReplay true when adding replayed KVs into memstore
3708    * @return the additional memory usage of the memstore caused by the
3709    * new entries.
3710    * @throws IOException
3711    */
3712   private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap,
3713     long mvccNum, boolean isInReplay) throws IOException {
3714     long size = 0;
3715 
3716     for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3717       byte[] family = e.getKey();
3718       List<Cell> cells = e.getValue();
3719       assert cells instanceof RandomAccess;
3720       Store store = getStore(family);
3721       int listSize = cells.size();
3722       for (int i=0; i < listSize; i++) {
3723         Cell cell = cells.get(i);
3724         if (cell.getSequenceId() == 0 || isInReplay) {
3725           CellUtil.setSequenceId(cell, mvccNum);
3726         }
3727         size += store.add(cell);
3728       }
3729     }
3730 
3731      return size;
3732    }
3733 
3734   /**
3735    * Remove all the keys listed in the map from the memstore. This method is
3736    * called when a Put/Delete has updated memstore but subsequently fails to update
3737    * the wal. This method is then invoked to rollback the memstore.
3738    */
3739   private void rollbackMemstore(List<Cell> memstoreCells) {
3740     int kvsRolledback = 0;
3741 
3742     for (Cell cell : memstoreCells) {
3743       byte[] family = CellUtil.cloneFamily(cell);
3744       Store store = getStore(family);
3745       store.rollback(cell);
3746       kvsRolledback++;
3747     }
3748     LOG.debug("rollbackMemstore rolled back " + kvsRolledback);
3749   }
3750 
3751   @Override
3752   public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException {
3753     for (byte[] family : families) {
3754       checkFamily(family);
3755     }
3756   }
3757 
3758   /**
3759    * During replay, there could exist column families which are removed between region server
3760    * failure and replay
3761    */
3762   private void removeNonExistentColumnFamilyForReplay(
3763       final Map<byte[], List<Cell>> familyMap) {
3764     List<byte[]> nonExistentList = null;
3765     for (byte[] family : familyMap.keySet()) {
3766       if (!this.htableDescriptor.hasFamily(family)) {
3767         if (nonExistentList == null) {
3768           nonExistentList = new ArrayList<byte[]>();
3769         }
3770         nonExistentList.add(family);
3771       }
3772     }
3773     if (nonExistentList != null) {
3774       for (byte[] family : nonExistentList) {
3775         // Perhaps schema was changed between crash and replay
3776         LOG.info("No family for " + Bytes.toString(family) + " omit from reply.");
3777         familyMap.remove(family);
3778       }
3779     }
3780   }
3781 
3782   @Override
3783   public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
3784       throws FailedSanityCheckException {
3785     if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
3786       return;
3787     }
3788     long maxTs = now + timestampSlop;
3789     for (List<Cell> kvs : familyMap.values()) {
3790       assert kvs instanceof RandomAccess;
3791       int listSize  = kvs.size();
3792       for (int i=0; i < listSize; i++) {
3793         Cell cell = kvs.get(i);
3794         // see if the user-side TS is out of range. latest = server-side
3795         long ts = cell.getTimestamp();
3796         if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
3797           throw new FailedSanityCheckException("Timestamp for KV out of range "
3798               + cell + " (too.new=" + timestampSlop + ")");
3799         }
3800       }
3801     }
3802   }
3803 
3804   /**
3805    * Append the given map of family->edits to a WALEdit data structure.
3806    * This does not write to the WAL itself.
3807    * @param familyMap map of family->edits
3808    * @param walEdit the destination entry to append into
3809    */
3810   private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
3811       WALEdit walEdit) {
3812     for (List<Cell> edits : familyMap.values()) {
3813       assert edits instanceof RandomAccess;
3814       int listSize = edits.size();
3815       for (int i=0; i < listSize; i++) {
3816         Cell cell = edits.get(i);
3817         walEdit.add(cell);
3818       }
3819     }
3820   }
3821 
3822   private void requestFlush() {
3823     if (this.rsServices == null) {
3824       return;
3825     }
3826     synchronized (writestate) {
3827       if (this.writestate.isFlushRequested()) {
3828         return;
3829       }
3830       writestate.flushRequested = true;
3831     }
3832     // Make request outside of synchronize block; HBASE-818.
3833     this.rsServices.getFlushRequester().requestFlush(this, false);
3834     if (LOG.isDebugEnabled()) {
3835       LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
3836     }
3837   }
3838 
3839   /*
3840    * @param size
3841    * @return True if size is over the flush threshold
3842    */
3843   private boolean isFlushSize(final long size) {
3844     return size > this.memstoreFlushSize;
3845   }
3846 
3847   /**
3848    * Read the edits put under this region by wal splitting process.  Put
3849    * the recovered edits back up into this region.
3850    *
3851    * <p>We can ignore any wal message that has a sequence ID that's equal to or
3852    * lower than minSeqId.  (Because we know such messages are already
3853    * reflected in the HFiles.)
3854    *
3855    * <p>While this is running we are putting pressure on memory yet we are
3856    * outside of our usual accounting because we are not yet an onlined region
3857    * (this stuff is being run as part of Region initialization).  This means
3858    * that if we're up against global memory limits, we'll not be flagged to flush
3859    * because we are not online. We can't be flushed by usual mechanisms anyways;
3860    * we're not yet online so our relative sequenceids are not yet aligned with
3861    * WAL sequenceids -- not till we come up online, post processing of split
3862    * edits.
3863    *
3864    * <p>But to help relieve memory pressure, at least manage our own heap size
3865    * flushing if are in excess of per-region limits.  Flushing, though, we have
3866    * to be careful and avoid using the regionserver/wal sequenceid.  Its running
3867    * on a different line to whats going on in here in this region context so if we
3868    * crashed replaying these edits, but in the midst had a flush that used the
3869    * regionserver wal with a sequenceid in excess of whats going on in here
3870    * in this region and with its split editlogs, then we could miss edits the
3871    * next time we go to recover. So, we have to flush inline, using seqids that
3872    * make sense in a this single region context only -- until we online.
3873    *
3874    * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of
3875    * the maxSeqId for the store to be applied, else its skipped.
3876    * @return the sequence id of the last edit added to this region out of the
3877    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3878    * @throws IOException
3879    */
3880   protected long replayRecoveredEditsIfAny(final Path regiondir,
3881       Map<byte[], Long> maxSeqIdInStores,
3882       final CancelableProgressable reporter, final MonitoredTask status)
3883       throws IOException {
3884     long minSeqIdForTheRegion = -1;
3885     for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
3886       if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
3887         minSeqIdForTheRegion = maxSeqIdInStore;
3888       }
3889     }
3890     long seqid = minSeqIdForTheRegion;
3891 
3892     FileSystem fs = this.fs.getFileSystem();
3893     NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
3894     if (LOG.isDebugEnabled()) {
3895       LOG.debug("Found " + (files == null ? 0 : files.size())
3896         + " recovered edits file(s) under " + regiondir);
3897     }
3898 
3899     if (files == null || files.isEmpty()) return seqid;
3900 
3901     for (Path edits: files) {
3902       if (edits == null || !fs.exists(edits)) {
3903         LOG.warn("Null or non-existent edits file: " + edits);
3904         continue;
3905       }
3906       if (isZeroLengthThenDelete(fs, edits)) continue;
3907 
3908       long maxSeqId;
3909       String fileName = edits.getName();
3910       maxSeqId = Math.abs(Long.parseLong(fileName));
3911       if (maxSeqId <= minSeqIdForTheRegion) {
3912         if (LOG.isDebugEnabled()) {
3913           String msg = "Maximum sequenceid for this wal is " + maxSeqId
3914             + " and minimum sequenceid for the region is " + minSeqIdForTheRegion
3915             + ", skipped the whole file, path=" + edits;
3916           LOG.debug(msg);
3917         }
3918         continue;
3919       }
3920 
3921       try {
3922         // replay the edits. Replay can return -1 if everything is skipped, only update
3923         // if seqId is greater
3924         seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter));
3925       } catch (IOException e) {
3926         boolean skipErrors = conf.getBoolean(
3927             HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
3928             conf.getBoolean(
3929                 "hbase.skip.errors",
3930                 HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
3931         if (conf.get("hbase.skip.errors") != null) {
3932           LOG.warn(
3933               "The property 'hbase.skip.errors' has been deprecated. Please use " +
3934               HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
3935         }
3936         if (skipErrors) {
3937           Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
3938           LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
3939               + "=true so continuing. Renamed " + edits +
3940               " as " + p, e);
3941         } else {
3942           throw e;
3943         }
3944       }
3945     }
3946     // The edits size added into rsAccounting during this replaying will not
3947     // be required any more. So just clear it.
3948     if (this.rsAccounting != null) {
3949       this.rsAccounting.clearRegionReplayEditsSize(getRegionInfo().getRegionName());
3950     }
3951     if (seqid > minSeqIdForTheRegion) {
3952       // Then we added some edits to memory. Flush and cleanup split edit files.
3953       internalFlushcache(null, seqid, stores.values(), status, false);
3954     }
3955     // Now delete the content of recovered edits.  We're done w/ them.
3956     if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
3957       // For debugging data loss issues!
3958       // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
3959       // column family. Have to fake out file type too by casting our recovered.edits as storefiles
3960       String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regiondir).getName();
3961       Set<StoreFile> fakeStoreFiles = new HashSet<StoreFile>(files.size());
3962       for (Path file: files) {
3963         fakeStoreFiles.add(new StoreFile(getRegionFileSystem().getFileSystem(), file, this.conf,
3964           null, null));
3965       }
3966       getRegionFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
3967     } else {
3968       for (Path file: files) {
3969         if (!fs.delete(file, false)) {
3970           LOG.error("Failed delete of " + file);
3971         } else {
3972           LOG.debug("Deleted recovered.edits file=" + file);
3973         }
3974       }
3975     }
3976     return seqid;
3977   }
3978 
3979   /*
3980    * @param edits File of recovered edits.
3981    * @param maxSeqIdInStores Maximum sequenceid found in each store.  Edits in wal
3982    * must be larger than this to be replayed for each store.
3983    * @param reporter
3984    * @return the sequence id of the last edit added to this region out of the
3985    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3986    * @throws IOException
3987    */
3988   private long replayRecoveredEdits(final Path edits,
3989       Map<byte[], Long> maxSeqIdInStores, final CancelableProgressable reporter)
3990     throws IOException {
3991     String msg = "Replaying edits from " + edits;
3992     LOG.info(msg);
3993     MonitoredTask status = TaskMonitor.get().createStatus(msg);
3994     FileSystem fs = this.fs.getFileSystem();
3995 
3996     status.setStatus("Opening recovered edits");
3997     WAL.Reader reader = null;
3998     try {
3999       reader = WALFactory.createReader(fs, edits, conf);
4000       long currentEditSeqId = -1;
4001       long currentReplaySeqId = -1;
4002       long firstSeqIdInLog = -1;
4003       long skippedEdits = 0;
4004       long editsCount = 0;
4005       long intervalEdits = 0;
4006       WAL.Entry entry;
4007       Store store = null;
4008       boolean reported_once = false;
4009       ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
4010 
4011       try {
4012         // How many edits seen before we check elapsed time
4013         int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
4014         // How often to send a progress report (default 1/2 master timeout)
4015         int period = this.conf.getInt("hbase.hstore.report.period", 300000);
4016         long lastReport = EnvironmentEdgeManager.currentTime();
4017 
4018         while ((entry = reader.next()) != null) {
4019           WALKey key = entry.getKey();
4020           WALEdit val = entry.getEdit();
4021 
4022           if (ng != null) { // some test, or nonces disabled
4023             ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
4024           }
4025 
4026           if (reporter != null) {
4027             intervalEdits += val.size();
4028             if (intervalEdits >= interval) {
4029               // Number of edits interval reached
4030               intervalEdits = 0;
4031               long cur = EnvironmentEdgeManager.currentTime();
4032               if (lastReport + period <= cur) {
4033                 status.setStatus("Replaying edits..." +
4034                     " skipped=" + skippedEdits +
4035                     " edits=" + editsCount);
4036                 // Timeout reached
4037                 if(!reporter.progress()) {
4038                   msg = "Progressable reporter failed, stopping replay";
4039                   LOG.warn(msg);
4040                   status.abort(msg);
4041                   throw new IOException(msg);
4042                 }
4043                 reported_once = true;
4044                 lastReport = cur;
4045               }
4046             }
4047           }
4048 
4049           if (firstSeqIdInLog == -1) {
4050             firstSeqIdInLog = key.getLogSeqNum();
4051           }
4052           if (currentEditSeqId > key.getLogSeqNum()) {
4053             // when this condition is true, it means we have a serious defect because we need to
4054             // maintain increasing SeqId for WAL edits per region
4055             LOG.error(getRegionInfo().getEncodedName() + " : "
4056                  + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
4057                 + "; edit=" + val);
4058           } else {
4059             currentEditSeqId = key.getLogSeqNum();
4060           }
4061           currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
4062             key.getOrigLogSeqNum() : currentEditSeqId;
4063 
4064           // Start coprocessor replay here. The coprocessor is for each WALEdit
4065           // instead of a KeyValue.
4066           if (coprocessorHost != null) {
4067             status.setStatus("Running pre-WAL-restore hook in coprocessors");
4068             if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
4069               // if bypass this wal entry, ignore it ...
4070               continue;
4071             }
4072           }
4073           boolean checkRowWithinBoundary = false;
4074           // Check this edit is for this region.
4075           if (!Bytes.equals(key.getEncodedRegionName(),
4076               this.getRegionInfo().getEncodedNameAsBytes())) {
4077             checkRowWithinBoundary = true;
4078           }
4079 
4080           boolean flush = false;
4081           for (Cell cell: val.getCells()) {
4082             // Check this edit is for me. Also, guard against writing the special
4083             // METACOLUMN info such as HBASE::CACHEFLUSH entries
4084             if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
4085               // if region names don't match, skipp replaying compaction marker
4086               if (!checkRowWithinBoundary) {
4087                 //this is a special edit, we should handle it
4088                 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
4089                 if (compaction != null) {
4090                   //replay the compaction
4091                   replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
4092                 }
4093               }
4094               skippedEdits++;
4095               continue;
4096             }
4097             // Figure which store the edit is meant for.
4098             if (store == null || !CellUtil.matchingFamily(cell, store.getFamily().getName())) {
4099               store = getStore(cell);
4100             }
4101             if (store == null) {
4102               // This should never happen.  Perhaps schema was changed between
4103               // crash and redeploy?
4104               LOG.warn("No family for " + cell);
4105               skippedEdits++;
4106               continue;
4107             }
4108             if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
4109               cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
4110               LOG.warn("Row of " + cell + " is not within region boundary");
4111               skippedEdits++;
4112               continue;
4113             }
4114             // Now, figure if we should skip this edit.
4115             if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
4116                 .getName())) {
4117               skippedEdits++;
4118               continue;
4119             }
4120             CellUtil.setSequenceId(cell, currentReplaySeqId);
4121 
4122             // Once we are over the limit, restoreEdit will keep returning true to
4123             // flush -- but don't flush until we've played all the kvs that make up
4124             // the WALEdit.
4125             flush |= restoreEdit(store, cell);
4126             editsCount++;
4127           }
4128           if (flush) {
4129             internalFlushcache(null, currentEditSeqId, stores.values(), status, false);
4130           }
4131 
4132           if (coprocessorHost != null) {
4133             coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
4134           }
4135         }
4136       } catch (EOFException eof) {
4137         Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4138         msg = "Encountered EOF. Most likely due to Master failure during " +
4139             "wal splitting, so we have this data in another edit.  " +
4140             "Continuing, but renaming " + edits + " as " + p;
4141         LOG.warn(msg, eof);
4142         status.abort(msg);
4143       } catch (IOException ioe) {
4144         // If the IOE resulted from bad file format,
4145         // then this problem is idempotent and retrying won't help
4146         if (ioe.getCause() instanceof ParseException) {
4147           Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4148           msg = "File corruption encountered!  " +
4149               "Continuing, but renaming " + edits + " as " + p;
4150           LOG.warn(msg, ioe);
4151           status.setStatus(msg);
4152         } else {
4153           status.abort(StringUtils.stringifyException(ioe));
4154           // other IO errors may be transient (bad network connection,
4155           // checksum exception on one datanode, etc).  throw & retry
4156           throw ioe;
4157         }
4158       }
4159       if (reporter != null && !reported_once) {
4160         reporter.progress();
4161       }
4162       msg = "Applied " + editsCount + ", skipped " + skippedEdits +
4163         ", firstSequenceIdInLog=" + firstSeqIdInLog +
4164         ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
4165       status.markComplete(msg);
4166       LOG.debug(msg);
4167       return currentEditSeqId;
4168     } finally {
4169       status.cleanup();
4170       if (reader != null) {
4171          reader.close();
4172       }
4173     }
4174   }
4175 
4176   /**
4177    * Call to complete a compaction. Its for the case where we find in the WAL a compaction
4178    * that was not finished.  We could find one recovering a WAL after a regionserver crash.
4179    * See HBASE-2331.
4180    */
4181   void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
4182       boolean removeFiles, long replaySeqId)
4183       throws IOException {
4184     try {
4185       checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
4186         "Compaction marker from WAL ", compaction);
4187     } catch (WrongRegionException wre) {
4188       if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4189         // skip the compaction marker since it is not for this region
4190         return;
4191       }
4192       throw wre;
4193     }
4194 
4195     synchronized (writestate) {
4196       if (replaySeqId < lastReplayedOpenRegionSeqId) {
4197         LOG.warn(getRegionInfo().getEncodedName() + " : "
4198             + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4199             + " because its sequence id " + replaySeqId + " is smaller than this regions "
4200             + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
4201         return;
4202       }
4203       if (replaySeqId < lastReplayedCompactionSeqId) {
4204         LOG.warn(getRegionInfo().getEncodedName() + " : "
4205             + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4206             + " because its sequence id " + replaySeqId + " is smaller than this regions "
4207             + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId);
4208         return;
4209       } else {
4210         lastReplayedCompactionSeqId = replaySeqId;
4211       }
4212 
4213       if (LOG.isDebugEnabled()) {
4214         LOG.debug(getRegionInfo().getEncodedName() + " : "
4215             + "Replaying compaction marker " + TextFormat.shortDebugString(compaction)
4216             + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId="
4217             + lastReplayedOpenRegionSeqId);
4218       }
4219 
4220       startRegionOperation(Operation.REPLAY_EVENT);
4221       try {
4222         Store store = this.getStore(compaction.getFamilyName().toByteArray());
4223         if (store == null) {
4224           LOG.warn(getRegionInfo().getEncodedName() + " : "
4225               + "Found Compaction WAL edit for deleted family:"
4226               + Bytes.toString(compaction.getFamilyName().toByteArray()));
4227           return;
4228         }
4229         store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
4230         logRegionFiles();
4231       } catch (FileNotFoundException ex) {
4232         LOG.warn(getRegionInfo().getEncodedName() + " : "
4233             + "At least one of the store files in compaction: "
4234             + TextFormat.shortDebugString(compaction)
4235             + " doesn't exist any more. Skip loading the file(s)", ex);
4236       } finally {
4237         closeRegionOperation(Operation.REPLAY_EVENT);
4238       }
4239     }
4240   }
4241 
4242   void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
4243     checkTargetRegion(flush.getEncodedRegionName().toByteArray(),
4244       "Flush marker from WAL ", flush);
4245 
4246     if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4247       return; // if primary nothing to do
4248     }
4249 
4250     if (LOG.isDebugEnabled()) {
4251       LOG.debug(getRegionInfo().getEncodedName() + " : "
4252           + "Replaying flush marker " + TextFormat.shortDebugString(flush));
4253     }
4254 
4255     startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close
4256     try {
4257       FlushAction action = flush.getAction();
4258       switch (action) {
4259       case START_FLUSH:
4260         replayWALFlushStartMarker(flush);
4261         break;
4262       case COMMIT_FLUSH:
4263         replayWALFlushCommitMarker(flush);
4264         break;
4265       case ABORT_FLUSH:
4266         replayWALFlushAbortMarker(flush);
4267         break;
4268       case CANNOT_FLUSH:
4269         replayWALFlushCannotFlushMarker(flush, replaySeqId);
4270         break;
4271       default:
4272         LOG.warn(getRegionInfo().getEncodedName() + " : " +
4273           "Received a flush event with unknown action, ignoring. " +
4274           TextFormat.shortDebugString(flush));
4275         break;
4276       }
4277 
4278       logRegionFiles();
4279     } finally {
4280       closeRegionOperation(Operation.REPLAY_EVENT);
4281     }
4282   }
4283 
4284   /** Replay the flush marker from primary region by creating a corresponding snapshot of
4285    * the store memstores, only if the memstores do not have a higher seqId from an earlier wal
4286    * edit (because the events may be coming out of order).
4287    */
4288   @VisibleForTesting
4289   PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
4290     long flushSeqId = flush.getFlushSequenceNumber();
4291 
4292     HashSet<Store> storesToFlush = new HashSet<Store>();
4293     for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
4294       byte[] family = storeFlush.getFamilyName().toByteArray();
4295       Store store = getStore(family);
4296       if (store == null) {
4297         LOG.warn(getRegionInfo().getEncodedName() + " : "
4298           + "Received a flush start marker from primary, but the family is not found. Ignoring"
4299           + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
4300         continue;
4301       }
4302       storesToFlush.add(store);
4303     }
4304 
4305     MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
4306 
4307     // we will use writestate as a coarse-grain lock for all the replay events
4308     // (flush, compaction, region open etc)
4309     synchronized (writestate) {
4310       try {
4311         if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
4312           LOG.warn(getRegionInfo().getEncodedName() + " : "
4313               + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4314               + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4315               + " of " + lastReplayedOpenRegionSeqId);
4316           return null;
4317         }
4318         if (numMutationsWithoutWAL.get() > 0) {
4319           numMutationsWithoutWAL.set(0);
4320           dataInMemoryWithoutWAL.set(0);
4321         }
4322 
4323         if (!writestate.flushing) {
4324           // we do not have an active snapshot and corresponding this.prepareResult. This means
4325           // we can just snapshot our memstores and continue as normal.
4326 
4327           // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal
4328           PrepareFlushResult prepareResult = internalPrepareFlushCache(null,
4329             flushSeqId, storesToFlush, status, false);
4330           if (prepareResult.result == null) {
4331             // save the PrepareFlushResult so that we can use it later from commit flush
4332             this.writestate.flushing = true;
4333             this.prepareFlushResult = prepareResult;
4334             status.markComplete("Flush prepare successful");
4335             if (LOG.isDebugEnabled()) {
4336               LOG.debug(getRegionInfo().getEncodedName() + " : "
4337                   + " Prepared flush with seqId:" + flush.getFlushSequenceNumber());
4338             }
4339           } else {
4340             // special case empty memstore. We will still save the flush result in this case, since
4341             // our memstore ie empty, but the primary is still flushing
4342             if (prepareResult.getResult().getResult() ==
4343                   FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
4344               this.writestate.flushing = true;
4345               this.prepareFlushResult = prepareResult;
4346               if (LOG.isDebugEnabled()) {
4347                 LOG.debug(getRegionInfo().getEncodedName() + " : "
4348                   + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
4349               }
4350             }
4351             status.abort("Flush prepare failed with " + prepareResult.result);
4352             // nothing much to do. prepare flush failed because of some reason.
4353           }
4354           return prepareResult;
4355         } else {
4356           // we already have an active snapshot.
4357           if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
4358             // They define the same flush. Log and continue.
4359             LOG.warn(getRegionInfo().getEncodedName() + " : "
4360                 + "Received a flush prepare marker with the same seqId: " +
4361                 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4362                 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4363             // ignore
4364           } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
4365             // We received a flush with a smaller seqNum than what we have prepared. We can only
4366             // ignore this prepare flush request.
4367             LOG.warn(getRegionInfo().getEncodedName() + " : "
4368                 + "Received a flush prepare marker with a smaller seqId: " +
4369                 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4370                 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4371             // ignore
4372           } else {
4373             // We received a flush with a larger seqNum than what we have prepared
4374             LOG.warn(getRegionInfo().getEncodedName() + " : "
4375                 + "Received a flush prepare marker with a larger seqId: " +
4376                 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4377                 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4378             // We do not have multiple active snapshots in the memstore or a way to merge current
4379             // memstore snapshot with the contents and resnapshot for now. We cannot take
4380             // another snapshot and drop the previous one because that will cause temporary
4381             // data loss in the secondary. So we ignore this for now, deferring the resolution
4382             // to happen when we see the corresponding flush commit marker. If we have a memstore
4383             // snapshot with x, and later received another prepare snapshot with y (where x < y),
4384             // when we see flush commit for y, we will drop snapshot for x, and can also drop all
4385             // the memstore edits if everything in memstore is < y. This is the usual case for
4386             // RS crash + recovery where we might see consequtive prepare flush wal markers.
4387             // Otherwise, this will cause more memory to be used in secondary replica until a
4388             // further prapare + commit flush is seen and replayed.
4389           }
4390         }
4391       } finally {
4392         status.cleanup();
4393         writestate.notifyAll();
4394       }
4395     }
4396     return null;
4397   }
4398 
4399   @VisibleForTesting
4400   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4401     justification="Intentional; post memstore flush")
4402   void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
4403     MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
4404 
4405     // check whether we have the memstore snapshot with the corresponding seqId. Replay to
4406     // secondary region replicas are in order, except for when the region moves or then the
4407     // region server crashes. In those cases, we may receive replay requests out of order from
4408     // the original seqIds.
4409     synchronized (writestate) {
4410       try {
4411         if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
4412           LOG.warn(getRegionInfo().getEncodedName() + " : "
4413             + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4414             + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4415             + " of " + lastReplayedOpenRegionSeqId);
4416           return;
4417         }
4418 
4419         if (writestate.flushing) {
4420           PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
4421           if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
4422             if (LOG.isDebugEnabled()) {
4423               LOG.debug(getRegionInfo().getEncodedName() + " : "
4424                   + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
4425                   + " and a previous prepared snapshot was found");
4426             }
4427             // This is the regular case where we received commit flush after prepare flush
4428             // corresponding to the same seqId.
4429             replayFlushInStores(flush, prepareFlushResult, true);
4430 
4431             // Set down the memstore size by amount of flush.
4432             this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
4433 
4434             this.prepareFlushResult = null;
4435             writestate.flushing = false;
4436           } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
4437             // This should not happen normally. However, lets be safe and guard against these cases
4438             // we received a flush commit with a smaller seqId than what we have prepared
4439             // we will pick the flush file up from this commit (if we have not seen it), but we
4440             // will not drop the memstore
4441             LOG.warn(getRegionInfo().getEncodedName() + " : "
4442                 + "Received a flush commit marker with smaller seqId: "
4443                 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
4444                 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
4445                 +"  prepared memstore snapshot");
4446             replayFlushInStores(flush, prepareFlushResult, false);
4447 
4448             // snapshot is not dropped, so memstore sizes should not be decremented
4449             // we still have the prepared snapshot, flushing should still be true
4450           } else {
4451             // This should not happen normally. However, lets be safe and guard against these cases
4452             // we received a flush commit with a larger seqId than what we have prepared
4453             // we will pick the flush file for this. We will also obtain the updates lock and
4454             // look for contents of the memstore to see whether we have edits after this seqId.
4455             // If not, we will drop all the memstore edits and the snapshot as well.
4456             LOG.warn(getRegionInfo().getEncodedName() + " : "
4457                 + "Received a flush commit marker with larger seqId: "
4458                 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " +
4459                 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
4460                 +" memstore snapshot");
4461 
4462             replayFlushInStores(flush, prepareFlushResult, true);
4463 
4464             // Set down the memstore size by amount of flush.
4465             this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
4466 
4467             // Inspect the memstore contents to see whether the memstore contains only edits
4468             // with seqId smaller than the flush seqId. If so, we can discard those edits.
4469             dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
4470 
4471             this.prepareFlushResult = null;
4472             writestate.flushing = false;
4473           }
4474           // If we were waiting for observing a flush or region opening event for not showing
4475           // partial data after a secondary region crash, we can allow reads now. We can only make
4476           // sure that we are not showing partial data (for example skipping some previous edits)
4477           // until we observe a full flush start and flush commit. So if we were not able to find
4478           // a previous flush we will not enable reads now.
4479           this.setReadsEnabled(true);
4480         } else {
4481           LOG.warn(getRegionInfo().getEncodedName() + " : "
4482               + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
4483               + ", but no previous prepared snapshot was found");
4484           // There is no corresponding prepare snapshot from before.
4485           // We will pick up the new flushed file
4486           replayFlushInStores(flush, null, false);
4487 
4488           // Inspect the memstore contents to see whether the memstore contains only edits
4489           // with seqId smaller than the flush seqId. If so, we can discard those edits.
4490           dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
4491         }
4492 
4493         status.markComplete("Flush commit successful");
4494 
4495         // Update the last flushed sequence id for region.
4496         this.maxFlushedSeqId = flush.getFlushSequenceNumber();
4497 
4498         // advance the mvcc read point so that the new flushed file is visible.
4499         mvcc.advanceTo(flush.getFlushSequenceNumber());
4500 
4501       } catch (FileNotFoundException ex) {
4502         LOG.warn(getRegionInfo().getEncodedName() + " : "
4503             + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
4504             + " doesn't exist any more. Skip loading the file(s)", ex);
4505       }
4506       finally {
4507         status.cleanup();
4508         writestate.notifyAll();
4509       }
4510     }
4511 
4512     // C. Finally notify anyone waiting on memstore to clear:
4513     // e.g. checkResources().
4514     synchronized (this) {
4515       notifyAll(); // FindBugs NN_NAKED_NOTIFY
4516     }
4517   }
4518 
4519   /**
4520    * Replays the given flush descriptor by opening the flush files in stores and dropping the
4521    * memstore snapshots if requested.
4522    * @param flush
4523    * @param prepareFlushResult
4524    * @param dropMemstoreSnapshot
4525    * @throws IOException
4526    */
4527   private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
4528       boolean dropMemstoreSnapshot)
4529       throws IOException {
4530     for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
4531       byte[] family = storeFlush.getFamilyName().toByteArray();
4532       Store store = getStore(family);
4533       if (store == null) {
4534         LOG.warn(getRegionInfo().getEncodedName() + " : "
4535             + "Received a flush commit marker from primary, but the family is not found."
4536             + "Ignoring StoreFlushDescriptor:" + storeFlush);
4537         continue;
4538       }
4539       List<String> flushFiles = storeFlush.getFlushOutputList();
4540       StoreFlushContext ctx = null;
4541       long startTime = EnvironmentEdgeManager.currentTime();
4542       if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
4543         ctx = store.createFlushContext(flush.getFlushSequenceNumber());
4544       } else {
4545         ctx = prepareFlushResult.storeFlushCtxs.get(family);
4546         startTime = prepareFlushResult.startTime;
4547       }
4548 
4549       if (ctx == null) {
4550         LOG.warn(getRegionInfo().getEncodedName() + " : "
4551             + "Unexpected: flush commit marker received from store "
4552             + Bytes.toString(family) + " but no associated flush context. Ignoring");
4553         continue;
4554       }
4555 
4556       ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush
4557 
4558       // Record latest flush time
4559       this.lastStoreFlushTimeMap.put(store, startTime);
4560     }
4561   }
4562 
4563   /**
4564    * Drops the memstore contents after replaying a flush descriptor or region open event replay
4565    * if the memstore edits have seqNums smaller than the given seq id
4566    * @throws IOException
4567    */
4568   private long dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException {
4569     long totalFreedSize = 0;
4570     this.updatesLock.writeLock().lock();
4571     try {
4572 
4573       long currentSeqId = mvcc.getReadPoint();
4574       if (seqId >= currentSeqId) {
4575         // then we can drop the memstore contents since everything is below this seqId
4576         LOG.info(getRegionInfo().getEncodedName() + " : "
4577             + "Dropping memstore contents as well since replayed flush seqId: "
4578             + seqId + " is greater than current seqId:" + currentSeqId);
4579 
4580         // Prepare flush (take a snapshot) and then abort (drop the snapshot)
4581         if (store == null) {
4582           for (Store s : stores.values()) {
4583             totalFreedSize += doDropStoreMemstoreContentsForSeqId(s, currentSeqId);
4584           }
4585         } else {
4586           totalFreedSize += doDropStoreMemstoreContentsForSeqId(store, currentSeqId);
4587         }
4588       } else {
4589         LOG.info(getRegionInfo().getEncodedName() + " : "
4590             + "Not dropping memstore contents since replayed flush seqId: "
4591             + seqId + " is smaller than current seqId:" + currentSeqId);
4592       }
4593     } finally {
4594       this.updatesLock.writeLock().unlock();
4595     }
4596     return totalFreedSize;
4597   }
4598 
4599   private long doDropStoreMemstoreContentsForSeqId(Store s, long currentSeqId) throws IOException {
4600     long snapshotSize = s.getFlushableSize();
4601     this.addAndGetGlobalMemstoreSize(-snapshotSize);
4602     StoreFlushContext ctx = s.createFlushContext(currentSeqId);
4603     ctx.prepare();
4604     ctx.abort();
4605     return snapshotSize;
4606   }
4607 
4608   private void replayWALFlushAbortMarker(FlushDescriptor flush) {
4609     // nothing to do for now. A flush abort will cause a RS abort which means that the region
4610     // will be opened somewhere else later. We will see the region open event soon, and replaying
4611     // that will drop the snapshot
4612   }
4613 
4614   private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
4615     synchronized (writestate) {
4616       if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
4617         LOG.warn(getRegionInfo().getEncodedName() + " : "
4618           + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4619           + " because its sequence id " + replaySeqId + " is smaller than this regions "
4620           + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
4621         return;
4622       }
4623 
4624       // If we were waiting for observing a flush or region opening event for not showing partial
4625       // data after a secondary region crash, we can allow reads now. This event means that the
4626       // primary was not able to flush because memstore is empty when we requested flush. By the
4627       // time we observe this, we are guaranteed to have up to date seqId with our previous
4628       // assignment.
4629       this.setReadsEnabled(true);
4630     }
4631   }
4632 
4633   @VisibleForTesting
4634   PrepareFlushResult getPrepareFlushResult() {
4635     return prepareFlushResult;
4636   }
4637 
4638   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4639       justification="Intentional; cleared the memstore")
4640   void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
4641     checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
4642       "RegionEvent marker from WAL ", regionEvent);
4643 
4644     startRegionOperation(Operation.REPLAY_EVENT);
4645     try {
4646       if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4647         return; // if primary nothing to do
4648       }
4649 
4650       if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
4651         // nothing to do on REGION_CLOSE for now.
4652         return;
4653       }
4654       if (regionEvent.getEventType() != EventType.REGION_OPEN) {
4655         LOG.warn(getRegionInfo().getEncodedName() + " : "
4656             + "Unknown region event received, ignoring :"
4657             + TextFormat.shortDebugString(regionEvent));
4658         return;
4659       }
4660 
4661       if (LOG.isDebugEnabled()) {
4662         LOG.debug(getRegionInfo().getEncodedName() + " : "
4663           + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent));
4664       }
4665 
4666       // we will use writestate as a coarse-grain lock for all the replay events
4667       synchronized (writestate) {
4668         // Replication can deliver events out of order when primary region moves or the region
4669         // server crashes, since there is no coordination between replication of different wal files
4670         // belonging to different region servers. We have to safe guard against this case by using
4671         // region open event's seqid. Since this is the first event that the region puts (after
4672         // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
4673         // smaller than this seqId
4674         if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
4675           this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
4676         } else {
4677           LOG.warn(getRegionInfo().getEncodedName() + " : "
4678             + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent)
4679             + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4680             + " of " + lastReplayedOpenRegionSeqId);
4681           return;
4682         }
4683 
4684         // region open lists all the files that the region has at the time of the opening. Just pick
4685         // all the files and drop prepared flushes and empty memstores
4686         for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
4687           // stores of primary may be different now
4688           byte[] family = storeDescriptor.getFamilyName().toByteArray();
4689           Store store = getStore(family);
4690           if (store == null) {
4691             LOG.warn(getRegionInfo().getEncodedName() + " : "
4692                 + "Received a region open marker from primary, but the family is not found. "
4693                 + "Ignoring. StoreDescriptor:" + storeDescriptor);
4694             continue;
4695           }
4696 
4697           long storeSeqId = store.getMaxSequenceId();
4698           List<String> storeFiles = storeDescriptor.getStoreFileList();
4699           try {
4700             store.refreshStoreFiles(storeFiles); // replace the files with the new ones
4701           } catch (FileNotFoundException ex) {
4702             LOG.warn(getRegionInfo().getEncodedName() + " : "
4703                     + "At least one of the store files: " + storeFiles
4704                     + " doesn't exist any more. Skip loading the file(s)", ex);
4705             continue;
4706           }
4707           if (store.getMaxSequenceId() != storeSeqId) {
4708             // Record latest flush time if we picked up new files
4709             lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
4710           }
4711 
4712           if (writestate.flushing) {
4713             // only drop memstore snapshots if they are smaller than last flush for the store
4714             if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
4715               StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
4716                   null : this.prepareFlushResult.storeFlushCtxs.get(family);
4717               if (ctx != null) {
4718                 long snapshotSize = store.getFlushableSize();
4719                 ctx.abort();
4720                 this.addAndGetGlobalMemstoreSize(-snapshotSize);
4721                 this.prepareFlushResult.storeFlushCtxs.remove(family);
4722               }
4723             }
4724           }
4725 
4726           // Drop the memstore contents if they are now smaller than the latest seen flushed file
4727           dropMemstoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
4728           if (storeSeqId > this.maxFlushedSeqId) {
4729             this.maxFlushedSeqId = storeSeqId;
4730           }
4731         }
4732 
4733         // if all stores ended up dropping their snapshots, we can safely drop the
4734         // prepareFlushResult
4735         dropPrepareFlushIfPossible();
4736 
4737         // advance the mvcc read point so that the new flushed file is visible.
4738         mvcc.await();
4739 
4740         // If we were waiting for observing a flush or region opening event for not showing partial
4741         // data after a secondary region crash, we can allow reads now.
4742         this.setReadsEnabled(true);
4743 
4744         // C. Finally notify anyone waiting on memstore to clear:
4745         // e.g. checkResources().
4746         synchronized (this) {
4747           notifyAll(); // FindBugs NN_NAKED_NOTIFY
4748         }
4749       }
4750       logRegionFiles();
4751     } finally {
4752       closeRegionOperation(Operation.REPLAY_EVENT);
4753     }
4754   }
4755 
4756   void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
4757     checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
4758       "BulkLoad marker from WAL ", bulkLoadEvent);
4759 
4760     if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4761       return; // if primary nothing to do
4762     }
4763 
4764     if (LOG.isDebugEnabled()) {
4765       LOG.debug(getRegionInfo().getEncodedName() + " : "
4766               +  "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent));
4767     }
4768     // check if multiple families involved
4769     boolean multipleFamilies = false;
4770     byte[] family = null;
4771     for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
4772       byte[] fam = storeDescriptor.getFamilyName().toByteArray();
4773       if (family == null) {
4774         family = fam;
4775       } else if (!Bytes.equals(family, fam)) {
4776         multipleFamilies = true;
4777         break;
4778       }
4779     }
4780 
4781     startBulkRegionOperation(multipleFamilies);
4782     try {
4783       // we will use writestate as a coarse-grain lock for all the replay events
4784       synchronized (writestate) {
4785         // Replication can deliver events out of order when primary region moves or the region
4786         // server crashes, since there is no coordination between replication of different wal files
4787         // belonging to different region servers. We have to safe guard against this case by using
4788         // region open event's seqid. Since this is the first event that the region puts (after
4789         // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
4790         // smaller than this seqId
4791         if (bulkLoadEvent.getBulkloadSeqNum() >= 0
4792             && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) {
4793           LOG.warn(getRegionInfo().getEncodedName() + " : "
4794               + "Skipping replaying bulkload event :"
4795               + TextFormat.shortDebugString(bulkLoadEvent)
4796               + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
4797               + " =" + lastReplayedOpenRegionSeqId);
4798 
4799           return;
4800         }
4801 
4802         for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
4803           // stores of primary may be different now
4804           family = storeDescriptor.getFamilyName().toByteArray();
4805           Store store = getStore(family);
4806           if (store == null) {
4807             LOG.warn(getRegionInfo().getEncodedName() + " : "
4808                     + "Received a bulk load marker from primary, but the family is not found. "
4809                     + "Ignoring. StoreDescriptor:" + storeDescriptor);
4810             continue;
4811           }
4812 
4813           List<String> storeFiles = storeDescriptor.getStoreFileList();
4814           for (String storeFile : storeFiles) {
4815             StoreFileInfo storeFileInfo = null;
4816             try {
4817               storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile);
4818               store.bulkLoadHFile(storeFileInfo);
4819             } catch(FileNotFoundException ex) {
4820               LOG.warn(getRegionInfo().getEncodedName() + " : "
4821                       + ((storeFileInfo != null) ? storeFileInfo.toString() :
4822                             (new Path(Bytes.toString(family), storeFile)).toString())
4823                       + " doesn't exist any more. Skip loading the file");
4824             }
4825           }
4826         }
4827       }
4828       if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
4829         mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
4830       }
4831     } finally {
4832       closeBulkRegionOperation();
4833     }
4834   }
4835 
4836   /**
4837    * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult
4838    */
4839   private void dropPrepareFlushIfPossible() {
4840     if (writestate.flushing) {
4841       boolean canDrop = true;
4842       if (prepareFlushResult.storeFlushCtxs != null) {
4843         for (Entry<byte[], StoreFlushContext> entry
4844             : prepareFlushResult.storeFlushCtxs.entrySet()) {
4845           Store store = getStore(entry.getKey());
4846           if (store == null) {
4847             continue;
4848           }
4849           if (store.getSnapshotSize() > 0) {
4850             canDrop = false;
4851             break;
4852           }
4853         }
4854       }
4855 
4856       // this means that all the stores in the region has finished flushing, but the WAL marker
4857       // may not have been written or we did not receive it yet.
4858       if (canDrop) {
4859         writestate.flushing = false;
4860         this.prepareFlushResult = null;
4861       }
4862     }
4863   }
4864 
4865   @Override
4866   public boolean refreshStoreFiles() throws IOException {
4867     return refreshStoreFiles(false);
4868   }
4869 
4870   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4871       justification="Notify is about post replay. Intentional")
4872   protected boolean refreshStoreFiles(boolean force) throws IOException {
4873     if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4874       return false; // if primary nothing to do
4875     }
4876 
4877     if (LOG.isDebugEnabled()) {
4878       LOG.debug(getRegionInfo().getEncodedName() + " : "
4879           + "Refreshing store files to see whether we can free up memstore");
4880     }
4881 
4882     long totalFreedSize = 0;
4883 
4884     long smallestSeqIdInStores = Long.MAX_VALUE;
4885 
4886     startRegionOperation(); // obtain region close lock
4887     try {
4888       Map<Store, Long> map = new HashMap<Store, Long>();
4889       synchronized (writestate) {
4890         for (Store store : getStores()) {
4891           // TODO: some stores might see new data from flush, while others do not which
4892           // MIGHT break atomic edits across column families.
4893           long maxSeqIdBefore = store.getMaxSequenceId();
4894 
4895           // refresh the store files. This is similar to observing a region open wal marker.
4896           store.refreshStoreFiles();
4897 
4898           long storeSeqId = store.getMaxSequenceId();
4899           if (storeSeqId < smallestSeqIdInStores) {
4900             smallestSeqIdInStores = storeSeqId;
4901           }
4902 
4903           // see whether we can drop the memstore or the snapshot
4904           if (storeSeqId > maxSeqIdBefore) {
4905 
4906             if (writestate.flushing) {
4907               // only drop memstore snapshots if they are smaller than last flush for the store
4908               if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
4909                 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
4910                     null : this.prepareFlushResult.storeFlushCtxs.get(store.getFamily().getName());
4911                 if (ctx != null) {
4912                   long snapshotSize = store.getFlushableSize();
4913                   ctx.abort();
4914                   this.addAndGetGlobalMemstoreSize(-snapshotSize);
4915                   this.prepareFlushResult.storeFlushCtxs.remove(store.getFamily().getName());
4916                   totalFreedSize += snapshotSize;
4917                 }
4918               }
4919             }
4920 
4921             map.put(store, storeSeqId);
4922           }
4923         }
4924 
4925         // if all stores ended up dropping their snapshots, we can safely drop the
4926         // prepareFlushResult
4927         dropPrepareFlushIfPossible();
4928 
4929         // advance the mvcc read point so that the new flushed files are visible.
4930           // either greater than flush seq number or they were already picked up via flush.
4931           for (Store s : getStores()) {
4932             mvcc.advanceTo(s.getMaxMemstoreTS());
4933           }
4934 
4935 
4936         // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely
4937         // skip all edits that are to be replayed in the future with that has a smaller seqId
4938         // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits
4939         // that we have picked the flush files for
4940         if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
4941           this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
4942         }
4943       }
4944       if (!map.isEmpty()) {
4945         if (!force) {
4946           for (Map.Entry<Store, Long> entry : map.entrySet()) {
4947             // Drop the memstore contents if they are now smaller than the latest seen flushed file
4948             totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey());
4949           }
4950         } else {
4951           synchronized (storeSeqIds) {
4952             // don't try to acquire write lock of updatesLock now
4953             storeSeqIds.add(map);
4954           }
4955         }
4956       }
4957       // C. Finally notify anyone waiting on memstore to clear:
4958       // e.g. checkResources().
4959       synchronized (this) {
4960         notifyAll(); // FindBugs NN_NAKED_NOTIFY
4961       }
4962       return totalFreedSize > 0;
4963     } finally {
4964       closeRegionOperation();
4965     }
4966   }
4967 
4968   private void logRegionFiles() {
4969     if (LOG.isTraceEnabled()) {
4970       LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
4971       for (Store s : stores.values()) {
4972         Collection<StoreFile> storeFiles = s.getStorefiles();
4973         if (storeFiles == null) continue;
4974         for (StoreFile sf : storeFiles) {
4975           LOG.trace(getRegionInfo().getEncodedName() + " : " + sf);
4976         }
4977       }
4978     }
4979   }
4980 
4981   /** Checks whether the given regionName is either equal to our region, or that
4982    * the regionName is the primary region to our corresponding range for the secondary replica.
4983    */
4984   private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
4985       throws WrongRegionException {
4986     if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
4987       return;
4988     }
4989 
4990     if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) &&
4991         Bytes.equals(encodedRegionName,
4992           this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) {
4993       return;
4994     }
4995 
4996     throw new WrongRegionException(exceptionMsg + payload
4997       + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
4998       + " does not match this region: " + this.getRegionInfo());
4999   }
5000 
5001   /**
5002    * Used by tests
5003    * @param s Store to add edit too.
5004    * @param cell Cell to add.
5005    * @return True if we should flush.
5006    */
5007   protected boolean restoreEdit(final Store s, final Cell cell) {
5008     long kvSize = s.add(cell);
5009     if (this.rsAccounting != null) {
5010       rsAccounting.addAndGetRegionReplayEditsSize(getRegionInfo().getRegionName(), kvSize);
5011     }
5012     return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
5013   }
5014 
5015   /*
5016    * @param fs
5017    * @param p File to check.
5018    * @return True if file was zero-length (and if so, we'll delete it in here).
5019    * @throws IOException
5020    */
5021   private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
5022       throws IOException {
5023     FileStatus stat = fs.getFileStatus(p);
5024     if (stat.getLen() > 0) return false;
5025     LOG.warn("File " + p + " is zero-length, deleting.");
5026     fs.delete(p, false);
5027     return true;
5028   }
5029 
5030   protected HStore instantiateHStore(final HColumnDescriptor family) throws IOException {
5031     return new HStore(this, family, this.conf);
5032   }
5033 
5034   @Override
5035   public Store getStore(final byte[] column) {
5036     return this.stores.get(column);
5037   }
5038 
5039   /**
5040    * Return HStore instance. Does not do any copy: as the number of store is limited, we
5041    *  iterate on the list.
5042    */
5043   private Store getStore(Cell cell) {
5044     for (Map.Entry<byte[], Store> famStore : stores.entrySet()) {
5045       if (Bytes.equals(
5046           cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
5047           famStore.getKey(), 0, famStore.getKey().length)) {
5048         return famStore.getValue();
5049       }
5050     }
5051 
5052     return null;
5053   }
5054 
5055   @Override
5056   public List<Store> getStores() {
5057     List<Store> list = new ArrayList<Store>(stores.size());
5058     list.addAll(stores.values());
5059     return list;
5060   }
5061 
5062   @Override
5063   public List<String> getStoreFileList(final byte [][] columns)
5064     throws IllegalArgumentException {
5065     List<String> storeFileNames = new ArrayList<String>();
5066     synchronized(closeLock) {
5067       for(byte[] column : columns) {
5068         Store store = this.stores.get(column);
5069         if (store == null) {
5070           throw new IllegalArgumentException("No column family : " +
5071               new String(column) + " available");
5072         }
5073         Collection<StoreFile> storeFiles = store.getStorefiles();
5074         if (storeFiles == null) continue;
5075         for (StoreFile storeFile: storeFiles) {
5076           storeFileNames.add(storeFile.getPath().toString());
5077         }
5078 
5079         logRegionFiles();
5080       }
5081     }
5082     return storeFileNames;
5083   }
5084 
5085   //////////////////////////////////////////////////////////////////////////////
5086   // Support code
5087   //////////////////////////////////////////////////////////////////////////////
5088 
5089   /** Make sure this is a valid row for the HRegion */
5090   void checkRow(final byte [] row, String op) throws IOException {
5091     if (!rowIsInRange(getRegionInfo(), row)) {
5092       throw new WrongRegionException("Requested row out of range for " +
5093           op + " on HRegion " + this + ", startKey='" +
5094           Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" +
5095           Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" +
5096           Bytes.toStringBinary(row) + "'");
5097     }
5098   }
5099 
5100 
5101   /**
5102    * Get an exclusive ( write lock ) lock on a given row.
5103    * @param row Which row to lock.
5104    * @return A locked RowLock. The lock is exclusive and already aqquired.
5105    * @throws IOException
5106    */
5107   public RowLock getRowLock(byte[] row) throws IOException {
5108     return getRowLock(row, false);
5109   }
5110 
5111   /**
5112    *
5113    * Get a row lock for the specified row. All locks are reentrant.
5114    *
5115    * Before calling this function make sure that a region operation has already been
5116    * started (the calling thread has already acquired the region-close-guard lock).
5117    * @param row The row actions will be performed against
5118    * @param readLock is the lock reader or writer. True indicates that a non-exlcusive
5119    *                 lock is requested
5120    */
5121   public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
5122     // Make sure the row is inside of this region before getting the lock for it.
5123     checkRow(row, "row lock");
5124     // create an object to use a a key in the row lock map
5125     HashedBytes rowKey = new HashedBytes(row);
5126 
5127     RowLockContext rowLockContext = null;
5128     RowLockImpl result = null;
5129     TraceScope traceScope = null;
5130 
5131     // If we're tracing start a span to show how long this took.
5132     if (Trace.isTracing()) {
5133       traceScope = Trace.startSpan("HRegion.getRowLock");
5134       traceScope.getSpan().addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock"));
5135     }
5136 
5137     try {
5138       // Keep trying until we have a lock or error out.
5139       // TODO: do we need to add a time component here?
5140       while (result == null) {
5141 
5142         // Try adding a RowLockContext to the lockedRows.
5143         // If we can add it then there's no other transactions currently running.
5144         rowLockContext = new RowLockContext(rowKey);
5145         RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext);
5146 
5147         // if there was a running transaction then there's already a context.
5148         if (existingContext != null) {
5149           rowLockContext = existingContext;
5150         }
5151 
5152         // Now try an get the lock.
5153         //
5154         // This can fail as
5155         if (readLock) {
5156           result = rowLockContext.newReadLock();
5157         } else {
5158           result = rowLockContext.newWriteLock();
5159         }
5160       }
5161       if (!result.getLock().tryLock(this.rowLockWaitDuration, TimeUnit.MILLISECONDS)) {
5162         if (traceScope != null) {
5163           traceScope.getSpan().addTimelineAnnotation("Failed to get row lock");
5164         }
5165         result = null;
5166         // Clean up the counts just in case this was the thing keeping the context alive.
5167         rowLockContext.cleanUp();
5168         throw new IOException("Timed out waiting for lock for row: " + rowKey);
5169       }
5170       rowLockContext.setThreadName(Thread.currentThread().getName());
5171       return result;
5172     } catch (InterruptedException ie) {
5173       LOG.warn("Thread interrupted waiting for lock on row: " + rowKey);
5174       InterruptedIOException iie = new InterruptedIOException();
5175       iie.initCause(ie);
5176       if (traceScope != null) {
5177         traceScope.getSpan().addTimelineAnnotation("Interrupted exception getting row lock");
5178       }
5179       Thread.currentThread().interrupt();
5180       throw iie;
5181     } finally {
5182       if (traceScope != null) {
5183         traceScope.close();
5184       }
5185     }
5186   }
5187 
5188   @Override
5189   public void releaseRowLocks(List<RowLock> rowLocks) {
5190     if (rowLocks != null) {
5191       for (RowLock rowLock : rowLocks) {
5192         rowLock.release();
5193       }
5194       rowLocks.clear();
5195     }
5196   }
5197 
5198   public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
5199     return lockedRows;
5200   }
5201 
5202   @VisibleForTesting
5203   class RowLockContext {
5204     private final HashedBytes row;
5205     final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
5206     final AtomicBoolean usable = new AtomicBoolean(true);
5207     final AtomicInteger count = new AtomicInteger(0);
5208     final Object lock = new Object();
5209     private String threadName;
5210 
5211     RowLockContext(HashedBytes row) {
5212       this.row = row;
5213     }
5214 
5215     RowLockImpl newWriteLock() {
5216       Lock l = readWriteLock.writeLock();
5217       return getRowLock(l);
5218     }
5219     RowLockImpl newReadLock() {
5220       Lock l = readWriteLock.readLock();
5221       return getRowLock(l);
5222     }
5223 
5224     private RowLockImpl getRowLock(Lock l) {
5225       count.incrementAndGet();
5226       synchronized (lock) {
5227         if (usable.get()) {
5228           return new RowLockImpl(this, l);
5229         } else {
5230           return null;
5231         }
5232       }
5233     }
5234 
5235     void cleanUp() {
5236       long c = count.decrementAndGet();
5237       if (c <= 0) {
5238         synchronized (lock) {
5239           if (count.get() <= 0 ){
5240             usable.set(false);
5241             RowLockContext removed = lockedRows.remove(row);
5242             assert removed == this: "we should never remove a different context";
5243           }
5244         }
5245       }
5246     }
5247 
5248     public void setThreadName(String threadName) {
5249       this.threadName = threadName;
5250     }
5251 
5252     @Override
5253     public String toString() {
5254       return "RowLockContext{" +
5255           "row=" + row +
5256           ", readWriteLock=" + readWriteLock +
5257           ", count=" + count +
5258           ", threadName=" + threadName +
5259           '}';
5260     }
5261   }
5262 
5263   /**
5264    * Class used to represent a lock on a row.
5265    */
5266   public static class RowLockImpl implements RowLock {
5267     private final RowLockContext context;
5268     private final Lock lock;
5269 
5270     public RowLockImpl(RowLockContext context, Lock lock) {
5271       this.context = context;
5272       this.lock = lock;
5273     }
5274 
5275     public Lock getLock() {
5276       return lock;
5277     }
5278 
5279     @VisibleForTesting
5280     public RowLockContext getContext() {
5281       return context;
5282     }
5283 
5284     @Override
5285     public void release() {
5286       lock.unlock();
5287       context.cleanUp();
5288     }
5289 
5290     @Override
5291     public String toString() {
5292       return "RowLockImpl{" +
5293           "context=" + context +
5294           ", lock=" + lock +
5295           '}';
5296     }
5297   }
5298 
5299   /**
5300    * Determines whether multiple column families are present
5301    * Precondition: familyPaths is not null
5302    *
5303    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
5304    */
5305   private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
5306     boolean multipleFamilies = false;
5307     byte[] family = null;
5308     for (Pair<byte[], String> pair : familyPaths) {
5309       byte[] fam = pair.getFirst();
5310       if (family == null) {
5311         family = fam;
5312       } else if (!Bytes.equals(family, fam)) {
5313         multipleFamilies = true;
5314         break;
5315       }
5316     }
5317     return multipleFamilies;
5318   }
5319 
5320   @Override
5321   public boolean bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
5322       BulkLoadListener bulkLoadListener) throws IOException {
5323     long seqId = -1;
5324     Map<byte[], List<Path>> storeFiles = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
5325     Preconditions.checkNotNull(familyPaths);
5326     // we need writeLock for multi-family bulk load
5327     startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
5328     try {
5329       this.writeRequestsCount.increment();
5330 
5331       // There possibly was a split that happened between when the split keys
5332       // were gathered and before the HRegion's write lock was taken.  We need
5333       // to validate the HFile region before attempting to bulk load all of them
5334       List<IOException> ioes = new ArrayList<IOException>();
5335       List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
5336       for (Pair<byte[], String> p : familyPaths) {
5337         byte[] familyName = p.getFirst();
5338         String path = p.getSecond();
5339 
5340         Store store = getStore(familyName);
5341         if (store == null) {
5342           IOException ioe = new org.apache.hadoop.hbase.DoNotRetryIOException(
5343               "No such column family " + Bytes.toStringBinary(familyName));
5344           ioes.add(ioe);
5345         } else {
5346           try {
5347             store.assertBulkLoadHFileOk(new Path(path));
5348           } catch (WrongRegionException wre) {
5349             // recoverable (file doesn't fit in region)
5350             failures.add(p);
5351           } catch (IOException ioe) {
5352             // unrecoverable (hdfs problem)
5353             ioes.add(ioe);
5354           }
5355         }
5356       }
5357 
5358       // validation failed because of some sort of IO problem.
5359       if (ioes.size() != 0) {
5360         IOException e = MultipleIOException.createIOException(ioes);
5361         LOG.error("There were one or more IO errors when checking if the bulk load is ok.", e);
5362         throw e;
5363       }
5364 
5365       // validation failed, bail out before doing anything permanent.
5366       if (failures.size() != 0) {
5367         StringBuilder list = new StringBuilder();
5368         for (Pair<byte[], String> p : failures) {
5369           list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
5370               .append(p.getSecond());
5371         }
5372         // problem when validating
5373         LOG.warn("There was a recoverable bulk load failure likely due to a" +
5374             " split.  These (family, HFile) pairs were not loaded: " + list);
5375         return false;
5376       }
5377 
5378       // We need to assign a sequential ID that's in between two memstores in order to preserve
5379       // the guarantee that all the edits lower than the highest sequential ID from all the
5380       // HFiles are flushed on disk. See HBASE-10958.  The sequence id returned when we flush is
5381       // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is
5382       // a sequence id that we can be sure is beyond the last hfile written).
5383       if (assignSeqId) {
5384         FlushResult fs = flushcache(true, false);
5385         if (fs.isFlushSucceeded()) {
5386           seqId = ((FlushResultImpl)fs).flushSequenceId;
5387         } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
5388           seqId = ((FlushResultImpl)fs).flushSequenceId;
5389         } else {
5390           throw new IOException("Could not bulk load with an assigned sequential ID because the "+
5391             "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason);
5392         }
5393       }
5394 
5395       for (Pair<byte[], String> p : familyPaths) {
5396         byte[] familyName = p.getFirst();
5397         String path = p.getSecond();
5398         Store store = getStore(familyName);
5399         try {
5400           String finalPath = path;
5401           if (bulkLoadListener != null) {
5402             finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
5403           }
5404           Path commitedStoreFile = store.bulkLoadHFile(finalPath, seqId);
5405 
5406           if(storeFiles.containsKey(familyName)) {
5407             storeFiles.get(familyName).add(commitedStoreFile);
5408           } else {
5409             List<Path> storeFileNames = new ArrayList<Path>();
5410             storeFileNames.add(commitedStoreFile);
5411             storeFiles.put(familyName, storeFileNames);
5412           }
5413           if (bulkLoadListener != null) {
5414             bulkLoadListener.doneBulkLoad(familyName, path);
5415           }
5416         } catch (IOException ioe) {
5417           // A failure here can cause an atomicity violation that we currently
5418           // cannot recover from since it is likely a failed HDFS operation.
5419 
5420           // TODO Need a better story for reverting partial failures due to HDFS.
5421           LOG.error("There was a partial failure due to IO when attempting to" +
5422               " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
5423           if (bulkLoadListener != null) {
5424             try {
5425               bulkLoadListener.failedBulkLoad(familyName, path);
5426             } catch (Exception ex) {
5427               LOG.error("Error while calling failedBulkLoad for family " +
5428                   Bytes.toString(familyName) + " with path " + path, ex);
5429             }
5430           }
5431           throw ioe;
5432         }
5433       }
5434 
5435       return true;
5436     } finally {
5437       if (wal != null && !storeFiles.isEmpty()) {
5438         // write a bulk load event when not all hfiles are loaded
5439         try {
5440           WALProtos.BulkLoadDescriptor loadDescriptor = ProtobufUtil.toBulkLoadDescriptor(
5441               this.getRegionInfo().getTable(),
5442               ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles, seqId);
5443           WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(),
5444               loadDescriptor, mvcc);
5445         } catch (IOException ioe) {
5446           if (this.rsServices != null) {
5447             // Have to abort region server because some hfiles has been loaded but we can't write
5448             // the event into WAL
5449             this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
5450           }
5451         }
5452       }
5453 
5454       closeBulkRegionOperation();
5455     }
5456   }
5457 
5458   @Override
5459   public boolean equals(Object o) {
5460     return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
5461                                                 ((HRegion) o).getRegionInfo().getRegionName());
5462   }
5463 
5464   @Override
5465   public int hashCode() {
5466     return Bytes.hashCode(getRegionInfo().getRegionName());
5467   }
5468 
5469   @Override
5470   public String toString() {
5471     return getRegionInfo().getRegionNameAsString();
5472   }
5473 
5474   /**
5475    * RegionScannerImpl is used to combine scanners from multiple Stores (aka column families).
5476    */
5477   class RegionScannerImpl implements RegionScanner {
5478     // Package local for testability
5479     KeyValueHeap storeHeap = null;
5480     /** Heap of key-values that are not essential for the provided filters and are thus read
5481      * on demand, if on-demand column family loading is enabled.*/
5482     KeyValueHeap joinedHeap = null;
5483     /**
5484      * If the joined heap data gathering is interrupted due to scan limits, this will
5485      * contain the row for which we are populating the values.*/
5486     protected Cell joinedContinuationRow = null;
5487     private boolean filterClosed = false;
5488 
5489     protected final int isScan;
5490     protected final byte[] stopRow;
5491     protected final HRegion region;
5492 
5493     private final long readPt;
5494     private final long maxResultSize;
5495     private final ScannerContext defaultScannerContext;
5496     private final FilterWrapper filter;
5497 
5498     @Override
5499     public HRegionInfo getRegionInfo() {
5500       return region.getRegionInfo();
5501     }
5502 
5503     RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
5504         throws IOException {
5505       this.region = region;
5506       this.maxResultSize = scan.getMaxResultSize();
5507       if (scan.hasFilter()) {
5508         this.filter = new FilterWrapper(scan.getFilter());
5509       } else {
5510         this.filter = null;
5511       }
5512 
5513       /**
5514        * By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default
5515        * scanner context that can be used to enforce the batch limit in the event that a
5516        * ScannerContext is not specified during an invocation of next/nextRaw
5517        */
5518       defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build();
5519 
5520       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) && !scan.isGetScan()) {
5521         this.stopRow = null;
5522       } else {
5523         this.stopRow = scan.getStopRow();
5524       }
5525       // If we are doing a get, we want to be [startRow,endRow] normally
5526       // it is [startRow,endRow) and if startRow=endRow we get nothing.
5527       this.isScan = scan.isGetScan() ? -1 : 0;
5528 
5529       // synchronize on scannerReadPoints so that nobody calculates
5530       // getSmallestReadPoint, before scannerReadPoints is updated.
5531       IsolationLevel isolationLevel = scan.getIsolationLevel();
5532       synchronized(scannerReadPoints) {
5533         this.readPt = getReadpoint(isolationLevel);
5534         scannerReadPoints.put(this, this.readPt);
5535       }
5536 
5537       // Here we separate all scanners into two lists - scanner that provide data required
5538       // by the filter to operate (scanners list) and all others (joinedScanners list).
5539       List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
5540       List<KeyValueScanner> joinedScanners = new ArrayList<KeyValueScanner>();
5541       // Store all already instantiated scanners for exception handling
5542       List<KeyValueScanner> instantiatedScanners = new ArrayList<KeyValueScanner>();
5543       // handle additionalScanners
5544       if (additionalScanners != null && !additionalScanners.isEmpty()) {
5545         scanners.addAll(additionalScanners);
5546         instantiatedScanners.addAll(additionalScanners);
5547       }
5548 
5549       try {
5550         for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
5551           Store store = stores.get(entry.getKey());
5552           KeyValueScanner scanner;
5553           try {
5554             scanner = store.getScanner(scan, entry.getValue(), this.readPt);
5555           } catch (FileNotFoundException e) {
5556             throw handleFileNotFound(e);
5557           }
5558           instantiatedScanners.add(scanner);
5559           if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
5560               || this.filter.isFamilyEssential(entry.getKey())) {
5561             scanners.add(scanner);
5562           } else {
5563             joinedScanners.add(scanner);
5564           }
5565         }
5566         initializeKVHeap(scanners, joinedScanners, region);
5567       } catch (Throwable t) {
5568         throw handleException(instantiatedScanners, t);
5569       }
5570     }
5571 
5572     protected void initializeKVHeap(List<KeyValueScanner> scanners,
5573         List<KeyValueScanner> joinedScanners, HRegion region)
5574         throws IOException {
5575       this.storeHeap = new KeyValueHeap(scanners, region.comparator);
5576       if (!joinedScanners.isEmpty()) {
5577         this.joinedHeap = new KeyValueHeap(joinedScanners, region.comparator);
5578       }
5579     }
5580 
5581     private IOException handleException(List<KeyValueScanner> instantiatedScanners,
5582         Throwable t) {
5583       // remove scaner read point before throw the exception
5584       scannerReadPoints.remove(this);
5585       if (storeHeap != null) {
5586         storeHeap.close();
5587         storeHeap = null;
5588         if (joinedHeap != null) {
5589           joinedHeap.close();
5590           joinedHeap = null;
5591         }
5592       } else {
5593         // close all already instantiated scanners before throwing the exception
5594         for (KeyValueScanner scanner : instantiatedScanners) {
5595           scanner.close();
5596         }
5597       }
5598       return t instanceof IOException ? (IOException) t : new IOException(t);
5599     }
5600 
5601     @Override
5602     public long getMaxResultSize() {
5603       return maxResultSize;
5604     }
5605 
5606     @Override
5607     public long getMvccReadPoint() {
5608       return this.readPt;
5609     }
5610 
5611     @Override
5612     public int getBatch() {
5613       return this.defaultScannerContext.getBatchLimit();
5614     }
5615 
5616     /**
5617      * Reset both the filter and the old filter.
5618      *
5619      * @throws IOException in case a filter raises an I/O exception.
5620      */
5621     protected void resetFilters() throws IOException {
5622       if (filter != null) {
5623         filter.reset();
5624       }
5625     }
5626 
5627     @Override
5628     public boolean next(List<Cell> outResults)
5629         throws IOException {
5630       // apply the batching limit by default
5631       return next(outResults, defaultScannerContext);
5632     }
5633 
5634     @Override
5635     public synchronized boolean next(List<Cell> outResults, ScannerContext scannerContext)
5636     throws IOException {
5637       if (this.filterClosed) {
5638         throw new UnknownScannerException("Scanner was closed (timed out?) " +
5639             "after we renewed it. Could be caused by a very slow scanner " +
5640             "or a lengthy garbage collection");
5641       }
5642       startRegionOperation(Operation.SCAN);
5643       readRequestsCount.increment();
5644       try {
5645         return nextRaw(outResults, scannerContext);
5646       } finally {
5647         closeRegionOperation(Operation.SCAN);
5648       }
5649     }
5650 
5651     @Override
5652     public boolean nextRaw(List<Cell> outResults) throws IOException {
5653       // Use the RegionScanner's context by default
5654       return nextRaw(outResults, defaultScannerContext);
5655     }
5656 
5657     @Override
5658     public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext)
5659         throws IOException {
5660       if (storeHeap == null) {
5661         // scanner is closed
5662         throw new UnknownScannerException("Scanner was closed");
5663       }
5664       boolean moreValues;
5665       if (outResults.isEmpty()) {
5666         // Usually outResults is empty. This is true when next is called
5667         // to handle scan or get operation.
5668         moreValues = nextInternal(outResults, scannerContext);
5669       } else {
5670         List<Cell> tmpList = new ArrayList<Cell>();
5671         moreValues = nextInternal(tmpList, scannerContext);
5672         outResults.addAll(tmpList);
5673       }
5674 
5675       // If the size limit was reached it means a partial Result is being returned. Returning a
5676       // partial Result means that we should not reset the filters; filters should only be reset in
5677       // between rows
5678       if (!scannerContext.midRowResultFormed()) resetFilters();
5679 
5680       if (isFilterDoneInternal()) {
5681         moreValues = false;
5682       }
5683       return moreValues;
5684     }
5685 
5686     /**
5687      * @return true if more cells exist after this batch, false if scanner is done
5688      */
5689     private boolean populateFromJoinedHeap(List<Cell> results, ScannerContext scannerContext)
5690             throws IOException {
5691       assert joinedContinuationRow != null;
5692       boolean moreValues =
5693           populateResult(results, this.joinedHeap, scannerContext,
5694           joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
5695           joinedContinuationRow.getRowLength());
5696 
5697       if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5698         // We are done with this row, reset the continuation.
5699         joinedContinuationRow = null;
5700       }
5701       // As the data is obtained from two independent heaps, we need to
5702       // ensure that result list is sorted, because Result relies on that.
5703       Collections.sort(results, comparator);
5704       return moreValues;
5705     }
5706 
5707     /**
5708      * Fetches records with currentRow into results list, until next row, batchLimit (if not -1) is
5709      * reached, or remainingResultSize (if not -1) is reaced
5710      * @param heap KeyValueHeap to fetch data from.It must be positioned on correct row before call.
5711      * @param scannerContext
5712      * @param currentRow Byte array with key we are fetching.
5713      * @param offset offset for currentRow
5714      * @param length length for currentRow
5715      * @return state of last call to {@link KeyValueHeap#next()}
5716      */
5717     private boolean populateResult(List<Cell> results, KeyValueHeap heap,
5718         ScannerContext scannerContext, byte[] currentRow, int offset, short length)
5719         throws IOException {
5720       Cell nextKv;
5721       boolean moreCellsInRow = false;
5722       boolean tmpKeepProgress = scannerContext.getKeepProgress();
5723       // Scanning between column families and thus the scope is between cells
5724       LimitScope limitScope = LimitScope.BETWEEN_CELLS;
5725       try {
5726         do {
5727           // We want to maintain any progress that is made towards the limits while scanning across
5728           // different column families. To do this, we toggle the keep progress flag on during calls
5729           // to the StoreScanner to ensure that any progress made thus far is not wiped away.
5730           scannerContext.setKeepProgress(true);
5731           heap.next(results, scannerContext);
5732           scannerContext.setKeepProgress(tmpKeepProgress);
5733 
5734           nextKv = heap.peek();
5735           moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length);
5736           if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
5737 
5738           if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
5739             return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
5740           } else if (scannerContext.checkSizeLimit(limitScope)) {
5741             ScannerContext.NextState state =
5742                 moreCellsInRow? NextState.SIZE_LIMIT_REACHED_MID_ROW: NextState.SIZE_LIMIT_REACHED;
5743             return scannerContext.setScannerState(state).hasMoreValues();
5744           } else if (scannerContext.checkTimeLimit(limitScope)) {
5745             ScannerContext.NextState state =
5746                 moreCellsInRow? NextState.TIME_LIMIT_REACHED_MID_ROW: NextState.TIME_LIMIT_REACHED;
5747             return scannerContext.setScannerState(state).hasMoreValues();
5748           }
5749         } while (moreCellsInRow);
5750       } catch (FileNotFoundException e) {
5751         throw handleFileNotFound(e);
5752       }
5753       return nextKv != null;
5754     }
5755 
5756     /**
5757      * Based on the nextKv in the heap, and the current row, decide whether or not there are more
5758      * cells to be read in the heap. If the row of the nextKv in the heap matches the current row
5759      * then there are more cells to be read in the row.
5760      * @param nextKv
5761      * @param currentRow
5762      * @param offset
5763      * @param length
5764      * @return true When there are more cells in the row to be read
5765      */
5766     private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset,
5767         short length) {
5768       return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length);
5769     }
5770 
5771     /*
5772      * @return True if a filter rules the scanner is over, done.
5773      */
5774     @Override
5775     public synchronized boolean isFilterDone() throws IOException {
5776       return isFilterDoneInternal();
5777     }
5778 
5779     private boolean isFilterDoneInternal() throws IOException {
5780       return this.filter != null && this.filter.filterAllRemaining();
5781     }
5782 
5783     private boolean nextInternal(List<Cell> results, ScannerContext scannerContext)
5784         throws IOException {
5785       if (!results.isEmpty()) {
5786         throw new IllegalArgumentException("First parameter should be an empty list");
5787       }
5788       if (scannerContext == null) {
5789         throw new IllegalArgumentException("Scanner context cannot be null");
5790       }
5791       RpcCallContext rpcCall = RpcServer.getCurrentCall();
5792 
5793       // Save the initial progress from the Scanner context in these local variables. The progress
5794       // may need to be reset a few times if rows are being filtered out so we save the initial
5795       // progress.
5796       int initialBatchProgress = scannerContext.getBatchProgress();
5797       long initialSizeProgress = scannerContext.getSizeProgress();
5798       long initialTimeProgress = scannerContext.getTimeProgress();
5799 
5800       // The loop here is used only when at some point during the next we determine
5801       // that due to effects of filters or otherwise, we have an empty row in the result.
5802       // Then we loop and try again. Otherwise, we must get out on the first iteration via return,
5803       // "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row,
5804       // and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow).
5805       while (true) {
5806         // Starting to scan a new row. Reset the scanner progress according to whether or not
5807         // progress should be kept.
5808         if (scannerContext.getKeepProgress()) {
5809           // Progress should be kept. Reset to initial values seen at start of method invocation.
5810           scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
5811             initialTimeProgress);
5812         } else {
5813           scannerContext.clearProgress();
5814         }
5815 
5816         if (rpcCall != null) {
5817           // If a user specifies a too-restrictive or too-slow scanner, the
5818           // client might time out and disconnect while the server side
5819           // is still processing the request. We should abort aggressively
5820           // in that case.
5821           long afterTime = rpcCall.disconnectSince();
5822           if (afterTime >= 0) {
5823             throw new CallerDisconnectedException(
5824                 "Aborting on region " + getRegionInfo().getRegionNameAsString() + ", call " +
5825                     this + " after " + afterTime + " ms, since " +
5826                     "caller disconnected");
5827           }
5828         }
5829 
5830         // Let's see what we have in the storeHeap.
5831         Cell current = this.storeHeap.peek();
5832 
5833         byte[] currentRow = null;
5834         int offset = 0;
5835         short length = 0;
5836         if (current != null) {
5837           currentRow = current.getRowArray();
5838           offset = current.getRowOffset();
5839           length = current.getRowLength();
5840         }
5841 
5842         boolean stopRow = isStopRow(currentRow, offset, length);
5843         // When has filter row is true it means that the all the cells for a particular row must be
5844         // read before a filtering decision can be made. This means that filters where hasFilterRow
5845         // run the risk of encountering out of memory errors in the case that they are applied to a
5846         // table that has very large rows.
5847         boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow();
5848 
5849         // If filter#hasFilterRow is true, partial results are not allowed since allowing them
5850         // would prevent the filters from being evaluated. Thus, if it is true, change the
5851         // scope of any limits that could potentially create partial results to
5852         // LimitScope.BETWEEN_ROWS so that those limits are not reached mid-row
5853         if (hasFilterRow) {
5854           if (LOG.isTraceEnabled()) {
5855             LOG.trace("filter#hasFilterRow is true which prevents partial results from being "
5856                 + " formed. Changing scope of limits that may create partials");
5857           }
5858           scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS);
5859           scannerContext.setTimeLimitScope(LimitScope.BETWEEN_ROWS);
5860         }
5861 
5862         // Check if we were getting data from the joinedHeap and hit the limit.
5863         // If not, then it's main path - getting results from storeHeap.
5864         if (joinedContinuationRow == null) {
5865           // First, check if we are at a stop row. If so, there are no more results.
5866           if (stopRow) {
5867             if (hasFilterRow) {
5868               filter.filterRowCells(results);
5869             }
5870             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5871           }
5872 
5873           // Check if rowkey filter wants to exclude this row. If so, loop to next.
5874           // Technically, if we hit limits before on this row, we don't need this call.
5875           if (filterRowKey(currentRow, offset, length)) {
5876             incrementCountOfRowsFilteredMetric(scannerContext);
5877             // early check, see HBASE-16296
5878             if (isFilterDoneInternal()) {
5879               return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5880             }
5881             // Typically the count of rows scanned is incremented inside #populateResult. However,
5882             // here we are filtering a row based purely on its row key, preventing us from calling
5883             // #populateResult. Thus, perform the necessary increment here to rows scanned metric
5884             incrementCountOfRowsScannedMetric(scannerContext);
5885             boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
5886             if (!moreRows) {
5887               return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5888             }
5889             results.clear();
5890             continue;
5891           }
5892 
5893           // Ok, we are good, let's try to get some results from the main heap.
5894           populateResult(results, this.storeHeap, scannerContext, currentRow, offset, length);
5895 
5896           if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5897             if (hasFilterRow) {
5898               throw new IncompatibleFilterException(
5899                   "Filter whose hasFilterRow() returns true is incompatible with scans that must "
5900                       + " stop mid-row because of a limit. ScannerContext:" + scannerContext);
5901             }
5902             return true;
5903           }
5904 
5905           Cell nextKv = this.storeHeap.peek();
5906           stopRow = nextKv == null ||
5907               isStopRow(nextKv.getRowArray(), nextKv.getRowOffset(), nextKv.getRowLength());
5908           // save that the row was empty before filters applied to it.
5909           final boolean isEmptyRow = results.isEmpty();
5910 
5911           // We have the part of the row necessary for filtering (all of it, usually).
5912           // First filter with the filterRow(List).
5913           FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;
5914           if (hasFilterRow) {
5915             ret = filter.filterRowCellsWithRet(results);
5916 
5917             // We don't know how the results have changed after being filtered. Must set progress
5918             // according to contents of results now. However, a change in the results should not
5919             // affect the time progress. Thus preserve whatever time progress has been made
5920             long timeProgress = scannerContext.getTimeProgress();
5921             if (scannerContext.getKeepProgress()) {
5922               scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
5923                 initialTimeProgress);
5924             } else {
5925               scannerContext.clearProgress();
5926             }
5927             scannerContext.setTimeProgress(timeProgress);
5928             scannerContext.incrementBatchProgress(results.size());
5929             for (Cell cell : results) {
5930               scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell));
5931             }
5932           }
5933 
5934           if (isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE || filterRow()) {
5935             incrementCountOfRowsFilteredMetric(scannerContext);
5936             results.clear();
5937             boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
5938             if (!moreRows) {
5939               return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5940             }
5941 
5942             // This row was totally filtered out, if this is NOT the last row,
5943             // we should continue on. Otherwise, nothing else to do.
5944             if (!stopRow) continue;
5945             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5946           }
5947 
5948           // Ok, we are done with storeHeap for this row.
5949           // Now we may need to fetch additional, non-essential data into row.
5950           // These values are not needed for filter to work, so we postpone their
5951           // fetch to (possibly) reduce amount of data loads from disk.
5952           if (this.joinedHeap != null) {
5953             boolean mayHaveData = joinedHeapMayHaveData(currentRow, offset, length);
5954             if (mayHaveData) {
5955               joinedContinuationRow = current;
5956               populateFromJoinedHeap(results, scannerContext);
5957 
5958               if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5959                 return true;
5960               }
5961             }
5962           }
5963         } else {
5964           // Populating from the joined heap was stopped by limits, populate some more.
5965           populateFromJoinedHeap(results, scannerContext);
5966           if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
5967             return true;
5968           }
5969         }
5970         // We may have just called populateFromJoinedMap and hit the limits. If that is
5971         // the case, we need to call it again on the next next() invocation.
5972         if (joinedContinuationRow != null) {
5973           return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
5974         }
5975 
5976         // Finally, we are done with both joinedHeap and storeHeap.
5977         // Double check to prevent empty rows from appearing in result. It could be
5978         // the case when SingleColumnValueExcludeFilter is used.
5979         if (results.isEmpty()) {
5980           incrementCountOfRowsFilteredMetric(scannerContext);
5981           boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
5982           if (!moreRows) {
5983             return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5984           }
5985           if (!stopRow) continue;
5986         }
5987 
5988         if (stopRow) {
5989           return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
5990         } else {
5991           return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
5992         }
5993       }
5994     }
5995 
5996     protected void incrementCountOfRowsFilteredMetric(ScannerContext scannerContext) {
5997       if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
5998 
5999       scannerContext.getMetrics().countOfRowsFiltered.incrementAndGet();
6000     }
6001 
6002     protected void incrementCountOfRowsScannedMetric(ScannerContext scannerContext) {
6003       if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
6004 
6005       scannerContext.getMetrics().countOfRowsScanned.incrementAndGet();
6006     }
6007 
6008     /**
6009      * @param currentRow
6010      * @param offset
6011      * @param length
6012      * @return true when the joined heap may have data for the current row
6013      * @throws IOException
6014      */
6015     private boolean joinedHeapMayHaveData(byte[] currentRow, int offset, short length)
6016         throws IOException {
6017       Cell nextJoinedKv = joinedHeap.peek();
6018       boolean matchCurrentRow =
6019           nextJoinedKv != null && CellUtil.matchingRow(nextJoinedKv, currentRow, offset, length);
6020       boolean matchAfterSeek = false;
6021 
6022       // If the next value in the joined heap does not match the current row, try to seek to the
6023       // correct row
6024       if (!matchCurrentRow) {
6025         Cell firstOnCurrentRow = KeyValueUtil.createFirstOnRow(currentRow, offset, length);
6026         boolean seekSuccessful = this.joinedHeap.requestSeek(firstOnCurrentRow, true, true);
6027         matchAfterSeek =
6028             seekSuccessful && joinedHeap.peek() != null
6029                 && CellUtil.matchingRow(joinedHeap.peek(), currentRow, offset, length);
6030       }
6031 
6032       return matchCurrentRow || matchAfterSeek;
6033     }
6034 
6035     /**
6036      * This function is to maintain backward compatibility for 0.94 filters. HBASE-6429 combines
6037      * both filterRow & filterRow(List<KeyValue> kvs) functions. While 0.94 code or older, it may
6038      * not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only returns
6039      * true when filterRow(List<KeyValue> kvs) is overridden not the filterRow(). Therefore, the
6040      * filterRow() will be skipped.
6041      */
6042     private boolean filterRow() throws IOException {
6043       // when hasFilterRow returns true, filter.filterRow() will be called automatically inside
6044       // filterRowCells(List<Cell> kvs) so we skip that scenario here.
6045       return filter != null && (!filter.hasFilterRow())
6046           && filter.filterRow();
6047     }
6048 
6049     private boolean filterRowKey(byte[] row, int offset, short length) throws IOException {
6050       return filter != null
6051           && filter.filterRowKey(row, offset, length);
6052     }
6053 
6054     protected boolean nextRow(ScannerContext scannerContext, byte[] currentRow, int offset,
6055         short length) throws IOException {
6056       assert this.joinedContinuationRow == null:
6057         "Trying to go to next row during joinedHeap read.";
6058       Cell next;
6059       while ((next = this.storeHeap.peek()) != null &&
6060              CellUtil.matchingRow(next, currentRow, offset, length)) {
6061         this.storeHeap.next(MOCKED_LIST);
6062       }
6063       resetFilters();
6064 
6065       // Calling the hook in CP which allows it to do a fast forward
6066       return this.region.getCoprocessorHost() == null
6067           || this.region.getCoprocessorHost()
6068               .postScannerFilterRow(this, currentRow, offset, length);
6069     }
6070 
6071     protected boolean isStopRow(byte[] currentRow, int offset, short length) {
6072       return currentRow == null ||
6073           (stopRow != null &&
6074           comparator.compareRows(stopRow, 0, stopRow.length,
6075             currentRow, offset, length) <= isScan);
6076     }
6077 
6078     @Override
6079     public synchronized void close() {
6080       if (storeHeap != null) {
6081         storeHeap.close();
6082         storeHeap = null;
6083       }
6084       if (joinedHeap != null) {
6085         joinedHeap.close();
6086         joinedHeap = null;
6087       }
6088       // no need to synchronize here.
6089       scannerReadPoints.remove(this);
6090       this.filterClosed = true;
6091     }
6092 
6093     KeyValueHeap getStoreHeapForTesting() {
6094       return storeHeap;
6095     }
6096 
6097     @Override
6098     public synchronized boolean reseek(byte[] row) throws IOException {
6099       if (row == null) {
6100         throw new IllegalArgumentException("Row cannot be null.");
6101       }
6102       boolean result = false;
6103       startRegionOperation();
6104       KeyValue kv = KeyValueUtil.createFirstOnRow(row);
6105       try {
6106         // use request seek to make use of the lazy seek option. See HBASE-5520
6107         result = this.storeHeap.requestSeek(kv, true, true);
6108         if (this.joinedHeap != null) {
6109           result = this.joinedHeap.requestSeek(kv, true, true) || result;
6110         }
6111       } catch (FileNotFoundException e) {
6112         throw handleFileNotFound(e);
6113       } finally {
6114         closeRegionOperation();
6115       }
6116       return result;
6117     }
6118 
6119     private IOException handleFileNotFound(FileNotFoundException fnfe) throws IOException {
6120       // tries to refresh the store files, otherwise shutdown the RS.
6121       // TODO: add support for abort() of a single region and trigger reassignment.
6122       try {
6123         region.refreshStoreFiles(true);
6124         return new IOException("unable to read store file");
6125       } catch (IOException e) {
6126         String msg = "a store file got lost: " + fnfe.getMessage();
6127         LOG.error(msg);
6128         LOG.error("unable to refresh store files", e);
6129         abortRegionServer(msg);
6130         return new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " closing");
6131       }
6132     }
6133 
6134     private void abortRegionServer(String msg) throws IOException {
6135       if (rsServices instanceof HRegionServer) {
6136         ((HRegionServer)rsServices).abort(msg);
6137       }
6138       throw new UnsupportedOperationException("not able to abort RS after: " + msg);
6139     }
6140   }
6141 
6142   // Utility methods
6143   /**
6144    * A utility method to create new instances of HRegion based on the
6145    * {@link HConstants#REGION_IMPL} configuration property.
6146    * @param tableDir qualified path of directory where region should be located,
6147    * usually the table directory.
6148    * @param wal The WAL is the outbound log for any updates to the HRegion
6149    * The wal file is a logfile from the previous execution that's
6150    * custom-computed for this HRegion. The HRegionServer computes and sorts the
6151    * appropriate wal info for this HRegion. If there is a previous file
6152    * (implying that the HRegion has been written-to before), then read it from
6153    * the supplied path.
6154    * @param fs is the filesystem.
6155    * @param conf is global configuration settings.
6156    * @param regionInfo - HRegionInfo that describes the region
6157    * is new), then read them from the supplied path.
6158    * @param htd the table descriptor
6159    * @return the new instance
6160    */
6161   static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
6162       Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
6163       RegionServerServices rsServices) {
6164     try {
6165       @SuppressWarnings("unchecked")
6166       Class<? extends HRegion> regionClass =
6167           (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
6168 
6169       Constructor<? extends HRegion> c =
6170           regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
6171               Configuration.class, HRegionInfo.class, HTableDescriptor.class,
6172               RegionServerServices.class);
6173 
6174       return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
6175     } catch (Throwable e) {
6176       // todo: what should I throw here?
6177       throw new IllegalStateException("Could not instantiate a region instance.", e);
6178     }
6179   }
6180 
6181   /**
6182    * Convenience method creating new HRegions. Used by createTable and by the
6183    * bootstrap code in the HMaster constructor.
6184    * Note, this method creates an {@link WAL} for the created region. It
6185    * needs to be closed explicitly.  Use {@link HRegion#getWAL()} to get
6186    * access.  <b>When done with a region created using this method, you will
6187    * need to explicitly close the {@link WAL} it created too; it will not be
6188    * done for you.  Not closing the wal will leave at least a daemon thread
6189    * running.</b>  Call {@link #closeHRegion(HRegion)} and it will do
6190    * necessary cleanup for you.
6191    * @param info Info for region to create.
6192    * @param rootDir Root directory for HBase instance
6193    * @return new HRegion
6194    *
6195    * @throws IOException
6196    */
6197   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6198       final Configuration conf, final HTableDescriptor hTableDescriptor)
6199   throws IOException {
6200     return createHRegion(info, rootDir, conf, hTableDescriptor, null);
6201   }
6202 
6203   /**
6204    * This will do the necessary cleanup a call to
6205    * {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)}
6206    * requires.  This method will close the region and then close its
6207    * associated {@link WAL} file.  You can still use it if you call the other createHRegion,
6208    * the one that takes an {@link WAL} instance but don't be surprised by the
6209    * call to the {@link WAL#close()} on the {@link WAL} the
6210    * HRegion was carrying.
6211    * @throws IOException
6212    */
6213   public static void closeHRegion(final HRegion r) throws IOException {
6214     if (r == null) return;
6215     r.close();
6216     if (r.getWAL() == null) return;
6217     r.getWAL().close();
6218   }
6219 
6220   /**
6221    * Convenience method creating new HRegions. Used by createTable.
6222    * The {@link WAL} for the created region needs to be closed explicitly.
6223    * Use {@link HRegion#getWAL()} to get access.
6224    *
6225    * @param info Info for region to create.
6226    * @param rootDir Root directory for HBase instance
6227    * @param wal shared WAL
6228    * @param initialize - true to initialize the region
6229    * @return new HRegion
6230    *
6231    * @throws IOException
6232    */
6233   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6234                                       final Configuration conf,
6235                                       final HTableDescriptor hTableDescriptor,
6236                                       final WAL wal,
6237                                       final boolean initialize)
6238       throws IOException {
6239     return createHRegion(info, rootDir, conf, hTableDescriptor,
6240         wal, initialize, false);
6241   }
6242 
6243   /**
6244    * Convenience method creating new HRegions. Used by createTable.
6245    * The {@link WAL} for the created region needs to be closed
6246    * explicitly, if it is not null.
6247    * Use {@link HRegion#getWAL()} to get access.
6248    *
6249    * @param info Info for region to create.
6250    * @param rootDir Root directory for HBase instance
6251    * @param wal shared WAL
6252    * @param initialize - true to initialize the region
6253    * @param ignoreWAL - true to skip generate new wal if it is null, mostly for createTable
6254    * @return new HRegion
6255    * @throws IOException
6256    */
6257   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6258                                       final Configuration conf,
6259                                       final HTableDescriptor hTableDescriptor,
6260                                       final WAL wal,
6261                                       final boolean initialize, final boolean ignoreWAL)
6262       throws IOException {
6263       Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6264       return createHRegion(info, rootDir, tableDir, conf, hTableDescriptor, wal, initialize,
6265           ignoreWAL);
6266   }
6267 
6268   /**
6269    * Convenience method creating new HRegions. Used by createTable.
6270    * The {@link WAL} for the created region needs to be closed
6271    * explicitly, if it is not null.
6272    * Use {@link HRegion#getWAL()} to get access.
6273    *
6274    * @param info Info for region to create.
6275    * @param rootDir Root directory for HBase instance
6276    * @param tableDir table directory
6277    * @param wal shared WAL
6278    * @param initialize - true to initialize the region
6279    * @param ignoreWAL - true to skip generate new wal if it is null, mostly for createTable
6280    * @return new HRegion
6281    * @throws IOException
6282    */
6283   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6284       final Path tableDir, final Configuration conf, final HTableDescriptor hTableDescriptor,
6285       final WAL wal, final boolean initialize, final boolean ignoreWAL)
6286   throws IOException {
6287     LOG.info("creating HRegion " + info.getTable().getNameAsString()
6288         + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
6289         " Table name == " + info.getTable().getNameAsString());
6290     FileSystem fs = FileSystem.get(conf);
6291     HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
6292     WAL effectiveWAL = wal;
6293     if (wal == null && !ignoreWAL) {
6294       // TODO HBASE-11983 There'll be no roller for this wal?
6295       // The WAL subsystem will use the default rootDir rather than the passed in rootDir
6296       // unless I pass along via the conf.
6297       Configuration confForWAL = new Configuration(conf);
6298       confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
6299       effectiveWAL = (new WALFactory(confForWAL,
6300           Collections.<WALActionsListener>singletonList(new MetricsWAL()),
6301           "hregion-" + RandomStringUtils.randomNumeric(8))).
6302             getWAL(info.getEncodedNameAsBytes());
6303     }
6304     HRegion region = HRegion.newHRegion(tableDir,
6305         effectiveWAL, fs, conf, info, hTableDescriptor, null);
6306     if (initialize) region.initialize(null);
6307     return region;
6308   }
6309 
6310   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6311                                       final Configuration conf,
6312                                       final HTableDescriptor hTableDescriptor,
6313                                       final WAL wal)
6314     throws IOException {
6315     return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
6316   }
6317 
6318 
6319   /**
6320    * Open a Region.
6321    * @param info Info for region to be opened.
6322    * @param wal WAL for region to use. This method will call
6323    * WAL#setSequenceNumber(long) passing the result of the call to
6324    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6325    * up.  HRegionStore does this every time it opens a new region.
6326    * @return new HRegion
6327    *
6328    * @throws IOException
6329    */
6330   public static HRegion openHRegion(final HRegionInfo info,
6331       final HTableDescriptor htd, final WAL wal,
6332       final Configuration conf)
6333   throws IOException {
6334     return openHRegion(info, htd, wal, conf, null, null);
6335   }
6336 
6337   /**
6338    * Open a Region.
6339    * @param info Info for region to be opened
6340    * @param htd the table descriptor
6341    * @param wal WAL for region to use. This method will call
6342    * WAL#setSequenceNumber(long) passing the result of the call to
6343    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6344    * up.  HRegionStore does this every time it opens a new region.
6345    * @param conf The Configuration object to use.
6346    * @param rsServices An interface we can request flushes against.
6347    * @param reporter An interface we can report progress against.
6348    * @return new HRegion
6349    *
6350    * @throws IOException
6351    */
6352   public static HRegion openHRegion(final HRegionInfo info,
6353     final HTableDescriptor htd, final WAL wal, final Configuration conf,
6354     final RegionServerServices rsServices,
6355     final CancelableProgressable reporter)
6356   throws IOException {
6357     return openHRegion(FSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
6358   }
6359 
6360   /**
6361    * Open a Region.
6362    * @param rootDir Root directory for HBase instance
6363    * @param info Info for region to be opened.
6364    * @param htd the table descriptor
6365    * @param wal WAL for region to use. This method will call
6366    * WAL#setSequenceNumber(long) passing the result of the call to
6367    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6368    * up.  HRegionStore does this every time it opens a new region.
6369    * @param conf The Configuration object to use.
6370    * @return new HRegion
6371    * @throws IOException
6372    */
6373   public static HRegion openHRegion(Path rootDir, final HRegionInfo info,
6374       final HTableDescriptor htd, final WAL wal, final Configuration conf)
6375   throws IOException {
6376     return openHRegion(rootDir, info, htd, wal, conf, null, null);
6377   }
6378 
6379   /**
6380    * Open a Region.
6381    * @param rootDir Root directory for HBase instance
6382    * @param info Info for region to be opened.
6383    * @param htd the table descriptor
6384    * @param wal WAL for region to use. This method will call
6385    * WAL#setSequenceNumber(long) passing the result of the call to
6386    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6387    * up.  HRegionStore does this every time it opens a new region.
6388    * @param conf The Configuration object to use.
6389    * @param rsServices An interface we can request flushes against.
6390    * @param reporter An interface we can report progress against.
6391    * @return new HRegion
6392    * @throws IOException
6393    */
6394   public static HRegion openHRegion(final Path rootDir, final HRegionInfo info,
6395       final HTableDescriptor htd, final WAL wal, final Configuration conf,
6396       final RegionServerServices rsServices,
6397       final CancelableProgressable reporter)
6398   throws IOException {
6399     FileSystem fs = null;
6400     if (rsServices != null) {
6401       fs = rsServices.getFileSystem();
6402     }
6403     if (fs == null) {
6404       fs = FileSystem.get(conf);
6405     }
6406     return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
6407   }
6408 
6409   /**
6410    * Open a Region.
6411    * @param conf The Configuration object to use.
6412    * @param fs Filesystem to use
6413    * @param rootDir Root directory for HBase instance
6414    * @param info Info for region to be opened.
6415    * @param htd the table descriptor
6416    * @param wal WAL for region to use. This method will call
6417    * WAL#setSequenceNumber(long) passing the result of the call to
6418    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6419    * up.  HRegionStore does this every time it opens a new region.
6420    * @return new HRegion
6421    * @throws IOException
6422    */
6423   public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
6424       final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal)
6425       throws IOException {
6426     return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
6427   }
6428 
6429   /**
6430    * Open a Region.
6431    * @param conf The Configuration object to use.
6432    * @param fs Filesystem to use
6433    * @param rootDir Root directory for HBase instance
6434    * @param info Info for region to be opened.
6435    * @param htd the table descriptor
6436    * @param wal WAL for region to use. This method will call
6437    * WAL#setSequenceNumber(long) passing the result of the call to
6438    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6439    * up.  HRegionStore does this every time it opens a new region.
6440    * @param rsServices An interface we can request flushes against.
6441    * @param reporter An interface we can report progress against.
6442    * @return new HRegion
6443    * @throws IOException
6444    */
6445   public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
6446       final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal,
6447       final RegionServerServices rsServices, final CancelableProgressable reporter)
6448       throws IOException {
6449     Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6450     return openHRegion(conf, fs, rootDir, tableDir, info, htd, wal, rsServices, reporter);
6451   }
6452 
6453   /**
6454    * Open a Region.
6455    * @param conf The Configuration object to use.
6456    * @param fs Filesystem to use
6457    * @param rootDir Root directory for HBase instance
6458    * @param info Info for region to be opened.
6459    * @param htd the table descriptor
6460    * @param wal WAL for region to use. This method will call
6461    * WAL#setSequenceNumber(long) passing the result of the call to
6462    * HRegion#getMinSequenceId() to ensure the wal id is properly kept
6463    * up.  HRegionStore does this every time it opens a new region.
6464    * @param rsServices An interface we can request flushes against.
6465    * @param reporter An interface we can report progress against.
6466    * @return new HRegion
6467    * @throws IOException
6468    */
6469   public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
6470       final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd,
6471       final WAL wal, final RegionServerServices rsServices,
6472       final CancelableProgressable reporter)
6473       throws IOException {
6474     if (info == null) throw new NullPointerException("Passed region info is null");
6475     if (LOG.isDebugEnabled()) {
6476       LOG.debug("Opening region: " + info);
6477     }
6478     HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
6479     return r.openHRegion(reporter);
6480   }
6481 
6482 
6483   /**
6484    * Useful when reopening a closed region (normally for unit tests)
6485    * @param other original object
6486    * @param reporter An interface we can report progress against.
6487    * @return new HRegion
6488    * @throws IOException
6489    */
6490   public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
6491       throws IOException {
6492     HRegionFileSystem regionFs = other.getRegionFileSystem();
6493     HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
6494         other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
6495     return r.openHRegion(reporter);
6496   }
6497 
6498   public static Region openHRegion(final Region other, final CancelableProgressable reporter)
6499         throws IOException {
6500     return openHRegion((HRegion)other, reporter);
6501   }
6502 
6503   /**
6504    * Open HRegion.
6505    * Calls initialize and sets sequenceId.
6506    * @return Returns <code>this</code>
6507    * @throws IOException
6508    */
6509   protected HRegion openHRegion(final CancelableProgressable reporter)
6510   throws IOException {
6511     // Refuse to open the region if we are missing local compression support
6512     checkCompressionCodecs();
6513     // Refuse to open the region if encryption configuration is incorrect or
6514     // codec support is missing
6515     checkEncryption();
6516     // Refuse to open the region if a required class cannot be loaded
6517     checkClassLoading();
6518     this.openSeqNum = initialize(reporter);
6519     this.mvcc.advanceTo(openSeqNum);
6520     if (wal != null && getRegionServerServices() != null && !writestate.readOnly
6521         && !recovering) {
6522       // Only write the region open event marker to WAL if (1) we are not read-only
6523       // (2) dist log replay is off or we are not recovering. In case region is
6524       // recovering, the open event will be written at setRecovering(false)
6525       writeRegionOpenMarker(wal, openSeqNum);
6526     }
6527     return this;
6528   }
6529 
6530   public static void warmupHRegion(final HRegionInfo info,
6531       final HTableDescriptor htd, final WAL wal, final Configuration conf,
6532       final RegionServerServices rsServices,
6533       final CancelableProgressable reporter)
6534       throws IOException {
6535 
6536     if (info == null) throw new NullPointerException("Passed region info is null");
6537 
6538     if (LOG.isDebugEnabled()) {
6539       LOG.debug("HRegion.Warming up region: " + info);
6540     }
6541 
6542     Path rootDir = FSUtils.getRootDir(conf);
6543     Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6544 
6545     FileSystem fs = null;
6546     if (rsServices != null) {
6547       fs = rsServices.getFileSystem();
6548     }
6549     if (fs == null) {
6550       fs = FileSystem.get(conf);
6551     }
6552 
6553     HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
6554     r.initializeWarmup(reporter);
6555   }
6556 
6557 
6558   private void checkCompressionCodecs() throws IOException {
6559     for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
6560       CompressionTest.testCompression(fam.getCompression());
6561       CompressionTest.testCompression(fam.getCompactionCompression());
6562     }
6563   }
6564 
6565   private void checkEncryption() throws IOException {
6566     for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
6567       EncryptionTest.testEncryption(conf, fam.getEncryptionType(), fam.getEncryptionKey());
6568     }
6569   }
6570 
6571   private void checkClassLoading() throws IOException {
6572     RegionSplitPolicy.getSplitPolicyClass(this.htableDescriptor, conf);
6573     RegionCoprocessorHost.testTableCoprocessorAttrs(conf, this.htableDescriptor);
6574   }
6575 
6576   /**
6577    * Create a daughter region from given a temp directory with the region data.
6578    * @param hri Spec. for daughter region to open.
6579    * @throws IOException
6580    */
6581   HRegion createDaughterRegionFromSplits(final HRegionInfo hri) throws IOException {
6582     // Move the files from the temporary .splits to the final /table/region directory
6583     fs.commitDaughterRegion(hri);
6584 
6585     // Create the daughter HRegion instance
6586     HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(), fs.getFileSystem(),
6587         this.getBaseConf(), hri, this.getTableDesc(), rsServices);
6588     r.readRequestsCount.set(this.getReadRequestsCount() / 2);
6589     r.writeRequestsCount.set(this.getWriteRequestsCount() / 2);
6590     return r;
6591   }
6592 
6593   /**
6594    * Create a merged region given a temp directory with the region data.
6595    * @param region_b another merging region
6596    * @return merged HRegion
6597    * @throws IOException
6598    */
6599   HRegion createMergedRegionFromMerges(final HRegionInfo mergedRegionInfo,
6600       final HRegion region_b) throws IOException {
6601     HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(),
6602         fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
6603         this.getTableDesc(), this.rsServices);
6604     r.readRequestsCount.set(this.getReadRequestsCount()
6605         + region_b.getReadRequestsCount());
6606     r.writeRequestsCount.set(this.getWriteRequestsCount()
6607 
6608         + region_b.getWriteRequestsCount());
6609     this.fs.commitMergedRegion(mergedRegionInfo);
6610     return r;
6611   }
6612 
6613   /**
6614    * Inserts a new region's meta information into the passed
6615    * <code>meta</code> region. Used by the HMaster bootstrap code adding
6616    * new table to hbase:meta table.
6617    *
6618    * @param meta hbase:meta HRegion to be updated
6619    * @param r HRegion to add to <code>meta</code>
6620    *
6621    * @throws IOException
6622    */
6623   // TODO remove since only test and merge use this
6624   public static void addRegionToMETA(final HRegion meta, final HRegion r) throws IOException {
6625     meta.checkResources();
6626     // The row key is the region name
6627     byte[] row = r.getRegionInfo().getRegionName();
6628     final long now = EnvironmentEdgeManager.currentTime();
6629     final List<Cell> cells = new ArrayList<Cell>(2);
6630     cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
6631       HConstants.REGIONINFO_QUALIFIER, now,
6632       r.getRegionInfo().toByteArray()));
6633     // Set into the root table the version of the meta table.
6634     cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
6635       HConstants.META_VERSION_QUALIFIER, now,
6636       Bytes.toBytes(HConstants.META_VERSION)));
6637     meta.put(row, HConstants.CATALOG_FAMILY, cells);
6638   }
6639 
6640   /**
6641    * Computes the Path of the HRegion
6642    *
6643    * @param tabledir qualified path for table
6644    * @param name ENCODED region name
6645    * @return Path of HRegion directory
6646    * @deprecated For tests only; to be removed.
6647    */
6648   @Deprecated
6649   public static Path getRegionDir(final Path tabledir, final String name) {
6650     return new Path(tabledir, name);
6651   }
6652 
6653   /**
6654    * Computes the Path of the HRegion
6655    *
6656    * @param rootdir qualified path of HBase root directory
6657    * @param info HRegionInfo for the region
6658    * @return qualified path of region directory
6659    * @deprecated For tests only; to be removed.
6660    */
6661   @Deprecated
6662   @VisibleForTesting
6663   public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
6664     return new Path(
6665       FSUtils.getTableDir(rootdir, info.getTable()), info.getEncodedName());
6666   }
6667 
6668   /**
6669    * Determines if the specified row is within the row range specified by the
6670    * specified HRegionInfo
6671    *
6672    * @param info HRegionInfo that specifies the row range
6673    * @param row row to be checked
6674    * @return true if the row is within the range specified by the HRegionInfo
6675    */
6676   public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
6677     return ((info.getStartKey().length == 0) ||
6678         (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
6679         ((info.getEndKey().length == 0) ||
6680             (Bytes.compareTo(info.getEndKey(), row) > 0));
6681   }
6682 
6683   public static boolean rowIsInRange(HRegionInfo info, final byte [] row, final int offset,
6684       final short length) {
6685     return ((info.getStartKey().length == 0) ||
6686         (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
6687           row, offset, length) <= 0)) &&
6688         ((info.getEndKey().length == 0) ||
6689           (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
6690   }
6691 
6692   /**
6693    * Merge two HRegions.  The regions must be adjacent and must not overlap.
6694    *
6695    * @return new merged HRegion
6696    * @throws IOException
6697    */
6698   public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
6699   throws IOException {
6700     HRegion a = srcA;
6701     HRegion b = srcB;
6702 
6703     // Make sure that srcA comes first; important for key-ordering during
6704     // write of the merged file.
6705     if (srcA.getRegionInfo().getStartKey() == null) {
6706       if (srcB.getRegionInfo().getStartKey() == null) {
6707         throw new IOException("Cannot merge two regions with null start key");
6708       }
6709       // A's start key is null but B's isn't. Assume A comes before B
6710     } else if ((srcB.getRegionInfo().getStartKey() == null) ||
6711       (Bytes.compareTo(srcA.getRegionInfo().getStartKey(),
6712         srcB.getRegionInfo().getStartKey()) > 0)) {
6713       a = srcB;
6714       b = srcA;
6715     }
6716 
6717     if (!(Bytes.compareTo(a.getRegionInfo().getEndKey(),
6718         b.getRegionInfo().getStartKey()) == 0)) {
6719       throw new IOException("Cannot merge non-adjacent regions");
6720     }
6721     return merge(a, b);
6722   }
6723 
6724   /**
6725    * Merge two regions whether they are adjacent or not.
6726    *
6727    * @param a region a
6728    * @param b region b
6729    * @return new merged region
6730    * @throws IOException
6731    */
6732   public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
6733     if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
6734       throw new IOException("Regions do not belong to the same table");
6735     }
6736 
6737     FileSystem fs = a.getRegionFileSystem().getFileSystem();
6738     // Make sure each region's cache is empty
6739     a.flush(true);
6740     b.flush(true);
6741 
6742     // Compact each region so we only have one store file per family
6743     a.compact(true);
6744     if (LOG.isDebugEnabled()) {
6745       LOG.debug("Files for region: " + a);
6746       a.getRegionFileSystem().logFileSystemState(LOG);
6747     }
6748     b.compact(true);
6749     if (LOG.isDebugEnabled()) {
6750       LOG.debug("Files for region: " + b);
6751       b.getRegionFileSystem().logFileSystemState(LOG);
6752     }
6753 
6754     RegionMergeTransactionImpl rmt = new RegionMergeTransactionImpl(a, b, true);
6755     if (!rmt.prepare(null)) {
6756       throw new IOException("Unable to merge regions " + a + " and " + b);
6757     }
6758     HRegionInfo mergedRegionInfo = rmt.getMergedRegionInfo();
6759     LOG.info("starting merge of regions: " + a + " and " + b
6760         + " into new region " + mergedRegionInfo.getRegionNameAsString()
6761         + " with start key <"
6762         + Bytes.toStringBinary(mergedRegionInfo.getStartKey())
6763         + "> and end key <"
6764         + Bytes.toStringBinary(mergedRegionInfo.getEndKey()) + ">");
6765     HRegion dstRegion;
6766     try {
6767       dstRegion = (HRegion)rmt.execute(null, null);
6768     } catch (IOException ioe) {
6769       rmt.rollback(null, null);
6770       throw new IOException("Failed merging region " + a + " and " + b
6771           + ", and successfully rolled back");
6772     }
6773     dstRegion.compact(true);
6774 
6775     if (LOG.isDebugEnabled()) {
6776       LOG.debug("Files for new region");
6777       dstRegion.getRegionFileSystem().logFileSystemState(LOG);
6778     }
6779 
6780     if (dstRegion.getRegionFileSystem().hasReferences(dstRegion.getTableDesc())) {
6781       throw new IOException("Merged region " + dstRegion
6782           + " still has references after the compaction, is compaction canceled?");
6783     }
6784 
6785     // Archiving the 'A' region
6786     HFileArchiver.archiveRegion(a.getBaseConf(), fs, a.getRegionInfo());
6787     // Archiving the 'B' region
6788     HFileArchiver.archiveRegion(b.getBaseConf(), fs, b.getRegionInfo());
6789 
6790     LOG.info("merge completed. New region is " + dstRegion);
6791     return dstRegion;
6792   }
6793 
6794   @Override
6795   public Result get(final Get get) throws IOException {
6796     checkRow(get.getRow(), "Get");
6797     // Verify families are all valid
6798     if (get.hasFamilies()) {
6799       for (byte [] family: get.familySet()) {
6800         checkFamily(family);
6801       }
6802     } else { // Adding all families to scanner
6803       for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
6804         get.addFamily(family);
6805       }
6806     }
6807     List<Cell> results = get(get, true);
6808     boolean stale = this.getRegionInfo().getReplicaId() != 0;
6809     return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
6810   }
6811 
6812   @Override
6813   public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
6814 
6815     List<Cell> results = new ArrayList<Cell>();
6816 
6817     // pre-get CP hook
6818     if (withCoprocessor && (coprocessorHost != null)) {
6819        if (coprocessorHost.preGet(get, results)) {
6820          return results;
6821        }
6822     }
6823 
6824     Scan scan = new Scan(get);
6825 
6826     RegionScanner scanner = null;
6827     try {
6828       scanner = getScanner(scan);
6829       scanner.next(results);
6830     } finally {
6831       if (scanner != null)
6832         scanner.close();
6833     }
6834 
6835     // post-get CP hook
6836     if (withCoprocessor && (coprocessorHost != null)) {
6837       coprocessorHost.postGet(get, results);
6838     }
6839 
6840     // do after lock
6841     if (this.metricsRegion != null) {
6842       long totalSize = 0L;
6843       for (Cell cell : results) {
6844         totalSize += CellUtil.estimatedSerializedSizeOf(cell);
6845       }
6846       this.metricsRegion.updateGet(totalSize);
6847     }
6848 
6849     return results;
6850   }
6851 
6852   @Override
6853   public void mutateRow(RowMutations rm) throws IOException {
6854     // Don't need nonces here - RowMutations only supports puts and deletes
6855     mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
6856   }
6857 
6858   /**
6859    * Perform atomic mutations within the region w/o nonces.
6860    * See {@link #mutateRowsWithLocks(Collection, Collection, long, long)}
6861    */
6862   public void mutateRowsWithLocks(Collection<Mutation> mutations,
6863       Collection<byte[]> rowsToLock) throws IOException {
6864     mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
6865   }
6866 
6867   /**
6868    * Perform atomic mutations within the region.
6869    * @param mutations The list of mutations to perform.
6870    * <code>mutations</code> can contain operations for multiple rows.
6871    * Caller has to ensure that all rows are contained in this region.
6872    * @param rowsToLock Rows to lock
6873    * @param nonceGroup Optional nonce group of the operation (client Id)
6874    * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
6875    * If multiple rows are locked care should be taken that
6876    * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
6877    * @throws IOException
6878    */
6879   @Override
6880   public void mutateRowsWithLocks(Collection<Mutation> mutations,
6881       Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
6882     MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
6883     processRowsWithLocks(proc, -1, nonceGroup, nonce);
6884   }
6885 
6886   /**
6887    * @return the current load statistics for the the region
6888    */
6889   public ClientProtos.RegionLoadStats getRegionStats() {
6890     if (!regionStatsEnabled) {
6891       return null;
6892     }
6893     ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
6894     stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreSize.get() * 100) / this
6895         .memstoreFlushSize)));
6896     stats.setHeapOccupancy((int)rsServices.getHeapMemoryManager().getHeapOccupancyPercent()*100);
6897     stats.setCompactionPressure((int)rsServices.getCompactionPressure()*100 > 100 ? 100 :
6898                 (int)rsServices.getCompactionPressure()*100);
6899     return stats.build();
6900   }
6901 
6902   @Override
6903   public void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException {
6904     processRowsWithLocks(processor, rowProcessorTimeout, HConstants.NO_NONCE,
6905       HConstants.NO_NONCE);
6906   }
6907 
6908   @Override
6909   public void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce)
6910       throws IOException {
6911     processRowsWithLocks(processor, rowProcessorTimeout, nonceGroup, nonce);
6912   }
6913 
6914   @Override
6915   public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout,
6916       long nonceGroup, long nonce) throws IOException {
6917 
6918     for (byte[] row : processor.getRowsToLock()) {
6919       checkRow(row, "processRowsWithLocks");
6920     }
6921     if (!processor.readOnly()) {
6922       checkReadOnly();
6923     }
6924     checkResources();
6925 
6926     startRegionOperation();
6927     WALEdit walEdit = new WALEdit();
6928 
6929     // 1. Run pre-process hook
6930     try {
6931       processor.preProcess(this, walEdit);
6932     } catch (IOException e) {
6933       closeRegionOperation();
6934       throw e;
6935     }
6936     // Short circuit the read only case
6937     if (processor.readOnly()) {
6938       try {
6939         long now = EnvironmentEdgeManager.currentTime();
6940         doProcessRowWithTimeout(
6941             processor, now, this, null, null, timeout);
6942         processor.postProcess(this, walEdit, true);
6943       } finally {
6944         closeRegionOperation();
6945       }
6946       return;
6947     }
6948 
6949     MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
6950     boolean locked;
6951     boolean walSyncSuccessful = false;
6952     List<RowLock> acquiredRowLocks;
6953     long addedSize = 0;
6954     List<Mutation> mutations = new ArrayList<Mutation>();
6955     Collection<byte[]> rowsToLock = processor.getRowsToLock();
6956     long mvccNum = 0;
6957     WALKey walKey = null;
6958     try {
6959       // 2. Acquire the row lock(s)
6960       acquiredRowLocks = new ArrayList<RowLock>(rowsToLock.size());
6961       for (byte[] row : rowsToLock) {
6962         // Attempt to lock all involved rows, throw if any lock times out
6963         // use a writer lock for mixed reads and writes
6964         acquiredRowLocks.add(getRowLock(row));
6965       }
6966       // 3. Region lock
6967       lock(this.updatesLock.readLock(), acquiredRowLocks.size() == 0 ? 1 : acquiredRowLocks.size());
6968       locked = true;
6969 
6970       long now = EnvironmentEdgeManager.currentTime();
6971       try {
6972         // 4. Let the processor scan the rows, generate mutations and add
6973         //    waledits
6974         doProcessRowWithTimeout(
6975             processor, now, this, mutations, walEdit, timeout);
6976 
6977         if (!mutations.isEmpty()) {
6978 
6979           // 5. Call the preBatchMutate hook
6980           processor.preBatchMutate(this, walEdit);
6981 
6982           long txid = 0;
6983           // 6. Append no sync
6984           if (!walEdit.isEmpty()) {
6985             // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
6986             walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
6987               this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
6988               processor.getClusterIds(), nonceGroup, nonce, mvcc);
6989             txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(),
6990                 walKey, walEdit, false);
6991           }
6992           if(walKey == null){
6993             // since we use wal sequence Id as mvcc, for SKIP_WAL changes we need a "faked" WALEdit
6994             // to get a sequence id assigned which is done by FSWALEntry#stampRegionSequenceId
6995             walKey = this.appendEmptyEdit(this.wal);
6996           }
6997 
6998           // 7. Start mvcc transaction
6999           writeEntry = walKey.getWriteEntry();
7000           mvccNum = walKey.getSequenceId();
7001 
7002 
7003 
7004           // 8. Apply to memstore
7005           for (Mutation m : mutations) {
7006             // Handle any tag based cell features
7007             rewriteCellTags(m.getFamilyCellMap(), m);
7008 
7009             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7010               Cell cell = cellScanner.current();
7011               CellUtil.setSequenceId(cell, mvccNum);
7012               Store store = getStore(cell);
7013               if (store == null) {
7014                 checkFamily(CellUtil.cloneFamily(cell));
7015                 // unreachable
7016               }
7017               addedSize += store.add(cell);
7018             }
7019           }
7020 
7021           // 9. Release region lock
7022           if (locked) {
7023             this.updatesLock.readLock().unlock();
7024             locked = false;
7025           }
7026 
7027           // 10. Release row lock(s)
7028           releaseRowLocks(acquiredRowLocks);
7029 
7030           // 11. Sync edit log
7031           if (txid != 0) {
7032             syncOrDefer(txid, getEffectiveDurability(processor.useDurability()));
7033           }
7034           walSyncSuccessful = true;
7035           // 12. call postBatchMutate hook
7036           processor.postBatchMutate(this);
7037         }
7038       } finally {
7039         // TODO: Make this method look like all other methods that are doing append/sync and
7040         // memstore rollback such as append and doMiniBatchMutation. Currently it is a little
7041         // different. Make them all share same code!
7042         if (!mutations.isEmpty() && !walSyncSuccessful) {
7043           LOG.warn("Wal sync failed. Roll back " + mutations.size() +
7044               " memstore keyvalues for row(s):" + StringUtils.byteToHexString(
7045               processor.getRowsToLock().iterator().next()) + "...");
7046           for (Mutation m : mutations) {
7047             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7048               Cell cell = cellScanner.current();
7049               getStore(cell).rollback(cell);
7050             }
7051           }
7052           if (writeEntry != null) {
7053             mvcc.complete(writeEntry);
7054             writeEntry = null;
7055           }
7056         }
7057         // 13. Roll mvcc forward
7058         if (writeEntry != null) {
7059           mvcc.completeAndWait(writeEntry);
7060         }
7061         if (locked) {
7062           this.updatesLock.readLock().unlock();
7063         }
7064         // release locks if some were acquired but another timed out
7065         releaseRowLocks(acquiredRowLocks);
7066       }
7067 
7068       // 14. Run post-process hook
7069       processor.postProcess(this, walEdit, walSyncSuccessful);
7070 
7071     } finally {
7072       closeRegionOperation();
7073       if (!mutations.isEmpty() &&
7074           isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize))) {
7075         requestFlush();
7076       }
7077     }
7078   }
7079 
7080   private void doProcessRowWithTimeout(final RowProcessor<?,?> processor,
7081                                        final long now,
7082                                        final HRegion region,
7083                                        final List<Mutation> mutations,
7084                                        final WALEdit walEdit,
7085                                        final long timeout) throws IOException {
7086     // Short circuit the no time bound case.
7087     if (timeout < 0) {
7088       try {
7089         processor.process(now, region, mutations, walEdit);
7090       } catch (IOException e) {
7091         LOG.warn("RowProcessor:" + processor.getClass().getName() +
7092             " throws Exception on row(s):" +
7093             Bytes.toStringBinary(
7094               processor.getRowsToLock().iterator().next()) + "...", e);
7095         throw e;
7096       }
7097       return;
7098     }
7099 
7100     // Case with time bound
7101     FutureTask<Void> task =
7102       new FutureTask<Void>(new Callable<Void>() {
7103         @Override
7104         public Void call() throws IOException {
7105           try {
7106             processor.process(now, region, mutations, walEdit);
7107             return null;
7108           } catch (IOException e) {
7109             LOG.warn("RowProcessor:" + processor.getClass().getName() +
7110                 " throws Exception on row(s):" +
7111                 Bytes.toStringBinary(
7112                     processor.getRowsToLock().iterator().next()) + "...", e);
7113             throw e;
7114           }
7115         }
7116       });
7117     rowProcessorExecutor.execute(task);
7118     try {
7119       task.get(timeout, TimeUnit.MILLISECONDS);
7120     } catch (TimeoutException te) {
7121       LOG.error("RowProcessor timeout:" + timeout + " ms on row(s):" +
7122           Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) +
7123           "...");
7124       throw new IOException(te);
7125     } catch (Exception e) {
7126       throw new IOException(e);
7127     }
7128   }
7129 
7130   /**
7131    * @param cell
7132    * @param tags
7133    * @return The passed-in List<Tag> but with the tags from <code>cell</code> added.
7134    */
7135   private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
7136     if (cell.getTagsLength() <= 0) return tags;
7137     List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags;
7138     Iterator<Tag> i =
7139         CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
7140     while (i.hasNext()) newTags.add(i.next());
7141     return newTags;
7142   }
7143 
7144   /**
7145    * Run a Get against passed in <code>store</code> on passed <code>row</code>, etc.
7146    * @param store
7147    * @param row
7148    * @param family
7149    * @param tr
7150    * @return Get result.
7151    * @throws IOException
7152    */
7153   private List<Cell> doGet(final Store store, final byte [] row,
7154       final Map.Entry<byte[], List<Cell>> family, final TimeRange tr)
7155   throws IOException {
7156     // Sort the cells so that they match the order that they
7157     // appear in the Get results. Otherwise, we won't be able to
7158     // find the existing values if the cells are not specified
7159     // in order by the client since cells are in an array list.
7160     Collections.sort(family.getValue(), store.getComparator());
7161     // Get previous values for all columns in this family
7162     Get get = new Get(row);
7163     for (Cell cell : family.getValue()) {
7164       get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell));
7165     }
7166     if (tr != null) get.setTimeRange(tr.getMin(), tr.getMax());
7167     return get(get, false);
7168   }
7169 
7170   public Result append(Append append) throws IOException {
7171     return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
7172   }
7173 
7174   // TODO: There's a lot of boiler plate code identical to increment.
7175   // We should refactor append and increment as local get-mutate-put
7176   // transactions, so all stores only go through one code path for puts.
7177 
7178   // dropMemstoreContentsForSeqId() would acquire write lock of updatesLock
7179   // We perform this operation outside of the read lock of updatesLock to avoid dead lock
7180   // See HBASE-16304
7181   @SuppressWarnings("unchecked")
7182   private void dropMemstoreContents() throws IOException {
7183     long totalFreedSize = 0;
7184     while (!storeSeqIds.isEmpty()) {
7185       Map<Store, Long> map = null;
7186       synchronized (storeSeqIds) {
7187         if (storeSeqIds.isEmpty()) break;
7188         map = storeSeqIds.remove(storeSeqIds.size()-1);
7189       }
7190       for (Map.Entry<Store, Long> entry : map.entrySet()) {
7191         // Drop the memstore contents if they are now smaller than the latest seen flushed file
7192         totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey());
7193       }
7194     }
7195     if (totalFreedSize > 0) {
7196       LOG.debug("Freed " + totalFreedSize + " bytes from memstore");
7197     }
7198   }
7199 
7200   @Override
7201   public Result append(Append mutate, long nonceGroup, long nonce) throws IOException {
7202     Operation op = Operation.APPEND;
7203     byte[] row = mutate.getRow();
7204     checkRow(row, op.toString());
7205     boolean flush = false;
7206     Durability durability = getEffectiveDurability(mutate.getDurability());
7207     boolean writeToWAL = durability != Durability.SKIP_WAL;
7208     WALEdit walEdits = null;
7209     List<Cell> allKVs = new ArrayList<Cell>(mutate.size());
7210     Map<Store, List<Cell>> tempMemstore = new HashMap<Store, List<Cell>>();
7211     long size = 0;
7212     long txid = 0;
7213     checkReadOnly();
7214     checkResources();
7215     // Lock row
7216     startRegionOperation(op);
7217     this.writeRequestsCount.increment();
7218     RowLock rowLock = null;
7219     WALKey walKey = null;
7220     boolean doRollBackMemstore = false;
7221     try {
7222       rowLock = getRowLock(row);
7223       assert rowLock != null;
7224       try {
7225         lock(this.updatesLock.readLock());
7226         try {
7227           // Wait for all prior MVCC transactions to finish - while we hold the row lock
7228           // (so that we are guaranteed to see the latest state when we do our Get)
7229           mvcc.await();
7230           if (this.coprocessorHost != null) {
7231             Result r = this.coprocessorHost.preAppendAfterRowLock(mutate);
7232             if (r!= null) {
7233               return r;
7234             }
7235           }
7236           long now = EnvironmentEdgeManager.currentTime();
7237           // Process each family
7238           for (Map.Entry<byte[], List<Cell>> family : mutate.getFamilyCellMap().entrySet()) {
7239             Store store = stores.get(family.getKey());
7240             List<Cell> kvs = new ArrayList<Cell>(family.getValue().size());
7241 
7242             List<Cell> results = doGet(store, row, family, null);
7243 
7244             // Iterate the input columns and update existing values if they were
7245             // found, otherwise add new column initialized to the append value
7246 
7247             // Avoid as much copying as possible. We may need to rewrite and
7248             // consolidate tags. Bytes are only copied once.
7249             // Would be nice if KeyValue had scatter/gather logic
7250             int idx = 0;
7251             for (Cell cell : family.getValue()) {
7252               Cell newCell;
7253               Cell oldCell = null;
7254               if (idx < results.size()
7255                   && CellUtil.matchingQualifier(results.get(idx), cell)) {
7256                 oldCell = results.get(idx);
7257                 long ts = Math.max(now, oldCell.getTimestamp());
7258 
7259                 // Process cell tags
7260                 // Make a union of the set of tags in the old and new KVs
7261                 List<Tag> tags = Tag.carryForwardTags(null, oldCell);
7262                 tags = Tag.carryForwardTags(tags, cell);
7263                 tags = carryForwardTTLTag(tags, mutate);
7264 
7265                 newCell = getNewCell(row, ts, cell, oldCell, Tag.fromList(tags));
7266 
7267                 idx++;
7268               } else {
7269                 // Append's KeyValue.Type==Put and ts==HConstants.LATEST_TIMESTAMP
7270                 CellUtil.updateLatestStamp(cell, now);
7271 
7272                 // Cell TTL handling
7273                 newCell = getNewCell(mutate, cell);
7274               }
7275 
7276               // Give coprocessors a chance to update the new cell
7277               if (coprocessorHost != null) {
7278                 newCell = coprocessorHost.postMutationBeforeWAL(RegionObserver.MutationType.APPEND,
7279                     mutate, oldCell, newCell);
7280               }
7281               kvs.add(newCell);
7282 
7283               // Append update to WAL
7284               if (writeToWAL) {
7285                 if (walEdits == null) {
7286                   walEdits = new WALEdit();
7287                 }
7288                 walEdits.add(newCell);
7289               }
7290             }
7291 
7292             //store the kvs to the temporary memstore before writing WAL
7293             tempMemstore.put(store, kvs);
7294           }
7295 
7296           // Actually write to WAL now
7297           if (walEdits != null && !walEdits.isEmpty()) {
7298             if (writeToWAL) {
7299               // Using default cluster id, as this can only happen in the originating
7300               // cluster. A slave cluster receives the final value (not the delta)
7301               // as a Put.
7302               // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
7303               walKey = new HLogKey(
7304                   getRegionInfo().getEncodedNameAsBytes(),
7305                   this.htableDescriptor.getTableName(),
7306                   WALKey.NO_SEQUENCE_ID,
7307                   nonceGroup,
7308                   nonce,
7309                   mvcc);
7310               txid =
7311                 this.wal.append(this.htableDescriptor, getRegionInfo(), walKey, walEdits, true);
7312             } else {
7313               recordMutationWithoutWal(mutate.getFamilyCellMap());
7314             }
7315           }
7316           if (walKey == null) {
7317             // Append a faked WALEdit in order for SKIP_WAL updates to get mvcc assigned
7318             walKey = this.appendEmptyEdit(this.wal);
7319           }
7320           // Do a get on the write entry... this will block until sequenceid is assigned... w/o it,
7321           // TestAtomicOperation fails.
7322           walKey.getWriteEntry();
7323 
7324           // Actually write to Memstore now
7325           if (!tempMemstore.isEmpty()) {
7326             for (Map.Entry<Store, List<Cell>> entry : tempMemstore.entrySet()) {
7327               Store store = entry.getKey();
7328               if (store.getFamily().getMaxVersions() == 1) {
7329                 // upsert if VERSIONS for this CF == 1
7330                 // Is this right? It immediately becomes visible? St.Ack 20150907
7331                 size += store.upsert(entry.getValue(), getSmallestReadPoint());
7332               } else {
7333                 // otherwise keep older versions around
7334                 for (Cell cell: entry.getValue()) {
7335                   // This stamping of sequenceid seems redundant; it is happening down in
7336                   // FSHLog when we consume edits off the ring buffer.
7337                   CellUtil.setSequenceId(cell, walKey.getWriteEntry().getWriteNumber());
7338                   size += store.add(cell);
7339                   doRollBackMemstore = true;
7340                 }
7341               }
7342               // We add to all KVs here whereas when doing increment, we do it
7343               // earlier... why?
7344               allKVs.addAll(entry.getValue());
7345             }
7346 
7347             size = this.addAndGetGlobalMemstoreSize(size);
7348             flush = isFlushSize(size);
7349           }
7350         } finally {
7351           this.updatesLock.readLock().unlock();
7352           // For increment/append, a region scanner for doing a get operation could throw 
7353           // FileNotFoundException. So we call dropMemstoreContents() in finally block
7354           // after releasing read lock
7355           dropMemstoreContents();
7356         }
7357 
7358       } finally {
7359         rowLock.release();
7360         rowLock = null;
7361       }
7362       // sync the transaction log outside the rowlock
7363       if(txid != 0){
7364         syncOrDefer(txid, durability);
7365       }
7366       doRollBackMemstore = false;
7367     } finally {
7368       if (rowLock != null) {
7369         rowLock.release();
7370       }
7371       // if the wal sync was unsuccessful, remove keys from memstore
7372       WriteEntry we = walKey != null? walKey.getWriteEntry(): null;
7373       if (doRollBackMemstore) {
7374         rollbackMemstore(allKVs);
7375         if (we != null) mvcc.complete(we);
7376       } else if (we != null) {
7377         mvcc.completeAndWait(we);
7378       }
7379 
7380       closeRegionOperation(op);
7381     }
7382 
7383     if (this.metricsRegion != null) {
7384       this.metricsRegion.updateAppend();
7385     }
7386 
7387     if (flush) {
7388       // Request a cache flush. Do it outside update lock.
7389       requestFlush();
7390     }
7391 
7392     return mutate.isReturnResults() ? Result.create(allKVs) : null;
7393   }
7394 
7395   private static Cell getNewCell(final byte [] row, final long ts, final Cell cell,
7396       final Cell oldCell, final byte [] tagBytes) {
7397     // allocate an empty cell once
7398     Cell newCell = new KeyValue(row.length, cell.getFamilyLength(),
7399         cell.getQualifierLength(), ts, KeyValue.Type.Put,
7400         oldCell.getValueLength() + cell.getValueLength(),
7401         tagBytes == null? 0: tagBytes.length);
7402     // copy in row, family, and qualifier
7403     System.arraycopy(cell.getRowArray(), cell.getRowOffset(),
7404       newCell.getRowArray(), newCell.getRowOffset(), cell.getRowLength());
7405     System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(),
7406       newCell.getFamilyArray(), newCell.getFamilyOffset(),
7407       cell.getFamilyLength());
7408     System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(),
7409       newCell.getQualifierArray(), newCell.getQualifierOffset(),
7410       cell.getQualifierLength());
7411     // copy in the value
7412     System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(),
7413       newCell.getValueArray(), newCell.getValueOffset(),
7414       oldCell.getValueLength());
7415     System.arraycopy(cell.getValueArray(), cell.getValueOffset(),
7416       newCell.getValueArray(),
7417       newCell.getValueOffset() + oldCell.getValueLength(),
7418       cell.getValueLength());
7419     // Copy in tag data
7420     if (tagBytes != null) {
7421       System.arraycopy(tagBytes, 0, newCell.getTagsArray(), newCell.getTagsOffset(),
7422         tagBytes.length);
7423     }
7424     return newCell;
7425   }
7426 
7427   private static Cell getNewCell(final Mutation mutate, final Cell cell) {
7428     Cell newCell = null;
7429     if (mutate.getTTL() != Long.MAX_VALUE) {
7430       // Add the new TTL tag
7431       newCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
7432           cell.getRowLength(),
7433         cell.getFamilyArray(), cell.getFamilyOffset(),
7434           cell.getFamilyLength(),
7435         cell.getQualifierArray(), cell.getQualifierOffset(),
7436           cell.getQualifierLength(),
7437         cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
7438         cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
7439         carryForwardTTLTag(mutate));
7440     } else {
7441       newCell = cell;
7442     }
7443     return newCell;
7444   }
7445 
7446   public Result increment(Increment increment) throws IOException {
7447     return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
7448   }
7449 
7450   // TODO: There's a lot of boiler plate code identical to append.
7451   // We should refactor append and increment as local get-mutate-put
7452   // transactions, so all stores only go through one code path for puts.
7453 
7454   // They are subtley different in quiet a few ways. This came out only
7455   // after study. I am not sure that many of the differences are intentional.
7456   // TODO: St.Ack 20150907
7457 
7458   @Override
7459   public Result increment(Increment mutation, long nonceGroup, long nonce)
7460   throws IOException {
7461     Operation op = Operation.INCREMENT;
7462     checkReadOnly();
7463     checkResources();
7464     checkRow(mutation.getRow(), op.toString());
7465     checkFamilies(mutation.getFamilyCellMap().keySet());
7466     startRegionOperation(op);
7467     this.writeRequestsCount.increment();
7468     try {
7469       // Which Increment is it? Narrow increment-only consistency or slow (default) and general
7470       // row-wide consistency.
7471 
7472       // So, difference between fastAndNarrowConsistencyIncrement and slowButConsistentIncrement is
7473       // that the former holds the row lock until the sync completes; this allows us to reason that
7474       // there are no other writers afoot when we read the current increment value. The row lock
7475       // means that we do not need to wait on mvcc reads to catch up to writes before we proceed
7476       // with the read, the root of the slowdown seen in HBASE-14460. The fast-path also does not
7477       // wait on mvcc to complete before returning to the client. We also reorder the write so that
7478       // the update of memstore happens AFTER sync returns; i.e. the write pipeline does less
7479       // zigzagging now.
7480       //
7481       // See the comment on INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY
7482       // for the constraints that apply when you take this code path; it is correct but only if
7483       // Increments are used mutating an Increment Cell; mixing concurrent Put+Delete and Increment
7484       // will yield indeterminate results.
7485       return doIncrement(mutation, nonceGroup, nonce);
7486     } finally {
7487       if (this.metricsRegion != null) this.metricsRegion.updateIncrement();
7488       closeRegionOperation(op);
7489     }
7490   }
7491 
7492   private Result doIncrement(Increment increment, long nonceGroup, long nonce) throws IOException {
7493     RowLock rowLock = null;
7494     WALKey walKey = null;
7495     boolean doRollBackMemstore = false;
7496     long accumulatedResultSize = 0;
7497     List<Cell> allKVs = new ArrayList<Cell>(increment.size());
7498     List<Cell> memstoreCells = new ArrayList<Cell>();
7499     Durability effectiveDurability = getEffectiveDurability(increment.getDurability());
7500     try {
7501       rowLock = getRowLock(increment.getRow());
7502       long txid = 0;
7503       try {
7504         lock(this.updatesLock.readLock());
7505         try {
7506           // Wait for all prior MVCC transactions to finish - while we hold the row lock
7507           // (so that we are guaranteed to see the latest increment)
7508           this.mvcc.await();
7509           if (this.coprocessorHost != null) {
7510             Result r = this.coprocessorHost.preIncrementAfterRowLock(increment);
7511             if (r != null) return r;
7512           }
7513           long now = EnvironmentEdgeManager.currentTime();
7514           final boolean writeToWAL = effectiveDurability != Durability.SKIP_WAL;
7515           WALEdit walEdits = null;
7516           // Process increments a Store/family at a time.
7517           // Accumulate edits for memstore to add later after we've added to WAL.
7518           Map<Store, List<Cell>> forMemStore = new HashMap<Store, List<Cell>>();
7519           for (Map.Entry<byte [], List<Cell>> entry: increment.getFamilyCellMap().entrySet()) {
7520             byte [] columnFamilyName = entry.getKey();
7521             List<Cell> increments = entry.getValue();
7522             Store store = this.stores.get(columnFamilyName);
7523             // Do increment for this store; be sure to 'sort' the increments first so increments
7524             // match order in which we get back current Cells when we get.
7525             List<Cell> results = applyIncrementsToColumnFamily(increment, columnFamilyName,
7526                 sort(increments, store.getComparator()), now,
7527                 MultiVersionConcurrencyControl.NO_WRITE_NUMBER, allKVs, null);
7528             if (!results.isEmpty()) {
7529               forMemStore.put(store, results);
7530               // Prepare WAL updates
7531               if (writeToWAL) {
7532                 if (walEdits == null) walEdits = new WALEdit();
7533                 walEdits.getCells().addAll(results);
7534               }
7535             }
7536           }
7537           // Actually write to WAL now. If walEdits is non-empty, we write the WAL.
7538           if (walEdits != null && !walEdits.isEmpty()) {
7539             // Using default cluster id, as this can only happen in the originating cluster.
7540             // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey
7541             // here instead of WALKey directly to support legacy coprocessors.
7542             walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
7543               this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce,
7544               getMVCC());
7545             txid =
7546               this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdits, true);
7547           } else {
7548             // Append a faked WALEdit in order for SKIP_WAL updates to get mvccNum assigned
7549             walKey = this.appendEmptyEdit(this.wal);
7550           }
7551           // Get WriteEntry. Will wait on assign of the sequence id.
7552           walKey.getWriteEntry();
7553 
7554           // Now write to memstore, a family at a time.
7555           for (Map.Entry<Store, List<Cell>> entry: forMemStore.entrySet()) {
7556             Store store = entry.getKey();
7557             List<Cell> results = entry.getValue();
7558             if (store.getFamily().getMaxVersions() == 1) {
7559               // Upsert if VERSIONS for this CF == 1
7560               accumulatedResultSize += store.upsert(results, getSmallestReadPoint());
7561               // TODO: St.Ack 20151222 Why no rollback in this case?
7562             } else {
7563               // Otherwise keep older versions around
7564               for (Cell cell: results) {
7565                 // Why we need this?
7566                 CellUtil.setSequenceId(cell, walKey.getWriteEntry().getWriteNumber());
7567                 accumulatedResultSize += store.add(cell);
7568                 doRollBackMemstore = true;
7569               }
7570             }
7571           }
7572         } finally {
7573           this.updatesLock.readLock().unlock();
7574           // For increment/append, a region scanner for doing a get operation could throw 
7575           // FileNotFoundException. So we call dropMemstoreContents() in finally block
7576           // after releasing read lock
7577           dropMemstoreContents();
7578         }
7579       } finally {
7580         rowLock.release();
7581         rowLock = null;
7582       }
7583       // sync the transaction log outside the rowlock
7584       if(txid != 0) {
7585         syncOrDefer(txid, durability);
7586       }
7587       doRollBackMemstore = false;
7588     } finally {
7589       if (rowLock != null) {
7590         rowLock.release();
7591       }
7592       // if the wal sync was unsuccessful, remove keys from memstore
7593       if (doRollBackMemstore) {
7594         rollbackMemstore(memstoreCells);
7595         if (walKey != null) mvcc.complete(walKey.getWriteEntry());
7596       } else {
7597         if (walKey != null) mvcc.completeAndWait(walKey.getWriteEntry());
7598       }
7599     }
7600 
7601     // Request a cache flush.  Do it outside update lock.
7602     if (isFlushSize(this.addAndGetGlobalMemstoreSize(accumulatedResultSize))) requestFlush();
7603     return increment.isReturnResults() ? Result.create(allKVs) : null;
7604   }
7605 
7606   /**
7607    * @return Sorted list of <code>cells</code> using <code>comparator</code>
7608    */
7609   private static List<Cell> sort(List<Cell> cells, final Comparator<Cell> comparator) {
7610     Collections.sort(cells, comparator);
7611     return cells;
7612   }
7613 
7614   /**
7615    * Apply increments to a column family.
7616    * @param sortedIncrements The passed in increments to apply MUST be sorted so that they match
7617    * the order that they appear in the Get results (get results will be sorted on return).
7618    * Otherwise, we won't be able to find the existing values if the cells are not specified in
7619    * order by the client since cells are in an array list.
7620    * @islation Isolation level to use when running the 'get'. Pass null for default.
7621    * @return Resulting increments after <code>sortedIncrements</code> have been applied to current
7622    * values (if any -- else passed increment is the final result).
7623    * @throws IOException
7624    */
7625   private List<Cell> applyIncrementsToColumnFamily(Increment increment, byte[] columnFamilyName,
7626       List<Cell> sortedIncrements, long now, long mvccNum, List<Cell> allKVs,
7627       final IsolationLevel isolation)
7628   throws IOException {
7629     List<Cell> results = new ArrayList<Cell>(sortedIncrements.size());
7630     byte [] row = increment.getRow();
7631     // Get previous values for all columns in this family
7632     List<Cell> currentValues =
7633         getIncrementCurrentValue(increment, columnFamilyName, sortedIncrements, isolation);
7634     // Iterate the input columns and update existing values if they were found, otherwise
7635     // add new column initialized to the increment amount
7636     int idx = 0;
7637     for (int i = 0; i < sortedIncrements.size(); i++) {
7638       Cell inc = sortedIncrements.get(i);
7639       long incrementAmount = getLongValue(inc);
7640       // If increment amount == 0, then don't write this Increment to the WAL.
7641       boolean writeBack = (incrementAmount != 0);
7642       // Carry forward any tags that might have been added by a coprocessor.
7643       List<Tag> tags = Tag.carryForwardTags(inc);
7644 
7645       Cell currentValue = null;
7646       long ts = now;
7647       if (idx < currentValues.size() && CellUtil.matchingQualifier(currentValues.get(idx), inc)) {
7648         currentValue = currentValues.get(idx);
7649         ts = Math.max(now, currentValue.getTimestamp());
7650         incrementAmount += getLongValue(currentValue);
7651         // Carry forward all tags
7652         tags = Tag.carryForwardTags(tags, currentValue);
7653         if (i < (sortedIncrements.size() - 1) &&
7654             !CellUtil.matchingQualifier(inc, sortedIncrements.get(i + 1))) idx++;
7655       }
7656 
7657       // Append new incremented KeyValue to list
7658       byte [] qualifier = CellUtil.cloneQualifier(inc);
7659       byte [] incrementAmountInBytes = Bytes.toBytes(incrementAmount);
7660       tags = carryForwardTTLTag(tags, increment);
7661 
7662       Cell newValue = new KeyValue(row, 0, row.length,
7663         columnFamilyName, 0, columnFamilyName.length,
7664         qualifier, 0, qualifier.length,
7665         ts, KeyValue.Type.Put,
7666         incrementAmountInBytes, 0, incrementAmountInBytes.length,
7667         tags);
7668 
7669       // Don't set an mvcc if none specified. The mvcc may be assigned later in case where we
7670       // write the memstore AFTER we sync our edit to the log.
7671       if (mvccNum != MultiVersionConcurrencyControl.NO_WRITE_NUMBER) {
7672         CellUtil.setSequenceId(newValue, mvccNum);
7673       }
7674 
7675       // Give coprocessors a chance to update the new cell
7676       if (coprocessorHost != null) {
7677         newValue = coprocessorHost.postMutationBeforeWAL(
7678             RegionObserver.MutationType.INCREMENT, increment, currentValue, newValue);
7679       }
7680       allKVs.add(newValue);
7681       if (writeBack) {
7682         results.add(newValue);
7683       }
7684     }
7685     return results;
7686   }
7687 
7688   /**
7689    * @return Get the long out of the passed in Cell
7690    * @throws DoNotRetryIOException
7691    */
7692   private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
7693     int len = cell.getValueLength();
7694     if (len != Bytes.SIZEOF_LONG) {
7695       // throw DoNotRetryIOException instead of IllegalArgumentException
7696       throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
7697     }
7698     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), len);
7699   }
7700 
7701   /**
7702    * Do a specific Get on passed <code>columnFamily</code> and column qualifiers
7703    * from <code>incrementCoordinates</code> only.
7704    * @param increment
7705    * @param columnFamily
7706    * @param incrementCoordinates
7707    * @return Return the Cells to Increment
7708    * @throws IOException
7709    */
7710   private List<Cell> getIncrementCurrentValue(final Increment increment, byte [] columnFamily,
7711       final List<Cell> increments, final IsolationLevel isolation)
7712   throws IOException {
7713     Get get = new Get(increment.getRow());
7714     if (isolation != null) get.setIsolationLevel(isolation);
7715     for (Cell cell: increments) {
7716       get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
7717     }
7718     TimeRange tr = increment.getTimeRange();
7719     if (tr != null) {
7720       get.setTimeRange(tr.getMin(), tr.getMax());
7721     }
7722     return get(get, false);
7723   }
7724 
7725   private static List<Tag> carryForwardTTLTag(final Mutation mutation) {
7726     return carryForwardTTLTag(null, mutation);
7727   }
7728 
7729   /**
7730    * @return Carry forward the TTL tag if the increment is carrying one
7731    */
7732   private static List<Tag> carryForwardTTLTag(final List<Tag> tagsOrNull,
7733       final Mutation mutation) {
7734     long ttl = mutation.getTTL();
7735