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