View Javadoc

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