View Javadoc

1   /*
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver;
21  
22  import java.io.EOFException;
23  import java.io.FileNotFoundException;
24  import java.io.IOException;
25  import java.io.InterruptedIOException;
26  import java.io.UnsupportedEncodingException;
27  import java.lang.reflect.Constructor;
28  import java.lang.reflect.InvocationTargetException;
29  import java.lang.reflect.Method;
30  import java.text.ParseException;
31  import java.util.AbstractList;
32  import java.util.ArrayList;
33  import java.util.Arrays;
34  import java.util.Collection;
35  import java.util.Collections;
36  import java.util.HashMap;
37  import java.util.List;
38  import java.util.Map;
39  import java.util.NavigableMap;
40  import java.util.NavigableSet;
41  import java.util.Random;
42  import java.util.Set;
43  import java.util.TreeMap;
44  import java.util.UUID;
45  import java.util.concurrent.Callable;
46  import java.util.concurrent.CompletionService;
47  import java.util.concurrent.ConcurrentHashMap;
48  import java.util.concurrent.ConcurrentSkipListMap;
49  import java.util.concurrent.CountDownLatch;
50  import java.util.concurrent.ExecutionException;
51  import java.util.concurrent.ExecutorCompletionService;
52  import java.util.concurrent.Future;
53  import java.util.concurrent.ThreadFactory;
54  import java.util.concurrent.ThreadPoolExecutor;
55  import java.util.concurrent.TimeUnit;
56  import java.util.concurrent.atomic.AtomicBoolean;
57  import java.util.concurrent.atomic.AtomicInteger;
58  import java.util.concurrent.atomic.AtomicLong;
59  import java.util.concurrent.locks.Lock;
60  import java.util.concurrent.locks.ReentrantReadWriteLock;
61  
62  import org.apache.commons.logging.Log;
63  import org.apache.commons.logging.LogFactory;
64  import org.apache.hadoop.conf.Configuration;
65  import org.apache.hadoop.fs.FSDataOutputStream;
66  import org.apache.hadoop.fs.FSDataInputStream;
67  import org.apache.hadoop.fs.FileStatus;
68  import org.apache.hadoop.fs.FileSystem;
69  import org.apache.hadoop.fs.Path;
70  import org.apache.hadoop.fs.permission.FsPermission;
71  import org.apache.hadoop.hbase.DoNotRetryIOException;
72  import org.apache.hadoop.hbase.DroppedSnapshotException;
73  import org.apache.hadoop.hbase.HBaseConfiguration;
74  import org.apache.hadoop.hbase.HBaseFileSystem;
75  import org.apache.hadoop.hbase.HColumnDescriptor;
76  import org.apache.hadoop.hbase.HConstants;
77  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
78  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
79  import org.apache.hadoop.hbase.HRegionInfo;
80  import org.apache.hadoop.hbase.HTableDescriptor;
81  import org.apache.hadoop.hbase.KeyValue;
82  import org.apache.hadoop.hbase.NotServingRegionException;
83  import org.apache.hadoop.hbase.RegionTooBusyException;
84  import org.apache.hadoop.hbase.UnknownScannerException;
85  import org.apache.hadoop.hbase.backup.HFileArchiver;
86  import org.apache.hadoop.hbase.client.Append;
87  import org.apache.hadoop.hbase.client.Durability;
88  import org.apache.hadoop.hbase.client.RowMutations;
89  import org.apache.hadoop.hbase.client.Delete;
90  import org.apache.hadoop.hbase.client.Get;
91  import org.apache.hadoop.hbase.client.Increment;
92  import org.apache.hadoop.hbase.client.IsolationLevel;
93  import org.apache.hadoop.hbase.client.Mutation;
94  import org.apache.hadoop.hbase.client.Put;
95  import org.apache.hadoop.hbase.client.Result;
96  import org.apache.hadoop.hbase.client.Row;
97  import org.apache.hadoop.hbase.client.RowLock;
98  import org.apache.hadoop.hbase.client.Scan;
99  import org.apache.hadoop.hbase.client.coprocessor.Exec;
100 import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
101 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
102 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
103 import org.apache.hadoop.hbase.filter.Filter;
104 import org.apache.hadoop.hbase.filter.FilterBase;
105 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
106 import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
107 import org.apache.hadoop.hbase.io.HFileLink;
108 import org.apache.hadoop.hbase.io.HeapSize;
109 import org.apache.hadoop.hbase.io.Reference;
110 import org.apache.hadoop.hbase.io.TimeRange;
111 import org.apache.hadoop.hbase.io.hfile.BlockCache;
112 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
113 import org.apache.hadoop.hbase.io.hfile.HFile;
114 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
115 import org.apache.hadoop.hbase.ipc.HBaseRPC;
116 import org.apache.hadoop.hbase.ipc.HBaseServer;
117 import org.apache.hadoop.hbase.ipc.RpcCallContext;
118 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
119 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
120 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
121 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
122 import org.apache.hadoop.hbase.regionserver.metrics.OperationMetrics;
123 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
124 import org.apache.hadoop.hbase.regionserver.wal.HLog;
125 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
126 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
127 import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
128 import org.apache.hadoop.hbase.util.Bytes;
129 import org.apache.hadoop.hbase.util.CancelableProgressable;
130 import org.apache.hadoop.hbase.util.ClassSize;
131 import org.apache.hadoop.hbase.util.CompressionTest;
132 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
133 import org.apache.hadoop.hbase.util.FSUtils;
134 import org.apache.hadoop.hbase.util.HashedBytes;
135 import org.apache.hadoop.hbase.util.Pair;
136 import org.apache.hadoop.hbase.util.Threads;
137 import org.apache.hadoop.hbase.util.Writables;
138 import org.apache.hadoop.io.MultipleIOException;
139 import org.apache.hadoop.io.Writable;
140 import org.apache.hadoop.util.StringUtils;
141 import org.cliffc.high_scale_lib.Counter;
142 
143 import com.google.common.base.Preconditions;
144 import com.google.common.collect.ClassToInstanceMap;
145 import com.google.common.collect.ImmutableList;
146 import com.google.common.collect.Lists;
147 import com.google.common.collect.Maps;
148 import com.google.common.collect.Sets;
149 import com.google.common.collect.MutableClassToInstanceMap;
150 
151 /**
152  * HRegion stores data for a certain region of a table.  It stores all columns
153  * for each row. A given table consists of one or more HRegions.
154  *
155  * <p>We maintain multiple HStores for a single HRegion.
156  *
157  * <p>An Store is a set of rows with some column data; together,
158  * they make up all the data for the rows.
159  *
160  * <p>Each HRegion has a 'startKey' and 'endKey'.
161  * <p>The first is inclusive, the second is exclusive (except for
162  * the final region)  The endKey of region 0 is the same as
163  * startKey for region 1 (if it exists).  The startKey for the
164  * first region is null. The endKey for the final region is null.
165  *
166  * <p>Locking at the HRegion level serves only one purpose: preventing the
167  * region from being closed (and consequently split) while other operations
168  * are ongoing. Each row level operation obtains both a row lock and a region
169  * read lock for the duration of the operation. While a scanner is being
170  * constructed, getScanner holds a read lock. If the scanner is successfully
171  * constructed, it holds a read lock until it is closed. A close takes out a
172  * write lock and consequently will block for ongoing operations and will block
173  * new operations from starting while the close is in progress.
174  *
175  * <p>An HRegion is defined by its table and its key extent.
176  *
177  * <p>It consists of at least one Store.  The number of Stores should be
178  * configurable, so that data which is accessed together is stored in the same
179  * Store.  Right now, we approximate that by building a single Store for
180  * each column family.  (This config info will be communicated via the
181  * tabledesc.)
182  *
183  * <p>The HTableDescriptor contains metainfo about the HRegion's table.
184  * regionName is a unique identifier for this HRegion. (startKey, endKey]
185  * defines the keyspace for this HRegion.
186  */
187 public class HRegion implements HeapSize { // , Writable{
188   public static final Log LOG = LogFactory.getLog(HRegion.class);
189   private static final String MERGEDIR = ".merges";
190 
191   public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY = "hbase.hregion.scan.loadColumnFamiliesOnDemand";
192 
193   final AtomicBoolean closed = new AtomicBoolean(false);
194   /* Closing can take some time; use the closing flag if there is stuff we don't
195    * want to do while in closing state; e.g. like offer this region up to the
196    * master as a region to close if the carrying regionserver is overloaded.
197    * Once set, it is never cleared.
198    */
199   final AtomicBoolean closing = new AtomicBoolean(false);
200 
201   //////////////////////////////////////////////////////////////////////////////
202   // Members
203   //////////////////////////////////////////////////////////////////////////////
204 
205   private final ConcurrentHashMap<HashedBytes, CountDownLatch> lockedRows =
206     new ConcurrentHashMap<HashedBytes, CountDownLatch>();
207   private final ConcurrentHashMap<Integer, HashedBytes> lockIds =
208     new ConcurrentHashMap<Integer, HashedBytes>();
209   private final AtomicInteger lockIdGenerator = new AtomicInteger(1);
210   static private Random rand = new Random();
211 
212   protected final Map<byte [], Store> stores =
213     new ConcurrentSkipListMap<byte [], Store>(Bytes.BYTES_RAWCOMPARATOR);
214 
215   // Registered region protocol handlers
216   private ClassToInstanceMap<CoprocessorProtocol>
217       protocolHandlers = MutableClassToInstanceMap.create();
218 
219   private Map<String, Class<? extends CoprocessorProtocol>>
220       protocolHandlerNames = Maps.newHashMap();
221 
222   /**
223    * Temporary subdirectory of the region directory used for compaction output.
224    */
225   public static final String REGION_TEMP_SUBDIR = ".tmp";
226 
227   //These variable are just used for getting data out of the region, to test on
228   //client side
229   // private int numStores = 0;
230   // private int [] storeSize = null;
231   // private byte [] name = null;
232 
233   final AtomicLong memstoreSize = new AtomicLong(0);
234 
235   // Debug possible data loss due to WAL off
236   final AtomicLong numPutsWithoutWAL = new AtomicLong(0);
237   final AtomicLong dataInMemoryWithoutWAL = new AtomicLong(0);
238 
239   final Counter readRequestsCount = new Counter();
240   final Counter writeRequestsCount = new Counter();
241   final Counter updatesBlockedMs = new Counter();
242 
243   /**
244    * The directory for the table this region is part of.
245    * This directory contains the directory for this region.
246    */
247   private final Path tableDir;
248 
249   private final HLog log;
250   private final FileSystem fs;
251   private final Configuration conf;
252   final Configuration baseConf;
253   private final int rowLockWaitDuration;
254   static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
255 
256   // The internal wait duration to acquire a lock before read/update
257   // from the region. It is not per row. The purpose of this wait time
258   // is to avoid waiting a long time while the region is busy, so that
259   // we can release the IPC handler soon enough to improve the
260   // availability of the region server. It can be adjusted by
261   // tuning configuration "hbase.busy.wait.duration".
262   final long busyWaitDuration;
263   static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
264 
265   // If updating multiple rows in one call, wait longer,
266   // i.e. waiting for busyWaitDuration * # of rows. However,
267   // we can limit the max multiplier.
268   final int maxBusyWaitMultiplier;
269 
270   // Max busy wait duration. There is no point to wait longer than the RPC
271   // purge timeout, when a RPC call will be terminated by the RPC engine.
272   final long maxBusyWaitDuration;
273 
274   private final HRegionInfo regionInfo;
275   private final Path regiondir;
276   KeyValue.KVComparator comparator;
277 
278   private ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
279   /**
280    * The default setting for whether to enable on-demand CF loading for
281    * scan requests to this region. Requests can override it.
282    */
283   private boolean isLoadingCfsOnDemandDefault = false;
284 
285   /**
286    * @return The smallest mvcc readPoint across all the scanners in this
287    * region. Writes older than this readPoint, are included  in every
288    * read operation.
289    */
290   public long getSmallestReadPoint() {
291     long minimumReadPoint;
292     // We need to ensure that while we are calculating the smallestReadPoint
293     // no new RegionScanners can grab a readPoint that we are unaware of.
294     // We achieve this by synchronizing on the scannerReadPoints object.
295     synchronized(scannerReadPoints) {
296       minimumReadPoint = mvcc.memstoreReadPoint();
297 
298       for (Long readPoint: this.scannerReadPoints.values()) {
299         if (readPoint < minimumReadPoint) {
300           minimumReadPoint = readPoint;
301         }
302       }
303     }
304     return minimumReadPoint;
305   }
306   /*
307    * Data structure of write state flags used coordinating flushes,
308    * compactions and closes.
309    */
310   static class WriteState {
311     // Set while a memstore flush is happening.
312     volatile boolean flushing = false;
313     // Set when a flush has been requested.
314     volatile boolean flushRequested = false;
315     // Number of compactions running.
316     volatile int compacting = 0;
317     // Gets set in close. If set, cannot compact or flush again.
318     volatile boolean writesEnabled = true;
319     // Set if region is read-only
320     volatile boolean readOnly = false;
321 
322     /**
323      * Set flags that make this region read-only.
324      *
325      * @param onOff flip value for region r/o setting
326      */
327     synchronized void setReadOnly(final boolean onOff) {
328       this.writesEnabled = !onOff;
329       this.readOnly = onOff;
330     }
331 
332     boolean isReadOnly() {
333       return this.readOnly;
334     }
335 
336     boolean isFlushRequested() {
337       return this.flushRequested;
338     }
339 
340     static final long HEAP_SIZE = ClassSize.align(
341         ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
342   }
343 
344   /**
345    * Objects from this class are created when flushing to describe all the different states that
346    * that method ends up in. The Result enum describes those states. The sequence id should only
347    * be specified if the flush was successful, and the failure message should only be speficied
348    * if it didn't flush.
349    */
350   public static class FlushResult {
351     enum Result {
352       FLUSHED_NO_COMPACTION_NEEDED,
353       FLUSHED_COMPACTION_NEEDED,
354       // Special case where a flush didn't run because there's nothing in the memstores. Used when
355       // bulk loading to know when we can still load even if a flush didn't happen.
356       CANNOT_FLUSH_MEMSTORE_EMPTY,
357       CANNOT_FLUSH
358       // Be careful adding more to this enum, look at the below methods to make sure
359     }
360 
361     final Result result;
362     final String failureReason;
363     final long flushSequenceId;
364 
365     /**
366      * Convenience constructor to use when the flush is successful, the failure message is set to
367      * null.
368      * @param result Expecting FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_COMPACTION_NEEDED.
369      * @param flushSequenceId Generated sequence id that comes right after the edits in the
370      *                        memstores.
371      */
372     FlushResult(Result result, long flushSequenceId) {
373       this(result, flushSequenceId, null);
374       assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
375           .FLUSHED_COMPACTION_NEEDED;
376     }
377 
378     /**
379      * Convenience constructor to use when we cannot flush.
380      * @param result Expecting CANNOT_FLUSH_MEMSTORE_EMPTY or CANNOT_FLUSH.
381      * @param failureReason Reason why we couldn't flush.
382      */
383     FlushResult(Result result, String failureReason) {
384       this(result, -1, failureReason);
385       assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH;
386     }
387 
388     /**
389      * Constructor with all the parameters.
390      * @param result Any of the Result.
391      * @param flushSequenceId Generated sequence id if the memstores were flushed else -1.
392      * @param failureReason Reason why we couldn't flush, or null.
393      */
394     FlushResult(Result result, long flushSequenceId, String failureReason) {
395       this.result = result;
396       this.flushSequenceId = flushSequenceId;
397       this.failureReason = failureReason;
398     }
399 
400     /**
401      * Convenience method, the equivalent of checking if result is
402      * FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_NO_COMPACTION_NEEDED.
403      * @return true if the memstores were flushed, else false.
404      */
405     public boolean isFlushSucceeded() {
406       return result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
407           .FLUSHED_COMPACTION_NEEDED;
408     }
409 
410     /**
411      * Convenience method, the equivalent of checking if result is FLUSHED_COMPACTION_NEEDED.
412      * @return True if the flush requested a compaction, else false (doesn't even mean it flushed).
413      */
414     public boolean isCompactionNeeded() {
415       return result == Result.FLUSHED_COMPACTION_NEEDED;
416     }
417   }
418 
419   final WriteState writestate = new WriteState();
420 
421   long memstoreFlushSize;
422   final long timestampSlop;
423   private volatile long lastFlushTime;
424   final RegionServerServices rsServices;
425   private RegionServerAccounting rsAccounting;
426   private List<Pair<Long, Long>> recentFlushes = new ArrayList<Pair<Long,Long>>();
427   private long flushCheckInterval;
428   private long blockingMemStoreSize;
429   final long threadWakeFrequency;
430   // Used to guard closes
431   final ReentrantReadWriteLock lock =
432     new ReentrantReadWriteLock();
433 
434   // Stop updates lock
435   private final ReentrantReadWriteLock updatesLock =
436     new ReentrantReadWriteLock();
437   private boolean splitRequest;
438   private byte[] explicitSplitPoint = null;
439 
440   private final MultiVersionConsistencyControl mvcc =
441       new MultiVersionConsistencyControl();
442 
443   // Coprocessor host
444   private RegionCoprocessorHost coprocessorHost;
445 
446   /**
447    * Name of the region info file that resides just under the region directory.
448    */
449   public final static String REGIONINFO_FILE = ".regioninfo";
450   private HTableDescriptor htableDescriptor = null;
451   private RegionSplitPolicy splitPolicy;
452   private final OperationMetrics opMetrics;
453   private final boolean deferredLogSyncDisabled;
454 
455   /**
456    * Should only be used for testing purposes
457    */
458   public HRegion(){
459     this.tableDir = null;
460     this.blockingMemStoreSize = 0L;
461     this.conf = null;
462     this.rowLockWaitDuration = DEFAULT_ROWLOCK_WAIT_DURATION;
463     this.rsServices = null;
464     this.baseConf = null;
465     this.fs = null;
466     this.timestampSlop = HConstants.LATEST_TIMESTAMP;
467     this.memstoreFlushSize = 0L;
468     this.log = null;
469     this.regiondir = null;
470     this.regionInfo = null;
471     this.htableDescriptor = null;
472     this.threadWakeFrequency = 0L;
473     this.coprocessorHost = null;
474     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
475     this.opMetrics = new OperationMetrics();
476 
477     this.maxBusyWaitDuration = 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
478     this.busyWaitDuration = DEFAULT_BUSY_WAIT_DURATION;
479     this.maxBusyWaitMultiplier = 2;
480     this.deferredLogSyncDisabled = false;
481   }
482 
483   
484   /**
485    * HRegion copy constructor. Useful when reopening a closed region (normally
486    * for unit tests)
487    * @param other original object
488    */
489   public HRegion(HRegion other) {
490     this(other.getTableDir(), other.getLog(), other.getFilesystem(),
491         other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
492   }
493   
494   /**
495    * HRegion constructor.  his constructor should only be used for testing and
496    * extensions.  Instances of HRegion should be instantiated with the
497    * {@link HRegion#newHRegion(Path, HLog, FileSystem, Configuration, HRegionInfo, HTableDescriptor, RegionServerServices)} method.
498    *
499    *
500    * @param tableDir qualified path of directory where region should be located,
501    * usually the table directory.
502    * @param log The HLog is the outbound log for any updates to the HRegion
503    * (There's a single HLog for all the HRegions on a single HRegionServer.)
504    * The log file is a logfile from the previous execution that's
505    * custom-computed for this HRegion. The HRegionServer computes and sorts the
506    * appropriate log info for this HRegion. If there is a previous log file
507    * (implying that the HRegion has been written-to before), then read it from
508    * the supplied path.
509    * @param fs is the filesystem.
510    * @param conf is global configuration settings.
511    * @param regionInfo - HRegionInfo that describes the region
512    * is new), then read them from the supplied path.
513    * @param rsServices reference to {@link RegionServerServices} or null
514    *
515    * @see HRegion#newHRegion(Path, HLog, FileSystem, Configuration, HRegionInfo, HTableDescriptor, RegionServerServices)
516    */
517   public HRegion(Path tableDir, HLog log, FileSystem fs, Configuration confParam,
518     final HRegionInfo regionInfo, final HTableDescriptor htd,
519       RegionServerServices rsServices) {
520     this.tableDir = tableDir;
521     this.comparator = regionInfo.getComparator();
522     this.log = log;
523     this.fs = fs;
524     if (confParam instanceof CompoundConfiguration) {
525        throw new IllegalArgumentException("Need original base configuration");
526     }
527     // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
528     this.baseConf = confParam;
529     if (htd != null) {
530       this.conf = new CompoundConfiguration().add(confParam).add(htd.getValues());
531     }
532     else {
533       this.conf = new CompoundConfiguration().add(confParam);
534     }
535     this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL,
536         DEFAULT_CACHE_FLUSH_INTERVAL);
537     this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
538                     DEFAULT_ROWLOCK_WAIT_DURATION);
539 
540     this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, false);
541     this.regionInfo = regionInfo;
542     this.htableDescriptor = htd;
543     this.rsServices = rsServices;
544     this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY,
545         10 * 1000);
546     String encodedNameStr = this.regionInfo.getEncodedName();
547     setHTableSpecificConf();
548     this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
549     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
550     this.opMetrics = new OperationMetrics(conf, this.regionInfo);
551 
552     this.busyWaitDuration = conf.getLong(
553       "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
554     this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
555     if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
556       throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
557         + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
558         + maxBusyWaitMultiplier + "). Their product should be positive");
559     }
560     this.maxBusyWaitDuration = conf.getLong("ipc.client.call.purge.timeout",
561       2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
562 
563     /*
564      * timestamp.slop provides a server-side constraint on the timestamp. This
565      * assumes that you base your TS around currentTimeMillis(). In this case,
566      * throw an error to the user if the user-specified TS is newer than now +
567      * slop. LATEST_TIMESTAMP == don't use this functionality
568      */
569     this.timestampSlop = conf.getLong(
570         "hbase.hregion.keyvalue.timestamp.slop.millisecs",
571         HConstants.LATEST_TIMESTAMP);
572     // When hbase.regionserver.optionallogflushinterval <= 0 , deferred log sync is disabled.
573     this.deferredLogSyncDisabled = conf.getLong("hbase.regionserver.optionallogflushinterval",
574         1 * 1000) <= 0;
575 
576     if (rsServices != null) {
577       this.rsAccounting = this.rsServices.getRegionServerAccounting();
578       // don't initialize coprocessors if not running within a regionserver
579       // TODO: revisit if coprocessors should load in other cases
580       this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
581     }
582     if (LOG.isDebugEnabled()) {
583       // Write out region name as string and its encoded name.
584       LOG.debug("Instantiated " + this);
585     }
586   }
587 
588   void setHTableSpecificConf() {
589     if (this.htableDescriptor == null) return;
590     LOG.info("Setting up tabledescriptor config now ...");
591     long flushSize = this.htableDescriptor.getMemStoreFlushSize();
592 
593     if (flushSize <= 0) {
594       flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
595         HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
596     }
597     this.memstoreFlushSize = flushSize;
598     this.blockingMemStoreSize = this.memstoreFlushSize *
599         conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
600   }
601 
602   /**
603    * Initialize this region.
604    * @return What the next sequence (edit) id should be.
605    * @throws IOException e
606    */
607   public long initialize() throws IOException {
608     return initialize(null);
609   }
610 
611   /**
612    * Initialize this region.
613    *
614    * @param reporter Tickle every so often if initialize is taking a while.
615    * @return What the next sequence (edit) id should be.
616    * @throws IOException e
617    */
618   public long initialize(final CancelableProgressable reporter)
619       throws IOException {
620 
621     MonitoredTask status = TaskMonitor.get().createStatus(
622         "Initializing region " + this);
623 
624     long nextSeqId = -1;
625     try {
626       nextSeqId = initializeRegionInternals(reporter, status);
627       return nextSeqId;
628     } finally {
629       // nextSeqid will be -1 if the initialization fails.
630       // At least it will be 0 otherwise.
631       if (nextSeqId == -1) {
632         status.abort("Exception during region " + this.getRegionNameAsString()
633             + " initialization.");
634       }
635     }
636   }
637 
638   private long initializeRegionInternals(final CancelableProgressable reporter,
639       MonitoredTask status) throws IOException, UnsupportedEncodingException {
640     if (coprocessorHost != null) {
641       status.setStatus("Running coprocessor pre-open hook");
642       coprocessorHost.preOpen();
643     }
644 
645     // Write HRI to a file in case we need to recover .META.
646     status.setStatus("Writing region info on filesystem");
647     checkRegioninfoOnFilesystem();
648 
649     // Remove temporary data left over from old regions
650     status.setStatus("Cleaning up temporary data from old regions");
651     cleanupTmpDir();
652 
653     // Load in all the HStores.
654     //
655     // Context: During replay we want to ensure that we do not lose any data. So, we
656     // have to be conservative in how we replay logs. For each store, we calculate
657     // the maxSeqId up to which the store was flushed. And, skip the edits which
658     // is equal to or lower than maxSeqId for each store.
659     Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
660         Bytes.BYTES_COMPARATOR);
661     long maxSeqId = -1;
662     // initialized to -1 so that we pick up MemstoreTS from column families
663     long maxMemstoreTS = -1;
664 
665     if (this.htableDescriptor != null &&
666         !htableDescriptor.getFamilies().isEmpty()) {
667       // initialize the thread pool for opening stores in parallel.
668       ThreadPoolExecutor storeOpenerThreadPool =
669         getStoreOpenAndCloseThreadPool(
670           "StoreOpenerThread-" + this.regionInfo.getRegionNameAsString());
671       CompletionService<Store> completionService =
672         new ExecutorCompletionService<Store>(storeOpenerThreadPool);
673 
674       // initialize each store in parallel
675       for (final HColumnDescriptor family : htableDescriptor.getFamilies()) {
676         status.setStatus("Instantiating store for column family " + family);
677         completionService.submit(new Callable<Store>() {
678           public Store call() throws IOException {
679             return instantiateHStore(tableDir, family);
680           }
681         });
682       }
683       boolean allStoresOpened = false;
684       try {
685         for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) {
686           Future<Store> future = completionService.take();
687           Store store = future.get();
688           this.stores.put(store.getColumnFamilyName().getBytes(), store);
689 
690           long storeSeqIdForReplay = store.getMaxSequenceId();
691           maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), storeSeqIdForReplay);
692           if (maxSeqId == -1 || storeSeqIdForReplay > maxSeqId) {
693             maxSeqId = storeSeqIdForReplay;
694           }
695           long maxStoreMemstoreTS = store.getMaxMemstoreTS();
696           if (maxStoreMemstoreTS > maxMemstoreTS) {
697             maxMemstoreTS = maxStoreMemstoreTS;
698           }
699         }
700         allStoresOpened = true;
701       } catch (InterruptedException e) {
702         throw new IOException(e);
703       } catch (ExecutionException e) {
704         throw new IOException(e.getCause());
705       } finally {
706         storeOpenerThreadPool.shutdownNow();
707         if (!allStoresOpened) {
708           // something went wrong, close all opened stores
709           LOG.error("Could not initialize all stores for the region=" + this);
710           for (Store store : this.stores.values()) {
711             try {
712               store.close();
713             } catch (IOException e) { 
714               LOG.warn(e.getMessage());
715             }
716           }
717         }
718       }
719     }
720     mvcc.initialize(maxMemstoreTS + 1);
721     // Recover any edits if available.
722     maxSeqId = Math.max(maxSeqId, replayRecoveredEditsIfAny(
723         this.regiondir, maxSeqIdInStores, reporter, status));
724 
725     status.setStatus("Cleaning up detritus from prior splits");
726     // Get rid of any splits or merges that were lost in-progress.  Clean out
727     // these directories here on open.  We may be opening a region that was
728     // being split but we crashed in the middle of it all.
729     SplitTransaction.cleanupAnySplitDetritus(this);
730     FSUtils.deleteDirectory(this.fs, new Path(regiondir, MERGEDIR));
731 
732     this.writestate.setReadOnly(this.htableDescriptor.isReadOnly());
733 
734     this.writestate.flushRequested = false;
735     this.writestate.compacting = 0;
736 
737     // Initialize split policy
738     this.splitPolicy = RegionSplitPolicy.create(this, conf);
739 
740     this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
741     // Use maximum of log sequenceid or that which was found in stores
742     // (particularly if no recovered edits, seqid will be -1).
743     long nextSeqid = maxSeqId + 1;
744     LOG.info("Onlined " + this.toString() + "; next sequenceid=" + nextSeqid);
745 
746     // A region can be reopened if failed a split; reset flags
747     this.closing.set(false);
748     this.closed.set(false);
749 
750     if (coprocessorHost != null) {
751       status.setStatus("Running coprocessor post-open hooks");
752       coprocessorHost.postOpen();
753     }
754 
755     status.markComplete("Region opened successfully");
756     return nextSeqid;
757   }
758 
759   /*
760    * Move any passed HStore files into place (if any).  Used to pick up split
761    * files and any merges from splits and merges dirs.
762    * @param initialFiles
763    * @throws IOException
764    */
765   static void moveInitialFilesIntoPlace(final FileSystem fs,
766     final Path initialFiles, final Path regiondir)
767   throws IOException {
768     if (initialFiles != null && fs.exists(initialFiles)) {
769       if (!HBaseFileSystem.renameDirForFileSystem(fs, initialFiles, regiondir)) {
770         LOG.warn("Unable to rename " + initialFiles + " to " + regiondir);
771       }
772     }
773   }
774 
775   /**
776    * @return True if this region has references.
777    */
778   public boolean hasReferences() {
779     for (Store store : this.stores.values()) {
780       for (StoreFile sf : store.getStorefiles()) {
781         // Found a reference, return.
782         if (sf.isReference()) return true;
783       }
784     }
785     return false;
786   }
787 
788   /**
789    * This function will return the HDFS blocks distribution based on the data
790    * captured when HFile is created
791    * @return The HDFS blocks distribution for the region.
792    */
793   public HDFSBlocksDistribution getHDFSBlocksDistribution() {
794     HDFSBlocksDistribution hdfsBlocksDistribution =
795       new HDFSBlocksDistribution();
796     synchronized (this.stores) {
797       for (Store store : this.stores.values()) {
798         for (StoreFile sf : store.getStorefiles()) {
799           HDFSBlocksDistribution storeFileBlocksDistribution =
800             sf.getHDFSBlockDistribution();
801           hdfsBlocksDistribution.add(storeFileBlocksDistribution);
802         }
803       }
804     }
805     return hdfsBlocksDistribution;
806   }
807 
808   /**
809    * This is a helper function to compute HDFS block distribution on demand
810    * @param conf configuration
811    * @param tableDescriptor HTableDescriptor of the table
812    * @param regionEncodedName encoded name of the region
813    * @return The HDFS blocks distribution for the given region.
814    * @throws IOException
815    */
816   static public HDFSBlocksDistribution computeHDFSBlocksDistribution(
817     Configuration conf, HTableDescriptor tableDescriptor,
818     String regionEncodedName) throws IOException {
819     Path tablePath = FSUtils.getTablePath(FSUtils.getRootDir(conf),
820       tableDescriptor.getName());
821     return computeHDFSBlocksDistribution(conf, tableDescriptor, regionEncodedName, tablePath);
822   }
823 
824   /**
825    * This is a helper function to compute HDFS block distribution on demand
826    * @param conf configuration
827    * @param tableDescriptor HTableDescriptor of the table
828    * @param regionEncodedName encoded name of the region
829    * @param tablePath The table's directory
830    * @return The HDFS blocks distribution for the given region.
831    * @throws IOException
832    */
833   static public HDFSBlocksDistribution computeHDFSBlocksDistribution(
834     Configuration conf, HTableDescriptor tableDescriptor,
835     String regionEncodedName, Path tablePath) throws IOException {
836     HDFSBlocksDistribution hdfsBlocksDistribution =
837       new HDFSBlocksDistribution();
838     FileSystem fs = tablePath.getFileSystem(conf);
839 
840     for (HColumnDescriptor family: tableDescriptor.getFamilies()) {
841       Path storeHomeDir = Store.getStoreHomedir(tablePath, regionEncodedName,
842       family.getName());
843       if (!fs.exists(storeHomeDir))continue;
844 
845       FileStatus[] hfilesStatus = null;
846       hfilesStatus = fs.listStatus(storeHomeDir);
847 
848       for (FileStatus hfileStatus : hfilesStatus) {
849         Path p = hfileStatus.getPath();
850         if (HFileLink.isHFileLink(p)) {
851           hfileStatus = new HFileLink(conf, p).getFileStatus(fs);
852         } else if (StoreFile.isReference(p)) {
853           p = StoreFile.getReferredToFile(p);
854           if (HFileLink.isHFileLink(p)) {
855             hfileStatus = new HFileLink(conf, p).getFileStatus(fs);
856           } else {
857             hfileStatus = fs.getFileStatus(p);
858           }
859         }
860         HDFSBlocksDistribution storeFileBlocksDistribution =
861           FSUtils.computeHDFSBlocksDistribution(fs, hfileStatus, 0,
862           hfileStatus.getLen());
863         hdfsBlocksDistribution.add(storeFileBlocksDistribution);
864       }
865     }
866     return hdfsBlocksDistribution;
867   }
868 
869   public AtomicLong getMemstoreSize() {
870     return memstoreSize;
871   }
872 
873   /**
874    * Increase the size of mem store in this region and the size of global mem
875    * store
876    * @param memStoreSize
877    * @return the size of memstore in this region
878    */
879   public long addAndGetGlobalMemstoreSize(long memStoreSize) {
880     if (this.rsAccounting != null) {
881       rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
882     }
883     return this.memstoreSize.addAndGet(memStoreSize);
884   }
885 
886   /*
887    * Write out an info file under the region directory.  Useful recovering
888    * mangled regions.
889    * @throws IOException
890    */
891   private void checkRegioninfoOnFilesystem() throws IOException {
892     checkRegioninfoOnFilesystem(this.regiondir);
893   }
894 
895   /**
896    * Write out an info file under the region directory. Useful recovering mangled regions.
897    * @param regiondir directory under which to write out the region info
898    * @throws IOException
899    */
900   private void checkRegioninfoOnFilesystem(Path regiondir) throws IOException {
901     writeRegioninfoOnFilesystem(regionInfo, regiondir, getFilesystem(), conf);
902   }
903 
904   /**
905    * Write out an info file under the region directory. Useful recovering mangled regions. If the
906    * regioninfo already exists on disk and there is information in the file, then we fast exit.
907    * @param regionInfo information about the region
908    * @param regiondir directory under which to write out the region info
909    * @param fs {@link FileSystem} on which to write the region info
910    * @param conf {@link Configuration} from which to extract specific file locations
911    * @throws IOException on unexpected error.
912    */
913   public static void writeRegioninfoOnFilesystem(HRegionInfo regionInfo, Path regiondir,
914       FileSystem fs, Configuration conf) throws IOException {
915     Path regioninfoPath = new Path(regiondir, REGIONINFO_FILE);
916     if (fs.exists(regioninfoPath)) {
917       if (fs.getFileStatus(regioninfoPath).getLen() > 0) {
918         return;
919       }
920 
921       LOG.info("Rewriting .regioninfo file at: " + regioninfoPath);
922       if (!fs.delete(regioninfoPath, false)) {
923         throw new IOException("Unable to remove existing " + regioninfoPath);
924       }
925     }
926 
927     // Create in tmpdir and then move into place in case we crash after
928     // create but before close.  If we don't successfully close the file,
929     // subsequent region reopens will fail the below because create is
930     // registered in NN.
931 
932     // first check to get the permissions
933     FsPermission perms = FSUtils.getFilePermissions(fs, conf,
934         HConstants.DATA_FILE_UMASK_KEY);
935 
936     // and then create the file
937     Path tmpPath = new Path(getTmpDir(regiondir), REGIONINFO_FILE);
938 
939     // if datanode crashes or if the RS goes down just before the close is called while trying to
940     // close the created regioninfo file in the .tmp directory then on next
941     // creation we will be getting AlreadyCreatedException.
942     // Hence delete and create the file if exists.
943     if (FSUtils.isExists(fs, tmpPath)) {
944       FSUtils.delete(fs, tmpPath, true);
945     }
946 
947     FSDataOutputStream out = FSUtils.create(fs, tmpPath, perms);
948 
949     try {
950       regionInfo.write(out);
951       out.write('\n');
952       out.write('\n');
953       out.write(Bytes.toBytes(regionInfo.toString()));
954     } finally {
955       out.close();
956     }
957     if (!HBaseFileSystem.renameDirForFileSystem(fs, tmpPath, regioninfoPath)) {
958       throw new IOException("Unable to rename " + tmpPath + " to " +
959         regioninfoPath);
960     }
961   }
962 
963   /**
964    * @param fs
965    * @param dir
966    * @return An HRegionInfo instance gotten from the <code>.regioninfo</code> file under region dir
967    * @throws IOException
968    */
969   public static HRegionInfo loadDotRegionInfoFileContent(final FileSystem fs, final Path dir)
970   throws IOException {
971     Path regioninfo = new Path(dir, HRegion.REGIONINFO_FILE);
972     if (!fs.exists(regioninfo)) throw new FileNotFoundException(regioninfo.toString());
973     FSDataInputStream in = fs.open(regioninfo);
974     try {
975       HRegionInfo hri = new HRegionInfo();
976       hri.readFields(in);
977       return hri;
978     } finally {
979       in.close();
980     }
981   }
982 
983   /** @return a HRegionInfo object for this region */
984   public HRegionInfo getRegionInfo() {
985     return this.regionInfo;
986   }
987 
988   /**
989    * @return Instance of {@link RegionServerServices} used by this HRegion.
990    * Can be null.
991    */
992   RegionServerServices getRegionServerServices() {
993     return this.rsServices;
994   }
995 
996   /** @return requestsCount for this region */
997   public long getRequestsCount() {
998     return this.readRequestsCount.get() + this.writeRequestsCount.get();
999   }
1000 
1001   /** @return readRequestsCount for this region */
1002   public long getReadRequestsCount() {
1003     return this.readRequestsCount.get();
1004   }
1005 
1006   /** @return writeRequestsCount for this region */
1007   public long getWriteRequestsCount() {
1008     return this.writeRequestsCount.get();
1009   }
1010 
1011   /** @return true if region is closed */
1012   public boolean isClosed() {
1013     return this.closed.get();
1014   }
1015 
1016   /**
1017    * @return True if closing process has started.
1018    */
1019   public boolean isClosing() {
1020     return this.closing.get();
1021   }
1022 
1023   /** @return true if region is available (not closed and not closing) */
1024   public boolean isAvailable() {
1025     return !isClosed() && !isClosing();
1026   }
1027 
1028   /** @return true if region is splittable */
1029   public boolean isSplittable() {
1030     return isAvailable() && !hasReferences();
1031   }
1032 
1033   boolean areWritesEnabled() {
1034     synchronized(this.writestate) {
1035       return this.writestate.writesEnabled;
1036     }
1037   }
1038 
1039    public MultiVersionConsistencyControl getMVCC() {
1040      return mvcc;
1041    }
1042 
1043    public boolean isLoadingCfsOnDemandDefault() {
1044      return this.isLoadingCfsOnDemandDefault;
1045    }
1046 
1047   /**
1048    * Close down this HRegion.  Flush the cache, shut down each HStore, don't
1049    * service any more calls.
1050    *
1051    * <p>This method could take some time to execute, so don't call it from a
1052    * time-sensitive thread.
1053    *
1054    * @return Vector of all the storage files that the HRegion's component
1055    * HStores make use of.  It's a list of all HStoreFile objects. Returns empty
1056    * vector if already closed and null if judged that it should not close.
1057    *
1058    * @throws IOException e
1059    */
1060   public List<StoreFile> close() throws IOException {
1061     return close(false);
1062   }
1063 
1064   private final Object closeLock = new Object();
1065 
1066   /** Conf key for the periodic flush interval */
1067   public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL = 
1068       "hbase.regionserver.optionalcacheflushinterval";
1069   /** Default interval for the memstore flush */
1070   public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
1071 
1072   /**
1073    * Close down this HRegion.  Flush the cache unless abort parameter is true,
1074    * Shut down each HStore, don't service any more calls.
1075    *
1076    * This method could take some time to execute, so don't call it from a
1077    * time-sensitive thread.
1078    *
1079    * @param abort true if server is aborting (only during testing)
1080    * @return Vector of all the storage files that the HRegion's component
1081    * HStores make use of.  It's a list of HStoreFile objects.  Can be null if
1082    * we are not to close at this time or we are already closed.
1083    *
1084    * @throws IOException e
1085    */
1086   public List<StoreFile> close(final boolean abort) throws IOException {
1087     // Only allow one thread to close at a time. Serialize them so dual
1088     // threads attempting to close will run up against each other.
1089     MonitoredTask status = TaskMonitor.get().createStatus(
1090         "Closing region " + this +
1091         (abort ? " due to abort" : ""));
1092 
1093     status.setStatus("Waiting for close lock");
1094     try {
1095       synchronized (closeLock) {
1096         return doClose(abort, status);
1097       }
1098     } finally {
1099       status.cleanup();
1100     }
1101   }
1102 
1103   private List<StoreFile> doClose(
1104       final boolean abort, MonitoredTask status)
1105   throws IOException {
1106     if (isClosed()) {
1107       LOG.warn("Region " + this + " already closed");
1108       return null;
1109     }
1110 
1111     if (coprocessorHost != null) {
1112       status.setStatus("Running coprocessor pre-close hooks");
1113       this.coprocessorHost.preClose(abort);
1114     }
1115 
1116     status.setStatus("Disabling compacts and flushes for region");
1117     synchronized (writestate) {
1118       // Disable compacting and flushing by background threads for this
1119       // region.
1120       writestate.writesEnabled = false;
1121       LOG.debug("Closing " + this + ": disabling compactions & flushes");
1122       waitForFlushesAndCompactions();
1123     }
1124     // If we were not just flushing, is it worth doing a preflush...one
1125     // that will clear out of the bulk of the memstore before we put up
1126     // the close flag?
1127     if (!abort && worthPreFlushing()) {
1128       status.setStatus("Pre-flushing region before close");
1129       LOG.info("Running close preflush of " + this.getRegionNameAsString());
1130       try {
1131         internalFlushcache(status);
1132       } catch (IOException ioe) {
1133         // Failed to flush the region. Keep going.
1134         status.setStatus("Failed pre-flush " + this + "; " + ioe.getMessage());
1135       }
1136     }
1137 
1138     this.closing.set(true);
1139     status.setStatus("Disabling writes for close");
1140     // block waiting for the lock for closing
1141     lock.writeLock().lock();
1142     try {
1143       if (this.isClosed()) {
1144         status.abort("Already got closed by another process");
1145         // SplitTransaction handles the null
1146         return null;
1147       }
1148       LOG.debug("Updates disabled for region " + this);
1149       // Don't flush the cache if we are aborting
1150       if (!abort) {
1151         int flushCount = 0;
1152         while (this.getMemstoreSize().get() > 0) {
1153           try {
1154             if (flushCount++ > 0) {
1155               int actualFlushes = flushCount - 1;
1156               if (actualFlushes > 5) {
1157                 // If we tried 5 times and are unable to clear memory, abort
1158                 // so we do not lose data
1159                 throw new DroppedSnapshotException("Failed clearing memory after " +
1160                   actualFlushes + " attempts on region: " + Bytes.toStringBinary(getRegionName()));
1161               } 
1162               LOG.info("Running extra flush, " + actualFlushes +
1163                 " (carrying snapshot?) " + this);
1164             }
1165             internalFlushcache(status);
1166           } catch (IOException ioe) {
1167             status.setStatus("Failed flush " + this + ", putting online again");
1168             synchronized (writestate) {
1169               writestate.writesEnabled = true;
1170             }
1171             // Have to throw to upper layers.  I can't abort server from here.
1172             throw ioe;
1173           }
1174         }
1175       }
1176 
1177       List<StoreFile> result = new ArrayList<StoreFile>();
1178       if (!stores.isEmpty()) {
1179         // initialize the thread pool for closing stores in parallel.
1180         ThreadPoolExecutor storeCloserThreadPool =
1181           getStoreOpenAndCloseThreadPool("StoreCloserThread-"
1182             + this.regionInfo.getRegionNameAsString());
1183         CompletionService<ImmutableList<StoreFile>> completionService =
1184           new ExecutorCompletionService<ImmutableList<StoreFile>>(
1185             storeCloserThreadPool);
1186 
1187         // close each store in parallel
1188         for (final Store store : stores.values()) {
1189           assert abort? true: store.getFlushableSize() == 0;
1190           completionService
1191               .submit(new Callable<ImmutableList<StoreFile>>() {
1192                 public ImmutableList<StoreFile> call() throws IOException {
1193                   return store.close();
1194                 }
1195               });
1196         }
1197         try {
1198           for (int i = 0; i < stores.size(); i++) {
1199             Future<ImmutableList<StoreFile>> future = completionService
1200                 .take();
1201             ImmutableList<StoreFile> storeFileList = future.get();
1202             result.addAll(storeFileList);
1203           }
1204         } catch (InterruptedException e) {
1205           throw new IOException(e);
1206         } catch (ExecutionException e) {
1207           throw new IOException(e.getCause());
1208         } finally {
1209           storeCloserThreadPool.shutdownNow();
1210         }
1211       }
1212       this.closed.set(true);
1213       if (memstoreSize.get() != 0) LOG.error("Memstore size is " + memstoreSize.get());
1214       if (coprocessorHost != null) {
1215         status.setStatus("Running coprocessor post-close hooks");
1216         this.coprocessorHost.postClose(abort);
1217       }
1218       this.opMetrics.closeMetrics(this.getRegionInfo().getEncodedName());
1219       status.markComplete("Closed");
1220       LOG.info("Closed " + this);
1221       return result;
1222     } finally {
1223       lock.writeLock().unlock();
1224     }
1225   }
1226 
1227   /**
1228    * Wait for all current flushes and compactions of the region to complete.
1229    * <p>
1230    * Exposed for TESTING.
1231    */
1232   public void waitForFlushesAndCompactions() {
1233     synchronized (writestate) {
1234       while (writestate.compacting > 0 || writestate.flushing) {
1235         LOG.debug("waiting for " + writestate.compacting + " compactions"
1236             + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
1237         try {
1238           writestate.wait();
1239         } catch (InterruptedException iex) {
1240           // essentially ignore and propagate the interrupt back up
1241           Thread.currentThread().interrupt();
1242         }
1243       }
1244     }
1245   }
1246 
1247   protected ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
1248       final String threadNamePrefix) {
1249     int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1250     int maxThreads = Math.min(numStores,
1251         conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1252             HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
1253     return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1254   }
1255 
1256   protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
1257       final String threadNamePrefix) {
1258     int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1259     int maxThreads = Math.max(1,
1260         conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1261             HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
1262             / numStores);
1263     return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1264   }
1265 
1266   static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
1267       final String threadNamePrefix) {
1268     return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
1269       new ThreadFactory() {
1270         private int count = 1;
1271 
1272         public Thread newThread(Runnable r) {
1273           return new Thread(r, threadNamePrefix + "-" + count++);
1274         }
1275       });
1276   }
1277 
1278    /**
1279     * @return True if its worth doing a flush before we put up the close flag.
1280     */
1281   private boolean worthPreFlushing() {
1282     return this.memstoreSize.get() >
1283       this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
1284   }
1285 
1286   //////////////////////////////////////////////////////////////////////////////
1287   // HRegion accessors
1288   //////////////////////////////////////////////////////////////////////////////
1289 
1290   /** @return start key for region */
1291   public byte [] getStartKey() {
1292     return this.regionInfo.getStartKey();
1293   }
1294 
1295   /** @return end key for region */
1296   public byte [] getEndKey() {
1297     return this.regionInfo.getEndKey();
1298   }
1299 
1300   /** @return region id */
1301   public long getRegionId() {
1302     return this.regionInfo.getRegionId();
1303   }
1304 
1305   /** @return region name */
1306   public byte [] getRegionName() {
1307     return this.regionInfo.getRegionName();
1308   }
1309 
1310   /** @return region name as string for logging */
1311   public String getRegionNameAsString() {
1312     return this.regionInfo.getRegionNameAsString();
1313   }
1314 
1315   /** @return HTableDescriptor for this region */
1316   public HTableDescriptor getTableDesc() {
1317     return this.htableDescriptor;
1318   }
1319 
1320   /** @return HLog in use for this region */
1321   public HLog getLog() {
1322     return this.log;
1323   }
1324 
1325   /** @return Configuration object */
1326   public Configuration getConf() {
1327     return this.conf;
1328   }
1329 
1330     /**
1331    * A split takes the config from the parent region & passes it to the daughter
1332    * region's constructor. If 'conf' was passed, you would end up using the HTD
1333    * of the parent region in addition to the new daughter HTD. Pass 'baseConf'
1334    * to the daughter regions to avoid this tricky dedupe problem.
1335    * @return Configuration object
1336    */
1337   Configuration getBaseConf() {
1338     return this.baseConf;
1339   }
1340 
1341   /** @return region directory Path */
1342   public Path getRegionDir() {
1343     return this.regiondir;
1344   }
1345 
1346   /**
1347    * Computes the Path of the HRegion
1348    *
1349    * @param tabledir qualified path for table
1350    * @param name ENCODED region name
1351    * @return Path of HRegion directory
1352    */
1353   public static Path getRegionDir(final Path tabledir, final String name) {
1354     return new Path(tabledir, name);
1355   }
1356 
1357   /** @return FileSystem being used by this region */
1358   public FileSystem getFilesystem() {
1359     return this.fs;
1360   }
1361 
1362   /** @return the last time the region was flushed */
1363   public long getLastFlushTime() {
1364     return this.lastFlushTime;
1365   }
1366 
1367   /** @return info about the last flushes <time, size> */
1368   public List<Pair<Long,Long>> getRecentFlushInfo() {
1369     this.lock.readLock().lock();
1370     List<Pair<Long,Long>> ret = this.recentFlushes;
1371     this.recentFlushes = new ArrayList<Pair<Long,Long>>();
1372     this.lock.readLock().unlock();
1373     return ret;
1374   }
1375 
1376   //////////////////////////////////////////////////////////////////////////////
1377   // HRegion maintenance.
1378   //
1379   // These methods are meant to be called periodically by the HRegionServer for
1380   // upkeep.
1381   //////////////////////////////////////////////////////////////////////////////
1382 
1383   /** @return returns size of largest HStore. */
1384   public long getLargestHStoreSize() {
1385     long size = 0;
1386     for (Store h: stores.values()) {
1387       long storeSize = h.getSize();
1388       if (storeSize > size) {
1389         size = storeSize;
1390       }
1391     }
1392     return size;
1393   }
1394 
1395   /*
1396    * Do preparation for pending compaction.
1397    * @throws IOException
1398    */
1399   void doRegionCompactionPrep() throws IOException {
1400   }
1401 
1402   /*
1403    * Removes the temporary directory for this Store.
1404    */
1405   private void cleanupTmpDir() throws IOException {
1406     FSUtils.deleteDirectory(this.fs, getTmpDir());
1407   }
1408 
1409   /**
1410    * Get the temporary directory for this region. This directory
1411    * will have its contents removed when the region is reopened.
1412    */
1413   Path getTmpDir() {
1414     return getTmpDir(getRegionDir());
1415   }
1416 
1417   static Path getTmpDir(Path regionDir) {
1418     return new Path(regionDir, REGION_TEMP_SUBDIR);
1419   }
1420 
1421   void triggerMajorCompaction() {
1422     for (Store h: stores.values()) {
1423       h.triggerMajorCompaction();
1424     }
1425   }
1426 
1427   /**
1428    * This is a helper function that compact all the stores synchronously
1429    * It is used by utilities and testing
1430    *
1431    * @param majorCompaction True to force a major compaction regardless of thresholds
1432    * @throws IOException e
1433    */
1434   public void compactStores(final boolean majorCompaction)
1435   throws IOException {
1436     if (majorCompaction) {
1437       this.triggerMajorCompaction();
1438     }
1439     compactStores();
1440   }
1441 
1442   /**
1443    * This is a helper function that compact all the stores synchronously
1444    * It is used by utilities and testing
1445    *
1446    * @throws IOException e
1447    */
1448   public void compactStores() throws IOException {
1449     for(Store s : getStores().values()) {
1450       CompactionRequest cr = s.requestCompaction();
1451       if(cr != null) {
1452         try {
1453           compact(cr);
1454         } finally {
1455           s.finishRequest(cr);
1456         }
1457       }
1458     }
1459   }
1460 
1461   /*
1462    * Called by compaction thread and after region is opened to compact the
1463    * HStores if necessary.
1464    *
1465    * <p>This operation could block for a long time, so don't call it from a
1466    * time-sensitive thread.
1467    *
1468    * Note that no locking is necessary at this level because compaction only
1469    * conflicts with a region split, and that cannot happen because the region
1470    * server does them sequentially and not in parallel.
1471    *
1472    * @param cr Compaction details, obtained by requestCompaction()
1473    * @return whether the compaction completed
1474    * @throws IOException e
1475    */
1476   public boolean compact(CompactionRequest cr)
1477   throws IOException {
1478     if (cr == null) {
1479       return false;
1480     }
1481     if (this.closing.get() || this.closed.get()) {
1482       LOG.debug("Skipping compaction on " + this + " because closing/closed");
1483       return false;
1484     }
1485     Preconditions.checkArgument(cr.getHRegion().equals(this));
1486     // block waiting for the lock for compaction
1487     lock.readLock().lock();
1488     MonitoredTask status = TaskMonitor.get().createStatus(
1489         "Compacting " + cr.getStore() + " in " + this);
1490     try {
1491       if (this.closed.get()) {
1492         LOG.debug("Skipping compaction on " + this + " because closed");
1493         return false;
1494       }
1495       boolean decr = true;
1496       try {
1497         synchronized (writestate) {
1498           if (writestate.writesEnabled) {
1499             ++writestate.compacting;
1500           } else {
1501             String msg = "NOT compacting region " + this + ". Writes disabled.";
1502             LOG.info(msg);
1503             status.abort(msg);
1504             decr = false;
1505             return false;
1506           }
1507         }
1508         LOG.info("Starting compaction on " + cr.getStore() + " in region "
1509             + this + (cr.getCompactSelection().isOffPeakCompaction()?" as an off-peak compaction":""));
1510         doRegionCompactionPrep();
1511         try {
1512           status.setStatus("Compacting store " + cr.getStore());
1513           cr.getStore().compact(cr);
1514         } catch (InterruptedIOException iioe) {
1515           String msg = "compaction interrupted by user";
1516           LOG.info(msg, iioe);
1517           status.abort(msg);
1518           return false;
1519         }
1520       } finally {
1521         if (decr) {
1522           synchronized (writestate) {
1523             --writestate.compacting;
1524             if (writestate.compacting <= 0) {
1525               writestate.notifyAll();
1526             }
1527           }
1528         }
1529       }
1530       status.markComplete("Compaction complete");
1531       return true;
1532     } finally {
1533       status.cleanup();
1534       lock.readLock().unlock();
1535     }
1536   }
1537 
1538   /**
1539    * Flush the cache.
1540    *
1541    * When this method is called the cache will be flushed unless:
1542    * <ol>
1543    *   <li>the cache is empty</li>
1544    *   <li>the region is closed.</li>
1545    *   <li>a flush is already in progress</li>
1546    *   <li>writes are disabled</li>
1547    * </ol>
1548    *
1549    * <p>This method may block for some time, so it should not be called from a
1550    * time-sensitive thread.
1551    *
1552    * @return true if the region needs compaction
1553    *
1554    * @throws IOException general io exceptions
1555    * @throws DroppedSnapshotException Thrown when replay of hlog is required
1556    * because a Snapshot was not properly persisted.
1557    */
1558   public FlushResult flushcache() throws IOException {
1559     // fail-fast instead of waiting on the lock
1560     if (this.closing.get()) {
1561       String msg = "Skipping flush on " + this + " because closing";
1562       LOG.debug(msg);
1563       return new FlushResult(FlushResult.Result.CANNOT_FLUSH, msg);
1564     }
1565     MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
1566     status.setStatus("Acquiring readlock on region");
1567     // block waiting for the lock for flushing cache
1568     lock.readLock().lock();
1569     try {
1570       if (this.closed.get()) {
1571         String msg = "Skipping flush on " + this + " because closed";
1572         LOG.debug(msg);
1573         status.abort(msg);
1574         return new FlushResult(FlushResult.Result.CANNOT_FLUSH, msg);
1575       }
1576       if (coprocessorHost != null) {
1577         status.setStatus("Running coprocessor pre-flush hooks");
1578         coprocessorHost.preFlush();
1579       }
1580       if (numPutsWithoutWAL.get() > 0) {
1581         numPutsWithoutWAL.set(0);
1582         dataInMemoryWithoutWAL.set(0);
1583       }
1584       synchronized (writestate) {
1585         if (!writestate.flushing && writestate.writesEnabled) {
1586           this.writestate.flushing = true;
1587         } else {
1588           if (LOG.isDebugEnabled()) {
1589             LOG.debug("NOT flushing memstore for region " + this
1590                 + ", flushing=" + writestate.flushing + ", writesEnabled="
1591                 + writestate.writesEnabled);
1592           }
1593           String msg = "Not flushing since "
1594               + (writestate.flushing ? "already flushing"
1595               : "writes not enabled");
1596           status.abort(msg);
1597           return new FlushResult(FlushResult.Result.CANNOT_FLUSH, msg);
1598         }
1599       }
1600       try {
1601         FlushResult fs = internalFlushcache(status);
1602 
1603         if (coprocessorHost != null) {
1604           status.setStatus("Running post-flush coprocessor hooks");
1605           coprocessorHost.postFlush();
1606         }
1607 
1608         status.markComplete("Flush successful");
1609         return fs;
1610       } finally {
1611         synchronized (writestate) {
1612           writestate.flushing = false;
1613           this.writestate.flushRequested = false;
1614           writestate.notifyAll();
1615         }
1616       }
1617     } finally {
1618       lock.readLock().unlock();
1619       status.cleanup();
1620     }
1621   }
1622 
1623   /**
1624    * Should the memstore be flushed now
1625    */
1626   boolean shouldFlush() {
1627     if (flushCheckInterval <= 0) { //disabled
1628       return false;
1629     }
1630     long now = EnvironmentEdgeManager.currentTimeMillis();
1631     //if we flushed in the recent past, we don't need to do again now
1632     if ((now - getLastFlushTime() < flushCheckInterval)) {
1633       return false;
1634     }
1635     //since we didn't flush in the recent past, flush now if certain conditions
1636     //are met. Return true on first such memstore hit.
1637     for (Store s : this.getStores().values()) {
1638       if (s.timeOfOldestEdit() < now - flushCheckInterval) {
1639         // we have an old enough edit in the memstore, flush
1640         return true;
1641       }
1642     }
1643     return false;
1644   }
1645 
1646   /**
1647    * Flush the memstore.
1648    *
1649    * Flushing the memstore is a little tricky. We have a lot of updates in the
1650    * memstore, all of which have also been written to the log. We need to
1651    * write those updates in the memstore out to disk, while being able to
1652    * process reads/writes as much as possible during the flush operation. Also,
1653    * the log has to state clearly the point in time at which the memstore was
1654    * flushed. (That way, during recovery, we know when we can rely on the
1655    * on-disk flushed structures and when we have to recover the memstore from
1656    * the log.)
1657    *
1658    * <p>So, we have a three-step process:
1659    *
1660    * <ul><li>A. Flush the memstore to the on-disk stores, noting the current
1661    * sequence ID for the log.<li>
1662    *
1663    * <li>B. Write a FLUSHCACHE-COMPLETE message to the log, using the sequence
1664    * ID that was current at the time of memstore-flush.</li>
1665    *
1666    * <li>C. Get rid of the memstore structures that are now redundant, as
1667    * they've been flushed to the on-disk HStores.</li>
1668    * </ul>
1669    * <p>This method is protected, but can be accessed via several public
1670    * routes.
1671    *
1672    * <p> This method may block for some time.
1673    * @param status
1674    *
1675    * @return true if the region needs compacting
1676    *
1677    * @throws IOException general io exceptions
1678    * @throws DroppedSnapshotException Thrown when replay of hlog is required
1679    * because a Snapshot was not properly persisted.
1680    */
1681   protected FlushResult internalFlushcache(MonitoredTask status)
1682       throws IOException {
1683     return internalFlushcache(this.log, -1, status);
1684   }
1685 
1686   /**
1687    * @param wal Null if we're NOT to go via hlog/wal.
1688    * @param myseqid The seqid to use if <code>wal</code> is null writing out
1689    * flush file.
1690    * @param status
1691    * @return true if the region needs compacting
1692    * @throws IOException
1693    * @see #internalFlushcache(MonitoredTask)
1694    */
1695   protected FlushResult internalFlushcache(
1696       final HLog wal, final long myseqid, MonitoredTask status)
1697   throws IOException {
1698     if (this.rsServices != null && this.rsServices.isAborted()) {
1699       // Don't flush when server aborting, it's unsafe
1700       throw new IOException("Aborting flush because server is abortted...");
1701     }
1702     final long startTime = EnvironmentEdgeManager.currentTimeMillis();
1703     // Clear flush flag.
1704     // Record latest flush time
1705     this.lastFlushTime = startTime;
1706     // If nothing to flush, return and avoid logging start/stop flush.
1707     if (this.memstoreSize.get() <= 0) {
1708       return new FlushResult(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush");
1709     }
1710     if (LOG.isDebugEnabled()) {
1711       LOG.debug("Started memstore flush for " + this +
1712         ", current region memstore size " +
1713         StringUtils.humanReadableInt(this.memstoreSize.get()) +
1714         ((wal != null)? "": "; wal is null, using passed sequenceid=" + myseqid));
1715     }
1716 
1717     // Stop updates while we snapshot the memstore of all stores. We only have
1718     // to do this for a moment.  Its quick.  The subsequent sequence id that
1719     // goes into the HLog after we've flushed all these snapshots also goes
1720     // into the info file that sits beside the flushed files.
1721     // We also set the memstore size to zero here before we allow updates
1722     // again so its value will represent the size of the updates received
1723     // during the flush
1724     long sequenceId = -1L;
1725     long completeSequenceId = -1L;
1726     MultiVersionConsistencyControl.WriteEntry w = null;
1727 
1728     // We have to take a write lock during snapshot, or else a write could
1729     // end up in both snapshot and memstore (makes it difficult to do atomic
1730     // rows then)
1731     status.setStatus("Obtaining lock to block concurrent updates");
1732     // block waiting for the lock for internal flush
1733     this.updatesLock.writeLock().lock();
1734     long totalFlushableSize = 0;
1735     status.setStatus("Preparing to flush by snapshotting stores");
1736     List<StoreFlusher> storeFlushers = new ArrayList<StoreFlusher>(stores.size());
1737     try {
1738       // Record the mvcc for all transactions in progress.
1739       w = mvcc.beginMemstoreInsert();
1740       mvcc.advanceMemstore(w);
1741 
1742       sequenceId = (wal == null)? myseqid:
1743         wal.startCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1744       completeSequenceId = this.getCompleteCacheFlushSequenceId(sequenceId);
1745 
1746       for (Store s : stores.values()) {
1747         totalFlushableSize += s.getFlushableSize();
1748         storeFlushers.add(s.getStoreFlusher(completeSequenceId));
1749       }
1750 
1751       // prepare flush (take a snapshot)
1752       for (StoreFlusher flusher : storeFlushers) {
1753         flusher.prepare();
1754       }
1755     } finally {
1756       this.updatesLock.writeLock().unlock();
1757     }
1758     String s = "Finished snapshotting " + this +
1759       ", syncing WAL and waiting on mvcc, flushsize=" + totalFlushableSize;
1760     status.setStatus(s);
1761     LOG.debug(s);
1762 
1763     // sync unflushed WAL changes when deferred log sync is enabled
1764     // see HBASE-8208 for details
1765     if (wal != null && isDeferredLogSyncEnabled()) {
1766       wal.sync();
1767     }
1768 
1769     // wait for all in-progress transactions to commit to HLog before
1770     // we can start the flush. This prevents
1771     // uncommitted transactions from being written into HFiles.
1772     // We have to block before we start the flush, otherwise keys that
1773     // were removed via a rollbackMemstore could be written to Hfiles.
1774     mvcc.waitForRead(w);
1775 
1776     status.setStatus("Flushing stores");
1777     LOG.debug("Finished snapshotting, commencing flushing stores");
1778 
1779     // Any failure from here on out will be catastrophic requiring server
1780     // restart so hlog content can be replayed and put back into the memstore.
1781     // Otherwise, the snapshot content while backed up in the hlog, it will not
1782     // be part of the current running servers state.
1783     boolean compactionRequested = false;
1784     try {
1785       // A.  Flush memstore to all the HStores.
1786       // Keep running vector of all store files that includes both old and the
1787       // just-made new flush store file. The new flushed file is still in the
1788       // tmp directory.
1789 
1790       for (StoreFlusher flusher : storeFlushers) {
1791         flusher.flushCache(status);
1792       }
1793 
1794       // Switch snapshot (in memstore) -> new hfile (thus causing
1795       // all the store scanners to reset/reseek).
1796       for (StoreFlusher flusher : storeFlushers) {
1797         boolean needsCompaction = flusher.commit(status);
1798         if (needsCompaction) {
1799           compactionRequested = true;
1800         }
1801       }
1802       storeFlushers.clear();
1803 
1804       // Set down the memstore size by amount of flush.
1805       this.addAndGetGlobalMemstoreSize(-totalFlushableSize);
1806     } catch (Throwable t) {
1807       // An exception here means that the snapshot was not persisted.
1808       // The hlog needs to be replayed so its content is restored to memstore.
1809       // Currently, only a server restart will do this.
1810       // We used to only catch IOEs but its possible that we'd get other
1811       // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
1812       // all and sundry.
1813       if (wal != null) {
1814         wal.abortCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1815       }
1816       DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
1817           Bytes.toStringBinary(getRegionName()));
1818       dse.initCause(t);
1819       status.abort("Flush failed: " + StringUtils.stringifyException(t));
1820       throw dse;
1821     }
1822 
1823     // If we get to here, the HStores have been written. If we get an
1824     // error in completeCacheFlush it will release the lock it is holding
1825 
1826     // B.  Write a FLUSHCACHE-COMPLETE message to the log.
1827     //     This tells future readers that the HStores were emitted correctly,
1828     //     and that all updates to the log for this regionName that have lower
1829     //     log-sequence-ids can be safely ignored.
1830     if (wal != null) {
1831       wal.completeCacheFlush(this.regionInfo.getEncodedNameAsBytes(),
1832         regionInfo.getTableName(), completeSequenceId,
1833         this.getRegionInfo().isMetaRegion());
1834     }
1835 
1836     // C. Finally notify anyone waiting on memstore to clear:
1837     // e.g. checkResources().
1838     synchronized (this) {
1839       notifyAll(); // FindBugs NN_NAKED_NOTIFY
1840     }
1841 
1842     long time = EnvironmentEdgeManager.currentTimeMillis() - startTime;
1843     long memstoresize = this.memstoreSize.get();
1844     String msg = "Finished memstore flush of ~" +
1845       StringUtils.humanReadableInt(totalFlushableSize) + "/" + totalFlushableSize +
1846       ", currentsize=" +
1847       StringUtils.humanReadableInt(memstoresize) + "/" + memstoresize +
1848       " for region " + this + " in " + time + "ms, sequenceid=" + sequenceId +
1849       ", compaction requested=" + compactionRequested +
1850       ((wal == null)? "; wal=null": "");
1851     LOG.info(msg);
1852     status.setStatus(msg);
1853     this.recentFlushes.add(new Pair<Long,Long>(time/1000, totalFlushableSize));
1854 
1855     return new FlushResult(compactionRequested ? FlushResult.Result.FLUSHED_COMPACTION_NEEDED :
1856         FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, sequenceId);
1857   }
1858 
1859    /**
1860    * Get the sequence number to be associated with this cache flush. Used by
1861    * TransactionalRegion to not complete pending transactions.
1862    *
1863    *
1864    * @param currentSequenceId
1865    * @return sequence id to complete the cache flush with
1866    */
1867   protected long getCompleteCacheFlushSequenceId(long currentSequenceId) {
1868     return currentSequenceId;
1869   }
1870 
1871   //////////////////////////////////////////////////////////////////////////////
1872   // get() methods for client use.
1873   //////////////////////////////////////////////////////////////////////////////
1874   /**
1875    * Return all the data for the row that matches <i>row</i> exactly,
1876    * or the one that immediately preceeds it, at or immediately before
1877    * <i>ts</i>.
1878    *
1879    * @param row row key
1880    * @return map of values
1881    * @throws IOException
1882    */
1883   Result getClosestRowBefore(final byte [] row)
1884   throws IOException{
1885     return getClosestRowBefore(row, HConstants.CATALOG_FAMILY);
1886   }
1887 
1888   /**
1889    * Return all the data for the row that matches <i>row</i> exactly,
1890    * or the one that immediately preceeds it, at or immediately before
1891    * <i>ts</i>.
1892    *
1893    * @param row row key
1894    * @param family column family to find on
1895    * @return map of values
1896    * @throws IOException read exceptions
1897    */
1898   public Result getClosestRowBefore(final byte [] row, final byte [] family)
1899   throws IOException {
1900     if (coprocessorHost != null) {
1901       Result result = new Result();
1902       if (coprocessorHost.preGetClosestRowBefore(row, family, result)) {
1903         return result;
1904       }
1905     }
1906     // look across all the HStores for this region and determine what the
1907     // closest key is across all column families, since the data may be sparse
1908     checkRow(row, "getClosestRowBefore");
1909     startRegionOperation();
1910     this.readRequestsCount.increment();
1911     this.opMetrics.setReadRequestCountMetrics(this.readRequestsCount.get());   
1912     try {
1913       Store store = getStore(family);
1914       // get the closest key. (HStore.getRowKeyAtOrBefore can return null)
1915       KeyValue key = store.getRowKeyAtOrBefore(row);
1916       Result result = null;
1917       if (key != null) {
1918         Get get = new Get(key.getRow());
1919         get.addFamily(family);
1920         result = get(get, null);
1921       }
1922       if (coprocessorHost != null) {
1923         coprocessorHost.postGetClosestRowBefore(row, family, result);
1924       }
1925       return result;
1926     } finally {
1927       closeRegionOperation();
1928     }
1929   }
1930 
1931   /**
1932    * Return an iterator that scans over the HRegion, returning the indicated
1933    * columns and rows specified by the {@link Scan}.
1934    * <p>
1935    * This Iterator must be closed by the caller.
1936    *
1937    * @param scan configured {@link Scan}
1938    * @return RegionScanner
1939    * @throws IOException read exceptions
1940    */
1941   public RegionScanner getScanner(Scan scan) throws IOException {
1942    return getScanner(scan, null);
1943   }
1944 
1945   void prepareScanner(Scan scan) throws IOException {
1946     if(!scan.hasFamilies()) {
1947       // Adding all families to scanner
1948       for(byte[] family: this.htableDescriptor.getFamiliesKeys()){
1949         scan.addFamily(family);
1950       }
1951     }
1952   }
1953 
1954   protected RegionScanner getScanner(Scan scan,
1955       List<KeyValueScanner> additionalScanners) throws IOException {
1956     startRegionOperation();
1957     try {
1958       // Verify families are all valid
1959       prepareScanner(scan);
1960       if(scan.hasFamilies()) {
1961         for(byte [] family : scan.getFamilyMap().keySet()) {
1962           checkFamily(family);
1963         }
1964       }
1965       return instantiateRegionScanner(scan, additionalScanners);
1966     } finally {
1967       closeRegionOperation();
1968     }
1969   }
1970 
1971   protected RegionScanner instantiateRegionScanner(Scan scan,
1972       List<KeyValueScanner> additionalScanners) throws IOException {
1973     return new RegionScannerImpl(scan, additionalScanners, this);
1974   }
1975 
1976   /*
1977    * @param delete The passed delete is modified by this method. WARNING!
1978    */
1979   private void prepareDelete(Delete delete) throws IOException {
1980     // Check to see if this is a deleteRow insert
1981     if(delete.getFamilyMap().isEmpty()){
1982       for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
1983         // Don't eat the timestamp
1984         delete.deleteFamily(family, delete.getTimeStamp());
1985       }
1986     } else {
1987       for(byte [] family : delete.getFamilyMap().keySet()) {
1988         if(family == null) {
1989           throw new NoSuchColumnFamilyException("Empty family is invalid");
1990         }
1991         checkFamily(family);
1992       }
1993     }
1994   }
1995 
1996   //////////////////////////////////////////////////////////////////////////////
1997   // set() methods for client use.
1998   //////////////////////////////////////////////////////////////////////////////
1999 
2000   /**
2001    * @param delete delete object
2002    * @param writeToWAL append to the write ahead lock or not
2003    * @throws IOException read exceptions
2004    */
2005   public void delete(Delete delete, boolean writeToWAL)
2006   throws IOException {
2007     delete(delete, null, writeToWAL);
2008   }
2009 
2010   /**
2011    * @param delete delete object
2012    * @param lockid existing lock id, or null for grab a lock
2013    * @param writeToWAL append to the write ahead lock or not
2014    * @throws IOException read exceptions
2015    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
2016    */
2017   public void delete(Delete delete, Integer lockid, boolean writeToWAL)
2018   throws IOException {
2019     checkReadOnly();
2020     checkResources();
2021     Integer lid = null;
2022     startRegionOperation();
2023     this.writeRequestsCount.increment();
2024     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2025     try {
2026       byte [] row = delete.getRow();
2027       // If we did not pass an existing row lock, obtain a new one
2028       lid = getLock(lockid, row, true);
2029 
2030       try {
2031         // All edits for the given row (across all column families) must happen atomically.
2032         prepareDelete(delete);
2033         internalDelete(delete, delete.getClusterId(), writeToWAL);
2034       } finally {
2035         if(lockid == null) releaseRowLock(lid);
2036       }
2037     } finally {
2038       closeRegionOperation();
2039     }
2040   }
2041 
2042   /**
2043    * This is used only by unit tests. Not required to be a public API.
2044    * @param familyMap map of family to edits for the given family.
2045    * @param writeToWAL
2046    * @throws IOException
2047    */
2048   void delete(Map<byte[], List<KeyValue>> familyMap, UUID clusterId,
2049       boolean writeToWAL) throws IOException {
2050     Delete delete = new Delete();
2051     delete.setFamilyMap(familyMap);
2052     delete.setClusterId(clusterId);
2053     delete.setWriteToWAL(writeToWAL);
2054     internalDelete(delete, clusterId, writeToWAL);
2055   }
2056 
2057   /**
2058    * Setup correct timestamps in the KVs in Delete object.
2059    * Caller should have the row and region locks.
2060    * @param familyMap
2061    * @param now
2062    * @throws IOException
2063    */
2064   private void prepareDeleteTimestamps(Map<byte[], List<KeyValue>> familyMap, byte[] byteNow)
2065       throws IOException {
2066     for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2067 
2068       byte[] family = e.getKey();
2069       List<KeyValue> kvs = e.getValue();
2070       Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
2071 
2072       for (KeyValue kv: kvs) {
2073         //  Check if time is LATEST, change to time of most recent addition if so
2074         //  This is expensive.
2075         if (kv.isLatestTimestamp() && kv.isDeleteType()) {
2076           byte[] qual = kv.getQualifier();
2077           if (qual == null) qual = HConstants.EMPTY_BYTE_ARRAY;
2078 
2079           Integer count = kvCount.get(qual);
2080           if (count == null) {
2081             kvCount.put(qual, 1);
2082           } else {
2083             kvCount.put(qual, count + 1);
2084           }
2085           count = kvCount.get(qual);
2086 
2087           Get get = new Get(kv.getRow());
2088           get.setMaxVersions(count);
2089           get.addColumn(family, qual);
2090 
2091           List<KeyValue> result = get(get, false);
2092 
2093           if (result.size() < count) {
2094             // Nothing to delete
2095             kv.updateLatestStamp(byteNow);
2096             continue;
2097           }
2098           if (result.size() > count) {
2099             throw new RuntimeException("Unexpected size: " + result.size());
2100           }
2101           KeyValue getkv = result.get(count - 1);
2102           Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(),
2103               getkv.getBuffer(), getkv.getTimestampOffset(), Bytes.SIZEOF_LONG);
2104         } else {
2105           kv.updateLatestStamp(byteNow);
2106         }
2107       }
2108     }
2109   }
2110 
2111   /**
2112    * @param delete The Delete command
2113    * @param clusterId UUID of the originating cluster (for replication).
2114    * @param writeToWAL
2115    * @throws IOException
2116    */
2117   private void internalDelete(Delete delete, UUID clusterId,
2118       boolean writeToWAL) throws IOException {
2119     Map<byte[], List<KeyValue>> familyMap = delete.getFamilyMap();
2120     WALEdit walEdit = new WALEdit();
2121     /* Run coprocessor pre hook outside of locks to avoid deadlock */
2122     if (coprocessorHost != null) {
2123       if (coprocessorHost.preDelete(delete, walEdit, writeToWAL)) {
2124         return;
2125       }
2126     }
2127 
2128     long now = EnvironmentEdgeManager.currentTimeMillis();
2129     byte [] byteNow = Bytes.toBytes(now);
2130     boolean flush = false;
2131 
2132     lock(updatesLock.readLock());
2133     try {
2134       prepareDeleteTimestamps(delete.getFamilyMap(), byteNow);
2135 
2136       if (writeToWAL) {
2137         // write/sync to WAL should happen before we touch memstore.
2138         //
2139         // If order is reversed, i.e. we write to memstore first, and
2140         // for some reason fail to write/sync to commit log, the memstore
2141         // will contain uncommitted transactions.
2142         //
2143         // bunch up all edits across all column families into a
2144         // single WALEdit.
2145         addFamilyMapToWALEdit(familyMap, walEdit);
2146         walEdit.addClusterIds(delete.getClusterIds());
2147         this.log.append(regionInfo, this.htableDescriptor.getName(),
2148             walEdit, clusterId, now, this.htableDescriptor);
2149       }
2150 
2151       // Now make changes to the memstore.
2152       long addedSize = applyFamilyMapToMemstore(familyMap, null);
2153       flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
2154 
2155     } finally {
2156       this.updatesLock.readLock().unlock();
2157     }
2158     // do after lock
2159     if (coprocessorHost != null) {
2160       coprocessorHost.postDelete(delete, walEdit, writeToWAL);
2161     }
2162     final long after = EnvironmentEdgeManager.currentTimeMillis();
2163     this.opMetrics.updateDeleteMetrics(familyMap.keySet(), after-now);
2164 
2165     if (flush) {
2166       // Request a cache flush.  Do it outside update lock.
2167       requestFlush();
2168     }
2169   }
2170 
2171   /**
2172    * @param put
2173    * @throws IOException
2174    */
2175   public void put(Put put) throws IOException {
2176     this.put(put, null, put.getWriteToWAL());
2177   }
2178 
2179   /**
2180    * @param put
2181    * @param writeToWAL
2182    * @throws IOException
2183    */
2184   public void put(Put put, boolean writeToWAL) throws IOException {
2185     this.put(put, null, writeToWAL);
2186   }
2187 
2188   /**
2189    * @param put
2190    * @param lockid
2191    * @throws IOException
2192    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
2193    */
2194   public void put(Put put, Integer lockid) throws IOException {
2195     this.put(put, lockid, put.getWriteToWAL());
2196   }
2197 
2198 
2199 
2200   /**
2201    * @param put
2202    * @param lockid
2203    * @param writeToWAL
2204    * @throws IOException
2205    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
2206    */
2207   public void put(Put put, Integer lockid, boolean writeToWAL)
2208   throws IOException {
2209     checkReadOnly();
2210 
2211     // Do a rough check that we have resources to accept a write.  The check is
2212     // 'rough' in that between the resource check and the call to obtain a
2213     // read lock, resources may run out.  For now, the thought is that this
2214     // will be extremely rare; we'll deal with it when it happens.
2215     checkResources();
2216     startRegionOperation();
2217     this.writeRequestsCount.increment();
2218     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2219     try {
2220       // We obtain a per-row lock, so other clients will block while one client
2221       // performs an update. The read lock is released by the client calling
2222       // #commit or #abort or if the HRegionServer lease on the lock expires.
2223       // See HRegionServer#RegionListener for how the expire on HRegionServer
2224       // invokes a HRegion#abort.
2225       byte [] row = put.getRow();
2226       // If we did not pass an existing row lock, obtain a new one
2227       Integer lid = getLock(lockid, row, true);
2228 
2229       try {
2230         // All edits for the given row (across all column families) must happen atomically.
2231         internalPut(put, put.getClusterId(), writeToWAL);
2232       } finally {
2233         if(lockid == null) releaseRowLock(lid);
2234       }
2235     } finally {
2236       closeRegionOperation();
2237     }
2238   }
2239 
2240   /**
2241    * Struct-like class that tracks the progress of a batch operation,
2242    * accumulating status codes and tracking the index at which processing
2243    * is proceeding.
2244    */
2245   private static class BatchOperationInProgress<T> {
2246     T[] operations;
2247     int nextIndexToProcess = 0;
2248     OperationStatus[] retCodeDetails;
2249     WALEdit[] walEditsFromCoprocessors;
2250 
2251     public BatchOperationInProgress(T[] operations) {
2252       this.operations = operations;
2253       this.retCodeDetails = new OperationStatus[operations.length];
2254       this.walEditsFromCoprocessors = new WALEdit[operations.length];
2255       Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
2256     }
2257 
2258     public boolean isDone() {
2259       return nextIndexToProcess == operations.length;
2260     }
2261   }
2262 
2263   /**
2264    * Perform a batch put with no pre-specified locks
2265    * @see HRegion#batchMutate(Pair[])
2266    */
2267   public OperationStatus[] put(Put[] puts) throws IOException {
2268     @SuppressWarnings("unchecked")
2269     Pair<Mutation, Integer> putsAndLocks[] = new Pair[puts.length];
2270 
2271     for (int i = 0; i < puts.length; i++) {
2272       putsAndLocks[i] = new Pair<Mutation, Integer>(puts[i], null);
2273     }
2274     return batchMutate(putsAndLocks);
2275   }
2276 
2277   /**
2278    * Perform a batch of puts.
2279    * @param putsAndLocks
2280    *          the list of puts paired with their requested lock IDs.
2281    * @return an array of OperationStatus which internally contains the OperationStatusCode and the
2282    *         exceptionMessage if any.
2283    * @throws IOException
2284    * @deprecated Instead use {@link HRegion#batchMutate(Pair[])}
2285    */
2286   @Deprecated
2287   public OperationStatus[] put(Pair<Put, Integer>[] putsAndLocks) throws IOException {
2288     Pair<Mutation, Integer>[] mutationsAndLocks = new Pair[putsAndLocks.length];
2289     System.arraycopy(putsAndLocks, 0, mutationsAndLocks, 0, putsAndLocks.length);
2290     return batchMutate(mutationsAndLocks);
2291   }
2292 
2293   /**
2294    * Perform a batch of mutations.
2295    * It supports only Put and Delete mutations and will ignore other types passed.
2296    * @param mutationsAndLocks
2297    *          the list of mutations paired with their requested lock IDs.
2298    * @return an array of OperationStatus which internally contains the
2299    *         OperationStatusCode and the exceptionMessage if any.
2300    * @throws IOException
2301    */
2302   public OperationStatus[] batchMutate(
2303       Pair<Mutation, Integer>[] mutationsAndLocks) throws IOException {
2304     BatchOperationInProgress<Pair<Mutation, Integer>> batchOp =
2305       new BatchOperationInProgress<Pair<Mutation,Integer>>(mutationsAndLocks);
2306 
2307     boolean initialized = false;
2308 
2309     while (!batchOp.isDone()) {
2310       checkReadOnly();
2311       checkResources();
2312 
2313       long newSize;
2314       startRegionOperation();
2315 
2316       try {
2317         if (!initialized) {
2318           this.writeRequestsCount.increment();
2319           this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2320           doPreMutationHook(batchOp);
2321           initialized = true;
2322         }
2323         long addedSize = doMiniBatchMutation(batchOp);
2324         newSize = this.addAndGetGlobalMemstoreSize(addedSize);
2325       } finally {
2326         closeRegionOperation();
2327       }
2328       if (isFlushSize(newSize)) {
2329         requestFlush();
2330       }
2331     }
2332     return batchOp.retCodeDetails;
2333   }
2334 
2335   private void doPreMutationHook(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp)
2336       throws IOException {
2337     /* Run coprocessor pre hook outside of locks to avoid deadlock */
2338     WALEdit walEdit = new WALEdit();
2339     if (coprocessorHost != null) {
2340       for (int i = 0; i < batchOp.operations.length; i++) {
2341         Pair<Mutation, Integer> nextPair = batchOp.operations[i];
2342         Mutation m = nextPair.getFirst();
2343         if (m instanceof Put) {
2344           if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
2345             // pre hook says skip this Put
2346             // mark as success and skip in doMiniBatchMutation
2347             batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2348           }
2349         } else if (m instanceof Delete) {
2350           if (coprocessorHost.preDelete((Delete) m, walEdit, m.getWriteToWAL())) {
2351             // pre hook says skip this Delete
2352             // mark as success and skip in doMiniBatchMutation
2353             batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2354           }
2355         } else {
2356           // In case of passing Append mutations along with the Puts and Deletes in batchMutate
2357           // mark the operation return code as failure so that it will not be considered in
2358           // the doMiniBatchMutation
2359           batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.FAILURE,
2360               "Put/Delete mutations only supported in batchMutate() now");
2361         }
2362         if (!walEdit.isEmpty()) {
2363           batchOp.walEditsFromCoprocessors[i] = walEdit;
2364           walEdit = new WALEdit();
2365         }
2366       }
2367     }
2368   }
2369 
2370   // The mutation will be either a Put or Delete.
2371   @SuppressWarnings("unchecked")
2372   private long doMiniBatchMutation(
2373       BatchOperationInProgress<Pair<Mutation, Integer>> batchOp) throws IOException {
2374 
2375     // The set of columnFamilies first seen for Put.
2376     Set<byte[]> putsCfSet = null;
2377     // variable to note if all Put items are for the same CF -- metrics related
2378     boolean putsCfSetConsistent = true;
2379     // The set of columnFamilies first seen for Delete.
2380     Set<byte[]> deletesCfSet = null;
2381     // variable to note if all Delete items are for the same CF -- metrics related
2382     boolean deletesCfSetConsistent = true;
2383     long startTimeMs = EnvironmentEdgeManager.currentTimeMillis();
2384 
2385     WALEdit walEdit = new WALEdit();
2386 
2387     MultiVersionConsistencyControl.WriteEntry w = null;
2388     long txid = 0;
2389     boolean doRollBackMemstore = false;
2390     boolean locked = false;
2391 
2392     /** Keep track of the locks we hold so we can release them in finally clause */
2393     List<Integer> acquiredLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
2394     Set<HashedBytes> rowsAlreadyLocked = Sets.newHashSet();
2395 
2396     // reference family maps directly so coprocessors can mutate them if desired
2397     Map<byte[],List<KeyValue>>[] familyMaps = new Map[batchOp.operations.length];
2398     // We try to set up a batch in the range [firstIndex,lastIndexExclusive)
2399     int firstIndex = batchOp.nextIndexToProcess;
2400     int lastIndexExclusive = firstIndex;
2401     boolean success = false;
2402     int noOfPuts = 0, noOfDeletes = 0;
2403     try {
2404       // ------------------------------------
2405       // STEP 1. Try to acquire as many locks as we can, and ensure
2406       // we acquire at least one.
2407       // ----------------------------------
2408       int numReadyToWrite = 0;
2409       long now = EnvironmentEdgeManager.currentTimeMillis();
2410       while (lastIndexExclusive < batchOp.operations.length) {
2411         Pair<Mutation, Integer> nextPair = batchOp.operations[lastIndexExclusive];
2412         Mutation mutation = nextPair.getFirst();
2413         Integer providedLockId = nextPair.getSecond();
2414 
2415         Map<byte[], List<KeyValue>> familyMap = mutation.getFamilyMap();
2416         // store the family map reference to allow for mutations
2417         familyMaps[lastIndexExclusive] = familyMap;
2418 
2419         // skip anything that "ran" already
2420         if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
2421             != OperationStatusCode.NOT_RUN) {
2422           lastIndexExclusive++;
2423           continue;
2424         }
2425 
2426         try {
2427           if (mutation instanceof Put) {
2428             checkFamilies(familyMap.keySet());
2429             checkTimestamps(mutation.getFamilyMap(), now);
2430           } else {
2431             prepareDelete((Delete) mutation);
2432           }
2433         } catch (NoSuchColumnFamilyException nscf) {
2434           LOG.warn("No such column family in batch mutation", nscf);
2435           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
2436               OperationStatusCode.BAD_FAMILY, nscf.getMessage());
2437           lastIndexExclusive++;
2438           continue;
2439         } catch (DoNotRetryIOException fsce) {
2440           // The only thing that throws a generic DoNotRetryIOException in the above code is
2441           // checkTimestamps so that DoNotRetryIOException means that timestamps were invalid.
2442           // If more checks are added, be sure to revisit this assumption.
2443           LOG.warn("Batch Mutation did not pass sanity check", fsce);
2444           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
2445               OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
2446           lastIndexExclusive++;
2447           continue;
2448         }
2449         // If we haven't got any rows in our batch, we should block to
2450         // get the next one.
2451         boolean shouldBlock = numReadyToWrite == 0;
2452         boolean failedToAcquire = false;
2453         Integer acquiredLockId = null;
2454         HashedBytes currentRow = new HashedBytes(mutation.getRow());
2455         try {
2456           if (providedLockId != null || !rowsAlreadyLocked.contains(currentRow)) {
2457             acquiredLockId = getLock(providedLockId, currentRow, shouldBlock);
2458             if (acquiredLockId == null) {
2459               failedToAcquire = true;
2460             } else if (providedLockId == null) {
2461               rowsAlreadyLocked.add(currentRow);
2462             }
2463           } 
2464         } catch (IOException ioe) {
2465           LOG.warn("Failed getting lock in batch put, row=" + currentRow, ioe);
2466           failedToAcquire = true;
2467         }
2468         if (failedToAcquire) {
2469           // We failed to grab another lock
2470           assert !shouldBlock : "Should never fail to get lock when blocking";
2471           break; // stop acquiring more rows for this batch
2472         }
2473         if (providedLockId == null) {
2474           acquiredLocks.add(acquiredLockId);
2475         }
2476         lastIndexExclusive++;
2477         numReadyToWrite++;
2478 
2479         if (mutation instanceof Put) {
2480           // If Column Families stay consistent through out all of the
2481           // individual puts then metrics can be reported as a mutliput across
2482           // column families in the first put.
2483           if (putsCfSet == null) {
2484             putsCfSet = mutation.getFamilyMap().keySet();
2485           } else {
2486             putsCfSetConsistent = putsCfSetConsistent
2487                 && mutation.getFamilyMap().keySet().equals(putsCfSet);
2488           }
2489         } else {
2490           if (deletesCfSet == null) {
2491             deletesCfSet = mutation.getFamilyMap().keySet();
2492           } else {
2493             deletesCfSetConsistent = deletesCfSetConsistent
2494                 && mutation.getFamilyMap().keySet().equals(deletesCfSet);
2495           }
2496         }
2497       }
2498 
2499       // we should record the timestamp only after we have acquired the rowLock,
2500       // otherwise, newer puts/deletes are not guaranteed to have a newer timestamp
2501       now = EnvironmentEdgeManager.currentTimeMillis();
2502       byte[] byteNow = Bytes.toBytes(now);
2503 
2504       // Nothing to put/delete -- an exception in the above such as NoSuchColumnFamily?
2505       if (numReadyToWrite <= 0) return 0L;
2506 
2507       // We've now grabbed as many mutations off the list as we can
2508 
2509       // ------------------------------------
2510       // STEP 2. Update any LATEST_TIMESTAMP timestamps
2511       // ----------------------------------
2512       for (int i = firstIndex; i < lastIndexExclusive; i++) {
2513         // skip invalid
2514         if (batchOp.retCodeDetails[i].getOperationStatusCode()
2515             != OperationStatusCode.NOT_RUN) continue;
2516         Mutation mutation = batchOp.operations[i].getFirst();
2517         if (mutation instanceof Put) {
2518           updateKVTimestamps(familyMaps[i].values(), byteNow);
2519           noOfPuts++;
2520         } else {
2521           prepareDeleteTimestamps(familyMaps[i], byteNow);
2522           noOfDeletes++;
2523         }
2524       }
2525 
2526       lock(this.updatesLock.readLock(), numReadyToWrite);
2527       locked = true;
2528 
2529       //
2530       // ------------------------------------
2531       // Acquire the latest mvcc number
2532       // ----------------------------------
2533       w = mvcc.beginMemstoreInsert();
2534 
2535       // calling the pre CP hook for batch mutation
2536       if (coprocessorHost != null) {
2537         MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp = 
2538           new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations, 
2539           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
2540         if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
2541       }
2542 
2543       // ------------------------------------
2544       // STEP 3. Write back to memstore
2545       // Write to memstore. It is ok to write to memstore
2546       // first without updating the HLog because we do not roll
2547       // forward the memstore MVCC. The MVCC will be moved up when
2548       // the complete operation is done. These changes are not yet
2549       // visible to scanners till we update the MVCC. The MVCC is
2550       // moved only when the sync is complete.
2551       // ----------------------------------
2552       long addedSize = 0;
2553       for (int i = firstIndex; i < lastIndexExclusive; i++) {
2554         if (batchOp.retCodeDetails[i].getOperationStatusCode()
2555             != OperationStatusCode.NOT_RUN) {
2556           continue;
2557         }
2558         doRollBackMemstore = true;
2559         addedSize += applyFamilyMapToMemstore(familyMaps[i], w);
2560       }
2561 
2562       // ------------------------------------
2563       // STEP 4. Build WAL edit
2564       // ----------------------------------
2565       Durability durability = Durability.USE_DEFAULT;
2566 
2567       for (int i = firstIndex; i < lastIndexExclusive; i++) {
2568         // Skip puts that were determined to be invalid during preprocessing
2569         if (batchOp.retCodeDetails[i].getOperationStatusCode()
2570             != OperationStatusCode.NOT_RUN) {
2571           continue;
2572         }
2573         batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2574 
2575         Mutation m = batchOp.operations[i].getFirst();
2576         Durability tmpDur = m.getDurability(); 
2577         if (tmpDur.ordinal() > durability.ordinal()) {
2578           durability = tmpDur;
2579         }
2580         if (tmpDur == Durability.SKIP_WAL) {
2581           if (m instanceof Put) {
2582             recordPutWithoutWal(m.getFamilyMap());
2583           }
2584           continue;
2585         }
2586 
2587         // Add WAL edits by CP
2588         WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
2589         if (fromCP != null) {
2590           for (KeyValue kv : fromCP.getKeyValues()) {
2591             walEdit.add(kv);
2592           }
2593         }
2594         addFamilyMapToWALEdit(familyMaps[i], walEdit);
2595       }
2596 
2597       // -------------------------
2598       // STEP 5. Append the edit to WAL. Do not sync wal.
2599       // -------------------------
2600       Mutation first = batchOp.operations[firstIndex].getFirst();
2601       walEdit.addClusterIds(first.getClusterIds());
2602       txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
2603                walEdit, first.getClusterId(), now, this.htableDescriptor);
2604 
2605       // -------------------------------
2606       // STEP 6. Release row locks, etc.
2607       // -------------------------------
2608       if (locked) {
2609         this.updatesLock.readLock().unlock();
2610         locked = false;
2611       }
2612       if (acquiredLocks != null) {
2613         for (Integer toRelease : acquiredLocks) {
2614           releaseRowLock(toRelease);
2615         }
2616         acquiredLocks = null;
2617         rowsAlreadyLocked = null;
2618       }
2619       // -------------------------
2620       // STEP 7. Sync wal.
2621       // -------------------------
2622       if (walEdit.size() > 0) {
2623         syncOrDefer(txid, durability);
2624       }
2625       doRollBackMemstore = false;
2626       // calling the post CP hook for batch mutation
2627       if (coprocessorHost != null) {
2628         MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp = 
2629           new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations, 
2630           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
2631         coprocessorHost.postBatchMutate(miniBatchOp);
2632       }
2633       
2634       // ------------------------------------------------------------------
2635       // STEP 8. Advance mvcc. This will make this put visible to scanners and getters.
2636       // ------------------------------------------------------------------
2637       if (w != null) {
2638         mvcc.completeMemstoreInsert(w);
2639         w = null;
2640       }
2641 
2642       // ------------------------------------
2643       // STEP 9. Run coprocessor post hooks. This should be done after the wal is
2644       // synced so that the coprocessor contract is adhered to.
2645       // ------------------------------------
2646       if (coprocessorHost != null) {
2647         for (int i = firstIndex; i < lastIndexExclusive; i++) {
2648           // only for successful puts
2649           if (batchOp.retCodeDetails[i].getOperationStatusCode()
2650               != OperationStatusCode.SUCCESS) {
2651             continue;
2652           }
2653           Mutation m = batchOp.operations[i].getFirst();
2654           if (m instanceof Put) {
2655             coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
2656           } else {
2657             coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
2658           }
2659         }
2660       }
2661       success = true;
2662       return addedSize;
2663     } finally {
2664 
2665       // if the wal sync was unsuccessful, remove keys from memstore
2666       if (doRollBackMemstore) {
2667         rollbackMemstore(batchOp, familyMaps, firstIndex, lastIndexExclusive);
2668       }
2669       if (w != null) mvcc.completeMemstoreInsert(w);
2670 
2671       if (locked) {
2672         this.updatesLock.readLock().unlock();
2673       }
2674 
2675       if (acquiredLocks != null) {
2676         for (Integer toRelease : acquiredLocks) {
2677           releaseRowLock(toRelease);
2678         }
2679       }
2680 
2681       // do after lock
2682       final long netTimeMs = EnvironmentEdgeManager.currentTimeMillis()- startTimeMs;
2683 
2684       // See if the column families were consistent through the whole thing.
2685       // if they were then keep them. If they were not then pass a null.
2686       // null will be treated as unknown.
2687       // Total time taken might be involving Puts and Deletes.
2688       // Split the time for puts and deletes based on the total number of Puts and Deletes.
2689       long timeTakenForPuts = 0;
2690       if (noOfPuts > 0) {
2691         // There were some Puts in the batch.
2692         double noOfMutations = noOfPuts + noOfDeletes;
2693         timeTakenForPuts = (long) (netTimeMs * (noOfPuts / noOfMutations));
2694         final Set<byte[]> keptCfs = putsCfSetConsistent ? putsCfSet : null;
2695         this.opMetrics.updateMultiPutMetrics(keptCfs, timeTakenForPuts);
2696       }
2697       if (noOfDeletes > 0) {
2698         // There were some Deletes in the batch.
2699         final Set<byte[]> keptCfs = deletesCfSetConsistent ? deletesCfSet : null;
2700         this.opMetrics.updateMultiDeleteMetrics(keptCfs, netTimeMs - timeTakenForPuts);
2701       }
2702       if (!success) {
2703         for (int i = firstIndex; i < lastIndexExclusive; i++) {
2704           if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
2705             batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
2706           }
2707         }
2708       }
2709       batchOp.nextIndexToProcess = lastIndexExclusive;
2710     }
2711   }
2712 
2713   //TODO, Think that gets/puts and deletes should be refactored a bit so that
2714   //the getting of the lock happens before, so that you would just pass it into
2715   //the methods. So in the case of checkAndMutate you could just do lockRow,
2716   //get, put, unlockRow or something
2717  /**
2718   *
2719   * @param row
2720   * @param family
2721   * @param qualifier
2722   * @param compareOp
2723   * @param comparator
2724   * @param writeToWAL
2725   * @throws IOException
2726   * @return true if the new put was execute, false otherwise
2727   */
2728  public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2729      CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2730      boolean writeToWAL)
2731  throws IOException {
2732    return checkAndMutate(row, family, qualifier, compareOp, comparator, w, null, writeToWAL);
2733  }
2734   
2735   /**
2736    *
2737    * @param row
2738    * @param family
2739    * @param qualifier
2740    * @param compareOp
2741    * @param comparator
2742    * @param lockId
2743    * @param writeToWAL
2744    * @throws IOException
2745    * @return true if the new put was execute, false otherwise
2746    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
2747    */
2748   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2749       CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2750       Integer lockId, boolean writeToWAL)
2751   throws IOException{
2752     checkReadOnly();
2753     //TODO, add check for value length or maybe even better move this to the
2754     //client if this becomes a global setting
2755     checkResources();
2756     boolean isPut = w instanceof Put;
2757     if (!isPut && !(w instanceof Delete))
2758       throw new DoNotRetryIOException("Action must be Put or Delete");
2759     Row r = (Row)w;
2760     if (!Bytes.equals(row, r.getRow())) {
2761       throw new DoNotRetryIOException("Action's getRow must match the passed row");
2762     }
2763 
2764     startRegionOperation();
2765     this.writeRequestsCount.increment();
2766     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2767     try {
2768       RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
2769       Get get = new Get(row, lock);
2770       checkFamily(family);
2771       get.addColumn(family, qualifier);
2772 
2773       // Lock row
2774       Integer lid = getLock(lockId, get.getRow(), true);
2775       // wait for all previous transactions to complete (with lock held)
2776       mvcc.completeMemstoreInsert(mvcc.beginMemstoreInsert());
2777       List<KeyValue> result = new ArrayList<KeyValue>();
2778       try {
2779         result = get(get, false);
2780 
2781         boolean valueIsNull = comparator.getValue() == null ||
2782           comparator.getValue().length == 0;
2783         boolean matches = false;
2784         if (result.size() == 0 && valueIsNull) {
2785           matches = true;
2786         } else if (result.size() > 0 && result.get(0).getValue().length == 0 &&
2787             valueIsNull) {
2788           matches = true;
2789         } else if (result.size() == 1 && !valueIsNull) {
2790           KeyValue kv = result.get(0);
2791           int compareResult = comparator.compareTo(kv.getBuffer(),
2792               kv.getValueOffset(), kv.getValueLength());
2793           switch (compareOp) {
2794           case LESS:
2795             matches = compareResult < 0;
2796             break;
2797           case LESS_OR_EQUAL:
2798             matches = compareResult <= 0;
2799             break;
2800           case EQUAL:
2801             matches = compareResult == 0;
2802             break;
2803           case NOT_EQUAL:
2804             matches = compareResult != 0;
2805             break;
2806           case GREATER_OR_EQUAL:
2807             matches = compareResult >= 0;
2808             break;
2809           case GREATER:
2810             matches = compareResult > 0;
2811             break;
2812           default:
2813             throw new RuntimeException("Unknown Compare op " + compareOp.name());
2814           }
2815         }
2816         //If matches put the new put or delete the new delete
2817         if (matches) {
2818           // All edits for the given row (across all column families) must
2819           // happen atomically.
2820           //
2821           // Using default cluster id, as this can only happen in the
2822           // originating cluster. A slave cluster receives the result as a Put
2823           // or Delete
2824           if (isPut) {
2825             internalPut(((Put) w), HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2826           } else {
2827             Delete d = (Delete)w;
2828             prepareDelete(d);
2829             internalDelete(d, HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2830           }
2831           return true;
2832         }
2833         return false;
2834       } finally {
2835         if(lockId == null) releaseRowLock(lid);
2836       }
2837     } finally {
2838       closeRegionOperation();
2839     }
2840   }
2841 
2842 
2843   /**
2844    * Complete taking the snapshot on the region. Writes the region info and adds references to the
2845    * working snapshot directory.
2846    *
2847    * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
2848    * arg.  (In the future other cancellable HRegion methods could eventually add a
2849    * {@link ForeignExceptionSnare}, or we could do something fancier).
2850    *
2851    * @param desc snasphot description object
2852    * @param exnSnare ForeignExceptionSnare that captures external exeptions in case we need to
2853    *   bail out.  This is allowed to be null and will just be ignored in that case.
2854    * @throws IOException if there is an external or internal error causing the snapshot to fail
2855    */
2856   public void addRegionToSnapshot(SnapshotDescription desc,
2857       ForeignExceptionSnare exnSnare) throws IOException {
2858     // This should be "fast" since we don't rewrite store files but instead
2859     // back up the store files by creating a reference
2860     Path rootDir = FSUtils.getRootDir(this.rsServices.getConfiguration());
2861     Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(desc, rootDir,
2862       regionInfo.getEncodedName());
2863 
2864     // 1. dump region meta info into the snapshot directory
2865     LOG.debug("Storing region-info for snapshot.");
2866     checkRegioninfoOnFilesystem(snapshotRegionDir);
2867 
2868     // 2. iterate through all the stores in the region
2869     LOG.debug("Creating references for hfiles");
2870 
2871     // This ensures that we have an atomic view of the directory as long as we have < ls limit
2872     // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files in
2873     // batches and may miss files being added/deleted. This could be more robust (iteratively
2874     // checking to see if we have all the files until we are sure), but the limit is currently 1000
2875     // files/batch, far more than the number of store files under a single column family.
2876     for (Store store : stores.values()) {
2877       // 2.1. build the snapshot reference directory for the store
2878       Path dstStoreDir = TakeSnapshotUtils.getStoreSnapshotDirectory(snapshotRegionDir,
2879         Bytes.toString(store.getFamily().getName()));
2880       List<StoreFile> storeFiles = store.getStorefiles();
2881       if (LOG.isDebugEnabled()) {
2882         LOG.debug("Adding snapshot references for " + storeFiles  + " hfiles");
2883       }
2884 
2885       // 2.2. iterate through all the store's files and create "references".
2886       int sz = storeFiles.size();
2887       for (int i = 0; i < sz; i++) {
2888         if (exnSnare != null) {
2889           exnSnare.rethrowException();
2890         }
2891         StoreFile storeFile = storeFiles.get(i);
2892         Path file = storeFile.getPath();
2893 
2894         LOG.debug("Creating reference for file (" + (i+1) + "/" + sz + ") : " + file);
2895         Path referenceFile = new Path(dstStoreDir, file.getName());
2896         boolean success = true;
2897         if (storeFile.isReference()) {
2898           // write the Reference object to the snapshot
2899           storeFile.getReference().write(fs, referenceFile);
2900         } else {
2901           // create "reference" to this store file.  It is intentionally an empty file -- all
2902           // necessary information is captured by its fs location and filename.  This allows us to
2903           // only figure out what needs to be done via a single nn operation (instead of having to
2904           // open and read the files as well).
2905           success = HBaseFileSystem.createNewFileOnFileSystem(fs, referenceFile);
2906         }
2907         if (!success) {
2908           throw new IOException("Failed to create reference file:" + referenceFile);
2909         }
2910       }
2911     }
2912   }
2913 
2914   /**
2915    * Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP} with the provided current
2916    * timestamp.
2917    */
2918   private void updateKVTimestamps(
2919       final Iterable<List<KeyValue>> keyLists, final byte[] now) {
2920     for (List<KeyValue> keys: keyLists) {
2921       if (keys == null) continue;
2922       for (KeyValue key : keys) {
2923         key.updateLatestStamp(now);
2924       }
2925     }
2926   }
2927 
2928   /*
2929    * Check if resources to support an update.
2930    *
2931    * Here we synchronize on HRegion, a broad scoped lock.  Its appropriate
2932    * given we're figuring in here whether this region is able to take on
2933    * writes.  This is only method with a synchronize (at time of writing),
2934    * this and the synchronize on 'this' inside in internalFlushCache to send
2935    * the notify.
2936    */
2937   private void checkResources()
2938       throws RegionTooBusyException, InterruptedIOException {
2939 
2940     // If catalog region, do not impose resource constraints or block updates.
2941     if (this.getRegionInfo().isMetaRegion()) return;
2942 
2943     boolean blocked = false;
2944     long startTime = 0;
2945     while (this.memstoreSize.get() > this.blockingMemStoreSize) {
2946       requestFlush();
2947       if (!blocked) {
2948         startTime = EnvironmentEdgeManager.currentTimeMillis();
2949         LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
2950           "' on region " + Bytes.toStringBinary(getRegionName()) +
2951           ": memstore size " +
2952           StringUtils.humanReadableInt(this.memstoreSize.get()) +
2953           " is >= than blocking " +
2954           StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
2955       }
2956       long now = EnvironmentEdgeManager.currentTimeMillis();
2957       long timeToWait = startTime + busyWaitDuration - now;
2958       if (timeToWait <= 0L) {
2959         final long totalTime = now - startTime;
2960         this.updatesBlockedMs.add(totalTime);
2961         LOG.info("Failed to unblock updates for region " + this + " '"
2962           + Thread.currentThread().getName() + "' in " + totalTime
2963           + "ms. The region is still busy.");
2964         throw new RegionTooBusyException("region is flushing");
2965       }
2966       blocked = true;
2967       synchronized(this) {
2968         try {
2969           wait(Math.min(timeToWait, threadWakeFrequency));
2970         } catch (InterruptedException ie) {
2971           final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2972           if (totalTime > 0) {
2973             this.updatesBlockedMs.add(totalTime);
2974           }
2975           LOG.info("Interrupted while waiting to unblock updates for region "
2976             + this + " '" + Thread.currentThread().getName() + "'");
2977           InterruptedIOException iie = new InterruptedIOException();
2978           iie.initCause(ie);
2979           throw iie;
2980         }
2981       }
2982     }
2983     if (blocked) {
2984       // Add in the blocked time if appropriate
2985       final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2986       if(totalTime > 0 ){
2987         this.updatesBlockedMs.add(totalTime);
2988       }
2989       LOG.info("Unblocking updates for region " + this + " '"
2990           + Thread.currentThread().getName() + "'");
2991     }
2992   }
2993 
2994   /**
2995    * @throws IOException Throws exception if region is in read-only mode.
2996    */
2997   protected void checkReadOnly() throws IOException {
2998     if (this.writestate.isReadOnly()) {
2999       throw new IOException("region is read only");
3000     }
3001   }
3002 
3003   /**
3004    * Add updates first to the hlog and then add values to memstore.
3005    * Warning: Assumption is caller has lock on passed in row.
3006    * @param family
3007    * @param edits Cell updates by column
3008    * @praram now
3009    * @throws IOException
3010    */
3011   private void put(byte [] family, List<KeyValue> edits)
3012   throws IOException {
3013     Map<byte[], List<KeyValue>> familyMap;
3014     familyMap = new HashMap<byte[], List<KeyValue>>();
3015 
3016     familyMap.put(family, edits);
3017     Put p = new Put();
3018     p.setFamilyMap(familyMap);
3019     p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
3020     p.setWriteToWAL(true);
3021     this.internalPut(p, HConstants.DEFAULT_CLUSTER_ID, true);
3022   }
3023 
3024   /**
3025    * Add updates first to the hlog (if writeToWal) and then add values to memstore.
3026    * Warning: Assumption is caller has lock on passed in row.
3027    * @param put The Put command
3028    * @param clusterId UUID of the originating cluster (for replication).
3029    * @param writeToWAL if true, then we should write to the log
3030    * @throws IOException
3031    */
3032   private void internalPut(Put put, UUID clusterId, boolean writeToWAL) throws IOException {
3033     Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
3034     WALEdit walEdit = new WALEdit();
3035     /* run pre put hook outside of lock to avoid deadlock */
3036     if (coprocessorHost != null) {
3037       if (coprocessorHost.prePut(put, walEdit, writeToWAL)) {
3038         return;
3039       }
3040     }
3041 
3042     long now = EnvironmentEdgeManager.currentTimeMillis();
3043     byte[] byteNow = Bytes.toBytes(now);
3044     boolean flush = false;
3045 
3046     lock(this.updatesLock.readLock());
3047     try {
3048       checkFamilies(familyMap.keySet());
3049       checkTimestamps(familyMap, now);
3050       updateKVTimestamps(familyMap.values(), byteNow);
3051       // write/sync to WAL should happen before we touch memstore.
3052       //
3053       // If order is reversed, i.e. we write to memstore first, and
3054       // for some reason fail to write/sync to commit log, the memstore
3055       // will contain uncommitted transactions.
3056       if (writeToWAL) {
3057         addFamilyMapToWALEdit(familyMap, walEdit);
3058         walEdit.addClusterIds(put.getClusterIds());
3059         this.log.append(regionInfo, this.htableDescriptor.getName(),
3060             walEdit, clusterId, now, this.htableDescriptor);
3061       } else {
3062         recordPutWithoutWal(familyMap);
3063       }
3064 
3065       long addedSize = applyFamilyMapToMemstore(familyMap, null);
3066       flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
3067     } finally {
3068       this.updatesLock.readLock().unlock();
3069     }
3070 
3071     if (coprocessorHost != null) {
3072       coprocessorHost.postPut(put, walEdit, writeToWAL);
3073     }
3074 
3075     // do after lock
3076     final long after = EnvironmentEdgeManager.currentTimeMillis();
3077     this.opMetrics.updatePutMetrics(familyMap.keySet(), after - now);
3078 
3079     if (flush) {
3080       // Request a cache flush.  Do it outside update lock.
3081       requestFlush();
3082     }
3083   }
3084 
3085   /**
3086    * Atomically apply the given map of family->edits to the memstore.
3087    * This handles the consistency control on its own, but the caller
3088    * should already have locked updatesLock.readLock(). This also does
3089    * <b>not</b> check the families for validity.
3090    *
3091    * @param familyMap Map of kvs per family
3092    * @param localizedWriteEntry The WriteEntry of the MVCC for this transaction.
3093    *        If null, then this method internally creates a mvcc transaction.
3094    * @return the additional memory usage of the memstore caused by the
3095    * new entries.
3096    */
3097   private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap,
3098     MultiVersionConsistencyControl.WriteEntry localizedWriteEntry) {
3099     long size = 0;
3100     boolean freemvcc = false;
3101 
3102     try {
3103       if (localizedWriteEntry == null) {
3104         localizedWriteEntry = mvcc.beginMemstoreInsert();
3105         freemvcc = true;
3106       }
3107 
3108       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
3109         byte[] family = e.getKey();
3110         List<KeyValue> edits = e.getValue();
3111 
3112         Store store = getStore(family);
3113         for (KeyValue kv: edits) {
3114           kv.setMemstoreTS(localizedWriteEntry.getWriteNumber());
3115           size += store.add(kv);
3116         }
3117       }
3118     } finally {
3119       if (freemvcc) {
3120         mvcc.completeMemstoreInsert(localizedWriteEntry);
3121       }
3122     }
3123 
3124      return size;
3125    }
3126 
3127   /**
3128    * Remove all the keys listed in the map from the memstore. This method is
3129    * called when a Put/Delete has updated memstore but subequently fails to update
3130    * the wal. This method is then invoked to rollback the memstore.
3131    */
3132   private void rollbackMemstore(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp,
3133                                 Map<byte[], List<KeyValue>>[] familyMaps,
3134                                 int start, int end) {
3135     int kvsRolledback = 0;
3136     for (int i = start; i < end; i++) {
3137       // skip over request that never succeeded in the first place.
3138       if (batchOp.retCodeDetails[i].getOperationStatusCode()
3139             != OperationStatusCode.SUCCESS) {
3140         continue;
3141       }
3142 
3143       // Rollback all the kvs for this row.
3144       Map<byte[], List<KeyValue>> familyMap  = familyMaps[i];
3145       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
3146         byte[] family = e.getKey();
3147         List<KeyValue> edits = e.getValue();
3148 
3149         // Remove those keys from the memstore that matches our
3150         // key's (row, cf, cq, timestamp, memstoreTS). The interesting part is
3151         // that even the memstoreTS has to match for keys that will be rolleded-back.
3152         Store store = getStore(family);
3153         for (KeyValue kv: edits) {
3154           store.rollback(kv);
3155           kvsRolledback++;
3156         }
3157       }
3158     }
3159     LOG.debug("rollbackMemstore rolled back " + kvsRolledback +
3160         " keyvalues from start:" + start + " to end:" + end);
3161   }
3162 
3163   /**
3164    * Check the collection of families for validity.
3165    * @throws NoSuchColumnFamilyException if a family does not exist.
3166    */
3167   private void checkFamilies(Collection<byte[]> families)
3168   throws NoSuchColumnFamilyException {
3169     for (byte[] family : families) {
3170       checkFamily(family);
3171     }
3172   }
3173 
3174   private void checkTimestamps(final Map<byte[], List<KeyValue>> familyMap,
3175       long now) throws DoNotRetryIOException {
3176     if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
3177       return;
3178     }
3179     long maxTs = now + timestampSlop;
3180     for (List<KeyValue> kvs : familyMap.values()) {
3181       for (KeyValue kv : kvs) {
3182         // see if the user-side TS is out of range. latest = server-side
3183         if (!kv.isLatestTimestamp() && kv.getTimestamp() > maxTs) {
3184           throw new DoNotRetryIOException("Timestamp for KV out of range "
3185               + kv + " (too.new=" + timestampSlop + ")");
3186         }
3187       }
3188     }
3189   }
3190 
3191   /**
3192    * Append the given map of family->edits to a WALEdit data structure.
3193    * This does not write to the HLog itself.
3194    * @param familyMap map of family->edits
3195    * @param walEdit the destination entry to append into
3196    */
3197   private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
3198       WALEdit walEdit) {
3199     for (List<KeyValue> edits : familyMap.values()) {
3200       for (KeyValue kv : edits) {
3201         walEdit.add(kv);
3202       }
3203     }
3204   }
3205 
3206   private void requestFlush() {
3207     if (this.rsServices == null) {
3208       return;
3209     }
3210     synchronized (writestate) {
3211       if (this.writestate.isFlushRequested()) {
3212         return;
3213       }
3214       writestate.flushRequested = true;
3215     }
3216     // Make request outside of synchronize block; HBASE-818.
3217     this.rsServices.getFlushRequester().requestFlush(this);
3218     if (LOG.isDebugEnabled()) {
3219       LOG.debug("Flush requested on " + this);
3220     }
3221   }
3222 
3223   /*
3224    * @param size
3225    * @return True if size is over the flush threshold
3226    */
3227   private boolean isFlushSize(final long size) {
3228     return size > this.memstoreFlushSize;
3229   }
3230 
3231   /**
3232    * Read the edits log put under this region by wal log splitting process.  Put
3233    * the recovered edits back up into this region.
3234    *
3235    * <p>We can ignore any log message that has a sequence ID that's equal to or
3236    * lower than minSeqId.  (Because we know such log messages are already
3237    * reflected in the HFiles.)
3238    *
3239    * <p>While this is running we are putting pressure on memory yet we are
3240    * outside of our usual accounting because we are not yet an onlined region
3241    * (this stuff is being run as part of Region initialization).  This means
3242    * that if we're up against global memory limits, we'll not be flagged to flush
3243    * because we are not online. We can't be flushed by usual mechanisms anyways;
3244    * we're not yet online so our relative sequenceids are not yet aligned with
3245    * HLog sequenceids -- not till we come up online, post processing of split
3246    * edits.
3247    *
3248    * <p>But to help relieve memory pressure, at least manage our own heap size
3249    * flushing if are in excess of per-region limits.  Flushing, though, we have
3250    * to be careful and avoid using the regionserver/hlog sequenceid.  Its running
3251    * on a different line to whats going on in here in this region context so if we
3252    * crashed replaying these edits, but in the midst had a flush that used the
3253    * regionserver log with a sequenceid in excess of whats going on in here
3254    * in this region and with its split editlogs, then we could miss edits the
3255    * next time we go to recover. So, we have to flush inline, using seqids that
3256    * make sense in a this single region context only -- until we online.
3257    *
3258    * @param regiondir
3259    * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of
3260    * the maxSeqId for the store to be applied, else its skipped.
3261    * @param reporter
3262    * @return the sequence id of the last edit added to this region out of the
3263    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3264    * @throws UnsupportedEncodingException
3265    * @throws IOException
3266    */
3267   protected long replayRecoveredEditsIfAny(final Path regiondir,
3268       Map<byte[], Long> maxSeqIdInStores,
3269       final CancelableProgressable reporter, final MonitoredTask status)
3270       throws UnsupportedEncodingException, IOException {
3271     long minSeqIdForTheRegion = -1;
3272     for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
3273       if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
3274         minSeqIdForTheRegion = maxSeqIdInStore;
3275       }
3276     }
3277     long seqid = minSeqIdForTheRegion;
3278     NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
3279     if (files == null || files.isEmpty()) return seqid;
3280 
3281     for (Path edits: files) {
3282       if (edits == null || !this.fs.exists(edits)) {
3283         LOG.warn("Null or non-existent edits file: " + edits);
3284         continue;
3285       }
3286       if (isZeroLengthThenDelete(this.fs, edits)) continue;
3287 
3288       long maxSeqId = Long.MAX_VALUE;
3289       String fileName = edits.getName();
3290       maxSeqId = Math.abs(Long.parseLong(fileName));
3291       if (maxSeqId <= minSeqIdForTheRegion) {
3292         String msg = "Maximum sequenceid for this log is " + maxSeqId
3293             + " and minimum sequenceid for the region is " + minSeqIdForTheRegion
3294             + ", skipped the whole file, path=" + edits;
3295         LOG.debug(msg);
3296         continue;
3297       }
3298 
3299       try {
3300         seqid = replayRecoveredEdits(edits, maxSeqIdInStores, reporter);
3301       } catch (IOException e) {
3302         boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
3303         if (skipErrors) {
3304           Path p = HLog.moveAsideBadEditsFile(fs, edits);
3305           LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits +
3306             " as " + p, e);
3307         } else {
3308           throw e;
3309         }
3310       }
3311       // The edits size added into rsAccounting during this replaying will not
3312       // be required any more. So just clear it.
3313       if (this.rsAccounting != null) {
3314         this.rsAccounting.clearRegionReplayEditsSize(this.regionInfo.getRegionName());
3315       }
3316     }
3317     if (seqid > minSeqIdForTheRegion) {
3318       // Then we added some edits to memory. Flush and cleanup split edit files.
3319       internalFlushcache(null, seqid, status);
3320     }
3321     // Now delete the content of recovered edits.  We're done w/ them.
3322     for (Path file: files) {
3323       if (!HBaseFileSystem.deleteFileFromFileSystem(fs, file)) {
3324         LOG.error("Failed delete of " + file);
3325       } else {
3326         LOG.debug("Deleted recovered.edits file=" + file);
3327       }
3328     }
3329     return seqid;
3330   }
3331 
3332   /*
3333    * @param edits File of recovered edits.
3334    * @param maxSeqIdInStores Maximum sequenceid found in each store.  Edits in log
3335    * must be larger than this to be replayed for each store.
3336    * @param reporter
3337    * @return the sequence id of the last edit added to this region out of the
3338    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3339    * @throws IOException
3340    */
3341   private long replayRecoveredEdits(final Path edits,
3342       Map<byte[], Long> maxSeqIdInStores, final CancelableProgressable reporter)
3343     throws IOException {
3344     String msg = "Replaying edits from " + edits;
3345     LOG.info(msg);
3346     MonitoredTask status = TaskMonitor.get().createStatus(msg);
3347 
3348     status.setStatus("Opening logs");
3349     HLog.Reader reader = null;
3350     try {
3351       reader = HLog.getReader(this.fs, edits, conf);
3352       long currentEditSeqId = -1;
3353       long firstSeqIdInLog = -1;
3354       long skippedEdits = 0;
3355       long editsCount = 0;
3356       long intervalEdits = 0;
3357       HLog.Entry entry;
3358       Store store = null;
3359       boolean reported_once = false;
3360 
3361       try {
3362         // How many edits seen before we check elapsed time
3363         int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
3364             2000);
3365         // How often to send a progress report (default 1/2 master timeout)
3366         int period = this.conf.getInt("hbase.hstore.report.period",
3367             this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
3368                 180000) / 2);
3369         long lastReport = EnvironmentEdgeManager.currentTimeMillis();
3370 
3371         while ((entry = reader.next()) != null) {
3372           HLogKey key = entry.getKey();
3373           WALEdit val = entry.getEdit();
3374 
3375           if (reporter != null) {
3376             intervalEdits += val.size();
3377             if (intervalEdits >= interval) {
3378               // Number of edits interval reached
3379               intervalEdits = 0;
3380               long cur = EnvironmentEdgeManager.currentTimeMillis();
3381               if (lastReport + period <= cur) {
3382                 status.setStatus("Replaying edits..." +
3383                     " skipped=" + skippedEdits +
3384                     " edits=" + editsCount);
3385                 // Timeout reached
3386                 if(!reporter.progress()) {
3387                   msg = "Progressable reporter failed, stopping replay";
3388                   LOG.warn(msg);
3389                   status.abort(msg);
3390                   throw new IOException(msg);
3391                 }
3392                 reported_once = true;
3393                 lastReport = cur;
3394               }
3395             }
3396           }
3397 
3398           // Start coprocessor replay here. The coprocessor is for each WALEdit
3399           // instead of a KeyValue.
3400           if (coprocessorHost != null) {
3401             status.setStatus("Running pre-WAL-restore hook in coprocessors");
3402             if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
3403               // if bypass this log entry, ignore it ...
3404               continue;
3405             }
3406           }
3407 
3408           if (firstSeqIdInLog == -1) {
3409             firstSeqIdInLog = key.getLogSeqNum();
3410           }
3411           boolean flush = false;
3412           for (KeyValue kv: val.getKeyValues()) {
3413             // Check this edit is for me. Also, guard against writing the special
3414             // METACOLUMN info such as HBASE::CACHEFLUSH entries
3415             if (kv.matchingFamily(HLog.METAFAMILY) ||
3416                 !Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
3417               skippedEdits++;
3418               continue;
3419                 }
3420             // Figure which store the edit is meant for.
3421             if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
3422               store = this.stores.get(kv.getFamily());
3423             }
3424             if (store == null) {
3425               // This should never happen.  Perhaps schema was changed between
3426               // crash and redeploy?
3427               LOG.warn("No family for " + kv);
3428               skippedEdits++;
3429               continue;
3430             }
3431             // Now, figure if we should skip this edit.
3432             if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
3433                 .getName())) {
3434               skippedEdits++;
3435               continue;
3436             }
3437             currentEditSeqId = key.getLogSeqNum();
3438             // Once we are over the limit, restoreEdit will keep returning true to
3439             // flush -- but don't flush until we've played all the kvs that make up
3440             // the WALEdit.
3441             flush = restoreEdit(store, kv);
3442             editsCount++;
3443           }
3444           if (flush) internalFlushcache(null, currentEditSeqId, status);
3445 
3446           if (coprocessorHost != null) {
3447             coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
3448           }
3449         }
3450       } catch (EOFException eof) {
3451         Path p = HLog.moveAsideBadEditsFile(fs, edits);
3452         msg = "Encountered EOF. Most likely due to Master failure during " +
3453             "log spliting, so we have this data in another edit.  " +
3454             "Continuing, but renaming " + edits + " as " + p;
3455         LOG.warn(msg, eof);
3456         status.abort(msg);
3457       } catch (IOException ioe) {
3458         // If the IOE resulted from bad file format,
3459         // then this problem is idempotent and retrying won't help
3460         if (ioe.getCause() instanceof ParseException) {
3461           Path p = HLog.moveAsideBadEditsFile(fs, edits);
3462           msg = "File corruption encountered!  " +
3463               "Continuing, but renaming " + edits + " as " + p;
3464           LOG.warn(msg, ioe);
3465           status.setStatus(msg);
3466         } else {
3467           status.abort(StringUtils.stringifyException(ioe));
3468           // other IO errors may be transient (bad network connection,
3469           // checksum exception on one datanode, etc).  throw & retry
3470           throw ioe;
3471         }
3472       }
3473       if (reporter != null && !reported_once) {
3474         reporter.progress();
3475       }
3476       msg = "Applied " + editsCount + ", skipped " + skippedEdits +
3477         ", firstSequenceidInLog=" + firstSeqIdInLog +
3478         ", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
3479       status.markComplete(msg);
3480       LOG.debug(msg);
3481       return currentEditSeqId;
3482     } finally {
3483       status.cleanup();
3484       if (reader != null) {
3485          reader.close();
3486       }
3487     }
3488   }
3489 
3490   /**
3491    * Used by tests
3492    * @param s Store to add edit too.
3493    * @param kv KeyValue to add.
3494    * @return True if we should flush.
3495    */
3496   protected boolean restoreEdit(final Store s, final KeyValue kv) {
3497     long kvSize = s.add(kv);
3498     if (this.rsAccounting != null) {
3499       rsAccounting.addAndGetRegionReplayEditsSize(this.regionInfo.getRegionName(), kvSize);
3500     }
3501     return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
3502   }
3503 
3504   /*
3505    * @param fs
3506    * @param p File to check.
3507    * @return True if file was zero-length (and if so, we'll delete it in here).
3508    * @throws IOException
3509    */
3510   private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
3511       throws IOException {
3512     FileStatus stat = fs.getFileStatus(p);
3513     if (stat.getLen() > 0) return false;
3514     LOG.warn("File " + p + " is zero-length, deleting.");
3515     HBaseFileSystem.deleteFileFromFileSystem(fs, p);
3516     return true;
3517   }
3518 
3519   protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
3520       throws IOException {
3521     return new Store(tableDir, this, c, this.fs, this.conf);
3522   }
3523 
3524   /**
3525    * Return HStore instance.
3526    * Use with caution.  Exposed for use of fixup utilities.
3527    * @param column Name of column family hosted by this region.
3528    * @return Store that goes with the family on passed <code>column</code>.
3529    * TODO: Make this lookup faster.
3530    */
3531   public Store getStore(final byte [] column) {
3532     return this.stores.get(column);
3533   }
3534 
3535   public Map<byte[], Store> getStores() {
3536     return this.stores;
3537   }
3538 
3539   /**
3540    * Return list of storeFiles for the set of CFs.
3541    * Uses closeLock to prevent the race condition where a region closes
3542    * in between the for loop - closing the stores one by one, some stores
3543    * will return 0 files.
3544    * @return List of storeFiles.
3545    */
3546   public List<String> getStoreFileList(final byte [][] columns)
3547     throws IllegalArgumentException {
3548     List<String> storeFileNames = new ArrayList<String>();
3549     synchronized(closeLock) {
3550       for(byte[] column : columns) {
3551         Store store = this.stores.get(column);
3552         if (store == null) {
3553           throw new IllegalArgumentException("No column family : " +
3554               new String(column) + " available");
3555         }
3556         List<StoreFile> storeFiles = store.getStorefiles();
3557         for (StoreFile storeFile: storeFiles) {
3558           storeFileNames.add(storeFile.getPath().toString());
3559         }
3560       }
3561     }
3562     return storeFileNames;
3563   }
3564   //////////////////////////////////////////////////////////////////////////////
3565   // Support code
3566   //////////////////////////////////////////////////////////////////////////////
3567 
3568   /** Make sure this is a valid row for the HRegion */
3569   void checkRow(final byte [] row, String op) throws IOException {
3570     if(!rowIsInRange(regionInfo, row)) {
3571       throw new WrongRegionException("Requested row out of range for " +
3572           op + " on HRegion " + this + ", startKey='" +
3573           Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
3574           Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
3575           Bytes.toStringBinary(row) + "'");
3576     }
3577   }
3578 
3579   /**
3580    * Obtain a lock on the given row.  Blocks until success.
3581    *
3582    * I know it's strange to have two mappings:
3583    * <pre>
3584    *   ROWS  ==> LOCKS
3585    * </pre>
3586    * as well as
3587    * <pre>
3588    *   LOCKS ==> ROWS
3589    * </pre>
3590    *
3591    * But it acts as a guard on the client; a miswritten client just can't
3592    * submit the name of a row and start writing to it; it must know the correct
3593    * lockid, which matches the lock list in memory.
3594    *
3595    * <p>It would be more memory-efficient to assume a correctly-written client,
3596    * which maybe we'll do in the future.
3597    *
3598    * @param row Name of row to lock.
3599    * @throws IOException
3600    * @return The id of the held lock.
3601    */
3602   public Integer obtainRowLock(final byte [] row) throws IOException {
3603     startRegionOperation();
3604     this.writeRequestsCount.increment();
3605     this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3606     try {
3607       return internalObtainRowLock(new HashedBytes(row), true);
3608     } finally {
3609       closeRegionOperation();
3610     }
3611   }
3612 
3613   /**
3614    * Obtains or tries to obtain the given row lock.
3615    * @param waitForLock if true, will block until the lock is available.
3616    *        Otherwise, just tries to obtain the lock and returns
3617    *        null if unavailable.
3618    */
3619   private Integer internalObtainRowLock(final HashedBytes rowKey, boolean waitForLock)
3620       throws IOException {
3621     checkRow(rowKey.getBytes(), "row lock");
3622     startRegionOperation();
3623     try {
3624       CountDownLatch rowLatch = new CountDownLatch(1);
3625 
3626       // loop until we acquire the row lock (unless !waitForLock)
3627       while (true) {
3628         CountDownLatch existingLatch = lockedRows.putIfAbsent(rowKey, rowLatch);
3629         if (existingLatch == null) {
3630           break;
3631         } else {
3632           // row already locked
3633           if (!waitForLock) {
3634             return null;
3635           }
3636           try {
3637             if (!existingLatch.await(this.rowLockWaitDuration,
3638                             TimeUnit.MILLISECONDS)) {
3639               throw new IOException("Timed out on getting lock for row=" + rowKey);
3640             }
3641           } catch (InterruptedException ie) {
3642             // Empty
3643           }
3644         }
3645       }
3646 
3647       // loop until we generate an unused lock id
3648       while (true) {
3649         Integer lockId = lockIdGenerator.incrementAndGet();
3650         HashedBytes existingRowKey = lockIds.putIfAbsent(lockId, rowKey);
3651         if (existingRowKey == null) {
3652           return lockId;
3653         } else {
3654           // lockId already in use, jump generator to a new spot
3655           lockIdGenerator.set(rand.nextInt());
3656         }
3657       }
3658     } finally {
3659       closeRegionOperation();
3660     }
3661   }
3662 
3663   /**
3664    * Used by unit tests.
3665    * @param lockid
3666    * @return Row that goes with <code>lockid</code>
3667    */
3668   byte[] getRowFromLock(final Integer lockid) {
3669     HashedBytes rowKey = lockIds.get(lockid);
3670     return rowKey == null ? null : rowKey.getBytes();
3671   }
3672 
3673   /**
3674    * Release the row lock!
3675    * @param lockId  The lock ID to release.
3676    */
3677   public void releaseRowLock(final Integer lockId) {
3678     if (lockId == null) return; // null lock id, do nothing
3679     HashedBytes rowKey = lockIds.remove(lockId);
3680     if (rowKey == null) {
3681       LOG.warn("Release unknown lockId: " + lockId);
3682       return;
3683     }
3684     CountDownLatch rowLatch = lockedRows.remove(rowKey);
3685     if (rowLatch == null) {
3686       LOG.error("Releases row not locked, lockId: " + lockId + " row: "
3687           + rowKey);
3688       return;
3689     }
3690     rowLatch.countDown();
3691   }
3692 
3693   /**
3694    * See if row is currently locked.
3695    * @param lockid
3696    * @return boolean
3697    */
3698   boolean isRowLocked(final Integer lockId) {
3699     return lockIds.containsKey(lockId);
3700   }
3701 
3702   /**
3703    * Returns existing row lock if found, otherwise
3704    * obtains a new row lock and returns it.
3705    * @param lockid requested by the user, or null if the user didn't already hold lock
3706    * @param row the row to lock
3707    * @param waitForLock if true, will block until the lock is available, otherwise will
3708    * simply return null if it could not acquire the lock.
3709    * @return lockid or null if waitForLock is false and the lock was unavailable.
3710    */
3711   public Integer getLock(Integer lockid, byte [] row, boolean waitForLock)
3712   throws IOException {
3713     return getLock(lockid, new HashedBytes(row), waitForLock);
3714   }
3715 
3716   /**
3717    * Returns existing row lock if found, otherwise
3718    * obtains a new row lock and returns it.
3719    * @param lockid requested by the user, or null if the user didn't already hold lock
3720    * @param row the row to lock
3721    * @param waitForLock if true, will block until the lock is available, otherwise will
3722    * simply return null if it could not acquire the lock.
3723    * @return lockid or null if waitForLock is false and the lock was unavailable.
3724    */
3725   protected Integer getLock(Integer lockid, HashedBytes row, boolean waitForLock)
3726   throws IOException {
3727     Integer lid;
3728     if (lockid == null) {
3729       lid = internalObtainRowLock(row, waitForLock);
3730     } else {
3731       HashedBytes rowFromLock = lockIds.get(lockid);
3732       if (!row.equals(rowFromLock)) {
3733         throw new IOException("Invalid row lock: LockId: " + lockid + " holds the lock for row: " + rowFromLock + " but wanted lock for row: " + row);
3734       }
3735       lid = lockid;
3736     }
3737     return lid;
3738   }
3739 
3740   /**
3741    * Determines whether multiple column families are present
3742    * Precondition: familyPaths is not null
3743    *
3744    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3745    */
3746   private static boolean hasMultipleColumnFamilies(
3747       List<Pair<byte[], String>> familyPaths) {
3748     boolean multipleFamilies = false;
3749     byte[] family = null;
3750     for (Pair<byte[], String> pair : familyPaths) {
3751       byte[] fam = pair.getFirst();
3752       if (family == null) {
3753         family = fam;
3754       } else if (!Bytes.equals(family, fam)) {
3755         multipleFamilies = true;
3756         break;
3757       }
3758     }
3759     return multipleFamilies;
3760   }
3761 
3762   /**
3763    * Attempts to atomically load a group of hfiles.  This is critical for loading
3764    * rows with multiple column families atomically.
3765    *
3766    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3767    * @return true if successful, false if failed recoverably
3768    * @throws IOException if failed unrecoverably.
3769    */
3770   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths) throws IOException {
3771     return bulkLoadHFiles(familyPaths, false);
3772   }
3773 
3774   /**
3775    * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple
3776    * column families atomically.
3777    * @param familyPaths List of Pair<byte[] column family, String hfilePath> * @param assignSeqNum
3778    *          should we assign sequence numbers
3779    * @return true if successful, false if failed recoverably
3780    * @throws IOException if failed unrecoverably.
3781    */
3782   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, boolean assignSeqId)
3783       throws IOException {
3784     return bulkLoadHFiles(familyPaths, null, assignSeqId);
3785   }
3786 
3787   /**
3788    * Attempts to atomically load a group of hfiles.  This is critical for loading
3789    * rows with multiple column families atomically.
3790    *
3791    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3792    * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
3793    * file about to be bulk loaded
3794    * @return true if successful, false if failed recoverably
3795    * @throws IOException if failed unrecoverably.
3796    */
3797   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
3798       BulkLoadListener bulkLoadListener) throws IOException {
3799     return bulkLoadHFiles(familyPaths, bulkLoadListener, false);
3800   }
3801 
3802   /**
3803    * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple
3804    * column families atomically.
3805    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3806    * @param bulkLoadListener Internal hooks enabling massaging/preparation of a file about to be
3807    *          bulk loaded * @param assignSeqNum should we assign sequence numbers
3808    * @return true if successful, false if failed recoverably
3809    * @throws IOException if failed unrecoverably.
3810    */
3811   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
3812       BulkLoadListener bulkLoadListener, boolean assignSeqId) throws IOException {
3813     Preconditions.checkNotNull(familyPaths);
3814     // we need writeLock for multi-family bulk load
3815     startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
3816     try {
3817       this.writeRequestsCount.increment();
3818       this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3819 
3820       // There possibly was a split that happend between when the split keys
3821       // were gathered and before the HRegion's write lock was taken.  We need
3822       // to validate the HFile region before attempting to bulk load all of them
3823       List<IOException> ioes = new ArrayList<IOException>();
3824       List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
3825       for (Pair<byte[], String> p : familyPaths) {
3826         byte[] familyName = p.getFirst();
3827         String path = p.getSecond();
3828 
3829         Store store = getStore(familyName);
3830         if (store == null) {
3831           IOException ioe = new DoNotRetryIOException(
3832               "No such column family " + Bytes.toStringBinary(familyName));
3833           ioes.add(ioe);
3834         } else {
3835           try {
3836             store.assertBulkLoadHFileOk(new Path(path));
3837           } catch (WrongRegionException wre) {
3838             // recoverable (file doesn't fit in region)
3839             failures.add(p);
3840           } catch (IOException ioe) {
3841             // unrecoverable (hdfs problem)
3842             ioes.add(ioe);
3843           }
3844         }
3845       }
3846 
3847       // validation failed because of some sort of IO problem.
3848       if (ioes.size() != 0) {
3849         IOException e = MultipleIOException.createIOException(ioes);
3850         LOG.error("There were one or more IO errors when checking if the bulk load is ok.", e);
3851         throw e;
3852       }
3853 
3854       // validation failed, bail out before doing anything permanent.
3855       if (failures.size() != 0) {
3856         StringBuilder list = new StringBuilder();
3857         for (Pair<byte[], String> p : failures) {
3858           list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
3859             .append(p.getSecond());
3860         }
3861         // problem when validating
3862         LOG.warn("There was a recoverable bulk load failure likely due to a" +
3863             " split.  These (family, HFile) pairs were not loaded: " + list);
3864         return false;
3865       }
3866 
3867       long seqId = -1;
3868       // We need to assign a sequential ID that's in between two memstores in order to preserve
3869       // the guarantee that all the edits lower than the highest sequential ID from all the
3870       // HFiles are flushed on disk. See HBASE-10958.
3871       if (assignSeqId) {
3872         FlushResult fs = this.flushcache();
3873         if (fs.isFlushSucceeded()) {
3874           seqId = fs.flushSequenceId;
3875         } else if (fs.result == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
3876           seqId = this.log.obtainSeqNum();
3877         } else {
3878           throw new IOException("Could not bulk load with an assigned sequential ID because the " +
3879               "flush didn't run. Reason for not flushing: " + fs.failureReason);
3880         }
3881       }
3882 
3883       for (Pair<byte[], String> p : familyPaths) {
3884         byte[] familyName = p.getFirst();
3885         String path = p.getSecond();
3886         Store store = getStore(familyName);
3887         try {
3888           String finalPath = path;
3889           if(bulkLoadListener != null) {
3890             finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
3891           }
3892           store.bulkLoadHFile(finalPath, seqId);
3893           if(bulkLoadListener != null) {
3894             bulkLoadListener.doneBulkLoad(familyName, path);
3895           }
3896         } catch (IOException ioe) {
3897           // a failure here causes an atomicity violation that we currently
3898           // cannot recover from since it is likely a failed hdfs operation.
3899 
3900           // TODO Need a better story for reverting partial failures due to HDFS.
3901           LOG.error("There was a partial failure due to IO when attempting to" +
3902               " load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond());
3903           if(bulkLoadListener != null) {
3904             try {
3905               bulkLoadListener.failedBulkLoad(familyName, path);
3906             } catch (Exception ex) {
3907               LOG.error("Error while calling failedBulkLoad for family "+
3908                   Bytes.toString(familyName)+" with path "+path, ex);
3909             }
3910           }
3911           throw ioe;
3912         }
3913       }
3914       return true;
3915     } finally {
3916       closeBulkRegionOperation();
3917     }
3918   }
3919 
3920   @Override
3921   public boolean equals(Object o) {
3922     if (!(o instanceof HRegion)) {
3923       return false;
3924     }
3925     return Bytes.equals(this.getRegionName(), ((HRegion) o).getRegionName());
3926   }
3927 
3928   @Override
3929   public int hashCode() {
3930     return Bytes.hashCode(this.getRegionName());
3931   }
3932 
3933   @Override
3934   public String toString() {
3935     return this.regionInfo.getRegionNameAsString();
3936   }
3937 
3938   /** @return Path of region base directory */
3939   public Path getTableDir() {
3940     return this.tableDir;
3941   }
3942 
3943   /**
3944    * RegionScannerImpl is used to combine scanners from multiple Stores (aka column families).
3945    */
3946   class RegionScannerImpl implements RegionScanner {
3947     // Package local for testability
3948     KeyValueHeap storeHeap = null;
3949     /** Heap of key-values that are not essential for the provided filters and are thus read
3950      * on demand, if on-demand column family loading is enabled.*/
3951     KeyValueHeap joinedHeap = null;
3952     /**
3953      * If the joined heap data gathering is interrupted due to scan limits, this will
3954      * contain the row for which we are populating the values.*/
3955     private KeyValue joinedContinuationRow = null;
3956     // KeyValue indicating that limit is reached when scanning
3957     private final KeyValue KV_LIMIT = new KeyValue();
3958     private final byte [] stopRow;
3959     private final Filter filter;
3960     private int batch;
3961     private int isScan;
3962     private boolean filterClosed = false;
3963     private long readPt;
3964     private HRegion region;
3965 
3966     public HRegionInfo getRegionInfo() {
3967       return regionInfo;
3968     }
3969     
3970     RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
3971         throws IOException {
3972       // DebugPrint.println("HRegionScanner.<init>");
3973       this.region = region;
3974       this.filter = scan.getFilter();
3975       this.batch = scan.getBatch();
3976       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
3977         this.stopRow = null;
3978       } else {
3979         this.stopRow = scan.getStopRow();
3980       }
3981       // If we are doing a get, we want to be [startRow,endRow] normally
3982       // it is [startRow,endRow) and if startRow=endRow we get nothing.
3983       this.isScan = scan.isGetScan() ? -1 : 0;
3984 
3985       // synchronize on scannerReadPoints so that nobody calculates
3986       // getSmallestReadPoint, before scannerReadPoints is updated.
3987       IsolationLevel isolationLevel = scan.getIsolationLevel();
3988       synchronized(scannerReadPoints) {
3989         if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
3990           // This scan can read even uncommitted transactions
3991           this.readPt = Long.MAX_VALUE;
3992           MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
3993         } else {
3994           this.readPt = MultiVersionConsistencyControl.resetThreadReadPoint(mvcc);
3995         }
3996         scannerReadPoints.put(this, this.readPt);
3997       }
3998 
3999       // Here we separate all scanners into two lists - scanner that provide data required
4000       // by the filter to operate (scanners list) and all others (joinedScanners list).
4001       List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
4002       List<KeyValueScanner> joinedScanners = new ArrayList<KeyValueScanner>();
4003       if (additionalScanners != null) {
4004         scanners.addAll(additionalScanners);
4005       }
4006 
4007       for (Map.Entry<byte[], NavigableSet<byte[]>> entry :
4008           scan.getFamilyMap().entrySet()) {
4009         Store store = stores.get(entry.getKey());
4010         KeyValueScanner scanner = store.getScanner(scan, entry.getValue());
4011         if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
4012           || FilterBase.isFamilyEssential(this.filter, entry.getKey())) {
4013           scanners.add(scanner);
4014         } else {
4015           joinedScanners.add(scanner);
4016         }
4017       }
4018       this.storeHeap = new KeyValueHeap(scanners, comparator);
4019       if (!joinedScanners.isEmpty()) {
4020         this.joinedHeap = new KeyValueHeap(joinedScanners, comparator);
4021       }
4022     }
4023 
4024     RegionScannerImpl(Scan scan, HRegion region) throws IOException {
4025       this(scan, null, region);
4026     }
4027 
4028     @Override
4029     public long getMvccReadPoint() {
4030       return this.readPt;
4031     }
4032     /**
4033      * Reset both the filter and the old filter.
4034      */
4035     protected void resetFilters() {
4036       if (filter != null) {
4037         filter.reset();
4038       }
4039     }
4040 
4041     @Override
4042     public boolean next(List<KeyValue> outResults, int limit)
4043         throws IOException {
4044       return next(outResults, limit, null);
4045     }
4046 
4047     @Override
4048     public synchronized boolean next(List<KeyValue> outResults, int limit,
4049         String metric) throws IOException {
4050       if (this.filterClosed) {
4051         throw new UnknownScannerException("Scanner was closed (timed out?) " +
4052             "after we renewed it. Could be caused by a very slow scanner " +
4053             "or a lengthy garbage collection");
4054       }
4055       startRegionOperation();
4056       readRequestsCount.increment();
4057       opMetrics.setReadRequestCountMetrics(readRequestsCount.get());
4058       try {
4059 
4060         // This could be a new thread from the last time we called next().
4061         MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
4062 
4063         return nextRaw(outResults, limit, metric);
4064       } finally {
4065         closeRegionOperation();
4066       }
4067     }
4068 
4069     @Override
4070     public boolean nextRaw(List<KeyValue> outResults, String metric)
4071         throws IOException {
4072       return nextRaw(outResults, batch, metric);
4073     }
4074 
4075     @Override
4076     public boolean nextRaw(List<KeyValue> outResults, int limit,
4077         String metric) throws IOException {
4078       boolean returnResult;
4079       if (outResults.isEmpty()) {
4080         // Usually outResults is empty. This is true when next is called
4081         // to handle scan or get operation.
4082         returnResult = nextInternal(outResults, limit, metric);
4083       } else {
4084         List<KeyValue> tmpList = new ArrayList<KeyValue>();
4085         returnResult = nextInternal(tmpList, limit, metric);
4086         outResults.addAll(tmpList);
4087       }
4088       resetFilters();
4089       if (isFilterDoneInternal()) {
4090         return false;
4091       }
4092       return returnResult;
4093     }
4094 
4095     @Override
4096     public boolean next(List<KeyValue> outResults)
4097         throws IOException {
4098       // apply the batching limit by default
4099       return next(outResults, batch, null);
4100     }
4101 
4102     @Override
4103     public boolean next(List<KeyValue> outResults, String metric)
4104         throws IOException {
4105       // apply the batching limit by default
4106       return next(outResults, batch, metric);
4107     }
4108 
4109     private void populateFromJoinedHeap(List<KeyValue> results, int limit, String metric)
4110         throws IOException {
4111       assert joinedContinuationRow != null;
4112       KeyValue kv = populateResult(results, this.joinedHeap, limit,
4113         joinedContinuationRow.getBuffer(), joinedContinuationRow.getRowOffset(),
4114         joinedContinuationRow.getRowLength(), metric);
4115       if (kv != KV_LIMIT) {
4116         // We are done with this row, reset the continuation.
4117         joinedContinuationRow = null;
4118       }
4119       // As the data is obtained from two independent heaps, we need to
4120       // ensure that result list is sorted, because Result relies on that.
4121       Collections.sort(results, comparator);
4122     }
4123 
4124     /**
4125      * Fetches records with this row into result list, until next row or limit (if not -1).
4126      * @param results
4127      * @param heap KeyValueHeap to fetch data from. It must be positioned on correct row before call.
4128      * @param limit Max amount of KVs to place in result list, -1 means no limit.
4129      * @param currentRow Byte array with key we are fetching.
4130      * @param offset offset for currentRow
4131      * @param length length for currentRow
4132      * @param metric Metric key to be passed into KeyValueHeap::next().
4133      * @return true if limit reached, false otherwise.
4134      */
4135     private KeyValue populateResult(List<KeyValue> results, KeyValueHeap heap, int limit,
4136         byte[] currentRow, int offset, short length, String metric) throws IOException {
4137       KeyValue nextKv;
4138       do {
4139         heap.next(results, limit - results.size(), metric);
4140         if (limit > 0 && results.size() == limit) {
4141           return KV_LIMIT;
4142         }
4143         nextKv = heap.peek();
4144       } while (nextKv != null && nextKv.matchingRow(currentRow, offset, length));
4145       return nextKv;
4146     }
4147 
4148     /*
4149      * @return True if a filter rules the scanner is over, done.
4150      */
4151     public synchronized boolean isFilterDone() {
4152       return isFilterDoneInternal();
4153     }
4154 
4155     private boolean isFilterDoneInternal() {
4156       return this.filter != null && this.filter.filterAllRemaining();
4157     }
4158 
4159     private boolean nextInternal(List<KeyValue> results, int limit, String metric)
4160     throws IOException {
4161       if (!results.isEmpty()) {
4162         throw new IllegalArgumentException("First parameter should be an empty list");
4163       }
4164       RpcCallContext rpcCall = HBaseServer.getCurrentCall();
4165       // The loop here is used only when at some point during the next we determine
4166       // that due to effects of filters or otherwise, we have an empty row in the result.
4167       // Then we loop and try again. Otherwise, we must get out on the first iteration via return,
4168       // "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row,
4169       // and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow).
4170       while (true) {
4171         if (rpcCall != null) {
4172           // If a user specifies a too-restrictive or too-slow scanner, the
4173           // client might time out and disconnect while the server side
4174           // is still processing the request. We should abort aggressively
4175           // in that case.
4176           rpcCall.throwExceptionIfCallerDisconnected();
4177         }
4178 
4179         // Let's see what we have in the storeHeap.
4180         KeyValue current = this.storeHeap.peek();
4181 
4182         byte[] currentRow = null;
4183         int offset = 0;
4184         short length = 0;
4185         if (current != null) {
4186           currentRow = current.getBuffer();
4187           offset = current.getRowOffset();
4188           length = current.getRowLength();
4189         }
4190         boolean stopRow = isStopRow(currentRow, offset, length);
4191         // Check if we were getting data from the joinedHeap abd hit the limit.
4192         // If not, then it's main path - getting results from storeHeap.
4193         if (joinedContinuationRow == null) {
4194           // First, check if we are at a stop row. If so, there are no more results.
4195           if (stopRow) {
4196             if (filter != null && filter.hasFilterRow()) {
4197               filter.filterRow(results);
4198             }
4199             if (filter != null && filter.filterRow()) {
4200               results.clear();
4201             }
4202             return false;
4203           }
4204 
4205           // Check if rowkey filter wants to exclude this row. If so, loop to next.
4206           // Techically, if we hit limits before on this row, we don't need this call.
4207           if (filterRowKey(currentRow, offset, length)) {
4208             results.clear();
4209             boolean moreRows = nextRow(currentRow, offset, length);
4210             if (!moreRows) return false;
4211             continue;
4212           }
4213 
4214           // Ok, we are good, let's try to get some results from the main heap.
4215           KeyValue nextKv = populateResult(results, this.storeHeap, limit, currentRow, offset,
4216               length, metric);
4217           if (nextKv == KV_LIMIT) {
4218             if (this.filter != null && filter.hasFilterRow()) {
4219               throw new IncompatibleFilterException(
4220                 "Filter whose hasFilterRow() returns true is incompatible with scan with limit!");
4221             }
4222             return true; // We hit the limit.
4223           }
4224           stopRow = nextKv == null
4225               || isStopRow(nextKv.getBuffer(), nextKv.getRowOffset(), nextKv.getRowLength());
4226           // save that the row was empty before filters applied to it.
4227           final boolean isEmptyRow = results.isEmpty();
4228 
4229           // We have the part of the row necessary for filtering (all of it, usually).
4230           // First filter with the filterRow(List).            
4231           if (filter != null && filter.hasFilterRow()) {
4232             filter.filterRow(results);
4233           }
4234 
4235           if (isEmptyRow || filterRow()) {
4236             results.clear();
4237             boolean moreRows = nextRow(currentRow, offset, length);
4238             if (!moreRows) return false;
4239 
4240             // This row was totally filtered out, if this is NOT the last row,
4241             // we should continue on. Otherwise, nothing else to do.
4242             if (!stopRow) continue;
4243             return false;
4244           }
4245 
4246           // Ok, we are done with storeHeap for this row.
4247           // Now we may need to fetch additional, non-essential data into row.
4248           // These values are not needed for filter to work, so we postpone their
4249           // fetch to (possibly) reduce amount of data loads from disk.
4250           if (this.joinedHeap != null) {
4251             KeyValue nextJoinedKv = joinedHeap.peek();
4252             // If joinedHeap is pointing to some other row, try to seek to a correct one.
4253             boolean mayHaveData =
4254               (nextJoinedKv != null && nextJoinedKv.matchingRow(currentRow, offset, length))
4255                 || (this.joinedHeap.requestSeek(
4256                     KeyValue.createFirstOnRow(currentRow, offset, length), true, true)
4257                   && joinedHeap.peek() != null
4258                   && joinedHeap.peek().matchingRow(currentRow, offset, length));
4259             if (mayHaveData) {
4260               joinedContinuationRow = current;
4261               populateFromJoinedHeap(results, limit, metric);
4262             }
4263           }
4264         } else {
4265           // Populating from the joined map was stopped by limits, populate some more.
4266           populateFromJoinedHeap(results, limit, metric);
4267         }
4268 
4269         // We may have just called populateFromJoinedMap and hit the limits. If that is
4270         // the case, we need to call it again on the next next() invocation.
4271         if (joinedContinuationRow != null) {
4272           return true;
4273         }
4274 
4275         // Finally, we are done with both joinedHeap and storeHeap.
4276         // Double check to prevent empty rows from appearing in result. It could be
4277         // the case when SingleValueExcludeFilter is used.
4278         if (results.isEmpty()) {
4279           boolean moreRows = nextRow(currentRow, offset, length);
4280           if (!moreRows) return false;
4281           if (!stopRow) continue;
4282         }
4283 
4284         // We are done. Return the result.
4285         return !stopRow;
4286       }
4287     }
4288 
4289     private boolean filterRow() {
4290       return filter != null
4291           && filter.filterRow();
4292     }
4293     private boolean filterRowKey(byte[] row, int offset, short length) {
4294       return filter != null
4295           && filter.filterRowKey(row, offset, length);
4296     }
4297 
4298     protected boolean nextRow(byte [] currentRow, int offset, short length) throws IOException {
4299       KeyValue next;
4300       while((next = this.storeHeap.peek()) != null && next.matchingRow(currentRow, offset, length)) {
4301         this.storeHeap.next(MOCKED_LIST);       
4302       }
4303       resetFilters();
4304       // Calling the hook in CP which allows it to do a fast forward
4305       if (this.region.getCoprocessorHost() != null) {
4306         return this.region.getCoprocessorHost().postScannerFilterRow(this, currentRow, offset,
4307             length);
4308       }
4309       return true;
4310     }
4311 
4312     private boolean isStopRow(byte [] currentRow, int offset, short length) {
4313       return currentRow == null ||
4314           (stopRow != null &&
4315           comparator.compareRows(stopRow, 0, stopRow.length,
4316               currentRow, offset, length) <= isScan);
4317     }
4318 
4319     @Override
4320     public synchronized void close() {
4321       if (storeHeap != null) {
4322         storeHeap.close();
4323         storeHeap = null;
4324       }
4325       if (joinedHeap != null) {
4326         joinedHeap.close();
4327         joinedHeap = null;
4328       }
4329       // no need to sychronize here.
4330       scannerReadPoints.remove(this);
4331       this.filterClosed = true;
4332     }
4333 
4334     KeyValueHeap getStoreHeapForTesting() {
4335       return storeHeap;
4336     }
4337 
4338     @Override
4339     public synchronized boolean reseek(byte[] row) throws IOException {
4340       if (row == null) {
4341         throw new IllegalArgumentException("Row cannot be null.");
4342       }
4343       boolean result = false;
4344       startRegionOperation();
4345       try {
4346         // This could be a new thread from the last time we called next().
4347         MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
4348         KeyValue kv = KeyValue.createFirstOnRow(row);
4349         // use request seek to make use of the lazy seek option. See HBASE-5520
4350         result = this.storeHeap.requestSeek(kv, true, true);
4351         if (this.joinedHeap != null) {
4352           result = this.joinedHeap.requestSeek(kv, true, true) || result;
4353         }
4354       } finally {
4355         closeRegionOperation();
4356       }
4357       return result;
4358     }
4359   }
4360 
4361   // Utility methods
4362   /**
4363    * A utility method to create new instances of HRegion based on the
4364    * {@link HConstants#REGION_IMPL} configuration property.
4365    * @param tableDir qualified path of directory where region should be located,
4366    * usually the table directory.
4367    * @param log The HLog is the outbound log for any updates to the HRegion
4368    * (There's a single HLog for all the HRegions on a single HRegionServer.)
4369    * The log file is a logfile from the previous execution that's
4370    * custom-computed for this HRegion. The HRegionServer computes and sorts the
4371    * appropriate log info for this HRegion. If there is a previous log file
4372    * (implying that the HRegion has been written-to before), then read it from
4373    * the supplied path.
4374    * @param fs is the filesystem.
4375    * @param conf is global configuration settings.
4376    * @param regionInfo - HRegionInfo that describes the region
4377    * is new), then read them from the supplied path.
4378    * @param htd
4379    * @param rsServices
4380    * @return the new instance
4381    */
4382   public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
4383       Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
4384       RegionServerServices rsServices) {
4385     try {
4386       @SuppressWarnings("unchecked")
4387       Class<? extends HRegion> regionClass =
4388           (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
4389 
4390       Constructor<? extends HRegion> c =
4391           regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
4392               Configuration.class, HRegionInfo.class, HTableDescriptor.class,
4393               RegionServerServices.class);
4394 
4395       return c.newInstance(tableDir, log, fs, conf, regionInfo, htd, rsServices);
4396     } catch (Throwable e) {
4397       // todo: what should I throw here?
4398       throw new IllegalStateException("Could not instantiate a region instance.", e);
4399     }
4400   }
4401 
4402   /**
4403    * Convenience method creating new HRegions. Used by createTable and by the
4404    * bootstrap code in the HMaster constructor.
4405    * Note, this method creates an {@link HLog} for the created region. It
4406    * needs to be closed explicitly.  Use {@link HRegion#getLog()} to get
4407    * access.  <b>When done with a region created using this method, you will
4408    * need to explicitly close the {@link HLog} it created too; it will not be
4409    * done for you.  Not closing the log will leave at least a daemon thread
4410    * running.</b>  Call {@link #closeHRegion(HRegion)} and it will do
4411    * necessary cleanup for you.
4412    * @param info Info for region to create.
4413    * @param rootDir Root directory for HBase instance
4414    * @param conf
4415    * @param hTableDescriptor
4416    * @return new HRegion
4417    *
4418    * @throws IOException
4419    */
4420   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4421       final Configuration conf, final HTableDescriptor hTableDescriptor)
4422   throws IOException {
4423     return createHRegion(info, rootDir, conf, hTableDescriptor, null);
4424   }
4425 
4426   /**
4427    * This will do the necessary cleanup a call to {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)}
4428    * requires.  This method will close the region and then close its
4429    * associated {@link HLog} file.  You use it if you call the other createHRegion,
4430    * the one that takes an {@link HLog} instance but don't be surprised by the
4431    * call to the {@link HLog#closeAndDelete()} on the {@link HLog} the
4432    * HRegion was carrying.
4433    * @param r
4434    * @throws IOException
4435    */
4436   public static void closeHRegion(final HRegion r) throws IOException {
4437     if (r == null) return;
4438     r.close();
4439     if (r.getLog() == null) return;
4440     r.getLog().closeAndDelete();
4441   }
4442 
4443   /**
4444    * Convenience method creating new HRegions. Used by createTable.
4445    * The {@link HLog} for the created region needs to be closed explicitly.
4446    * Use {@link HRegion#getLog()} to get access.
4447    *
4448    * @param info Info for region to create.
4449    * @param rootDir Root directory for HBase instance
4450    * @param conf
4451    * @param hTableDescriptor
4452    * @param hlog shared HLog
4453    * @param boolean initialize - true to initialize the region
4454    * @return new HRegion
4455    *
4456    * @throws IOException
4457    */
4458   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4459                                       final Configuration conf,
4460                                       final HTableDescriptor hTableDescriptor,
4461                                       final HLog hlog,
4462                                       final boolean initialize)
4463       throws IOException {
4464     return createHRegion(info, rootDir, conf, hTableDescriptor,
4465         hlog, initialize, false);
4466   }
4467 
4468   /**
4469    * Convenience method creating new HRegions. Used by createTable.
4470    * The {@link HLog} for the created region needs to be closed
4471    * explicitly, if it is not null.
4472    * Use {@link HRegion#getLog()} to get access.
4473    *
4474    * @param info Info for region to create.
4475    * @param rootDir Root directory for HBase instance
4476    * @param conf
4477    * @param hTableDescriptor
4478    * @param hlog shared HLog
4479    * @param boolean initialize - true to initialize the region
4480    * @param boolean ignoreHLog
4481       - true to skip generate new hlog if it is null, mostly for createTable
4482    * @return new HRegion
4483    *
4484    * @throws IOException
4485    */
4486   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4487                                       final Configuration conf,
4488                                       final HTableDescriptor hTableDescriptor,
4489                                       final HLog hlog,
4490                                       final boolean initialize, final boolean ignoreHLog)
4491       throws IOException {
4492     LOG.info("creating HRegion " + info.getTableNameAsString()
4493         + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
4494         " Table name == " + info.getTableNameAsString());
4495 
4496     Path tableDir =
4497         HTableDescriptor.getTableDir(rootDir, info.getTableName());
4498     Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
4499     FileSystem fs = FileSystem.get(conf);
4500     HBaseFileSystem.makeDirOnFileSystem(fs, regionDir);
4501     // Write HRI to a file in case we need to recover .META.
4502     writeRegioninfoOnFilesystem(info, regionDir, fs, conf);
4503     HLog effectiveHLog = hlog;
4504     if (hlog == null && !ignoreHLog) {
4505       effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
4506           new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
4507     }
4508     HRegion region = HRegion.newHRegion(tableDir,
4509         effectiveHLog, fs, conf, info, hTableDescriptor, null);
4510     if (initialize) {
4511       region.initialize();
4512     }
4513     return region;
4514   }
4515 
4516   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4517                                       final Configuration conf,
4518                                       final HTableDescriptor hTableDescriptor,
4519                                       final HLog hlog)
4520     throws IOException {
4521     return createHRegion(info, rootDir, conf, hTableDescriptor, hlog, true);
4522   }
4523 
4524   /**
4525    * Open a Region.
4526    * @param info Info for region to be opened.
4527    * @param wal HLog for region to use. This method will call
4528    * HLog#setSequenceNumber(long) passing the result of the call to
4529    * HRegion#getMinSequenceId() to ensure the log id is properly kept
4530    * up.  HRegionStore does this every time it opens a new region.
4531    * @param conf
4532    * @return new HRegion
4533    *
4534    * @throws IOException
4535    */
4536   public static HRegion openHRegion(final HRegionInfo info,
4537       final HTableDescriptor htd, final HLog wal,
4538       final Configuration conf)
4539   throws IOException {
4540     return openHRegion(info, htd, wal, conf, null, null);
4541   }
4542 
4543   /**
4544    * Open a Region.
4545    * @param info Info for region to be opened
4546    * @param htd
4547    * @param wal HLog for region to use. This method will call
4548    * HLog#setSequenceNumber(long) passing the result of the call to
4549    * HRegion#getMinSequenceId() to ensure the log id is properly kept
4550    * up.  HRegionStore does this every time it opens a new region.
4551    * @param conf
4552    * @param rsServices An interface we can request flushes against.
4553    * @param reporter An interface we can report progress against.
4554    * @return new HRegion
4555    *
4556    * @throws IOException
4557    */
4558   public static HRegion openHRegion(final HRegionInfo info,
4559     final HTableDescriptor htd, final HLog wal, final Configuration conf,
4560     final RegionServerServices rsServices,
4561     final CancelableProgressable reporter)
4562   throws IOException {
4563     if (LOG.isDebugEnabled()) {
4564       LOG.debug("Opening region: " + info);
4565     }
4566     if (info == null) {
4567       throw new NullPointerException("Passed region info is null");
4568     }
4569     Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
4570       info.getTableName());
4571     FileSystem fs = null;
4572     if (rsServices != null) {
4573       fs = rsServices.getFileSystem();
4574     }
4575     if (fs == null) {
4576       fs = FileSystem.get(conf);
4577     }
4578     HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info,
4579       htd, rsServices);
4580     return r.openHRegion(reporter);
4581   }
4582 
4583   public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
4584       final HTableDescriptor htd, final HLog wal, final Configuration conf)
4585   throws IOException {
4586     return openHRegion(tableDir, info, htd, wal, conf, null, null);
4587   }
4588 
4589   /**
4590    * Open a Region.
4591    * @param tableDir Table directory
4592    * @param info Info for region to be opened.
4593    * @param wal HLog for region to use. This method will call
4594    * HLog#setSequenceNumber(long) passing the result of the call to
4595    * HRegion#getMinSequenceId() to ensure the log id is properly kept
4596    * up.  HRegionStore does this every time it opens a new region.
4597    * @param conf
4598    * @param reporter An interface we can report progress against.
4599    * @return new HRegion
4600    *
4601    * @throws IOException
4602    */
4603   public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
4604       final HTableDescriptor htd, final HLog wal, final Configuration conf,
4605       final RegionServerServices rsServices,
4606       final CancelableProgressable reporter)
4607   throws IOException {
4608     if (info == null) throw new NullPointerException("Passed region info is null");
4609     LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
4610     if (LOG.isDebugEnabled()) {
4611       LOG.debug("Opening region: " + info);
4612     }
4613     Path dir = HTableDescriptor.getTableDir(tableDir,
4614         info.getTableName());
4615     HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
4616         htd, rsServices);
4617     return r.openHRegion(reporter);
4618   }
4619 
4620 
4621   /**
4622    * Open HRegion.
4623    * Calls initialize and sets sequenceid.
4624    * @param reporter
4625    * @return Returns <code>this</code>
4626    * @throws IOException
4627    */
4628   protected HRegion openHRegion(final CancelableProgressable reporter)
4629   throws IOException {
4630     checkCompressionCodecs();
4631 
4632     long seqid = initialize(reporter);
4633     if (this.log != null) {
4634       this.log.setSequenceNumber(seqid);
4635     }
4636     return this;
4637   }
4638 
4639   private void checkCompressionCodecs() throws IOException {
4640     for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
4641       CompressionTest.testCompression(fam.getCompression());
4642       CompressionTest.testCompression(fam.getCompactionCompression());
4643     }
4644   }
4645 
4646   /**
4647    * Inserts a new region's meta information into the passed
4648    * <code>meta</code> region. Used by the HMaster bootstrap code adding
4649    * new table to ROOT table.
4650    *
4651    * @param meta META HRegion to be updated
4652    * @param r HRegion to add to <code>meta</code>
4653    *
4654    * @throws IOException
4655    */
4656   public static void addRegionToMETA(HRegion meta, HRegion r)
4657   throws IOException {
4658     meta.checkResources();
4659     // The row key is the region name
4660     byte[] row = r.getRegionName();
4661     Integer lid = meta.obtainRowLock(row);
4662     try {
4663       final long now = EnvironmentEdgeManager.currentTimeMillis();
4664       final List<KeyValue> edits = new ArrayList<KeyValue>(2);
4665       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4666         HConstants.REGIONINFO_QUALIFIER, now,
4667         Writables.getBytes(r.getRegionInfo())));
4668       // Set into the root table the version of the meta table.
4669       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4670         HConstants.META_VERSION_QUALIFIER, now,
4671         Bytes.toBytes(HConstants.META_VERSION)));
4672       meta.put(HConstants.CATALOG_FAMILY, edits);
4673     } finally {
4674       meta.releaseRowLock(lid);
4675     }
4676   }
4677 
4678   /**
4679    * Deletes all the files for a HRegion
4680    *
4681    * @param fs the file system object
4682    * @param rootdir qualified path of HBase root directory
4683    * @param info HRegionInfo for region to be deleted
4684    * @throws IOException
4685    */
4686   public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
4687   throws IOException {
4688     deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
4689   }
4690 
4691   private static void deleteRegion(FileSystem fs, Path regiondir)
4692   throws IOException {
4693     if (LOG.isDebugEnabled()) {
4694       LOG.debug("DELETING region " + regiondir.toString());
4695     }
4696     if (!HBaseFileSystem.deleteDirFromFileSystem(fs, regiondir)) {
4697       LOG.warn("Failed delete of " + regiondir);
4698     }
4699   }
4700 
4701   /**
4702    * Computes the Path of the HRegion
4703    *
4704    * @param rootdir qualified path of HBase root directory
4705    * @param info HRegionInfo for the region
4706    * @return qualified path of region directory
4707    */
4708   public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
4709     return new Path(
4710       HTableDescriptor.getTableDir(rootdir, info.getTableName()),
4711                                    info.getEncodedName());
4712   }
4713 
4714   /**
4715    * Determines if the specified row is within the row range specified by the
4716    * specified HRegionInfo
4717    *
4718    * @param info HRegionInfo that specifies the row range
4719    * @param row row to be checked
4720    * @return true if the row is within the range specified by the HRegionInfo
4721    */
4722   public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
4723     return ((info.getStartKey().length == 0) ||
4724         (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
4725         ((info.getEndKey().length == 0) ||
4726             (Bytes.compareTo(info.getEndKey(), row) > 0));
4727   }
4728 
4729   /**
4730    * Make the directories for a specific column family
4731    *
4732    * @param fs the file system
4733    * @param tabledir base directory where region will live (usually the table dir)
4734    * @param hri
4735    * @param colFamily the column family
4736    * @throws IOException
4737    */
4738   public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
4739     final HRegionInfo hri, byte [] colFamily)
4740   throws IOException {
4741     Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
4742     if (!HBaseFileSystem.makeDirOnFileSystem(fs, dir)) {
4743       LOG.warn("Failed to create " + dir);
4744     }
4745   }
4746 
4747   /**
4748    * Merge two HRegions.  The regions must be adjacent and must not overlap.
4749    *
4750    * @param srcA
4751    * @param srcB
4752    * @return new merged HRegion
4753    * @throws IOException
4754    */
4755   public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
4756   throws IOException {
4757     HRegion a = srcA;
4758     HRegion b = srcB;
4759 
4760     // Make sure that srcA comes first; important for key-ordering during
4761     // write of the merged file.
4762     if (srcA.getStartKey() == null) {
4763       if (srcB.getStartKey() == null) {
4764         throw new IOException("Cannot merge two regions with null start key");
4765       }
4766       // A's start key is null but B's isn't. Assume A comes before B
4767     } else if ((srcB.getStartKey() == null) ||
4768       (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
4769       a = srcB;
4770       b = srcA;
4771     }
4772 
4773     if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) {
4774       throw new IOException("Cannot merge non-adjacent regions");
4775     }
4776     return merge(a, b);
4777   }
4778 
4779   /**
4780    * Merge two regions whether they are adjacent or not.
4781    *
4782    * @param a region a
4783    * @param b region b
4784    * @return new merged region
4785    * @throws IOException
4786    */
4787   public static HRegion merge(HRegion a, HRegion b)
4788   throws IOException {
4789     if (!a.getRegionInfo().getTableNameAsString().equals(
4790         b.getRegionInfo().getTableNameAsString())) {
4791       throw new IOException("Regions do not belong to the same table");
4792     }
4793 
4794     FileSystem fs = a.getFilesystem();
4795 
4796     // Make sure each region's cache is empty
4797 
4798     a.flushcache();
4799     b.flushcache();
4800 
4801     // Compact each region so we only have one store file per family
4802 
4803     a.compactStores(true);
4804     if (LOG.isDebugEnabled()) {
4805       LOG.debug("Files for region: " + a);
4806       listPaths(fs, a.getRegionDir());
4807     }
4808     b.compactStores(true);
4809     if (LOG.isDebugEnabled()) {
4810       LOG.debug("Files for region: " + b);
4811       listPaths(fs, b.getRegionDir());
4812     }
4813 
4814     Configuration conf = a.getBaseConf();
4815     HTableDescriptor tabledesc = a.getTableDesc();
4816     HLog log = a.getLog();
4817     Path tableDir = a.getTableDir();
4818     // Presume both are of same region type -- i.e. both user or catalog
4819     // table regions.  This way can use comparator.
4820     final byte[] startKey =
4821       (a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
4822            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4823        || b.comparator.matchingRows(b.getStartKey(), 0,
4824               b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
4825               HConstants.EMPTY_BYTE_ARRAY.length))
4826       ? HConstants.EMPTY_BYTE_ARRAY
4827       : (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
4828              b.getStartKey(), 0, b.getStartKey().length) <= 0
4829          ? a.getStartKey()
4830          : b.getStartKey());
4831     final byte[] endKey =
4832       (a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
4833            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4834        || a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
4835               HConstants.EMPTY_BYTE_ARRAY, 0,
4836               HConstants.EMPTY_BYTE_ARRAY.length))
4837       ? HConstants.EMPTY_BYTE_ARRAY
4838       : (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
4839              b.getEndKey(), 0, b.getEndKey().length) <= 0
4840          ? b.getEndKey()
4841          : a.getEndKey());
4842 
4843     HRegionInfo newRegionInfo =
4844         new HRegionInfo(tabledesc.getName(), startKey, endKey);
4845     LOG.info("Creating new region " + newRegionInfo.toString());
4846     String encodedName = newRegionInfo.getEncodedName();
4847     Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
4848     if(fs.exists(newRegionDir)) {
4849       throw new IOException("Cannot merge; target file collision at " +
4850           newRegionDir);
4851     }
4852     HBaseFileSystem.makeDirOnFileSystem(fs, newRegionDir);
4853 
4854     LOG.info("starting merge of regions: " + a + " and " + b +
4855       " into new region " + newRegionInfo.toString() +
4856         " with start key <" + Bytes.toStringBinary(startKey) + "> and end key <" +
4857         Bytes.toStringBinary(endKey) + ">");
4858 
4859     // Move HStoreFiles under new region directory
4860     Map<byte [], List<StoreFile>> byFamily =
4861       new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
4862     byFamily = filesByFamily(byFamily, a.close());
4863     byFamily = filesByFamily(byFamily, b.close());
4864     for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
4865       byte [] colFamily = es.getKey();
4866       makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily);
4867       // Because we compacted the source regions we should have no more than two
4868       // HStoreFiles per family and there will be no reference store
4869       List<StoreFile> srcFiles = es.getValue();
4870       for (StoreFile hsf: srcFiles) {
4871         StoreFile.rename(fs, hsf.getPath(),
4872           StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir,
4873             newRegionInfo.getEncodedName(), colFamily)));
4874       }
4875     }
4876     if (LOG.isDebugEnabled()) {
4877       LOG.debug("Files for new region");
4878       listPaths(fs, newRegionDir);
4879     }
4880     HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf,
4881         newRegionInfo, a.getTableDesc(), null);
4882     long totalReadRequestCount = a.readRequestsCount.get() + b.readRequestsCount.get();
4883     dstRegion.readRequestsCount.set(totalReadRequestCount);
4884     dstRegion.opMetrics.setReadRequestCountMetrics(totalReadRequestCount);
4885     
4886     long totalWriteRequestCount = a.writeRequestsCount.get() + b.writeRequestsCount.get();
4887     dstRegion.writeRequestsCount.set(totalWriteRequestCount);
4888     dstRegion.opMetrics.setWriteRequestCountMetrics(totalWriteRequestCount);
4889     
4890     dstRegion.initialize();
4891     dstRegion.compactStores();
4892     if (LOG.isDebugEnabled()) {
4893       LOG.debug("Files for new region");
4894       listPaths(fs, dstRegion.getRegionDir());
4895     }
4896 
4897     // delete out the 'A' region
4898     HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(a.getConf()),
4899         a.getTableDir(), a.getRegionDir());
4900     // delete out the 'B' region
4901     HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(b.getConf()),
4902         b.getTableDir(), b.getRegionDir());
4903 
4904     LOG.info("merge completed. New region is " + dstRegion);
4905 
4906     return dstRegion;
4907   }
4908 
4909   /*
4910    * Fills a map with a vector of store files keyed by column family.
4911    * @param byFamily Map to fill.
4912    * @param storeFiles Store files to process.
4913    * @param family
4914    * @return Returns <code>byFamily</code>
4915    */
4916   private static Map<byte [], List<StoreFile>> filesByFamily(
4917       Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
4918     for (StoreFile src: storeFiles) {
4919       byte [] family = src.getFamily();
4920       List<StoreFile> v = byFamily.get(family);
4921       if (v == null) {
4922         v = new ArrayList<StoreFile>();
4923         byFamily.put(family, v);
4924       }
4925       v.add(src);
4926     }
4927     return byFamily;
4928   }
4929 
4930   /**
4931    * @return True if needs a mojor compaction.
4932    * @throws IOException
4933    */
4934   boolean isMajorCompaction() throws IOException {
4935     for (Store store: this.stores.values()) {
4936       if (store.isMajorCompaction()) {
4937         return true;
4938       }
4939     }
4940     return false;
4941   }
4942 
4943   /*
4944    * List the files under the specified directory
4945    *
4946    * @param fs
4947    * @param dir
4948    * @throws IOException
4949    */
4950   private static void listPaths(FileSystem fs, Path dir) throws IOException {
4951     if (LOG.isDebugEnabled()) {
4952       FileStatus[] stats = FSUtils.listStatus(fs, dir, null);
4953       if (stats == null || stats.length == 0) {
4954         return;
4955       }
4956       for (int i = 0; i < stats.length; i++) {
4957         String path = stats[i].getPath().toString();
4958         if (stats[i].isDir()) {
4959           LOG.debug("d " + path);
4960           listPaths(fs, stats[i].getPath());
4961         } else {
4962           LOG.debug("f " + path + " size=" + stats[i].getLen());
4963         }
4964       }
4965     }
4966   }
4967 
4968 
4969   //
4970   // HBASE-880
4971   //
4972   /**
4973    * @param get get object
4974    * @return result
4975    * @throws IOException read exceptions
4976    */
4977   public Result get(final Get get) throws IOException {
4978     return get(get, null);
4979   }
4980 
4981   /**
4982    * @param get get object
4983    * @param lockid existing lock id, or null for no previous lock
4984    * @return result
4985    * @throws IOException read exceptions
4986    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
4987    */
4988   public Result get(final Get get, final Integer lockid) throws IOException {
4989     checkRow(get.getRow(), "Get");
4990     // Verify families are all valid
4991     if (get.hasFamilies()) {
4992       for (byte [] family: get.familySet()) {
4993         checkFamily(family);
4994       }
4995     } else { // Adding all families to scanner
4996       for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
4997         get.addFamily(family);
4998       }
4999     }
5000     List<KeyValue> results = get(get, true);
5001     return new Result(results);
5002   }
5003 
5004   /*
5005    * Do a get based on the get parameter.
5006    * @param withCoprocessor invoke coprocessor or not. We don't want to
5007    * always invoke cp for this private method.
5008    */
5009   private List<KeyValue> get(Get get, boolean withCoprocessor)
5010   throws IOException {
5011     long now = EnvironmentEdgeManager.currentTimeMillis();
5012 
5013     List<KeyValue> results = new ArrayList<KeyValue>();
5014 
5015     // pre-get CP hook
5016     if (withCoprocessor && (coprocessorHost != null)) {
5017        if (coprocessorHost.preGet(get, results)) {
5018          return results;
5019        }
5020     }
5021 
5022     Scan scan = new Scan(get);
5023 
5024     RegionScanner scanner = null;
5025     try {
5026       scanner = getScanner(scan);
5027       scanner.next(results, SchemaMetrics.METRIC_GETSIZE);
5028     } finally {
5029       if (scanner != null)
5030         scanner.close();
5031     }
5032 
5033     // post-get CP hook
5034     if (withCoprocessor && (coprocessorHost != null)) {
5035       coprocessorHost.postGet(get, results);
5036     }
5037 
5038     // do after lock
5039     final long after = EnvironmentEdgeManager.currentTimeMillis();
5040     this.opMetrics.updateGetMetrics(get.familySet(), after - now);
5041 
5042     return results;
5043   }
5044 
5045   public void mutateRow(RowMutations rm) throws IOException {
5046     mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
5047   }
5048 
5049   /**
5050    * Perform atomic mutations within the region.
5051    * @param mutations The list of mutations to perform.
5052    * <code>mutations</code> can contain operations for multiple rows.
5053    * Caller has to ensure that all rows are contained in this region.
5054    * @param rowsToLock Rows to lock
5055    * If multiple rows are locked care should be taken that
5056    * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
5057    * @throws IOException
5058    */
5059   public void mutateRowsWithLocks(Collection<Mutation> mutations,
5060       Collection<byte[]> rowsToLock) throws IOException {
5061     boolean flush = false;
5062 
5063     checkReadOnly();
5064     checkResources();
5065 
5066     startRegionOperation();
5067     List<Integer> acquiredLocks = null;
5068     try {
5069       // 1. run all pre-hooks before the atomic operation
5070       // if any pre hook indicates "bypass", bypass the entire operation
5071 
5072       // one WALEdit is used for all edits.
5073       WALEdit walEdit = new WALEdit();
5074       if (coprocessorHost != null) {
5075         for (Mutation m : mutations) {
5076           if (m instanceof Put) {
5077             if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
5078               // by pass everything
5079               return;
5080             }
5081           } else if (m instanceof Delete) {
5082             Delete d = (Delete) m;
5083             prepareDelete(d);
5084             if (coprocessorHost.preDelete(d, walEdit, d.getWriteToWAL())) {
5085               // by pass everything
5086               return;
5087             }
5088           }
5089         }
5090       }
5091 
5092       long txid = 0;
5093       boolean walSyncSuccessful = false;
5094       boolean memstoreUpdated = false;
5095       boolean locked = false;
5096 
5097       // 2. acquire the row lock(s)
5098       acquiredLocks = new ArrayList<Integer>(rowsToLock.size());
5099       for (byte[] row : rowsToLock) {
5100         // attempt to lock all involved rows, fail if one lock times out
5101         Integer lid = getLock(null, row, true);
5102         if (lid == null) {
5103           throw new IOException("Failed to acquire lock on "
5104               + Bytes.toStringBinary(row));
5105         }
5106         acquiredLocks.add(lid);
5107       }
5108 
5109       // 3. acquire the region lock
5110       lock(this.updatesLock.readLock(), acquiredLocks.size());
5111       locked = true;
5112 
5113       // 4. Get a mvcc write number
5114       MultiVersionConsistencyControl.WriteEntry w = mvcc.beginMemstoreInsert();
5115 
5116       long now = EnvironmentEdgeManager.currentTimeMillis();
5117       byte[] byteNow = Bytes.toBytes(now);
5118       Durability durability = Durability.USE_DEFAULT;
5119       try {
5120         // 5. Check mutations and apply edits to a single WALEdit
5121         for (Mutation m : mutations) {
5122           if (m instanceof Put) {
5123             Map<byte[], List<KeyValue>> familyMap = m.getFamilyMap();
5124             checkFamilies(familyMap.keySet());
5125             checkTimestamps(familyMap, now);
5126             updateKVTimestamps(familyMap.values(), byteNow);
5127           } else if (m instanceof Delete) {
5128             Delete d = (Delete) m;
5129             prepareDelete(d);
5130             prepareDeleteTimestamps(d.getFamilyMap(), byteNow);
5131           } else {
5132             throw new DoNotRetryIOException(
5133                 "Action must be Put or Delete. But was: "
5134                     + m.getClass().getName());
5135           }
5136           Durability tmpDur = m.getDurability(); 
5137           if (tmpDur.ordinal() > durability.ordinal()) {
5138             durability = tmpDur;
5139           }
5140           if (tmpDur != Durability.SKIP_WAL) {
5141             addFamilyMapToWALEdit(m.getFamilyMap(), walEdit);
5142           }
5143         }
5144 
5145         // 6. append all edits at once (don't sync)
5146         if (walEdit.size() > 0) {
5147           txid = this.log.appendNoSync(regionInfo,
5148               this.htableDescriptor.getName(), walEdit,
5149               HConstants.DEFAULT_CLUSTER_ID, now, this.htableDescriptor);
5150         }
5151 
5152         // 7. apply to memstore
5153         long addedSize = 0;
5154         memstoreUpdated = true;
5155         for (Mutation m : mutations) {
5156           addedSize += applyFamilyMapToMemstore(m.getFamilyMap(), w);
5157         }
5158         flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
5159 
5160         // 8. release region and row lock(s)
5161         this.updatesLock.readLock().unlock();
5162         locked = false;
5163         if (acquiredLocks != null) {
5164           for (Integer lid : acquiredLocks) {
5165             releaseRowLock(lid);
5166           }
5167           acquiredLocks = null;
5168         }
5169 
5170         // 9. sync WAL if required
5171         if (walEdit.size() > 0) {
5172           syncOrDefer(txid, durability);
5173         }
5174         walSyncSuccessful = true;
5175 
5176         // 10. advance mvcc
5177         mvcc.completeMemstoreInsert(w);
5178         w = null;
5179 
5180         // 11. run coprocessor post host hooks
5181         // after the WAL is sync'ed and all locks are released
5182         // (similar to doMiniBatchPut)
5183         if (coprocessorHost != null) {
5184           for (Mutation m : mutations) {
5185             if (m instanceof Put) {
5186               coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
5187             } else if (m instanceof Delete) {
5188               coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
5189             }
5190           }
5191         }
5192       } finally {
5193         // 12. clean up if needed
5194         if (memstoreUpdated && !walSyncSuccessful) {
5195           int kvsRolledback = 0;
5196           for (Mutation m : mutations) {
5197             for (Map.Entry<byte[], List<KeyValue>> e : m.getFamilyMap()
5198                 .entrySet()) {
5199               List<KeyValue> kvs = e.getValue();
5200               byte[] family = e.getKey();
5201               Store store = getStore(family);
5202               // roll back each kv
5203               for (KeyValue kv : kvs) {
5204                 store.rollback(kv);
5205                 kvsRolledback++;
5206               }
5207             }
5208           }
5209           LOG.info("mutateRowWithLocks: rolled back " + kvsRolledback
5210               + " KeyValues");
5211         }
5212 
5213         if (w != null) {
5214           mvcc.completeMemstoreInsert(w);
5215         }
5216 
5217         if (locked) {
5218           this.updatesLock.readLock().unlock();
5219         }
5220 
5221         if (acquiredLocks != null) {
5222           for (Integer lid : acquiredLocks) {
5223             releaseRowLock(lid);
5224           }
5225         }
5226       }
5227     } finally {
5228       if (flush) {
5229         // 13. Flush cache if needed. Do it outside update lock.
5230         requestFlush();
5231       }
5232       closeRegionOperation();
5233     }
5234   }
5235 
5236   // TODO: There's a lot of boiler plate code identical
5237   // to increment... See how to better unify that.
5238 
5239   /**
5240   *
5241   * Perform one or more append operations on a row.
5242   * <p>
5243   * Appends performed are done under row lock but reads do not take locks out
5244   * so this can be seen partially complete by gets and scans.
5245   *
5246   * @param append
5247   * @param writeToWAL
5248   * @return new keyvalues after increment
5249   * @throws IOException
5250   */
5251  public Result append(Append append, boolean writeToWAL)
5252      throws IOException {
5253    return append(append, null, writeToWAL);
5254  }
5255   /**
5256    *
5257    * Perform one or more append operations on a row.
5258    * <p>
5259    * Appends performed are done under row lock but reads do not take locks out
5260    * so this can be seen partially complete by gets and scans.
5261    *
5262    * @param append
5263    * @param lockid
5264    * @param writeToWAL
5265    * @return new keyvalues after increment
5266    * @throws IOException
5267    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
5268    */
5269   public Result append(Append append, Integer lockid, boolean writeToWAL)
5270       throws IOException {
5271     // TODO: Use MVCC to make this set of appends atomic to reads
5272     byte[] row = append.getRow();
5273     checkRow(row, "append");
5274     boolean flush = false;
5275     WALEdit walEdits = null;
5276     List<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
5277     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
5278     long before = EnvironmentEdgeManager.currentTimeMillis();
5279     long size = 0;
5280     long txid = 0;
5281 
5282     checkReadOnly();
5283     // Lock row
5284     startRegionOperation();
5285     this.writeRequestsCount.increment();
5286     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5287     try {
5288       Integer lid = getLock(lockid, row, true);
5289       lock(this.updatesLock.readLock());
5290       try {
5291         long now = EnvironmentEdgeManager.currentTimeMillis();
5292         // Process each family
5293         for (Map.Entry<byte[], List<KeyValue>> family : append.getFamilyMap()
5294             .entrySet()) {
5295 
5296           Store store = stores.get(family.getKey());
5297           Collections.sort(family.getValue(), store.getComparator());
5298           List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
5299 
5300           // Get previous values for all columns in this family
5301           Get get = new Get(row);
5302           for (KeyValue kv : family.getValue()) {
5303             get.addColumn(family.getKey(), kv.getQualifier());
5304           }
5305           List<KeyValue> results = get(get, false);
5306 
5307           // Iterate the input columns and update existing values if they were
5308           // found, otherwise add new column initialized to the append value
5309 
5310           // Avoid as much copying as possible. Every byte is copied at most
5311           // once.
5312           // Would be nice if KeyValue had scatter/gather logic
5313           int idx = 0;
5314           for (KeyValue kv : family.getValue()) {
5315             KeyValue newKV;
5316             if (idx < results.size()
5317                 && results.get(idx).matchingQualifier(kv.getBuffer(),
5318                     kv.getQualifierOffset(), kv.getQualifierLength())) {
5319               KeyValue oldKv = results.get(idx);
5320               // allocate an empty kv once
5321               newKV = new KeyValue(row.length, kv.getFamilyLength(),
5322                   kv.getQualifierLength(), now, KeyValue.Type.Put,
5323                   oldKv.getValueLength() + kv.getValueLength());
5324               // copy in the value
5325               System.arraycopy(oldKv.getBuffer(), oldKv.getValueOffset(),
5326                   newKV.getBuffer(), newKV.getValueOffset(),
5327                   oldKv.getValueLength());
5328               System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5329                   newKV.getBuffer(),
5330                   newKV.getValueOffset() + oldKv.getValueLength(),
5331                   kv.getValueLength());
5332               idx++;
5333             } else {
5334               // allocate an empty kv once
5335               newKV = new KeyValue(row.length, kv.getFamilyLength(),
5336                   kv.getQualifierLength(), now, KeyValue.Type.Put,
5337                   kv.getValueLength());
5338               // copy in the value
5339               System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5340                   newKV.getBuffer(), newKV.getValueOffset(),
5341                   kv.getValueLength());
5342             }
5343             // copy in row, family, and qualifier
5344             System.arraycopy(kv.getBuffer(), kv.getRowOffset(),
5345                 newKV.getBuffer(), newKV.getRowOffset(), kv.getRowLength());
5346             System.arraycopy(kv.getBuffer(), kv.getFamilyOffset(),
5347                 newKV.getBuffer(), newKV.getFamilyOffset(),
5348                 kv.getFamilyLength());
5349             System.arraycopy(kv.getBuffer(), kv.getQualifierOffset(),
5350                 newKV.getBuffer(), newKV.getQualifierOffset(),
5351                 kv.getQualifierLength());
5352 
5353             kvs.add(newKV);
5354 
5355             // Append update to WAL
5356             if (writeToWAL) {
5357               if (walEdits == null) {
5358                 walEdits = new WALEdit();
5359               }
5360               walEdits.add(newKV);
5361             }
5362           }
5363 
5364           // store the kvs to the temporary memstore before writing HLog
5365           tempMemstore.put(store, kvs);
5366         }
5367 
5368         // Actually write to WAL now
5369         if (writeToWAL) {
5370           // Using default cluster id, as this can only happen in the orginating
5371           // cluster. A slave cluster receives the final value (not the delta)
5372           // as a Put.
5373           txid = this.log.appendNoSync(regionInfo,
5374               this.htableDescriptor.getName(), walEdits,
5375               HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5376               this.htableDescriptor);
5377         }
5378         // Actually write to Memstore now
5379         for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5380           Store store = entry.getKey();
5381           size += store.upsert(entry.getValue());
5382           allKVs.addAll(entry.getValue());
5383         }
5384         size = this.addAndGetGlobalMemstoreSize(size);
5385         flush = isFlushSize(size);
5386       } finally {
5387         this.updatesLock.readLock().unlock();
5388         releaseRowLock(lid);
5389       }
5390       if (writeToWAL) {
5391         // sync the transaction log outside the rowlock
5392         syncOrDefer(txid, append.getDurability());
5393       }
5394     } finally {
5395       closeRegionOperation();
5396     }
5397 
5398 
5399     long after = EnvironmentEdgeManager.currentTimeMillis();
5400     this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
5401 
5402     if (flush) {
5403       // Request a cache flush. Do it outside update lock.
5404       requestFlush();
5405     }
5406 
5407     return append.isReturnResults() ? new Result(allKVs) : null;
5408   }
5409 
5410   /**
5411   *
5412   * Perform one or more increment operations on a row.
5413   * <p>
5414   * Increments performed are done under row lock but reads do not take locks
5415   * out so this can be seen partially complete by gets and scans.
5416   * @param increment
5417   * @param writeToWAL
5418   * @return new keyvalues after increment
5419   * @throws IOException
5420   */
5421   public Result increment(Increment increment, boolean writeToWAL)
5422   throws IOException {
5423     return increment(increment, null, writeToWAL);
5424   }
5425 
5426   /**
5427    *
5428    * Perform one or more increment operations on a row.
5429    * <p>
5430    * Increments performed are done under row lock but reads do not take locks
5431    * out so this can be seen partially complete by gets and scans.
5432    * @param increment
5433    * @param lockid
5434    * @param writeToWAL
5435    * @return new keyvalues after increment
5436    * @throws IOException
5437    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
5438 
5439    */
5440   public Result increment(Increment increment, Integer lockid,
5441       boolean writeToWAL)
5442   throws IOException {
5443     // TODO: Use MVCC to make this set of increments atomic to reads
5444     byte [] row = increment.getRow();
5445     checkRow(row, "increment");
5446     TimeRange tr = increment.getTimeRange();
5447     boolean flush = false;
5448     WALEdit walEdits = null;
5449     List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
5450     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
5451     long before = EnvironmentEdgeManager.currentTimeMillis();
5452     long size = 0;
5453     long txid = 0;
5454 
5455     checkReadOnly();
5456     // Lock row
5457     startRegionOperation();
5458     this.writeRequestsCount.increment();
5459     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5460     try {
5461       Integer lid = getLock(lockid, row, true);
5462       lock(this.updatesLock.readLock());
5463       try {
5464         long now = EnvironmentEdgeManager.currentTimeMillis();
5465         // Process each family
5466         for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
5467           increment.getFamilyMap().entrySet()) {
5468 
5469           Store store = stores.get(family.getKey());
5470           List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
5471 
5472           // Get previous values for all columns in this family
5473           Get get = new Get(row);
5474           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5475             get.addColumn(family.getKey(), column.getKey());
5476           }
5477           get.setTimeRange(tr.getMin(), tr.getMax());
5478           List<KeyValue> results = get(get, false);
5479 
5480           // Iterate the input columns and update existing values if they were
5481           // found, otherwise add new column initialized to the increment amount
5482           int idx = 0;
5483           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5484             long amount = column.getValue();
5485             if (idx < results.size() &&
5486                 results.get(idx).matchingQualifier(column.getKey())) {
5487               KeyValue kv = results.get(idx);
5488               if(kv.getValueLength() == Bytes.SIZEOF_LONG) {
5489                 amount += Bytes.toLong(kv.getBuffer(), kv.getValueOffset(), Bytes.SIZEOF_LONG);
5490               } else {
5491                 // throw DoNotRetryIOException instead of IllegalArgumentException
5492                 throw new DoNotRetryIOException(
5493                     "Attempted to increment field that isn't 64 bits wide");
5494               }
5495               idx++;
5496             }
5497 
5498             // Append new incremented KeyValue to list
5499             KeyValue newKV = new KeyValue(row, family.getKey(), column.getKey(),
5500                 now, Bytes.toBytes(amount));
5501             kvs.add(newKV);
5502 
5503             // Append update to WAL
5504             if (writeToWAL) {
5505               if (walEdits == null) {
5506                 walEdits = new WALEdit();
5507               }
5508               walEdits.add(newKV);
5509             }
5510           }
5511 
5512           //store the kvs to the temporary memstore before writing HLog
5513           tempMemstore.put(store, kvs);
5514         }
5515 
5516         // Actually write to WAL now
5517         if (writeToWAL) {
5518           // Using default cluster id, as this can only happen in the orginating
5519           // cluster. A slave cluster receives the final value (not the delta)
5520           // as a Put.
5521           txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5522               walEdits, HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5523               this.htableDescriptor);
5524         }
5525 
5526         //Actually write to Memstore now
5527         for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5528           Store store = entry.getKey();
5529           size += store.upsert(entry.getValue());
5530           allKVs.addAll(entry.getValue());
5531         }
5532         size = this.addAndGetGlobalMemstoreSize(size);
5533         flush = isFlushSize(size);
5534       } finally {
5535         this.updatesLock.readLock().unlock();
5536         releaseRowLock(lid);
5537       }
5538       if (writeToWAL) {
5539         // sync the transaction log outside the rowlock
5540         syncOrDefer(txid, Durability.USE_DEFAULT);
5541       }
5542     } finally {
5543       closeRegionOperation();
5544       long after = EnvironmentEdgeManager.currentTimeMillis();
5545       this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
5546     }
5547 
5548     if (flush) {
5549       // Request a cache flush.  Do it outside update lock.
5550       requestFlush();
5551     }
5552 
5553     return new Result(allKVs);
5554   }
5555 
5556   /**
5557    * @param row
5558    * @param family
5559    * @param qualifier
5560    * @param amount
5561    * @param writeToWAL
5562    * @return The new value.
5563    * @throws IOException
5564    */
5565   public long incrementColumnValue(byte [] row, byte [] family,
5566       byte [] qualifier, long amount, boolean writeToWAL)
5567   throws IOException {
5568     // to be used for metrics
5569     long before = EnvironmentEdgeManager.currentTimeMillis();
5570 
5571     checkRow(row, "increment");
5572     boolean flush = false;
5573     boolean wrongLength = false;
5574     long txid = 0;
5575     // Lock row
5576     long result = amount;
5577     startRegionOperation();
5578     this.writeRequestsCount.increment();
5579     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5580     try {
5581       Integer lid = obtainRowLock(row);
5582       lock(this.updatesLock.readLock());
5583       try {
5584         Store store = stores.get(family);
5585 
5586         // Get the old value:
5587         Get get = new Get(row);
5588         get.addColumn(family, qualifier);
5589 
5590         // we don't want to invoke coprocessor in this case; ICV is wrapped
5591         // in HRegionServer, so we leave getLastIncrement alone
5592         List<KeyValue> results = get(get, false);
5593 
5594         if (!results.isEmpty()) {
5595           KeyValue kv = results.get(0);
5596           if(kv.getValueLength() == Bytes.SIZEOF_LONG){
5597             byte [] buffer = kv.getBuffer();
5598             int valueOffset = kv.getValueOffset();
5599             result += Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG);
5600           }
5601           else{
5602             wrongLength = true;
5603           }
5604         }
5605         if(!wrongLength){
5606           // build the KeyValue now:
5607           KeyValue newKv = new KeyValue(row, family,
5608             qualifier, EnvironmentEdgeManager.currentTimeMillis(),
5609             Bytes.toBytes(result));
5610 
5611           // now log it:
5612           if (writeToWAL) {
5613             long now = EnvironmentEdgeManager.currentTimeMillis();
5614             WALEdit walEdit = new WALEdit();
5615             walEdit.add(newKv);
5616             // Using default cluster id, as this can only happen in the
5617             // orginating cluster. A slave cluster receives the final value (not
5618             // the delta) as a Put.
5619             txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5620                 walEdit, HConstants.DEFAULT_CLUSTER_ID, now,
5621                 this.htableDescriptor);
5622           }
5623 
5624           // Now request the ICV to the store, this will set the timestamp
5625           // appropriately depending on if there is a value in memcache or not.
5626           // returns the change in the size of the memstore from operation
5627           long size = store.updateColumnValue(row, family, qualifier, result);
5628 
5629           size = this.addAndGetGlobalMemstoreSize(size);
5630           flush = isFlushSize(size);
5631         }
5632       } finally {
5633         this.updatesLock.readLock().unlock();
5634         releaseRowLock(lid);
5635       }
5636       if (writeToWAL) {
5637         // sync the transaction log outside the rowlock
5638         syncOrDefer(txid, Durability.USE_DEFAULT);
5639       }
5640     } finally {
5641       closeRegionOperation();
5642     }
5643 
5644     // do after lock
5645     long after = EnvironmentEdgeManager.currentTimeMillis();
5646     this.opMetrics.updateIncrementColumnValueMetrics(family, after - before);
5647 
5648     if (flush) {
5649       // Request a cache flush.  Do it outside update lock.
5650       requestFlush();
5651     }
5652     if(wrongLength){
5653       throw new DoNotRetryIOException(
5654           "Attempted to increment field that isn't 64 bits wide");
5655     }
5656     return result;
5657   }
5658 
5659 
5660   //
5661   // New HBASE-880 Helpers
5662   //
5663 
5664   private void checkFamily(final byte [] family)
5665   throws NoSuchColumnFamilyException {
5666     if (!this.htableDescriptor.hasFamily(family)) {
5667       throw new NoSuchColumnFamilyException("Column family " +
5668           Bytes.toString(family) + " does not exist in region " + this
5669           + " in table " + this.htableDescriptor);
5670   	}
5671   }
5672 
5673   public static final long FIXED_OVERHEAD = ClassSize.align(
5674       ClassSize.OBJECT +
5675       ClassSize.ARRAY +
5676       36 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
5677       (8 * Bytes.SIZEOF_LONG) +
5678       Bytes.SIZEOF_BOOLEAN);
5679 
5680   public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
5681       ClassSize.OBJECT + // closeLock
5682       (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
5683       (3 * ClassSize.ATOMIC_LONG) + // memStoreSize, numPutsWithoutWAL, dataInMemoryWithoutWAL
5684       ClassSize.ATOMIC_INTEGER + // lockIdGenerator
5685       (3 * ClassSize.CONCURRENT_HASHMAP) +  // lockedRows, lockIds, scannerReadPoints
5686       WriteState.HEAP_SIZE + // writestate
5687       ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
5688       (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
5689       ClassSize.ARRAYLIST + // recentFlushes
5690       MultiVersionConsistencyControl.FIXED_SIZE // mvcc
5691       ;
5692 
5693   @Override
5694   public long heapSize() {
5695     long heapSize = DEEP_OVERHEAD;
5696     for(Store store : this.stores.values()) {
5697       heapSize += store.heapSize();
5698     }
5699     // this does not take into account row locks, recent flushes, mvcc entries
5700     return heapSize;
5701   }
5702 
5703   /*
5704    * This method calls System.exit.
5705    * @param message Message to print out.  May be null.
5706    */
5707   private static void printUsageAndExit(final String message) {
5708     if (message != null && message.length() > 0) System.out.println(message);
5709     System.out.println("Usage: HRegion CATLALOG_TABLE_DIR [major_compact]");
5710     System.out.println("Options:");
5711     System.out.println(" major_compact  Pass this option to major compact " +
5712       "passed region.");
5713     System.out.println("Default outputs scan of passed region.");
5714     System.exit(1);
5715   }
5716 
5717   /**
5718    * Registers a new CoprocessorProtocol subclass and instance to
5719    * be available for handling {@link HRegion#exec(Exec)} calls.
5720    *
5721    * <p>
5722    * Only a single protocol type/handler combination may be registered per
5723    * region.
5724    * After the first registration, subsequent calls with the same protocol type
5725    * will fail with a return value of {@code false}.
5726    * </p>
5727    * @param protocol a {@code CoprocessorProtocol} subinterface defining the
5728    * protocol methods
5729    * @param handler an instance implementing the interface
5730    * @param <T> the protocol type
5731    * @return {@code true} if the registration was successful, {@code false}
5732    * otherwise
5733    */
5734   public <T extends CoprocessorProtocol> boolean registerProtocol(
5735       Class<T> protocol, T handler) {
5736 
5737     /* No stacking of protocol handlers is currently allowed.  The
5738      * first to claim wins!
5739      */
5740     if (protocolHandlers.containsKey(protocol)) {
5741       LOG.error("Protocol "+protocol.getName()+
5742           " already registered, rejecting request from "+
5743           handler
5744       );
5745       return false;
5746     }
5747 
5748     protocolHandlers.putInstance(protocol, handler);
5749     protocolHandlerNames.put(protocol.getName(), protocol);
5750     if (LOG.isDebugEnabled()) {
5751       LOG.debug("Registered protocol handler: region="+
5752           Bytes.toStringBinary(getRegionName())+" protocol="+protocol.getName());
5753     }
5754     return true;
5755   }
5756 
5757   /**
5758    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
5759    * method using the registered protocol handlers.
5760    * {@link CoprocessorProtocol} implementations must be registered via the
5761    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
5762    * method before they are available.
5763    *
5764    * @param call an {@code Exec} instance identifying the protocol, method name,
5765    *     and parameters for the method invocation
5766    * @return an {@code ExecResult} instance containing the region name of the
5767    *     invocation and the return value
5768    * @throws IOException if no registered protocol handler is found or an error
5769    *     occurs during the invocation
5770    * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
5771    */
5772   public ExecResult exec(Exec call)
5773       throws IOException {
5774     Class<? extends CoprocessorProtocol> protocol = call.getProtocol();
5775     if (protocol == null) {
5776       String protocolName = call.getProtocolName();
5777       if (LOG.isTraceEnabled()) {
5778         LOG.trace("Received dynamic protocol exec call with protocolName " + protocolName);
5779       }
5780       // detect the actual protocol class
5781       protocol  = protocolHandlerNames.get(protocolName);
5782       if (protocol == null) {
5783         throw new HBaseRPC.UnknownProtocolException(protocol,
5784             "No matching handler for protocol "+protocolName+
5785             " in region "+Bytes.toStringBinary(getRegionName()));
5786       }
5787     }
5788     if (!protocolHandlers.containsKey(protocol)) {
5789       throw new HBaseRPC.UnknownProtocolException(protocol,
5790           "No matching handler for protocol "+protocol.getName()+
5791           " in region "+Bytes.toStringBinary(getRegionName()));
5792     }
5793 
5794     CoprocessorProtocol handler = protocolHandlers.getInstance(protocol);
5795     Object value;
5796 
5797     try {
5798       Method method = protocol.getMethod(
5799           call.getMethodName(), call.getParameterClasses());
5800       method.setAccessible(true);
5801 
5802       value = method.invoke(handler, call.getParameters());
5803     } catch (InvocationTargetException e) {
5804       Throwable target = e.getTargetException();
5805       if (target instanceof IOException) {
5806         throw (IOException)target;
5807       }
5808       IOException ioe = new IOException(target.toString());
5809       ioe.setStackTrace(target.getStackTrace());
5810       throw ioe;
5811     } catch (Throwable e) {
5812       if (!(e instanceof IOException)) {
5813         LOG.error("Unexpected throwable object ", e);
5814       }
5815       IOException ioe = new IOException(e.toString());
5816       ioe.setStackTrace(e.getStackTrace());
5817       throw ioe;
5818     }
5819 
5820     return new ExecResult(getRegionName(), value);
5821   }
5822 
5823   /*
5824    * Process table.
5825    * Do major compaction or list content.
5826    * @param fs
5827    * @param p
5828    * @param log
5829    * @param c
5830    * @param majorCompact
5831    * @throws IOException
5832    */
5833   private static void processTable(final FileSystem fs, final Path p,
5834       final HLog log, final Configuration c,
5835       final boolean majorCompact)
5836   throws IOException {
5837     HRegion region = null;
5838     String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
5839     String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
5840     // Currently expects tables have one region only.
5841     if (p.getName().startsWith(rootStr)) {
5842       region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
5843         HTableDescriptor.ROOT_TABLEDESC, null);
5844     } else if (p.getName().startsWith(metaStr)) {
5845       region = HRegion.newHRegion(p, log, fs, c,
5846         HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
5847     } else {
5848       throw new IOException("Not a known catalog table: " + p.toString());
5849     }
5850     try {
5851       region.initialize();
5852       if (majorCompact) {
5853         region.compactStores(true);
5854       } else {
5855         // Default behavior
5856         Scan scan = new Scan();
5857         // scan.addFamily(HConstants.CATALOG_FAMILY);
5858         RegionScanner scanner = region.getScanner(scan);
5859         try {
5860           List<KeyValue> kvs = new ArrayList<KeyValue>();
5861           boolean done = false;
5862           do {
5863             kvs.clear();
5864             done = scanner.next(kvs);
5865             if (kvs.size() > 0) LOG.info(kvs);
5866           } while (done);
5867         } finally {
5868           scanner.close();
5869         }
5870       }
5871     } finally {
5872       region.close();
5873     }
5874   }
5875 
5876   boolean shouldForceSplit() {
5877     return this.splitRequest;
5878   }
5879 
5880   byte[] getExplicitSplitPoint() {
5881     return this.explicitSplitPoint;
5882   }
5883 
5884   void forceSplit(byte[] sp) {
5885     // NOTE : this HRegion will go away after the forced split is successfull
5886     //        therefore, no reason to clear this value
5887     this.splitRequest = true;
5888     if (sp != null) {
5889       this.explicitSplitPoint = sp;
5890     }
5891   }
5892 
5893   void clearSplit_TESTS_ONLY() {
5894     this.splitRequest = false;
5895   }
5896 
5897   /**
5898    * Give the region a chance to prepare before it is split.
5899    */
5900   protected void prepareToSplit() {
5901     // nothing
5902   }
5903 
5904   /**
5905    * Return the splitpoint. null indicates the region isn't splittable
5906    * If the splitpoint isn't explicitly specified, it will go over the stores
5907    * to find the best splitpoint. Currently the criteria of best splitpoint
5908    * is based on the size of the store.
5909    */
5910   public byte[] checkSplit() {
5911     // Can't split ROOT/META
5912     if (this.regionInfo.isMetaTable()) {
5913       if (shouldForceSplit()) {
5914         LOG.warn("Cannot split root/meta regions in HBase 0.20 and above");
5915       }
5916       return null;
5917     }
5918 
5919     if (!splitPolicy.shouldSplit()) {
5920       return null;
5921     }
5922 
5923     byte[] ret = splitPolicy.getSplitPoint();
5924 
5925     if (ret != null) {
5926       try {
5927         checkRow(ret, "calculated split");
5928       } catch (IOException e) {
5929         LOG.error("Ignoring invalid split", e);
5930         return null;
5931       }
5932     }
5933     return ret;
5934   }
5935 
5936   /**
5937    * @return The priority that this region should have in the compaction queue
5938    */
5939   public int getCompactPriority() {
5940     int count = Integer.MAX_VALUE;
5941     for(Store store : stores.values()) {
5942       count = Math.min(count, store.getCompactPriority());
5943     }
5944     return count;
5945   }
5946 
5947   /**
5948    * Checks every store to see if one has too many
5949    * store files
5950    * @return true if any store has too many store files
5951    */
5952   public boolean needsCompaction() {
5953     for(Store store : stores.values()) {
5954       if(store.needsCompaction()) {
5955         return true;
5956       }
5957     }
5958     return false;
5959   }
5960 
5961   /** @return the coprocessor host */
5962   public RegionCoprocessorHost getCoprocessorHost() {
5963     return coprocessorHost;
5964   }
5965 
5966   /*
5967    * Set the read request count defined in opMetrics
5968    * @param value absolute value of read request count
5969    */
5970   public void setOpMetricsReadRequestCount(long value)
5971   {
5972     this.opMetrics.setReadRequestCountMetrics(value);
5973   }
5974   
5975   /*
5976    * Set the write request count defined in opMetrics
5977    * @param value absolute value of write request count
5978    */
5979   public void setOpMetricsWriteRequestCount(long value)
5980   {
5981     this.opMetrics.setWriteRequestCountMetrics(value);
5982   }
5983   
5984   /** @param coprocessorHost the new coprocessor host */
5985   public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
5986     this.coprocessorHost = coprocessorHost;
5987   }
5988 
5989   /**
5990    * This method needs to be called before any public call that reads or
5991    * modifies data. It has to be called just before a try.
5992    * #closeRegionOperation needs to be called in the try's finally block
5993    * Acquires a read lock and checks if the region is closing or closed.
5994    * @throws NotServingRegionException when the region is closing or closed
5995    * @throws RegionTooBusyException if failed to get the lock in time
5996    * @throws InterruptedIOException if interrupted while waiting for a lock
5997    */
5998   public void startRegionOperation()
5999       throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
6000     if (this.closing.get()) {
6001       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
6002           " is closing");
6003     }
6004     lock(lock.readLock());
6005     if (this.closed.get()) {
6006       lock.readLock().unlock();
6007       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
6008           " is closed");
6009     }
6010   }
6011 
6012   /**
6013    * Closes the lock. This needs to be called in the finally block corresponding
6014    * to the try block of #startRegionOperation
6015    */
6016   public void closeRegionOperation(){
6017     lock.readLock().unlock();
6018   }
6019 
6020   /**
6021    * This method needs to be called before any public call that reads or
6022    * modifies stores in bulk. It has to be called just before a try.
6023    * #closeBulkRegionOperation needs to be called in the try's finally block
6024    * Acquires a writelock and checks if the region is closing or closed.
6025    * @throws NotServingRegionException when the region is closing or closed
6026    * @throws RegionTooBusyException if failed to get the lock in time
6027    * @throws InterruptedIOException if interrupted while waiting for a lock
6028    */
6029   private void startBulkRegionOperation(boolean writeLockNeeded)
6030       throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
6031     if (this.closing.get()) {
6032       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
6033           " is closing");
6034     }
6035     if (writeLockNeeded) lock(lock.writeLock());
6036     else lock(lock.readLock());
6037     if (this.closed.get()) {
6038       if (writeLockNeeded) lock.writeLock().unlock();
6039       else lock.readLock().unlock();
6040       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
6041           " is closed");
6042     }
6043   }
6044 
6045   /**
6046    * Closes the lock. This needs to be called in the finally block corresponding
6047    * to the try block of #startRegionOperation
6048    */
6049   private void closeBulkRegionOperation() {
6050     if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
6051     else lock.readLock().unlock();
6052   }
6053 
6054   /**
6055    * Update counters for numer of puts without wal and the size of possible data loss.
6056    * These information are exposed by the region server metrics.
6057    */
6058   private void recordPutWithoutWal(final Map<byte [], List<KeyValue>> familyMap) {
6059     if (numPutsWithoutWAL.getAndIncrement() == 0) {
6060       LOG.info("writing data to region " + this +
6061                " with WAL disabled. Data may be lost in the event of a crash.");
6062     }
6063 
6064     long putSize = 0;
6065     for (List<KeyValue> edits : familyMap.values()) {
6066       for (KeyValue kv : edits) {
6067         putSize += kv.getKeyLength() + kv.getValueLength();
6068       }
6069     }
6070 
6071     dataInMemoryWithoutWAL.addAndGet(putSize);
6072   }
6073 
6074   private void lock(final Lock lock)
6075       throws RegionTooBusyException, InterruptedIOException {
6076     lock(lock, 1);
6077   }
6078 
6079   /**
6080    * Try to acquire a lock.  Throw RegionTooBusyException
6081    * if failed to get the lock in time. Throw InterruptedIOException
6082    * if interrupted while waiting for the lock.
6083    */
6084   private void lock(final Lock lock, final int multiplier)
6085       throws RegionTooBusyException, InterruptedIOException {
6086     try {
6087       final long waitTime = Math.min(maxBusyWaitDuration,
6088         busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
6089       if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
6090         throw new RegionTooBusyException(
6091           "failed to get a lock in " + waitTime + " ms. " +
6092             "regionName=" + (this.getRegionInfo() == null ? "unknown" :
6093             this.getRegionInfo().getRegionNameAsString()) +
6094             ", server=" + (this.getRegionServerServices() == null ? "unknown" :
6095             this.getRegionServerServices().getServerName()));
6096       }
6097     } catch (InterruptedException ie) {
6098       LOG.info("Interrupted while waiting for a lock");
6099       InterruptedIOException iie = new InterruptedIOException();
6100       iie.initCause(ie);
6101       throw iie;
6102     }
6103   }
6104 
6105   /**
6106    * Calls sync with the given transaction ID if the region's table is not
6107    * deferring it.
6108    * @param txid should sync up to which transaction
6109    * @throws IOException If anything goes wrong with DFS
6110    */
6111   private void syncOrDefer(long txid, Durability durability) throws IOException {
6112     if (this.getRegionInfo().isMetaRegion()) {
6113       this.log.sync(txid);
6114     } else {
6115       switch(durability) {
6116       case USE_DEFAULT:
6117         // do what CF defaults to
6118         if (!isDeferredLogSyncEnabled()) {
6119           this.log.sync(txid);
6120         }
6121         break;
6122       case SKIP_WAL:
6123         // nothing do to
6124         break;
6125       case ASYNC_WAL:
6126         // defer the sync, unless we globally can't
6127         if (this.deferredLogSyncDisabled) {
6128           this.log.sync(txid);
6129         }
6130         break;
6131       case SYNC_WAL:
6132       case FSYNC_WAL:
6133         // sync the WAL edit (SYNC and FSYNC treated the same for now)
6134         this.log.sync(txid);
6135         break;
6136       }
6137     }
6138   }
6139 
6140   /**
6141    * check if current region is deferred sync enabled.
6142    */
6143   private boolean isDeferredLogSyncEnabled() {
6144     return (this.htableDescriptor.isDeferredLogFlush() && !this.deferredLogSyncDisabled);
6145   }
6146 
6147   /**
6148    * A mocked list implementaion - discards all updates.
6149    */
6150   private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
6151 
6152     @Override
6153     public void add(int index, KeyValue element) {
6154       // do nothing
6155     }
6156 
6157     @Override
6158     public boolean addAll(int index, Collection<? extends KeyValue> c) {
6159       return false; // this list is never changed as a result of an update
6160     }
6161 
6162     @Override
6163     public KeyValue get(int index) {
6164       throw new UnsupportedOperationException();
6165     }
6166 
6167     @Override
6168     public int size() {
6169       return 0;
6170     }
6171   };
6172 
6173   /**
6174    * Facility for dumping and compacting catalog tables.
6175    * Only does catalog tables since these are only tables we for sure know
6176    * schema on.  For usage run:
6177    * <pre>
6178    *   ./bin/hbase org.apache.hadoop.hbase.regionserver.HRegion
6179    * </pre>
6180    * @param args
6181    * @throws IOException
6182    */
6183   public static void main(String[] args) throws IOException {
6184     if (args.length < 1) {
6185       printUsageAndExit(null);
6186     }
6187     boolean majorCompact = false;
6188     if (args.length > 1) {
6189       if (!args[1].toLowerCase().startsWith("major")) {
6190         printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
6191       }
6192       majorCompact = true;
6193     }
6194     final Path tableDir = new Path(args[0]);
6195     final Configuration c = HBaseConfiguration.create();
6196     final FileSystem fs = FileSystem.get(c);
6197     final Path logdir = new Path(c.get("hbase.tmp.dir"),
6198         "hlog" + tableDir.getName()
6199         + EnvironmentEdgeManager.currentTimeMillis());
6200     final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
6201         HConstants.HREGION_OLDLOGDIR_NAME);
6202     final HLog log = new HLog(fs, logdir, oldLogDir, c);
6203     try {
6204       processTable(fs, tableDir, log, c, majorCompact);
6205     } finally {
6206        log.close();
6207        // TODO: is this still right?
6208        BlockCache bc = new CacheConfig(c).getBlockCache();
6209        if (bc != null) bc.shutdown();
6210     }
6211   }
6212 
6213   /**
6214    * Listener class to enable callers of
6215    * bulkLoadHFile() to perform any necessary
6216    * pre/post processing of a given bulkload call
6217    */
6218   public static interface BulkLoadListener {
6219 
6220     /**
6221      * Called before an HFile is actually loaded
6222      * @param family family being loaded to
6223      * @param srcPath path of HFile
6224      * @return final path to be used for actual loading
6225      * @throws IOException
6226      */
6227     String prepareBulkLoad(byte[] family, String srcPath) throws IOException;
6228 
6229     /**
6230      * Called after a successful HFile load
6231      * @param family family being loaded to
6232      * @param srcPath path of HFile
6233      * @throws IOException
6234      */
6235     void doneBulkLoad(byte[] family, String srcPath) throws IOException;
6236 
6237     /**
6238      * Called after a failed HFile load
6239      * @param family family being loaded to
6240      * @param srcPath path of HFile
6241      * @throws IOException
6242      */
6243     void failedBulkLoad(byte[] family, String srcPath) throws IOException;
6244 
6245   }
6246 }