View Javadoc

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