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