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