View Javadoc

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