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