View Javadoc

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