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.IOException;
24  import java.io.InterruptedIOException;
25  import java.io.UnsupportedEncodingException;
26  import java.lang.reflect.Constructor;
27  import java.lang.reflect.InvocationTargetException;
28  import java.lang.reflect.Method;
29  import java.text.ParseException;
30  import java.util.AbstractList;
31  import java.util.ArrayList;
32  import java.util.Arrays;
33  import java.util.Collection;
34  import java.util.Collections;
35  import java.util.HashMap;
36  import java.util.List;
37  import java.util.Map;
38  import java.util.NavigableMap;
39  import java.util.NavigableSet;
40  import java.util.Random;
41  import java.util.TreeMap;
42  import java.util.UUID;
43  import java.util.concurrent.ConcurrentHashMap;
44  import java.util.concurrent.ConcurrentSkipListMap;
45  import java.util.concurrent.CountDownLatch;
46  import java.util.concurrent.TimeUnit;
47  import java.util.concurrent.atomic.AtomicBoolean;
48  import java.util.concurrent.atomic.AtomicInteger;
49  import java.util.concurrent.atomic.AtomicLong;
50  import java.util.concurrent.locks.ReentrantReadWriteLock;
51  
52  import org.apache.commons.logging.Log;
53  import org.apache.commons.logging.LogFactory;
54  import org.apache.hadoop.conf.Configuration;
55  import org.apache.hadoop.fs.FSDataOutputStream;
56  import org.apache.hadoop.fs.FileStatus;
57  import org.apache.hadoop.fs.FileSystem;
58  import org.apache.hadoop.fs.Path;
59  import org.apache.hadoop.hbase.DoNotRetryIOException;
60  import org.apache.hadoop.hbase.DroppedSnapshotException;
61  import org.apache.hadoop.hbase.HBaseConfiguration;
62  import org.apache.hadoop.hbase.HColumnDescriptor;
63  import org.apache.hadoop.hbase.HConstants;
64  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
65  import org.apache.hadoop.hbase.HRegionInfo;
66  import org.apache.hadoop.hbase.HTableDescriptor;
67  import org.apache.hadoop.hbase.KeyValue;
68  import org.apache.hadoop.hbase.NotServingRegionException;
69  import org.apache.hadoop.hbase.UnknownScannerException;
70  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
71  import org.apache.hadoop.hbase.client.Delete;
72  import org.apache.hadoop.hbase.client.Get;
73  import org.apache.hadoop.hbase.client.Increment;
74  import org.apache.hadoop.hbase.client.Put;
75  import org.apache.hadoop.hbase.client.Result;
76  import org.apache.hadoop.hbase.client.Row;
77  import org.apache.hadoop.hbase.client.RowLock;
78  import org.apache.hadoop.hbase.client.Scan;
79  import org.apache.hadoop.hbase.client.coprocessor.Exec;
80  import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
81  import org.apache.hadoop.hbase.filter.Filter;
82  import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
83  import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
84  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
85  import org.apache.hadoop.hbase.io.HeapSize;
86  import org.apache.hadoop.hbase.io.TimeRange;
87  import org.apache.hadoop.hbase.io.hfile.BlockCache;
88  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
89  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
90  import org.apache.hadoop.hbase.ipc.HBaseRPC;
91  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
92  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
93  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
94  import org.apache.hadoop.hbase.regionserver.wal.HLog;
95  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
96  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
97  import org.apache.hadoop.hbase.util.Bytes;
98  import org.apache.hadoop.hbase.util.CancelableProgressable;
99  import org.apache.hadoop.hbase.util.ClassSize;
100 import org.apache.hadoop.hbase.util.CompressionTest;
101 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
102 import org.apache.hadoop.hbase.util.FSUtils;
103 import org.apache.hadoop.hbase.util.HashedBytes;
104 import org.apache.hadoop.hbase.util.Pair;
105 import org.apache.hadoop.hbase.util.Writables;
106 import org.apache.hadoop.io.MultipleIOException;
107 import org.apache.hadoop.io.Writable;
108 import org.apache.hadoop.util.StringUtils;
109 import org.cliffc.high_scale_lib.Counter;
110 
111 import com.google.common.base.Preconditions;
112 import com.google.common.collect.ClassToInstanceMap;
113 import com.google.common.collect.Lists;
114 import com.google.common.collect.MutableClassToInstanceMap;
115 
116 /**
117  * HRegion stores data for a certain region of a table.  It stores all columns
118  * for each row. A given table consists of one or more HRegions.
119  *
120  * <p>We maintain multiple HStores for a single HRegion.
121  *
122  * <p>An Store is a set of rows with some column data; together,
123  * they make up all the data for the rows.
124  *
125  * <p>Each HRegion has a 'startKey' and 'endKey'.
126  * <p>The first is inclusive, the second is exclusive (except for
127  * the final region)  The endKey of region 0 is the same as
128  * startKey for region 1 (if it exists).  The startKey for the
129  * first region is null. The endKey for the final region is null.
130  *
131  * <p>Locking at the HRegion level serves only one purpose: preventing the
132  * region from being closed (and consequently split) while other operations
133  * are ongoing. Each row level operation obtains both a row lock and a region
134  * read lock for the duration of the operation. While a scanner is being
135  * constructed, getScanner holds a read lock. If the scanner is successfully
136  * constructed, it holds a read lock until it is closed. A close takes out a
137  * write lock and consequently will block for ongoing operations and will block
138  * new operations from starting while the close is in progress.
139  *
140  * <p>An HRegion is defined by its table and its key extent.
141  *
142  * <p>It consists of at least one Store.  The number of Stores should be
143  * configurable, so that data which is accessed together is stored in the same
144  * Store.  Right now, we approximate that by building a single Store for
145  * each column family.  (This config info will be communicated via the
146  * tabledesc.)
147  *
148  * <p>The HTableDescriptor contains metainfo about the HRegion's table.
149  * regionName is a unique identifier for this HRegion. (startKey, endKey]
150  * defines the keyspace for this HRegion.
151  */
152 public class HRegion implements HeapSize { // , Writable{
153   public static final Log LOG = LogFactory.getLog(HRegion.class);
154   static final String MERGEDIR = "merges";
155 
156   final AtomicBoolean closed = new AtomicBoolean(false);
157   /* Closing can take some time; use the closing flag if there is stuff we don't
158    * want to do while in closing state; e.g. like offer this region up to the
159    * master as a region to close if the carrying regionserver is overloaded.
160    * Once set, it is never cleared.
161    */
162   final AtomicBoolean closing = new AtomicBoolean(false);
163 
164   //////////////////////////////////////////////////////////////////////////////
165   // Members
166   //////////////////////////////////////////////////////////////////////////////
167 
168   private final ConcurrentHashMap<HashedBytes, CountDownLatch> lockedRows =
169     new ConcurrentHashMap<HashedBytes, CountDownLatch>();
170   private final ConcurrentHashMap<Integer, HashedBytes> lockIds =
171     new ConcurrentHashMap<Integer, HashedBytes>();
172   private final AtomicInteger lockIdGenerator = new AtomicInteger(1);
173   static private Random rand = new Random();
174 
175   protected final Map<byte [], Store> stores =
176     new ConcurrentSkipListMap<byte [], Store>(Bytes.BYTES_RAWCOMPARATOR);
177 
178   // Registered region protocol handlers
179   private ClassToInstanceMap<CoprocessorProtocol>
180       protocolHandlers = MutableClassToInstanceMap.create();
181 
182   //These variable are just used for getting data out of the region, to test on
183   //client side
184   // private int numStores = 0;
185   // private int [] storeSize = null;
186   // private byte [] name = null;
187 
188   final AtomicLong memstoreSize = new AtomicLong(0);
189 
190   final Counter readRequestsCount = new Counter();
191   final Counter writeRequestsCount = new Counter();
192 
193   /**
194    * The directory for the table this region is part of.
195    * This directory contains the directory for this region.
196    */
197   final Path tableDir;
198 
199   final HLog log;
200   final FileSystem fs;
201   final Configuration conf;
202   final int rowLockWaitDuration;
203   static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
204   final HRegionInfo regionInfo;
205   final Path regiondir;
206   KeyValue.KVComparator comparator;
207 
208   private ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
209 
210   /*
211    * @return The smallest mvcc readPoint across all the scanners in this
212    * region. Writes older than this readPoint, are included  in every
213    * read operation.
214    */
215   public long getSmallestReadPoint() {
216     long minimumReadPoint;
217     // We need to ensure that while we are calculating the smallestReadPoint
218     // no new RegionScanners can grab a readPoint that we are unaware of.
219     // We achieve this by synchronizing on the scannerReadPoints object.
220     synchronized(scannerReadPoints) {
221       minimumReadPoint = mvcc.memstoreReadPoint();
222 
223       for (Long readPoint: this.scannerReadPoints.values()) {
224         if (readPoint < minimumReadPoint) {
225           minimumReadPoint = readPoint;
226         }
227       }
228     }
229     return minimumReadPoint;
230   }
231   /*
232    * Data structure of write state flags used coordinating flushes,
233    * compactions and closes.
234    */
235   static class WriteState {
236     // Set while a memstore flush is happening.
237     volatile boolean flushing = false;
238     // Set when a flush has been requested.
239     volatile boolean flushRequested = false;
240     // Number of compactions running.
241     volatile int compacting = 0;
242     // Gets set in close. If set, cannot compact or flush again.
243     volatile boolean writesEnabled = true;
244     // Set if region is read-only
245     volatile boolean readOnly = false;
246 
247     /**
248      * Set flags that make this region read-only.
249      *
250      * @param onOff flip value for region r/o setting
251      */
252     synchronized void setReadOnly(final boolean onOff) {
253       this.writesEnabled = !onOff;
254       this.readOnly = onOff;
255     }
256 
257     boolean isReadOnly() {
258       return this.readOnly;
259     }
260 
261     boolean isFlushRequested() {
262       return this.flushRequested;
263     }
264 
265     static final long HEAP_SIZE = ClassSize.align(
266         ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
267   }
268 
269   final WriteState writestate = new WriteState();
270 
271   long memstoreFlushSize;
272   private volatile long lastFlushTime;
273   final RegionServerServices rsServices;
274   private List<Pair<Long, Long>> recentFlushes = new ArrayList<Pair<Long,Long>>();
275   private long blockingMemStoreSize;
276   final long threadWakeFrequency;
277   // Used to guard closes
278   final ReentrantReadWriteLock lock =
279     new ReentrantReadWriteLock();
280 
281   // Stop updates lock
282   private final ReentrantReadWriteLock updatesLock =
283     new ReentrantReadWriteLock();
284   private boolean splitRequest;
285   private byte[] explicitSplitPoint = null;
286 
287   private final MultiVersionConsistencyControl mvcc =
288       new MultiVersionConsistencyControl();
289 
290   // Coprocessor host
291   private RegionCoprocessorHost coprocessorHost;
292 
293   /**
294    * Name of the region info file that resides just under the region directory.
295    */
296   public final static String REGIONINFO_FILE = ".regioninfo";
297   private HTableDescriptor htableDescriptor = null;
298   private RegionSplitPolicy splitPolicy;
299 
300   /**
301    * Should only be used for testing purposes
302    */
303   public HRegion(){
304     this.tableDir = null;
305     this.blockingMemStoreSize = 0L;
306     this.conf = null;
307     this.rowLockWaitDuration = DEFAULT_ROWLOCK_WAIT_DURATION;
308     this.rsServices = null;
309     this.fs = null;
310     this.memstoreFlushSize = 0L;
311     this.log = null;
312     this.regiondir = null;
313     this.regionInfo = null;
314     this.htableDescriptor = null;
315     this.threadWakeFrequency = 0L;
316     this.coprocessorHost = null;
317     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
318   }
319 
320   /**
321    * HRegion constructor.  his constructor should only be used for testing and
322    * extensions.  Instances of HRegion should be instantiated with the
323    * {@link HRegion#newHRegion(Path, HLog, FileSystem, Configuration, HRegionInfo, HTableDescriptor, RegionServerServices)} method.
324    *
325    *
326    * @param tableDir qualified path of directory where region should be located,
327    * usually the table directory.
328    * @param log The HLog is the outbound log for any updates to the HRegion
329    * (There's a single HLog for all the HRegions on a single HRegionServer.)
330    * The log file is a logfile from the previous execution that's
331    * custom-computed for this HRegion. The HRegionServer computes and sorts the
332    * appropriate log info for this HRegion. If there is a previous log file
333    * (implying that the HRegion has been written-to before), then read it from
334    * the supplied path.
335    * @param fs is the filesystem.
336    * @param conf is global configuration settings.
337    * @param regionInfo - HRegionInfo that describes the region
338    * is new), then read them from the supplied path.
339    * @param rsServices reference to {@link RegionServerServices} or null
340    *
341    * @see HRegion#newHRegion(Path, HLog, FileSystem, Configuration, HRegionInfo, HTableDescriptor, RegionServerServices)
342    */
343   public HRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf,
344       HRegionInfo regionInfo, final HTableDescriptor htd,
345       RegionServerServices rsServices) {
346     this.tableDir = tableDir;
347     this.comparator = regionInfo.getComparator();
348     this.log = log;
349     this.fs = fs;
350     this.conf = conf;
351     this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
352                     DEFAULT_ROWLOCK_WAIT_DURATION);
353     this.regionInfo = regionInfo;
354     this.htableDescriptor = htd;
355     this.rsServices = rsServices;
356     this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY,
357         10 * 1000);
358     String encodedNameStr = this.regionInfo.getEncodedName();
359     setHTableSpecificConf();
360     this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
361     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
362 
363     // don't initialize coprocessors if not running within a regionserver
364     // TODO: revisit if coprocessors should load in other cases
365     if (rsServices != null) {
366       this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
367     }
368     if (LOG.isDebugEnabled()) {
369       // Write out region name as string and its encoded name.
370       LOG.debug("Instantiated " + this);
371     }
372   }
373 
374   void setHTableSpecificConf() {
375     if (this.htableDescriptor == null) return;
376     LOG.info("Setting up tabledescriptor config now ...");
377     long flushSize = this.htableDescriptor.getMemStoreFlushSize();
378     if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
379       flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
380          HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
381     }
382     this.memstoreFlushSize = flushSize;
383     this.blockingMemStoreSize = this.memstoreFlushSize *
384         conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
385   }
386 
387   /**
388    * Initialize this region.
389    * @return What the next sequence (edit) id should be.
390    * @throws IOException e
391    */
392   public long initialize() throws IOException {
393     return initialize(null);
394   }
395 
396   /**
397    * Initialize this region.
398    *
399    * @param reporter Tickle every so often if initialize is taking a while.
400    * @return What the next sequence (edit) id should be.
401    * @throws IOException e
402    */
403   public long initialize(final CancelableProgressable reporter)
404   throws IOException {
405 
406     MonitoredTask status = TaskMonitor.get().createStatus(
407         "Initializing region " + this);
408 
409     if (coprocessorHost != null) {
410       status.setStatus("Running coprocessor pre-open hook");
411       coprocessorHost.preOpen();
412     }
413     // A region can be reopened if failed a split; reset flags
414     this.closing.set(false);
415     this.closed.set(false);
416 
417     // Write HRI to a file in case we need to recover .META.
418     status.setStatus("Writing region info on filesystem");
419     checkRegioninfoOnFilesystem();
420 
421     // Remove temporary data left over from old regions
422     status.setStatus("Cleaning up temporary data from old regions");
423     cleanupTmpDir();
424 
425     // Load in all the HStores.
426     // Get minimum of the maxSeqId across all the store.
427     //
428     // Context: During replay we want to ensure that we do not lose any data. So, we
429     // have to be conservative in how we replay logs. For each store, we calculate
430     // the maxSeqId up to which the store was flushed. But, since different stores
431     // could have a different maxSeqId, we choose the
432     // minimum across all the stores.
433     // This could potentially result in duplication of data for stores that are ahead
434     // of others. ColumnTrackers in the ScanQueryMatchers do the de-duplication, so we
435     // do not have to worry.
436     // TODO: If there is a store that was never flushed in a long time, we could replay
437     // a lot of data. Currently, this is not a problem because we flush all the stores at
438     // the same time. If we move to per-cf flushing, we might want to revisit this and send
439     // in a vector of maxSeqIds instead of sending in a single number, which has to be the
440     // min across all the max.
441     long minSeqId = -1;
442     long maxSeqId = -1;
443     // initialized to -1 so that we pick up MemstoreTS from column families
444     long maxMemstoreTS = -1;
445     for (HColumnDescriptor c : this.htableDescriptor.getFamilies()) {
446       status.setStatus("Instantiating store for column family " + c);
447       Store store = instantiateHStore(this.tableDir, c);
448       this.stores.put(c.getName(), store);
449       long storeSeqId = store.getMaxSequenceId();
450       if (minSeqId == -1 || storeSeqId < minSeqId) {
451         minSeqId = storeSeqId;
452       }
453       if (maxSeqId == -1 || storeSeqId > maxSeqId) {
454         maxSeqId = storeSeqId;
455       }
456       long maxStoreMemstoreTS = store.getMaxMemstoreTS();
457       if (maxStoreMemstoreTS > maxMemstoreTS) {
458         maxMemstoreTS = maxStoreMemstoreTS;
459       }
460     }
461     mvcc.initialize(maxMemstoreTS + 1);
462     // Recover any edits if available.
463     maxSeqId = Math.max(maxSeqId, replayRecoveredEditsIfAny(
464         this.regiondir, minSeqId, reporter, status));
465 
466     status.setStatus("Cleaning up detritus from prior splits");
467     // Get rid of any splits or merges that were lost in-progress.  Clean out
468     // these directories here on open.  We may be opening a region that was
469     // being split but we crashed in the middle of it all.
470     SplitTransaction.cleanupAnySplitDetritus(this);
471     FSUtils.deleteDirectory(this.fs, new Path(regiondir, MERGEDIR));
472 
473     this.writestate.setReadOnly(this.htableDescriptor.isReadOnly());
474 
475     this.writestate.compacting = 0;
476 
477     // Initialize split policy
478     this.splitPolicy = RegionSplitPolicy.create(this, conf);
479 
480     this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
481     // Use maximum of log sequenceid or that which was found in stores
482     // (particularly if no recovered edits, seqid will be -1).
483     long nextSeqid = maxSeqId + 1;
484     LOG.info("Onlined " + this.toString() + "; next sequenceid=" + nextSeqid);
485 
486 
487     if (coprocessorHost != null) {
488       status.setStatus("Running coprocessor post-open hooks");
489       coprocessorHost.postOpen();
490     }
491     status.markComplete("Region opened successfully");
492     return nextSeqid;
493   }
494 
495   /*
496    * Move any passed HStore files into place (if any).  Used to pick up split
497    * files and any merges from splits and merges dirs.
498    * @param initialFiles
499    * @throws IOException
500    */
501   static void moveInitialFilesIntoPlace(final FileSystem fs,
502     final Path initialFiles, final Path regiondir)
503   throws IOException {
504     if (initialFiles != null && fs.exists(initialFiles)) {
505       if (!fs.rename(initialFiles, regiondir)) {
506         LOG.warn("Unable to rename " + initialFiles + " to " + regiondir);
507       }
508     }
509   }
510 
511   /**
512    * @return True if this region has references.
513    */
514   public boolean hasReferences() {
515     for (Store store : this.stores.values()) {
516       for (StoreFile sf : store.getStorefiles()) {
517         // Found a reference, return.
518         if (sf.isReference()) return true;
519       }
520     }
521     return false;
522   }
523 
524   /**
525    * This function will return the HDFS blocks distribution based on the data
526    * captured when HFile is created
527    * @return The HDFS blocks distribution for the region.
528    */
529   public HDFSBlocksDistribution getHDFSBlocksDistribution() {
530     HDFSBlocksDistribution hdfsBlocksDistribution =
531       new HDFSBlocksDistribution();
532     synchronized (this.stores) {
533       for (Store store : this.stores.values()) {
534         for (StoreFile sf : store.getStorefiles()) {
535           HDFSBlocksDistribution storeFileBlocksDistribution =
536             sf.getHDFSBlockDistribution();
537           hdfsBlocksDistribution.add(storeFileBlocksDistribution);
538         }
539       }
540     }
541     return hdfsBlocksDistribution;
542   }
543 
544   /**
545    * This is a helper function to compute HDFS block distribution on demand
546    * @param conf configuration
547    * @param tableDescriptor HTableDescriptor of the table
548    * @param regionEncodedName encoded name of the region
549    * @return The HDFS blocks distribution for the given region.
550  * @throws IOException
551    */
552   static public HDFSBlocksDistribution computeHDFSBlocksDistribution(
553     Configuration conf, HTableDescriptor tableDescriptor,
554     String regionEncodedName) throws IOException {
555     HDFSBlocksDistribution hdfsBlocksDistribution =
556       new HDFSBlocksDistribution();
557     Path tablePath = FSUtils.getTablePath(FSUtils.getRootDir(conf),
558       tableDescriptor.getName());
559     FileSystem fs = tablePath.getFileSystem(conf);
560 
561     for (HColumnDescriptor family: tableDescriptor.getFamilies()) {
562       Path storeHomeDir = Store.getStoreHomedir(tablePath, regionEncodedName,
563       family.getName());
564       if (!fs.exists(storeHomeDir))continue;
565 
566       FileStatus[] hfilesStatus = null;
567       hfilesStatus = fs.listStatus(storeHomeDir);
568 
569       for (FileStatus hfileStatus : hfilesStatus) {
570         HDFSBlocksDistribution storeFileBlocksDistribution =
571           FSUtils.computeHDFSBlocksDistribution(fs, hfileStatus, 0,
572           hfileStatus.getLen());
573         hdfsBlocksDistribution.add(storeFileBlocksDistribution);
574       }
575     }
576     return hdfsBlocksDistribution;
577   }
578 
579   public AtomicLong getMemstoreSize() {
580     return memstoreSize;
581   }
582 
583   /**
584    * Increase the size of mem store in this region and the size of global mem
585    * store
586    * @param memStoreSize
587    * @return the size of memstore in this region
588    */
589   public long addAndGetGlobalMemstoreSize(long memStoreSize) {
590     if (this.rsServices != null) {
591       RegionServerAccounting rsAccounting =
592         this.rsServices.getRegionServerAccounting();
593 
594       if (rsAccounting != null) {
595         rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
596       }
597     }
598     return this.memstoreSize.getAndAdd(memStoreSize);
599   }
600 
601   /*
602    * Write out an info file under the region directory.  Useful recovering
603    * mangled regions.
604    * @throws IOException
605    */
606   private void checkRegioninfoOnFilesystem() throws IOException {
607     Path regioninfoPath = new Path(this.regiondir, REGIONINFO_FILE);
608     if (this.fs.exists(regioninfoPath) &&
609         this.fs.getFileStatus(regioninfoPath).getLen() > 0) {
610       return;
611     }
612     // Create in tmpdir and then move into place in case we crash after
613     // create but before close.  If we don't successfully close the file,
614     // subsequent region reopens will fail the below because create is
615     // registered in NN.
616     Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
617     FSDataOutputStream out = this.fs.create(tmpPath, true);
618     try {
619       this.regionInfo.write(out);
620       out.write('\n');
621       out.write('\n');
622       out.write(Bytes.toBytes(this.regionInfo.toString()));
623     } finally {
624       out.close();
625     }
626     if (!fs.rename(tmpPath, regioninfoPath)) {
627       throw new IOException("Unable to rename " + tmpPath + " to " +
628         regioninfoPath);
629     }
630   }
631 
632   /** @return a HRegionInfo object for this region */
633   public HRegionInfo getRegionInfo() {
634     return this.regionInfo;
635   }
636 
637   /** @return requestsCount for this region */
638   public long getRequestsCount() {
639     return this.readRequestsCount.get() + this.writeRequestsCount.get();
640   }
641 
642   /** @return readRequestsCount for this region */
643   public long getReadRequestsCount() {
644     return this.readRequestsCount.get();
645   }
646 
647   /** @return writeRequestsCount for this region */
648   public long getWriteRequestsCount() {
649     return this.writeRequestsCount.get();
650   }
651 
652   /** @return true if region is closed */
653   public boolean isClosed() {
654     return this.closed.get();
655   }
656 
657   /**
658    * @return True if closing process has started.
659    */
660   public boolean isClosing() {
661     return this.closing.get();
662   }
663 
664   boolean areWritesEnabled() {
665     synchronized(this.writestate) {
666       return this.writestate.writesEnabled;
667     }
668   }
669 
670    public MultiVersionConsistencyControl getMVCC() {
671      return mvcc;
672    }
673 
674   /**
675    * Close down this HRegion.  Flush the cache, shut down each HStore, don't
676    * service any more calls.
677    *
678    * <p>This method could take some time to execute, so don't call it from a
679    * time-sensitive thread.
680    *
681    * @return Vector of all the storage files that the HRegion's component
682    * HStores make use of.  It's a list of all HStoreFile objects. Returns empty
683    * vector if already closed and null if judged that it should not close.
684    *
685    * @throws IOException e
686    */
687   public List<StoreFile> close() throws IOException {
688     return close(false);
689   }
690 
691   private final Object closeLock = new Object();
692 
693   /**
694    * Close down this HRegion.  Flush the cache unless abort parameter is true,
695    * Shut down each HStore, don't service any more calls.
696    *
697    * This method could take some time to execute, so don't call it from a
698    * time-sensitive thread.
699    *
700    * @param abort true if server is aborting (only during testing)
701    * @return Vector of all the storage files that the HRegion's component
702    * HStores make use of.  It's a list of HStoreFile objects.  Can be null if
703    * we are not to close at this time or we are already closed.
704    *
705    * @throws IOException e
706    */
707   public List<StoreFile> close(final boolean abort) throws IOException {
708     // Only allow one thread to close at a time. Serialize them so dual
709     // threads attempting to close will run up against each other.
710     MonitoredTask status = TaskMonitor.get().createStatus(
711         "Closing region " + this +
712         (abort ? " due to abort" : ""));
713 
714     status.setStatus("Waiting for close lock");
715     try {
716       synchronized (closeLock) {
717         return doClose(abort, status);
718       }
719     } finally {
720       status.cleanup();
721     }
722   }
723 
724   private List<StoreFile> doClose(
725       final boolean abort, MonitoredTask status)
726   throws IOException {
727     if (isClosed()) {
728       LOG.warn("Region " + this + " already closed");
729       return null;
730     }
731 
732     if (coprocessorHost != null) {
733       status.setStatus("Running coprocessor pre-close hooks");
734       this.coprocessorHost.preClose(abort);
735     }
736 
737     status.setStatus("Disabling compacts and flushes for region");
738     boolean wasFlushing = false;
739     synchronized (writestate) {
740       // Disable compacting and flushing by background threads for this
741       // region.
742       writestate.writesEnabled = false;
743       wasFlushing = writestate.flushing;
744       LOG.debug("Closing " + this + ": disabling compactions & flushes");
745       while (writestate.compacting > 0 || writestate.flushing) {
746         LOG.debug("waiting for " + writestate.compacting + " compactions" +
747             (writestate.flushing ? " & cache flush" : "") +
748             " to complete for region " + this);
749         try {
750           writestate.wait();
751         } catch (InterruptedException iex) {
752           // continue
753         }
754       }
755     }
756     // If we were not just flushing, is it worth doing a preflush...one
757     // that will clear out of the bulk of the memstore before we put up
758     // the close flag?
759     if (!abort && !wasFlushing && worthPreFlushing()) {
760       status.setStatus("Pre-flushing region before close");
761       LOG.info("Running close preflush of " + this.getRegionNameAsString());
762       internalFlushcache(status);
763     }
764 
765     this.closing.set(true);
766     status.setStatus("Disabling writes for close");
767     lock.writeLock().lock();
768     try {
769       if (this.isClosed()) {
770         status.abort("Already got closed by another process");
771         // SplitTransaction handles the null
772         return null;
773       }
774       LOG.debug("Updates disabled for region " + this);
775       // Don't flush the cache if we are aborting
776       if (!abort) {
777         internalFlushcache(status);
778       }
779 
780       List<StoreFile> result = new ArrayList<StoreFile>();
781       for (Store store : stores.values()) {
782         result.addAll(store.close());
783       }
784       this.closed.set(true);
785 
786       if (coprocessorHost != null) {
787         status.setStatus("Running coprocessor post-close hooks");
788         this.coprocessorHost.postClose(abort);
789       }
790       status.markComplete("Closed");
791       LOG.info("Closed " + this);
792       return result;
793     } finally {
794       lock.writeLock().unlock();
795     }
796   }
797 
798    /**
799     * @return True if its worth doing a flush before we put up the close flag.
800     */
801   private boolean worthPreFlushing() {
802     return this.memstoreSize.get() >
803       this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
804   }
805 
806   //////////////////////////////////////////////////////////////////////////////
807   // HRegion accessors
808   //////////////////////////////////////////////////////////////////////////////
809 
810   /** @return start key for region */
811   public byte [] getStartKey() {
812     return this.regionInfo.getStartKey();
813   }
814 
815   /** @return end key for region */
816   public byte [] getEndKey() {
817     return this.regionInfo.getEndKey();
818   }
819 
820   /** @return region id */
821   public long getRegionId() {
822     return this.regionInfo.getRegionId();
823   }
824 
825   /** @return region name */
826   public byte [] getRegionName() {
827     return this.regionInfo.getRegionName();
828   }
829 
830   /** @return region name as string for logging */
831   public String getRegionNameAsString() {
832     return this.regionInfo.getRegionNameAsString();
833   }
834 
835   /** @return HTableDescriptor for this region */
836   public HTableDescriptor getTableDesc() {
837     return this.htableDescriptor;
838   }
839 
840   /** @return HLog in use for this region */
841   public HLog getLog() {
842     return this.log;
843   }
844 
845   /** @return Configuration object */
846   public Configuration getConf() {
847     return this.conf;
848   }
849 
850   /** @return region directory Path */
851   public Path getRegionDir() {
852     return this.regiondir;
853   }
854 
855   /**
856    * Computes the Path of the HRegion
857    *
858    * @param tabledir qualified path for table
859    * @param name ENCODED region name
860    * @return Path of HRegion directory
861    */
862   public static Path getRegionDir(final Path tabledir, final String name) {
863     return new Path(tabledir, name);
864   }
865 
866   /** @return FileSystem being used by this region */
867   public FileSystem getFilesystem() {
868     return this.fs;
869   }
870 
871   /** @return the last time the region was flushed */
872   public long getLastFlushTime() {
873     return this.lastFlushTime;
874   }
875 
876   /** @return info about the last flushes <time, size> */
877   public List<Pair<Long,Long>> getRecentFlushInfo() {
878     this.lock.readLock().lock();
879     List<Pair<Long,Long>> ret = this.recentFlushes;
880     this.recentFlushes = new ArrayList<Pair<Long,Long>>();
881     this.lock.readLock().unlock();
882     return ret;
883   }
884 
885   //////////////////////////////////////////////////////////////////////////////
886   // HRegion maintenance.
887   //
888   // These methods are meant to be called periodically by the HRegionServer for
889   // upkeep.
890   //////////////////////////////////////////////////////////////////////////////
891 
892   /** @return returns size of largest HStore. */
893   public long getLargestHStoreSize() {
894     long size = 0;
895     for (Store h: stores.values()) {
896       long storeSize = h.getSize();
897       if (storeSize > size) {
898         size = storeSize;
899       }
900     }
901     return size;
902   }
903 
904   /*
905    * Do preparation for pending compaction.
906    * @throws IOException
907    */
908   void doRegionCompactionPrep() throws IOException {
909   }
910 
911   /*
912    * Removes the temporary directory for this Store.
913    */
914   private void cleanupTmpDir() throws IOException {
915     FSUtils.deleteDirectory(this.fs, getTmpDir());
916   }
917 
918   /**
919    * Get the temporary diretory for this region. This directory
920    * will have its contents removed when the region is reopened.
921    */
922   Path getTmpDir() {
923     return new Path(getRegionDir(), ".tmp");
924   }
925 
926   void triggerMajorCompaction() {
927     for (Store h: stores.values()) {
928       h.triggerMajorCompaction();
929     }
930   }
931 
932   /**
933    * This is a helper function that compact all the stores synchronously
934    * It is used by utilities and testing
935    *
936    * @param majorCompaction True to force a major compaction regardless of thresholds
937    * @throws IOException e
938    */
939   void compactStores(final boolean majorCompaction)
940   throws IOException {
941     if (majorCompaction) {
942       this.triggerMajorCompaction();
943     }
944     compactStores();
945   }
946 
947   /**
948    * This is a helper function that compact all the stores synchronously
949    * It is used by utilities and testing
950    *
951    * @throws IOException e
952    */
953   public void compactStores() throws IOException {
954     for(Store s : getStores().values()) {
955       CompactionRequest cr = s.requestCompaction();
956       if(cr != null) {
957         try {
958           compact(cr);
959         } finally {
960           s.finishRequest(cr);
961         }
962       }
963     }
964   }
965 
966   /*
967    * Called by compaction thread and after region is opened to compact the
968    * HStores if necessary.
969    *
970    * <p>This operation could block for a long time, so don't call it from a
971    * time-sensitive thread.
972    *
973    * Note that no locking is necessary at this level because compaction only
974    * conflicts with a region split, and that cannot happen because the region
975    * server does them sequentially and not in parallel.
976    *
977    * @param cr Compaction details, obtained by requestCompaction()
978    * @return whether the compaction completed
979    * @throws IOException e
980    */
981   public boolean compact(CompactionRequest cr)
982   throws IOException {
983     if (cr == null) {
984       return false;
985     }
986     if (this.closing.get() || this.closed.get()) {
987       LOG.debug("Skipping compaction on " + this + " because closing/closed");
988       return false;
989     }
990     Preconditions.checkArgument(cr.getHRegion().equals(this));
991     lock.readLock().lock();
992     MonitoredTask status = TaskMonitor.get().createStatus(
993         "Compacting " + cr.getStore() + " in " + this);
994     try {
995       if (this.closed.get()) {
996         LOG.debug("Skipping compaction on " + this + " because closed");
997         return false;
998       }
999       boolean decr = true;
1000       try {
1001         synchronized (writestate) {
1002           if (writestate.writesEnabled) {
1003             ++writestate.compacting;
1004           } else {
1005             String msg = "NOT compacting region " + this + ". Writes disabled.";
1006             LOG.info(msg);
1007             status.abort(msg);
1008             decr = false;
1009             return false;
1010           }
1011         }
1012         LOG.info("Starting compaction on " + cr.getStore() + " in region "
1013             + this);
1014         doRegionCompactionPrep();
1015         try {
1016           status.setStatus("Compacting store " + cr.getStore());
1017           cr.getStore().compact(cr);
1018         } catch (InterruptedIOException iioe) {
1019           String msg = "compaction interrupted by user";
1020           LOG.info(msg, iioe);
1021           status.abort(msg);
1022           return false;
1023         }
1024       } finally {
1025         if (decr) {
1026           synchronized (writestate) {
1027             --writestate.compacting;
1028             if (writestate.compacting <= 0) {
1029               writestate.notifyAll();
1030             }
1031           }
1032         }
1033       }
1034       status.markComplete("Compaction complete");
1035       return true;
1036     } finally {
1037       status.cleanup();
1038       lock.readLock().unlock();
1039     }
1040   }
1041 
1042   /**
1043    * Flush the cache.
1044    *
1045    * When this method is called the cache will be flushed unless:
1046    * <ol>
1047    *   <li>the cache is empty</li>
1048    *   <li>the region is closed.</li>
1049    *   <li>a flush is already in progress</li>
1050    *   <li>writes are disabled</li>
1051    * </ol>
1052    *
1053    * <p>This method may block for some time, so it should not be called from a
1054    * time-sensitive thread.
1055    *
1056    * @return true if cache was flushed
1057    *
1058    * @throws IOException general io exceptions
1059    * @throws DroppedSnapshotException Thrown when replay of hlog is required
1060    * because a Snapshot was not properly persisted.
1061    */
1062   public boolean flushcache() throws IOException {
1063     // fail-fast instead of waiting on the lock
1064     if (this.closing.get()) {
1065       LOG.debug("Skipping flush on " + this + " because closing");
1066       return false;
1067     }
1068     MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
1069     status.setStatus("Acquiring readlock on region");
1070     lock.readLock().lock();
1071     try {
1072       if (this.closed.get()) {
1073         LOG.debug("Skipping flush on " + this + " because closed");
1074         status.abort("Skipped: closed");
1075         return false;
1076       }
1077       if (coprocessorHost != null) {
1078         status.setStatus("Running coprocessor pre-flush hooks");
1079         coprocessorHost.preFlush();
1080       }
1081       try {
1082         synchronized (writestate) {
1083           if (!writestate.flushing && writestate.writesEnabled) {
1084             this.writestate.flushing = true;
1085           } else {
1086             if (LOG.isDebugEnabled()) {
1087               LOG.debug("NOT flushing memstore for region " + this +
1088                   ", flushing=" +
1089                   writestate.flushing + ", writesEnabled=" +
1090                   writestate.writesEnabled);
1091             }
1092             status.abort("Not flushing since " +
1093                 (writestate.flushing ? "already flushing" : "writes not enabled"));
1094             return false;
1095           }
1096         }
1097         boolean result = internalFlushcache(status);
1098 
1099         if (coprocessorHost != null) {
1100           status.setStatus("Running post-flush coprocessor hooks");
1101           coprocessorHost.postFlush();
1102         }
1103 
1104         status.markComplete("Flush successful");
1105         return result;
1106       } finally {
1107         synchronized (writestate) {
1108           writestate.flushing = false;
1109           this.writestate.flushRequested = false;
1110           writestate.notifyAll();
1111         }
1112       }
1113     } finally {
1114       lock.readLock().unlock();
1115       status.cleanup();
1116     }
1117   }
1118 
1119   /**
1120    * Flush the memstore.
1121    *
1122    * Flushing the memstore is a little tricky. We have a lot of updates in the
1123    * memstore, all of which have also been written to the log. We need to
1124    * write those updates in the memstore out to disk, while being able to
1125    * process reads/writes as much as possible during the flush operation. Also,
1126    * the log has to state clearly the point in time at which the memstore was
1127    * flushed. (That way, during recovery, we know when we can rely on the
1128    * on-disk flushed structures and when we have to recover the memstore from
1129    * the log.)
1130    *
1131    * <p>So, we have a three-step process:
1132    *
1133    * <ul><li>A. Flush the memstore to the on-disk stores, noting the current
1134    * sequence ID for the log.<li>
1135    *
1136    * <li>B. Write a FLUSHCACHE-COMPLETE message to the log, using the sequence
1137    * ID that was current at the time of memstore-flush.</li>
1138    *
1139    * <li>C. Get rid of the memstore structures that are now redundant, as
1140    * they've been flushed to the on-disk HStores.</li>
1141    * </ul>
1142    * <p>This method is protected, but can be accessed via several public
1143    * routes.
1144    *
1145    * <p> This method may block for some time.
1146    * @param status
1147    *
1148    * @return true if the region needs compacting
1149    *
1150    * @throws IOException general io exceptions
1151    * @throws DroppedSnapshotException Thrown when replay of hlog is required
1152    * because a Snapshot was not properly persisted.
1153    */
1154   protected boolean internalFlushcache(MonitoredTask status) throws IOException {
1155     return internalFlushcache(this.log, -1, status);
1156   }
1157 
1158   /**
1159    * @param wal Null if we're NOT to go via hlog/wal.
1160    * @param myseqid The seqid to use if <code>wal</code> is null writing out
1161    * flush file.
1162    * @param status
1163    * @return true if the region needs compacting
1164    * @throws IOException
1165    * @see #internalFlushcache(MonitoredTask)
1166    */
1167   protected boolean internalFlushcache(
1168       final HLog wal, final long myseqid, MonitoredTask status)
1169   throws IOException {
1170     final long startTime = EnvironmentEdgeManager.currentTimeMillis();
1171     // Clear flush flag.
1172     // Record latest flush time
1173     this.lastFlushTime = startTime;
1174     // If nothing to flush, return and avoid logging start/stop flush.
1175     if (this.memstoreSize.get() <= 0) {
1176       return false;
1177     }
1178     if (LOG.isDebugEnabled()) {
1179       LOG.debug("Started memstore flush for " + this +
1180         ", current region memstore size " +
1181         StringUtils.humanReadableInt(this.memstoreSize.get()) +
1182         ((wal != null)? "": "; wal is null, using passed sequenceid=" + myseqid));
1183     }
1184 
1185     // Stop updates while we snapshot the memstore of all stores. We only have
1186     // to do this for a moment.  Its quick.  The subsequent sequence id that
1187     // goes into the HLog after we've flushed all these snapshots also goes
1188     // into the info file that sits beside the flushed files.
1189     // We also set the memstore size to zero here before we allow updates
1190     // again so its value will represent the size of the updates received
1191     // during the flush
1192     long sequenceId = -1L;
1193     long completeSequenceId = -1L;
1194     MultiVersionConsistencyControl.WriteEntry w = null;
1195 
1196     // We have to take a write lock during snapshot, or else a write could
1197     // end up in both snapshot and memstore (makes it difficult to do atomic
1198     // rows then)
1199     status.setStatus("Obtaining lock to block concurrent updates");
1200     this.updatesLock.writeLock().lock();
1201     long flushsize = this.memstoreSize.get();
1202     status.setStatus("Preparing to flush by snapshotting stores");
1203     List<StoreFlusher> storeFlushers = new ArrayList<StoreFlusher>(stores.size());
1204     try {
1205       // Record the mvcc for all transactions in progress.
1206       w = mvcc.beginMemstoreInsert();
1207       mvcc.advanceMemstore(w);
1208 
1209       sequenceId = (wal == null)? myseqid:
1210         wal.startCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1211       completeSequenceId = this.getCompleteCacheFlushSequenceId(sequenceId);
1212 
1213       for (Store s : stores.values()) {
1214         storeFlushers.add(s.getStoreFlusher(completeSequenceId));
1215       }
1216 
1217       // prepare flush (take a snapshot)
1218       for (StoreFlusher flusher : storeFlushers) {
1219         flusher.prepare();
1220       }
1221     } finally {
1222       this.updatesLock.writeLock().unlock();
1223     }
1224     String s = "Finished snapshotting " + this +
1225       ", commencing wait for mvcc, flushsize=" + flushsize;
1226     status.setStatus(s);
1227     LOG.debug(s);
1228 
1229     // wait for all in-progress transactions to commit to HLog before
1230     // we can start the flush. This prevents
1231     // uncommitted transactions from being written into HFiles.
1232     // We have to block before we start the flush, otherwise keys that
1233     // were removed via a rollbackMemstore could be written to Hfiles.
1234     mvcc.waitForRead(w);
1235 
1236     // Any failure from here on out will be catastrophic requiring server
1237     // restart so hlog content can be replayed and put back into the memstore.
1238     // Otherwise, the snapshot content while backed up in the hlog, it will not
1239     // be part of the current running servers state.
1240     boolean compactionRequested = false;
1241     try {
1242       // A.  Flush memstore to all the HStores.
1243       // Keep running vector of all store files that includes both old and the
1244       // just-made new flush store file.
1245 
1246       for (StoreFlusher flusher : storeFlushers) {
1247         flusher.flushCache(status);
1248       }
1249       // Switch snapshot (in memstore) -> new hfile (thus causing
1250       // all the store scanners to reset/reseek).
1251       for (StoreFlusher flusher : storeFlushers) {
1252         boolean needsCompaction = flusher.commit();
1253         if (needsCompaction) {
1254           compactionRequested = true;
1255         }
1256       }
1257       storeFlushers.clear();
1258 
1259       // Set down the memstore size by amount of flush.
1260       this.addAndGetGlobalMemstoreSize(-flushsize);
1261     } catch (Throwable t) {
1262       // An exception here means that the snapshot was not persisted.
1263       // The hlog needs to be replayed so its content is restored to memstore.
1264       // Currently, only a server restart will do this.
1265       // We used to only catch IOEs but its possible that we'd get other
1266       // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
1267       // all and sundry.
1268       if (wal != null) {
1269         wal.abortCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1270       }
1271       DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
1272           Bytes.toStringBinary(getRegionName()));
1273       dse.initCause(t);
1274       status.abort("Flush failed: " + StringUtils.stringifyException(t));
1275       throw dse;
1276     }
1277 
1278     // If we get to here, the HStores have been written. If we get an
1279     // error in completeCacheFlush it will release the lock it is holding
1280 
1281     // B.  Write a FLUSHCACHE-COMPLETE message to the log.
1282     //     This tells future readers that the HStores were emitted correctly,
1283     //     and that all updates to the log for this regionName that have lower
1284     //     log-sequence-ids can be safely ignored.
1285     if (wal != null) {
1286       wal.completeCacheFlush(this.regionInfo.getEncodedNameAsBytes(),
1287         regionInfo.getTableName(), completeSequenceId,
1288         this.getRegionInfo().isMetaRegion());
1289     }
1290 
1291     // C. Finally notify anyone waiting on memstore to clear:
1292     // e.g. checkResources().
1293     synchronized (this) {
1294       notifyAll(); // FindBugs NN_NAKED_NOTIFY
1295     }
1296 
1297     long time = EnvironmentEdgeManager.currentTimeMillis() - startTime;
1298     long memstoresize = this.memstoreSize.get();
1299     String msg = "Finished memstore flush of ~" +
1300       StringUtils.humanReadableInt(flushsize) + "/" + flushsize +
1301       ", currentsize=" +
1302       StringUtils.humanReadableInt(memstoresize) + "/" + memstoresize +
1303       " for region " + this + " in " + time + "ms, sequenceid=" + sequenceId +
1304       ", compaction requested=" + compactionRequested +
1305       ((wal == null)? "; wal=null": "");
1306     LOG.info(msg);
1307     status.setStatus(msg);
1308     this.recentFlushes.add(new Pair<Long,Long>(time/1000, flushsize));
1309 
1310     return compactionRequested;
1311   }
1312 
1313    /**
1314    * Get the sequence number to be associated with this cache flush. Used by
1315    * TransactionalRegion to not complete pending transactions.
1316    *
1317    *
1318    * @param currentSequenceId
1319    * @return sequence id to complete the cache flush with
1320    */
1321   protected long getCompleteCacheFlushSequenceId(long currentSequenceId) {
1322     return currentSequenceId;
1323   }
1324 
1325   //////////////////////////////////////////////////////////////////////////////
1326   // get() methods for client use.
1327   //////////////////////////////////////////////////////////////////////////////
1328   /**
1329    * Return all the data for the row that matches <i>row</i> exactly,
1330    * or the one that immediately preceeds it, at or immediately before
1331    * <i>ts</i>.
1332    *
1333    * @param row row key
1334    * @return map of values
1335    * @throws IOException
1336    */
1337   Result getClosestRowBefore(final byte [] row)
1338   throws IOException{
1339     return getClosestRowBefore(row, HConstants.CATALOG_FAMILY);
1340   }
1341 
1342   /**
1343    * Return all the data for the row that matches <i>row</i> exactly,
1344    * or the one that immediately preceeds it, at or immediately before
1345    * <i>ts</i>.
1346    *
1347    * @param row row key
1348    * @param family column family to find on
1349    * @return map of values
1350    * @throws IOException read exceptions
1351    */
1352   public Result getClosestRowBefore(final byte [] row, final byte [] family)
1353   throws IOException {
1354     if (coprocessorHost != null) {
1355       Result result = new Result();
1356       if (coprocessorHost.preGetClosestRowBefore(row, family, result)) {
1357         return result;
1358       }
1359     }
1360     // look across all the HStores for this region and determine what the
1361     // closest key is across all column families, since the data may be sparse
1362     checkRow(row, "getClosestRowBefore");
1363     startRegionOperation();
1364     this.readRequestsCount.increment();
1365     try {
1366       Store store = getStore(family);
1367       // get the closest key. (HStore.getRowKeyAtOrBefore can return null)
1368       KeyValue key = store.getRowKeyAtOrBefore(row);
1369       Result result = null;
1370       if (key != null) {
1371         Get get = new Get(key.getRow());
1372         get.addFamily(family);
1373         result = get(get, null);
1374       }
1375       if (coprocessorHost != null) {
1376         coprocessorHost.postGetClosestRowBefore(row, family, result);
1377       }
1378       return result;
1379     } finally {
1380       closeRegionOperation();
1381     }
1382   }
1383 
1384   /**
1385    * Return an iterator that scans over the HRegion, returning the indicated
1386    * columns and rows specified by the {@link Scan}.
1387    * <p>
1388    * This Iterator must be closed by the caller.
1389    *
1390    * @param scan configured {@link Scan}
1391    * @return RegionScanner
1392    * @throws IOException read exceptions
1393    */
1394   public RegionScanner getScanner(Scan scan) throws IOException {
1395    return getScanner(scan, null);
1396   }
1397 
1398   void prepareScanner(Scan scan) throws IOException {
1399     if(!scan.hasFamilies()) {
1400       // Adding all families to scanner
1401       for(byte[] family: this.htableDescriptor.getFamiliesKeys()){
1402         scan.addFamily(family);
1403       }
1404     }
1405   }
1406 
1407   protected RegionScanner getScanner(Scan scan,
1408       List<KeyValueScanner> additionalScanners) throws IOException {
1409     startRegionOperation();
1410     this.readRequestsCount.increment();
1411     try {
1412       // Verify families are all valid
1413       prepareScanner(scan);
1414       if(scan.hasFamilies()) {
1415         for(byte [] family : scan.getFamilyMap().keySet()) {
1416           checkFamily(family);
1417         }
1418       }
1419       return instantiateRegionScanner(scan, additionalScanners);
1420     } finally {
1421       closeRegionOperation();
1422     }
1423   }
1424 
1425   protected RegionScanner instantiateRegionScanner(Scan scan,
1426       List<KeyValueScanner> additionalScanners) throws IOException {
1427     return new RegionScannerImpl(scan, additionalScanners);
1428   }
1429 
1430   /*
1431    * @param delete The passed delete is modified by this method. WARNING!
1432    */
1433   private void prepareDelete(Delete delete) throws IOException {
1434     // Check to see if this is a deleteRow insert
1435     if(delete.getFamilyMap().isEmpty()){
1436       for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
1437         // Don't eat the timestamp
1438         delete.deleteFamily(family, delete.getTimeStamp());
1439       }
1440     } else {
1441       for(byte [] family : delete.getFamilyMap().keySet()) {
1442         if(family == null) {
1443           throw new NoSuchColumnFamilyException("Empty family is invalid");
1444         }
1445         checkFamily(family);
1446       }
1447     }
1448   }
1449 
1450   //////////////////////////////////////////////////////////////////////////////
1451   // set() methods for client use.
1452   //////////////////////////////////////////////////////////////////////////////
1453   /**
1454    * @param delete delete object
1455    * @param lockid existing lock id, or null for grab a lock
1456    * @param writeToWAL append to the write ahead lock or not
1457    * @throws IOException read exceptions
1458    */
1459   public void delete(Delete delete, Integer lockid, boolean writeToWAL)
1460   throws IOException {
1461     checkReadOnly();
1462     checkResources();
1463     Integer lid = null;
1464     startRegionOperation();
1465     this.writeRequestsCount.increment();
1466     try {
1467       byte [] row = delete.getRow();
1468       // If we did not pass an existing row lock, obtain a new one
1469       lid = getLock(lockid, row, true);
1470 
1471       try {
1472         // All edits for the given row (across all column families) must happen atomically.
1473         prepareDelete(delete);
1474         internalDelete(delete, delete.getClusterId(), writeToWAL);
1475       } finally {
1476         if(lockid == null) releaseRowLock(lid);
1477       }
1478     } finally {
1479       closeRegionOperation();
1480     }
1481   }
1482 
1483   /**
1484    * @param familyMap map of family to edits for the given family.
1485    * @param writeToWAL
1486    * @throws IOException
1487    */
1488   public void delete(Map<byte[], List<KeyValue>> familyMap, UUID clusterId,
1489       boolean writeToWAL) throws IOException {
1490     Delete delete = new Delete();
1491     delete.setFamilyMap(familyMap);
1492     delete.setClusterId(clusterId);
1493     delete.setWriteToWAL(writeToWAL);
1494     internalDelete(delete, clusterId, writeToWAL);
1495   }
1496 
1497   /**
1498    * @param familyMap map of family to edits for the given family.
1499    * @param writeToWAL
1500    * @throws IOException
1501    */
1502   private void internalDelete(Delete delete, UUID clusterId,
1503       boolean writeToWAL) throws IOException {
1504     Map<byte[], List<KeyValue>> familyMap = delete.getFamilyMap();
1505     WALEdit walEdit = new WALEdit();
1506     /* Run coprocessor pre hook outside of locks to avoid deadlock */
1507     if (coprocessorHost != null) {
1508       if (coprocessorHost.preDelete(delete, walEdit, writeToWAL)) {
1509         return;
1510       }
1511     }
1512 
1513     long now = EnvironmentEdgeManager.currentTimeMillis();
1514     byte [] byteNow = Bytes.toBytes(now);
1515     boolean flush = false;
1516 
1517     updatesLock.readLock().lock();
1518     try {
1519       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
1520 
1521         byte[] family = e.getKey();
1522         List<KeyValue> kvs = e.getValue();
1523         Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
1524 
1525         for (KeyValue kv: kvs) {
1526           //  Check if time is LATEST, change to time of most recent addition if so
1527           //  This is expensive.
1528           if (kv.isLatestTimestamp() && kv.isDeleteType()) {
1529             byte[] qual = kv.getQualifier();
1530             if (qual == null) qual = HConstants.EMPTY_BYTE_ARRAY;
1531 
1532             Integer count = kvCount.get(qual);
1533             if (count == null) {
1534               kvCount.put(qual, 1);
1535             } else {
1536               kvCount.put(qual, count + 1);
1537             }
1538             count = kvCount.get(qual);
1539 
1540             Get get = new Get(kv.getRow());
1541             get.setMaxVersions(count);
1542             get.addColumn(family, qual);
1543 
1544             List<KeyValue> result = get(get, false);
1545 
1546             if (result.size() < count) {
1547               // Nothing to delete
1548               kv.updateLatestStamp(byteNow);
1549               continue;
1550             }
1551             if (result.size() > count) {
1552               throw new RuntimeException("Unexpected size: " + result.size());
1553             }
1554             KeyValue getkv = result.get(count - 1);
1555             Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(),
1556                 getkv.getBuffer(), getkv.getTimestampOffset(), Bytes.SIZEOF_LONG);
1557           } else {
1558             kv.updateLatestStamp(byteNow);
1559           }
1560         }
1561       }
1562 
1563       if (writeToWAL) {
1564         // write/sync to WAL should happen before we touch memstore.
1565         //
1566         // If order is reversed, i.e. we write to memstore first, and
1567         // for some reason fail to write/sync to commit log, the memstore
1568         // will contain uncommitted transactions.
1569         //
1570         // bunch up all edits across all column families into a
1571         // single WALEdit.
1572         addFamilyMapToWALEdit(familyMap, walEdit);
1573         this.log.append(regionInfo, this.htableDescriptor.getName(),
1574             walEdit, clusterId, now, this.htableDescriptor);
1575       }
1576 
1577       // Now make changes to the memstore.
1578       long addedSize = applyFamilyMapToMemstore(familyMap);
1579       flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
1580 
1581       if (coprocessorHost != null) {
1582         coprocessorHost.postDelete(delete, walEdit, writeToWAL);
1583       }
1584     } finally {
1585       this.updatesLock.readLock().unlock();
1586     }
1587     if (flush) {
1588       // Request a cache flush.  Do it outside update lock.
1589       requestFlush();
1590     }
1591   }
1592 
1593   /**
1594    * @param put
1595    * @throws IOException
1596    */
1597   public void put(Put put) throws IOException {
1598     this.put(put, null, put.getWriteToWAL());
1599   }
1600 
1601   /**
1602    * @param put
1603    * @param writeToWAL
1604    * @throws IOException
1605    */
1606   public void put(Put put, boolean writeToWAL) throws IOException {
1607     this.put(put, null, writeToWAL);
1608   }
1609 
1610   /**
1611    * @param put
1612    * @param lockid
1613    * @throws IOException
1614    */
1615   public void put(Put put, Integer lockid) throws IOException {
1616     this.put(put, lockid, put.getWriteToWAL());
1617   }
1618 
1619 
1620 
1621   /**
1622    * @param put
1623    * @param lockid
1624    * @param writeToWAL
1625    * @throws IOException
1626    */
1627   public void put(Put put, Integer lockid, boolean writeToWAL)
1628   throws IOException {
1629     checkReadOnly();
1630 
1631     // Do a rough check that we have resources to accept a write.  The check is
1632     // 'rough' in that between the resource check and the call to obtain a
1633     // read lock, resources may run out.  For now, the thought is that this
1634     // will be extremely rare; we'll deal with it when it happens.
1635     checkResources();
1636     startRegionOperation();
1637     this.writeRequestsCount.increment();
1638     try {
1639       // We obtain a per-row lock, so other clients will block while one client
1640       // performs an update. The read lock is released by the client calling
1641       // #commit or #abort or if the HRegionServer lease on the lock expires.
1642       // See HRegionServer#RegionListener for how the expire on HRegionServer
1643       // invokes a HRegion#abort.
1644       byte [] row = put.getRow();
1645       // If we did not pass an existing row lock, obtain a new one
1646       Integer lid = getLock(lockid, row, true);
1647 
1648       try {
1649         // All edits for the given row (across all column families) must happen atomically.
1650         internalPut(put, put.getClusterId(), writeToWAL);
1651       } finally {
1652         if(lockid == null) releaseRowLock(lid);
1653       }
1654     } finally {
1655       closeRegionOperation();
1656     }
1657   }
1658 
1659   /**
1660    * Struct-like class that tracks the progress of a batch operation,
1661    * accumulating status codes and tracking the index at which processing
1662    * is proceeding.
1663    */
1664   private static class BatchOperationInProgress<T> {
1665     T[] operations;
1666     int nextIndexToProcess = 0;
1667     OperationStatus[] retCodeDetails;
1668 
1669     public BatchOperationInProgress(T[] operations) {
1670       this.operations = operations;
1671       this.retCodeDetails = new OperationStatus[operations.length];
1672       Arrays.fill(this.retCodeDetails, new OperationStatus(
1673           OperationStatusCode.NOT_RUN));
1674     }
1675 
1676     public boolean isDone() {
1677       return nextIndexToProcess == operations.length;
1678     }
1679   }
1680 
1681   /**
1682    * Perform a batch put with no pre-specified locks
1683    * @see HRegion#put(Pair[])
1684    */
1685   public OperationStatus[] put(Put[] puts) throws IOException {
1686     @SuppressWarnings("unchecked")
1687     Pair<Put, Integer> putsAndLocks[] = new Pair[puts.length];
1688 
1689     for (int i = 0; i < puts.length; i++) {
1690       putsAndLocks[i] = new Pair<Put, Integer>(puts[i], null);
1691     }
1692     return put(putsAndLocks);
1693   }
1694 
1695   /**
1696    * Perform a batch of puts.
1697    *
1698    * @param putsAndLocks
1699    *          the list of puts paired with their requested lock IDs.
1700    * @return an array of OperationStatus which internally contains the
1701    *         OperationStatusCode and the exceptionMessage if any.
1702    * @throws IOException
1703    */
1704   public OperationStatus[] put(
1705       Pair<Put, Integer>[] putsAndLocks) throws IOException {
1706     BatchOperationInProgress<Pair<Put, Integer>> batchOp =
1707       new BatchOperationInProgress<Pair<Put,Integer>>(putsAndLocks);
1708 
1709     while (!batchOp.isDone()) {
1710       checkReadOnly();
1711       checkResources();
1712 
1713       long newSize;
1714       startRegionOperation();
1715       this.writeRequestsCount.increment();
1716       try {
1717         long addedSize = doMiniBatchPut(batchOp);
1718         newSize = this.addAndGetGlobalMemstoreSize(addedSize);
1719       } finally {
1720         closeRegionOperation();
1721       }
1722       if (isFlushSize(newSize)) {
1723         requestFlush();
1724       }
1725     }
1726     return batchOp.retCodeDetails;
1727   }
1728 
1729   @SuppressWarnings("unchecked")
1730   private long doMiniBatchPut(
1731       BatchOperationInProgress<Pair<Put, Integer>> batchOp) throws IOException {
1732 
1733     WALEdit walEdit = new WALEdit();
1734     /* Run coprocessor pre hook outside of locks to avoid deadlock */
1735     if (coprocessorHost != null) {
1736       for (int i = 0; i < batchOp.operations.length; i++) {
1737         Pair<Put, Integer> nextPair = batchOp.operations[i];
1738         Put put = nextPair.getFirst();
1739         if (coprocessorHost.prePut(put, walEdit, put.getWriteToWAL())) {
1740           // pre hook says skip this Put
1741           // mark as success and skip below
1742           batchOp.retCodeDetails[i] = new OperationStatus(
1743               OperationStatusCode.SUCCESS);
1744         }
1745       }
1746     }
1747 
1748     long now = EnvironmentEdgeManager.currentTimeMillis();
1749     byte[] byteNow = Bytes.toBytes(now);
1750     boolean locked = false;
1751 
1752     /** Keep track of the locks we hold so we can release them in finally clause */
1753     List<Integer> acquiredLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
1754     // reference family maps directly so coprocessors can mutate them if desired
1755     Map<byte[],List<KeyValue>>[] familyMaps = new Map[batchOp.operations.length];
1756     // We try to set up a batch in the range [firstIndex,lastIndexExclusive)
1757     int firstIndex = batchOp.nextIndexToProcess;
1758     int lastIndexExclusive = firstIndex;
1759     boolean success = false;
1760     try {
1761       // ------------------------------------
1762       // STEP 1. Try to acquire as many locks as we can, and ensure
1763       // we acquire at least one.
1764       // ----------------------------------
1765       int numReadyToWrite = 0;
1766       while (lastIndexExclusive < batchOp.operations.length) {
1767         Pair<Put, Integer> nextPair = batchOp.operations[lastIndexExclusive];
1768         Put put = nextPair.getFirst();
1769         Integer providedLockId = nextPair.getSecond();
1770 
1771         Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
1772         // store the family map reference to allow for mutations
1773         familyMaps[lastIndexExclusive] = familyMap;
1774 
1775         // skip anything that "ran" already
1776         if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
1777             != OperationStatusCode.NOT_RUN) {
1778           lastIndexExclusive++;
1779           continue;
1780         }
1781 
1782         // Check the families in the put. If bad, skip this one.
1783         try {
1784           checkFamilies(familyMap.keySet());
1785         } catch (NoSuchColumnFamilyException nscf) {
1786           LOG.warn("No such column family in batch put", nscf);
1787           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
1788               OperationStatusCode.BAD_FAMILY, nscf.getMessage());
1789           lastIndexExclusive++;
1790           continue;
1791         }
1792 
1793         // If we haven't got any rows in our batch, we should block to
1794         // get the next one.
1795         boolean shouldBlock = numReadyToWrite == 0;
1796         Integer acquiredLockId = getLock(providedLockId, put.getRow(), shouldBlock);
1797         if (acquiredLockId == null) {
1798           // We failed to grab another lock
1799           assert !shouldBlock : "Should never fail to get lock when blocking";
1800           break; // stop acquiring more rows for this batch
1801         }
1802         if (providedLockId == null) {
1803           acquiredLocks.add(acquiredLockId);
1804         }
1805         lastIndexExclusive++;
1806         numReadyToWrite++;
1807       }
1808       // Nothing to put -- an exception in the above such as NoSuchColumnFamily?
1809       if (numReadyToWrite <= 0) return 0L;
1810 
1811       // We've now grabbed as many puts off the list as we can
1812 
1813       // ------------------------------------
1814       // STEP 2. Update any LATEST_TIMESTAMP timestamps
1815       // ----------------------------------
1816       for (int i = firstIndex; i < lastIndexExclusive; i++) {
1817         // skip invalid
1818         if (batchOp.retCodeDetails[i].getOperationStatusCode()
1819             != OperationStatusCode.NOT_RUN) continue;
1820 
1821         updateKVTimestamps(
1822             familyMaps[i].values(),
1823             byteNow);
1824       }
1825 
1826 
1827       this.updatesLock.readLock().lock();
1828       locked = true;
1829 
1830       // ------------------------------------
1831       // STEP 3. Write to WAL
1832       // ----------------------------------
1833       for (int i = firstIndex; i < lastIndexExclusive; i++) {
1834         // Skip puts that were determined to be invalid during preprocessing
1835         if (batchOp.retCodeDetails[i].getOperationStatusCode()
1836             != OperationStatusCode.NOT_RUN) {
1837           continue;
1838         }
1839 
1840         Put p = batchOp.operations[i].getFirst();
1841         if (!p.getWriteToWAL()) continue;
1842         addFamilyMapToWALEdit(familyMaps[i], walEdit);
1843       }
1844 
1845       // Append the edit to WAL
1846       Put first = batchOp.operations[firstIndex].getFirst();
1847       this.log.append(regionInfo, this.htableDescriptor.getName(),
1848           walEdit, first.getClusterId(), now, this.htableDescriptor);
1849 
1850       // ------------------------------------
1851       // STEP 4. Write back to memstore
1852       // ----------------------------------
1853       long addedSize = 0;
1854       for (int i = firstIndex; i < lastIndexExclusive; i++) {
1855         if (batchOp.retCodeDetails[i].getOperationStatusCode()
1856             != OperationStatusCode.NOT_RUN) {
1857           continue;
1858         }
1859         addedSize += applyFamilyMapToMemstore(familyMaps[i]);
1860         batchOp.retCodeDetails[i] = new OperationStatus(
1861             OperationStatusCode.SUCCESS);
1862       }
1863 
1864       // ------------------------------------
1865       // STEP 5. Run coprocessor post hooks
1866       // ------------------------------------
1867       if (coprocessorHost != null) {
1868         for (int i = firstIndex; i < lastIndexExclusive; i++) {
1869           // only for successful puts
1870           if (batchOp.retCodeDetails[i].getOperationStatusCode()
1871               != OperationStatusCode.SUCCESS) {
1872             continue;
1873           }
1874           Put p = batchOp.operations[i].getFirst();
1875           coprocessorHost.postPut(p, walEdit, p.getWriteToWAL());
1876         }
1877       }
1878 
1879       success = true;
1880       return addedSize;
1881     } finally {
1882       if (locked)
1883         this.updatesLock.readLock().unlock();
1884 
1885       for (Integer toRelease : acquiredLocks) {
1886         releaseRowLock(toRelease);
1887       }
1888       if (!success) {
1889         for (int i = firstIndex; i < lastIndexExclusive; i++) {
1890           if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
1891             batchOp.retCodeDetails[i] = new OperationStatus(
1892                 OperationStatusCode.FAILURE);
1893           }
1894         }
1895       }
1896       batchOp.nextIndexToProcess = lastIndexExclusive;
1897     }
1898   }
1899 
1900   //TODO, Think that gets/puts and deletes should be refactored a bit so that
1901   //the getting of the lock happens before, so that you would just pass it into
1902   //the methods. So in the case of checkAndMutate you could just do lockRow,
1903   //get, put, unlockRow or something
1904   /**
1905    *
1906    * @param row
1907    * @param family
1908    * @param qualifier
1909    * @param compareOp
1910    * @param comparator
1911    * @param lockId
1912    * @param writeToWAL
1913    * @throws IOException
1914    * @return true if the new put was execute, false otherwise
1915    */
1916   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
1917       CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
1918       Integer lockId, boolean writeToWAL)
1919   throws IOException{
1920     checkReadOnly();
1921     //TODO, add check for value length or maybe even better move this to the
1922     //client if this becomes a global setting
1923     checkResources();
1924     boolean isPut = w instanceof Put;
1925     if (!isPut && !(w instanceof Delete))
1926       throw new DoNotRetryIOException("Action must be Put or Delete");
1927     Row r = (Row)w;
1928     if (!Bytes.equals(row, r.getRow())) {
1929       throw new DoNotRetryIOException("Action's getRow must match the passed row");
1930     }
1931 
1932     startRegionOperation();
1933     this.writeRequestsCount.increment();
1934     try {
1935       RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
1936       Get get = new Get(row, lock);
1937       checkFamily(family);
1938       get.addColumn(family, qualifier);
1939 
1940       // Lock row
1941       Integer lid = getLock(lockId, get.getRow(), true);
1942       List<KeyValue> result = new ArrayList<KeyValue>();
1943       try {
1944         result = get(get, false);
1945 
1946         boolean valueIsNull = comparator.getValue() == null ||
1947           comparator.getValue().length == 0;
1948         boolean matches = false;
1949         if (result.size() == 0 && valueIsNull) {
1950           matches = true;
1951         } else if (result.size() == 1 && !valueIsNull) {
1952           int compareResult = comparator.compareTo(result.get(0).getValue());
1953           switch (compareOp) {
1954           case LESS:
1955             matches = compareResult <= 0;
1956             break;
1957           case LESS_OR_EQUAL:
1958             matches = compareResult < 0;
1959             break;
1960           case EQUAL:
1961             matches = compareResult == 0;
1962             break;
1963           case NOT_EQUAL:
1964             matches = compareResult != 0;
1965             break;
1966           case GREATER_OR_EQUAL:
1967             matches = compareResult > 0;
1968             break;
1969           case GREATER:
1970             matches = compareResult >= 0;
1971             break;
1972           default:
1973             throw new RuntimeException("Unknown Compare op " + compareOp.name());
1974           }
1975         }
1976         //If matches put the new put or delete the new delete
1977         if (matches) {
1978           // All edits for the given row (across all column families) must
1979           // happen atomically.
1980           //
1981           // Using default cluster id, as this can only happen in the
1982           // originating cluster. A slave cluster receives the result as a Put
1983           // or Delete
1984           if (isPut) {
1985             internalPut(((Put)w), HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
1986           } else {
1987             Delete d = (Delete)w;
1988             prepareDelete(d);
1989             internalDelete(d, HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
1990           }
1991           return true;
1992         }
1993         return false;
1994       } finally {
1995         if(lockId == null) releaseRowLock(lid);
1996       }
1997     } finally {
1998       closeRegionOperation();
1999     }
2000   }
2001 
2002 
2003   /**
2004    * Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP}
2005    * with the provided current timestamp.
2006    */
2007   private void updateKVTimestamps(
2008       final Iterable<List<KeyValue>> keyLists, final byte[] now) {
2009     for (List<KeyValue> keys: keyLists) {
2010       if (keys == null) continue;
2011       for (KeyValue key : keys) {
2012         key.updateLatestStamp(now);
2013       }
2014     }
2015   }
2016 
2017   /*
2018    * Check if resources to support an update.
2019    *
2020    * Here we synchronize on HRegion, a broad scoped lock.  Its appropriate
2021    * given we're figuring in here whether this region is able to take on
2022    * writes.  This is only method with a synchronize (at time of writing),
2023    * this and the synchronize on 'this' inside in internalFlushCache to send
2024    * the notify.
2025    */
2026   private void checkResources() {
2027 
2028     // If catalog region, do not impose resource constraints or block updates.
2029     if (this.getRegionInfo().isMetaRegion()) return;
2030 
2031     boolean blocked = false;
2032     while (this.memstoreSize.get() > this.blockingMemStoreSize) {
2033       requestFlush();
2034       if (!blocked) {
2035         LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
2036           "' on region " + Bytes.toStringBinary(getRegionName()) +
2037           ": memstore size " +
2038           StringUtils.humanReadableInt(this.memstoreSize.get()) +
2039           " is >= than blocking " +
2040           StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
2041       }
2042       blocked = true;
2043       synchronized(this) {
2044         try {
2045           wait(threadWakeFrequency);
2046         } catch (InterruptedException e) {
2047           // continue;
2048         }
2049       }
2050     }
2051     if (blocked) {
2052       LOG.info("Unblocking updates for region " + this + " '"
2053           + Thread.currentThread().getName() + "'");
2054     }
2055   }
2056 
2057   /**
2058    * @throws IOException Throws exception if region is in read-only mode.
2059    */
2060   protected void checkReadOnly() throws IOException {
2061     if (this.writestate.isReadOnly()) {
2062       throw new IOException("region is read only");
2063     }
2064   }
2065 
2066   /**
2067    * Add updates first to the hlog and then add values to memstore.
2068    * Warning: Assumption is caller has lock on passed in row.
2069    * @param family
2070    * @param edits Cell updates by column
2071    * @praram now
2072    * @throws IOException
2073    */
2074   private void put(byte [] family, List<KeyValue> edits)
2075   throws IOException {
2076     Map<byte[], List<KeyValue>> familyMap;
2077     familyMap = new HashMap<byte[], List<KeyValue>>();
2078 
2079     familyMap.put(family, edits);
2080     Put p = new Put();
2081     p.setFamilyMap(familyMap);
2082     p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
2083     p.setWriteToWAL(true);
2084     this.internalPut(p, HConstants.DEFAULT_CLUSTER_ID, true);
2085   }
2086 
2087   /**
2088    * Add updates first to the hlog (if writeToWal) and then add values to memstore.
2089    * Warning: Assumption is caller has lock on passed in row.
2090    * @param put The Put command
2091    * @param writeToWAL if true, then we should write to the log
2092    * @throws IOException
2093    */
2094   private void internalPut(Put put, UUID clusterId,
2095       boolean writeToWAL) throws IOException {
2096     Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
2097     WALEdit walEdit = new WALEdit();
2098     /* run pre put hook outside of lock to avoid deadlock */
2099     if (coprocessorHost != null) {
2100       if (coprocessorHost.prePut(put, walEdit, writeToWAL)) {
2101         return;
2102       }
2103     }
2104 
2105     long now = EnvironmentEdgeManager.currentTimeMillis();
2106     byte[] byteNow = Bytes.toBytes(now);
2107     boolean flush = false;
2108 
2109     this.updatesLock.readLock().lock();
2110     try {
2111       checkFamilies(familyMap.keySet());
2112       updateKVTimestamps(familyMap.values(), byteNow);
2113       // write/sync to WAL should happen before we touch memstore.
2114       //
2115       // If order is reversed, i.e. we write to memstore first, and
2116       // for some reason fail to write/sync to commit log, the memstore
2117       // will contain uncommitted transactions.
2118       if (writeToWAL) {
2119         addFamilyMapToWALEdit(familyMap, walEdit);
2120         this.log.append(regionInfo, this.htableDescriptor.getName(),
2121             walEdit, clusterId, now, this.htableDescriptor);
2122       }
2123 
2124       long addedSize = applyFamilyMapToMemstore(familyMap);
2125       flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
2126     } finally {
2127       this.updatesLock.readLock().unlock();
2128     }
2129 
2130     if (coprocessorHost != null) {
2131       coprocessorHost.postPut(put, walEdit, writeToWAL);
2132     }
2133 
2134     if (flush) {
2135       // Request a cache flush.  Do it outside update lock.
2136       requestFlush();
2137     }
2138   }
2139 
2140   /**
2141    * Atomically apply the given map of family->edits to the memstore.
2142    * This handles the consistency control on its own, but the caller
2143    * should already have locked updatesLock.readLock(). This also does
2144    * <b>not</b> check the families for validity.
2145    *
2146    * @return the additional memory usage of the memstore caused by the
2147    * new entries.
2148    */
2149   private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap) {
2150     MultiVersionConsistencyControl.WriteEntry w = null;
2151     long size = 0;
2152     try {
2153       w = mvcc.beginMemstoreInsert();
2154 
2155       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2156         byte[] family = e.getKey();
2157         List<KeyValue> edits = e.getValue();
2158 
2159         Store store = getStore(family);
2160         for (KeyValue kv: edits) {
2161           kv.setMemstoreTS(w.getWriteNumber());
2162           size += store.add(kv);
2163         }
2164       }
2165     } finally {
2166       mvcc.completeMemstoreInsert(w);
2167     }
2168     return size;
2169   }
2170 
2171   /**
2172    * Check the collection of families for validity.
2173    * @throws NoSuchColumnFamilyException if a family does not exist.
2174    */
2175   private void checkFamilies(Collection<byte[]> families)
2176   throws NoSuchColumnFamilyException {
2177     for (byte[] family : families) {
2178       checkFamily(family);
2179     }
2180   }
2181 
2182   /**
2183    * Append the given map of family->edits to a WALEdit data structure.
2184    * This does not write to the HLog itself.
2185    * @param familyMap map of family->edits
2186    * @param walEdit the destination entry to append into
2187    */
2188   private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
2189       WALEdit walEdit) {
2190     for (List<KeyValue> edits : familyMap.values()) {
2191       for (KeyValue kv : edits) {
2192         walEdit.add(kv);
2193       }
2194     }
2195   }
2196 
2197   private void requestFlush() {
2198     if (this.rsServices == null) {
2199       return;
2200     }
2201     synchronized (writestate) {
2202       if (this.writestate.isFlushRequested()) {
2203         return;
2204       }
2205       writestate.flushRequested = true;
2206     }
2207     // Make request outside of synchronize block; HBASE-818.
2208     this.rsServices.getFlushRequester().requestFlush(this);
2209     if (LOG.isDebugEnabled()) {
2210       LOG.debug("Flush requested on " + this);
2211     }
2212   }
2213 
2214   /*
2215    * @param size
2216    * @return True if size is over the flush threshold
2217    */
2218   private boolean isFlushSize(final long size) {
2219     return size > this.memstoreFlushSize;
2220   }
2221 
2222   /**
2223    * Read the edits log put under this region by wal log splitting process.  Put
2224    * the recovered edits back up into this region.
2225    *
2226    * <p>We can ignore any log message that has a sequence ID that's equal to or
2227    * lower than minSeqId.  (Because we know such log messages are already
2228    * reflected in the HFiles.)
2229    *
2230    * <p>While this is running we are putting pressure on memory yet we are
2231    * outside of our usual accounting because we are not yet an onlined region
2232    * (this stuff is being run as part of Region initialization).  This means
2233    * that if we're up against global memory limits, we'll not be flagged to flush
2234    * because we are not online. We can't be flushed by usual mechanisms anyways;
2235    * we're not yet online so our relative sequenceids are not yet aligned with
2236    * HLog sequenceids -- not till we come up online, post processing of split
2237    * edits.
2238    *
2239    * <p>But to help relieve memory pressure, at least manage our own heap size
2240    * flushing if are in excess of per-region limits.  Flushing, though, we have
2241    * to be careful and avoid using the regionserver/hlog sequenceid.  Its running
2242    * on a different line to whats going on in here in this region context so if we
2243    * crashed replaying these edits, but in the midst had a flush that used the
2244    * regionserver log with a sequenceid in excess of whats going on in here
2245    * in this region and with its split editlogs, then we could miss edits the
2246    * next time we go to recover. So, we have to flush inline, using seqids that
2247    * make sense in a this single region context only -- until we online.
2248    *
2249    * @param regiondir
2250    * @param minSeqId Any edit found in split editlogs needs to be in excess of
2251    * this minSeqId to be applied, else its skipped.
2252    * @param reporter
2253    * @return the sequence id of the last edit added to this region out of the
2254    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
2255    * @throws UnsupportedEncodingException
2256    * @throws IOException
2257    */
2258   protected long replayRecoveredEditsIfAny(final Path regiondir,
2259       final long minSeqId, final CancelableProgressable reporter,
2260       final MonitoredTask status)
2261       throws UnsupportedEncodingException, IOException {
2262     long seqid = minSeqId;
2263     NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
2264     if (files == null || files.isEmpty()) return seqid;
2265     boolean checkSafeToSkip = true;
2266     for (Path edits: files) {
2267       if (edits == null || !this.fs.exists(edits)) {
2268         LOG.warn("Null or non-existent edits file: " + edits);
2269         continue;
2270       }
2271       if (isZeroLengthThenDelete(this.fs, edits)) continue;
2272 
2273       if (checkSafeToSkip) {
2274         Path higher = files.higher(edits);
2275         long maxSeqId = Long.MAX_VALUE;
2276         if (higher != null) {
2277           // Edit file name pattern, HLog.EDITFILES_NAME_PATTERN: "-?[0-9]+"
2278           String fileName = higher.getName();
2279           maxSeqId = Math.abs(Long.parseLong(fileName));
2280         }
2281         if (maxSeqId <= minSeqId) {
2282           String msg = "Maximum possible sequenceid for this log is " + maxSeqId
2283               + ", skipped the whole file, path=" + edits;
2284           LOG.debug(msg);
2285           continue;
2286         } else {
2287           checkSafeToSkip = false;
2288         }
2289       }
2290 
2291       try {
2292         seqid = replayRecoveredEdits(edits, seqid, reporter);
2293       } catch (IOException e) {
2294         boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
2295         if (skipErrors) {
2296           Path p = HLog.moveAsideBadEditsFile(fs, edits);
2297           LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits +
2298             " as " + p, e);
2299         } else {
2300           throw e;
2301         }
2302       }
2303     }
2304     if (seqid > minSeqId) {
2305       // Then we added some edits to memory. Flush and cleanup split edit files.
2306       internalFlushcache(null, seqid, status);
2307     }
2308     // Now delete the content of recovered edits.  We're done w/ them.
2309     for (Path file: files) {
2310       if (!this.fs.delete(file, false)) {
2311         LOG.error("Failed delete of " + file);
2312       } else {
2313         LOG.debug("Deleted recovered.edits file=" + file);
2314       }
2315     }
2316     return seqid;
2317   }
2318 
2319   /*
2320    * @param edits File of recovered edits.
2321    * @param minSeqId Minimum sequenceid found in a store file.  Edits in log
2322    * must be larger than this to be replayed.
2323    * @param reporter
2324    * @return the sequence id of the last edit added to this region out of the
2325    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
2326    * @throws IOException
2327    */
2328   private long replayRecoveredEdits(final Path edits,
2329       final long minSeqId, final CancelableProgressable reporter)
2330     throws IOException {
2331     String msg = "Replaying edits from " + edits + "; minSequenceid=" +
2332       minSeqId + "; path=" + edits;
2333     LOG.info(msg);
2334     MonitoredTask status = TaskMonitor.get().createStatus(msg);
2335 
2336     status.setStatus("Opening logs");
2337     HLog.Reader reader = HLog.getReader(this.fs, edits, conf);
2338     try {
2339       long currentEditSeqId = minSeqId;
2340       long firstSeqIdInLog = -1;
2341       long skippedEdits = 0;
2342       long editsCount = 0;
2343       long intervalEdits = 0;
2344       HLog.Entry entry;
2345       Store store = null;
2346       boolean reported_once = false;
2347 
2348       try {
2349         // How many edits seen before we check elapsed time
2350         int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
2351             2000);
2352         // How often to send a progress report (default 1/2 master timeout)
2353         int period = this.conf.getInt("hbase.hstore.report.period",
2354             this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
2355                 180000) / 2);
2356         long lastReport = EnvironmentEdgeManager.currentTimeMillis();
2357 
2358         while ((entry = reader.next()) != null) {
2359           HLogKey key = entry.getKey();
2360           WALEdit val = entry.getEdit();
2361 
2362           if (reporter != null) {
2363             intervalEdits += val.size();
2364             if (intervalEdits >= interval) {
2365               // Number of edits interval reached
2366               intervalEdits = 0;
2367               long cur = EnvironmentEdgeManager.currentTimeMillis();
2368               if (lastReport + period <= cur) {
2369                 status.setStatus("Replaying edits..." +
2370                     " skipped=" + skippedEdits +
2371                     " edits=" + editsCount);
2372                 // Timeout reached
2373                 if(!reporter.progress()) {
2374                   msg = "Progressable reporter failed, stopping replay";
2375                   LOG.warn(msg);
2376                   status.abort(msg);
2377                   throw new IOException(msg);
2378                 }
2379                 reported_once = true;
2380                 lastReport = cur;
2381               }
2382             }
2383           }
2384 
2385           // Start coprocessor replay here. The coprocessor is for each WALEdit
2386           // instead of a KeyValue.
2387           if (coprocessorHost != null) {
2388             status.setStatus("Running pre-WAL-restore hook in coprocessors");
2389             if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
2390               // if bypass this log entry, ignore it ...
2391               continue;
2392             }
2393           }
2394 
2395           if (firstSeqIdInLog == -1) {
2396             firstSeqIdInLog = key.getLogSeqNum();
2397           }
2398           // Now, figure if we should skip this edit.
2399           if (key.getLogSeqNum() <= currentEditSeqId) {
2400             skippedEdits++;
2401             continue;
2402           }
2403           currentEditSeqId = key.getLogSeqNum();
2404           boolean flush = false;
2405           for (KeyValue kv: val.getKeyValues()) {
2406             // Check this edit is for me. Also, guard against writing the special
2407             // METACOLUMN info such as HBASE::CACHEFLUSH entries
2408             if (kv.matchingFamily(HLog.METAFAMILY) ||
2409                 !Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
2410               skippedEdits++;
2411               continue;
2412                 }
2413             // Figure which store the edit is meant for.
2414             if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
2415               store = this.stores.get(kv.getFamily());
2416             }
2417             if (store == null) {
2418               // This should never happen.  Perhaps schema was changed between
2419               // crash and redeploy?
2420               LOG.warn("No family for " + kv);
2421               skippedEdits++;
2422               continue;
2423             }
2424             // Once we are over the limit, restoreEdit will keep returning true to
2425             // flush -- but don't flush until we've played all the kvs that make up
2426             // the WALEdit.
2427             flush = restoreEdit(store, kv);
2428             editsCount++;
2429           }
2430           if (flush) internalFlushcache(null, currentEditSeqId, status);
2431 
2432           if (coprocessorHost != null) {
2433             coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
2434           }
2435         }
2436       } catch (EOFException eof) {
2437         Path p = HLog.moveAsideBadEditsFile(fs, edits);
2438         msg = "Encountered EOF. Most likely due to Master failure during " +
2439             "log spliting, so we have this data in another edit.  " +
2440             "Continuing, but renaming " + edits + " as " + p;
2441         LOG.warn(msg, eof);
2442         status.abort(msg);
2443       } catch (IOException ioe) {
2444         // If the IOE resulted from bad file format,
2445         // then this problem is idempotent and retrying won't help
2446         if (ioe.getCause() instanceof ParseException) {
2447           Path p = HLog.moveAsideBadEditsFile(fs, edits);
2448           msg = "File corruption encountered!  " +
2449               "Continuing, but renaming " + edits + " as " + p;
2450           LOG.warn(msg, ioe);
2451           status.setStatus(msg);
2452         } else {
2453           status.abort(StringUtils.stringifyException(ioe));
2454           // other IO errors may be transient (bad network connection,
2455           // checksum exception on one datanode, etc).  throw & retry
2456           throw ioe;
2457         }
2458       }
2459       if (reporter != null && !reported_once) {
2460         reporter.progress();
2461       }
2462       msg = "Applied " + editsCount + ", skipped " + skippedEdits +
2463         ", firstSequenceidInLog=" + firstSeqIdInLog +
2464         ", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
2465       status.markComplete(msg);
2466       LOG.debug(msg);
2467       return currentEditSeqId;
2468     } finally {
2469       reader.close();
2470       status.cleanup();
2471     }
2472   }
2473 
2474   /**
2475    * Used by tests
2476    * @param s Store to add edit too.
2477    * @param kv KeyValue to add.
2478    * @return True if we should flush.
2479    */
2480   protected boolean restoreEdit(final Store s, final KeyValue kv) {
2481     return isFlushSize(this.addAndGetGlobalMemstoreSize(s.add(kv)));
2482   }
2483 
2484   /*
2485    * @param fs
2486    * @param p File to check.
2487    * @return True if file was zero-length (and if so, we'll delete it in here).
2488    * @throws IOException
2489    */
2490   private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
2491       throws IOException {
2492     FileStatus stat = fs.getFileStatus(p);
2493     if (stat.getLen() > 0) return false;
2494     LOG.warn("File " + p + " is zero-length, deleting.");
2495     fs.delete(p, false);
2496     return true;
2497   }
2498 
2499   protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
2500       throws IOException {
2501     return new Store(tableDir, this, c, this.fs, this.conf);
2502   }
2503 
2504   /**
2505    * Return HStore instance.
2506    * Use with caution.  Exposed for use of fixup utilities.
2507    * @param column Name of column family hosted by this region.
2508    * @return Store that goes with the family on passed <code>column</code>.
2509    * TODO: Make this lookup faster.
2510    */
2511   public Store getStore(final byte [] column) {
2512     return this.stores.get(column);
2513   }
2514 
2515   public Map<byte[], Store> getStores() {
2516     return this.stores;
2517   }
2518 
2519   //////////////////////////////////////////////////////////////////////////////
2520   // Support code
2521   //////////////////////////////////////////////////////////////////////////////
2522 
2523   /** Make sure this is a valid row for the HRegion */
2524   void checkRow(final byte [] row, String op) throws IOException {
2525     if(!rowIsInRange(regionInfo, row)) {
2526       throw new WrongRegionException("Requested row out of range for " +
2527           op + " on HRegion " + this + ", startKey='" +
2528           Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
2529           Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
2530           Bytes.toStringBinary(row) + "'");
2531     }
2532   }
2533 
2534   /**
2535    * Obtain a lock on the given row.  Blocks until success.
2536    *
2537    * I know it's strange to have two mappings:
2538    * <pre>
2539    *   ROWS  ==> LOCKS
2540    * </pre>
2541    * as well as
2542    * <pre>
2543    *   LOCKS ==> ROWS
2544    * </pre>
2545    *
2546    * But it acts as a guard on the client; a miswritten client just can't
2547    * submit the name of a row and start writing to it; it must know the correct
2548    * lockid, which matches the lock list in memory.
2549    *
2550    * <p>It would be more memory-efficient to assume a correctly-written client,
2551    * which maybe we'll do in the future.
2552    *
2553    * @param row Name of row to lock.
2554    * @throws IOException
2555    * @return The id of the held lock.
2556    */
2557   public Integer obtainRowLock(final byte [] row) throws IOException {
2558     startRegionOperation();
2559     this.writeRequestsCount.increment();
2560     try {
2561       return internalObtainRowLock(row, true);
2562     } finally {
2563       closeRegionOperation();
2564     }
2565   }
2566 
2567   /**
2568    * Obtains or tries to obtain the given row lock.
2569    * @param waitForLock if true, will block until the lock is available.
2570    *        Otherwise, just tries to obtain the lock and returns
2571    *        null if unavailable.
2572    */
2573   private Integer internalObtainRowLock(final byte[] row, boolean waitForLock)
2574       throws IOException {
2575     checkRow(row, "row lock");
2576     startRegionOperation();
2577     try {
2578       HashedBytes rowKey = new HashedBytes(row);
2579       CountDownLatch rowLatch = new CountDownLatch(1);
2580 
2581       // loop until we acquire the row lock (unless !waitForLock)
2582       while (true) {
2583         CountDownLatch existingLatch = lockedRows.putIfAbsent(rowKey, rowLatch);
2584         if (existingLatch == null) {
2585           break;
2586         } else {
2587           // row already locked
2588           if (!waitForLock) {
2589             return null;
2590           }
2591           try {
2592             if (!existingLatch.await(this.rowLockWaitDuration,
2593                             TimeUnit.MILLISECONDS)) {
2594                 return null;
2595             }
2596           } catch (InterruptedException ie) {
2597             // Empty
2598           }
2599         }
2600       }
2601 
2602       // loop until we generate an unused lock id
2603       while (true) {
2604         Integer lockId = lockIdGenerator.incrementAndGet();
2605         HashedBytes existingRowKey = lockIds.putIfAbsent(lockId, rowKey);
2606         if (existingRowKey == null) {
2607           return lockId;
2608         } else {
2609           // lockId already in use, jump generator to a new spot
2610           lockIdGenerator.set(rand.nextInt());
2611         }
2612       }
2613     } finally {
2614       closeRegionOperation();
2615     }
2616   }
2617 
2618   /**
2619    * Used by unit tests.
2620    * @param lockid
2621    * @return Row that goes with <code>lockid</code>
2622    */
2623   byte[] getRowFromLock(final Integer lockid) {
2624     HashedBytes rowKey = lockIds.get(lockid);
2625     return rowKey == null ? null : rowKey.getBytes();
2626   }
2627 
2628   /**
2629    * Release the row lock!
2630    * @param lockId  The lock ID to release.
2631    */
2632   public void releaseRowLock(final Integer lockId) {
2633     HashedBytes rowKey = lockIds.remove(lockId);
2634     if (rowKey == null) {
2635       LOG.warn("Release unknown lockId: " + lockId);
2636       return;
2637     }
2638     CountDownLatch rowLatch = lockedRows.remove(rowKey);
2639     if (rowLatch == null) {
2640       LOG.error("Releases row not locked, lockId: " + lockId + " row: "
2641           + rowKey);
2642       return;
2643     }
2644     rowLatch.countDown();
2645   }
2646 
2647   /**
2648    * See if row is currently locked.
2649    * @param lockid
2650    * @return boolean
2651    */
2652   boolean isRowLocked(final Integer lockId) {
2653     return lockIds.containsKey(lockId);
2654   }
2655 
2656   /**
2657    * Returns existing row lock if found, otherwise
2658    * obtains a new row lock and returns it.
2659    * @param lockid requested by the user, or null if the user didn't already hold lock
2660    * @param row the row to lock
2661    * @param waitForLock if true, will block until the lock is available, otherwise will
2662    * simply return null if it could not acquire the lock.
2663    * @return lockid or null if waitForLock is false and the lock was unavailable.
2664    */
2665   private Integer getLock(Integer lockid, byte [] row, boolean waitForLock)
2666   throws IOException {
2667     Integer lid = null;
2668     if (lockid == null) {
2669       lid = internalObtainRowLock(row, waitForLock);
2670     } else {
2671       if (!isRowLocked(lockid)) {
2672         throw new IOException("Invalid row lock");
2673       }
2674       lid = lockid;
2675     }
2676     return lid;
2677   }
2678 
2679   /**
2680    * Determines whether multiple column families are present
2681    * Precondition: familyPaths is not null
2682    *
2683    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
2684    */
2685   private static boolean hasMultipleColumnFamilies(
2686       List<Pair<byte[], String>> familyPaths) {
2687     boolean multipleFamilies = false;
2688     byte[] family = null;
2689     for (Pair<byte[], String> pair : familyPaths) {
2690       byte[] fam = pair.getFirst();
2691       if (family == null) {
2692         family = fam;
2693       } else if (!Bytes.equals(family, fam)) {
2694         multipleFamilies = true;
2695         break;
2696       }
2697     }
2698     return multipleFamilies;
2699   }
2700 
2701   /**
2702    * Attempts to atomically load a group of hfiles.  This is critical for loading
2703    * rows with multiple column families atomically.
2704    *
2705    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
2706    * @return true if successful, false if failed recoverably
2707    * @throws IOException if failed unrecoverably.
2708    */
2709   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths)
2710   throws IOException {
2711     Preconditions.checkNotNull(familyPaths);
2712     // we need writeLock for multi-family bulk load
2713     startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
2714     try {
2715       this.writeRequestsCount.increment();
2716 
2717       // There possibly was a split that happend between when the split keys
2718       // were gathered and before the HReiogn's write lock was taken.  We need
2719       // to validate the HFile region before attempting to bulk load all of them
2720       List<IOException> ioes = new ArrayList<IOException>();
2721       List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
2722       for (Pair<byte[], String> p : familyPaths) {
2723         byte[] familyName = p.getFirst();
2724         String path = p.getSecond();
2725 
2726         Store store = getStore(familyName);
2727         if (store == null) {
2728           IOException ioe = new DoNotRetryIOException(
2729               "No such column family " + Bytes.toStringBinary(familyName));
2730           ioes.add(ioe);
2731           failures.add(p);
2732         }
2733 
2734         try {
2735           store.assertBulkLoadHFileOk(new Path(path));
2736         } catch (WrongRegionException wre) {
2737           // recoverable (file doesn't fit in region)
2738           failures.add(p);
2739         } catch (IOException ioe) {
2740           // unrecoverable (hdfs problem)
2741           ioes.add(ioe);
2742         }
2743       }
2744 
2745 
2746       // validation failed, bail out before doing anything permanent.
2747       if (failures.size() != 0) {
2748         StringBuilder list = new StringBuilder();
2749         for (Pair<byte[], String> p : failures) {
2750           list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
2751             .append(p.getSecond());
2752         }
2753         // problem when validating
2754         LOG.warn("There was a recoverable bulk load failure likely due to a" +
2755             " split.  These (family, HFile) pairs were not loaded: " + list);
2756         return false;
2757       }
2758 
2759       // validation failed because of some sort of IO problem.
2760       if (ioes.size() != 0) {
2761         LOG.error("There were IO errors when checking if bulk load is ok.  " +
2762             "throwing exception!");
2763         throw MultipleIOException.createIOException(ioes);
2764       }
2765 
2766       for (Pair<byte[], String> p : familyPaths) {
2767         byte[] familyName = p.getFirst();
2768         String path = p.getSecond();
2769         Store store = getStore(familyName);
2770         try {
2771           store.bulkLoadHFile(path);
2772         } catch (IOException ioe) {
2773           // a failure here causes an atomicity violation that we currently
2774           // cannot recover from since it is likely a failed hdfs operation.
2775 
2776           // TODO Need a better story for reverting partial failures due to HDFS.
2777           LOG.error("There was a partial failure due to IO when attempting to" +
2778               " load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond());
2779           throw ioe;
2780         }
2781       }
2782       return true;
2783     } finally {
2784       closeBulkRegionOperation();
2785     }
2786   }
2787 
2788   @Override
2789   public boolean equals(Object o) {
2790     if (!(o instanceof HRegion)) {
2791       return false;
2792     }
2793     return Bytes.equals(this.getRegionName(), ((HRegion) o).getRegionName());
2794   }
2795 
2796   @Override
2797   public int hashCode() {
2798     return Bytes.hashCode(this.getRegionName());
2799   }
2800 
2801   @Override
2802   public String toString() {
2803     return this.regionInfo.getRegionNameAsString();
2804   }
2805 
2806   /** @return Path of region base directory */
2807   public Path getTableDir() {
2808     return this.tableDir;
2809   }
2810 
2811   /**
2812    * RegionScannerImpl is used to combine scanners from multiple Stores (aka column families).
2813    */
2814   class RegionScannerImpl implements RegionScanner {
2815     // Package local for testability
2816     KeyValueHeap storeHeap = null;
2817     private final byte [] stopRow;
2818     private Filter filter;
2819     private List<KeyValue> results = new ArrayList<KeyValue>();
2820     private int batch;
2821     private int isScan;
2822     private boolean filterClosed = false;
2823     private long readPt;
2824 
2825     public HRegionInfo getRegionInfo() {
2826       return regionInfo;
2827     }
2828     RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
2829       //DebugPrint.println("HRegionScanner.<init>");
2830 
2831       this.filter = scan.getFilter();
2832       this.batch = scan.getBatch();
2833       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
2834         this.stopRow = null;
2835       } else {
2836         this.stopRow = scan.getStopRow();
2837       }
2838       // If we are doing a get, we want to be [startRow,endRow] normally
2839       // it is [startRow,endRow) and if startRow=endRow we get nothing.
2840       this.isScan = scan.isGetScan() ? -1 : 0;
2841 
2842       // synchronize on scannerReadPoints so that nobody calculates
2843       // getSmallestReadPoint, before scannerReadPoints is updated.
2844       synchronized(scannerReadPoints) {
2845         this.readPt = MultiVersionConsistencyControl.resetThreadReadPoint(mvcc);
2846         scannerReadPoints.put(this, this.readPt);
2847       }
2848 
2849       List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
2850       if (additionalScanners != null) {
2851         scanners.addAll(additionalScanners);
2852       }
2853 
2854       for (Map.Entry<byte[], NavigableSet<byte[]>> entry :
2855           scan.getFamilyMap().entrySet()) {
2856         Store store = stores.get(entry.getKey());
2857         StoreScanner scanner = store.getScanner(scan, entry.getValue());
2858         scanners.add(scanner);
2859       }
2860       this.storeHeap = new KeyValueHeap(scanners, comparator);
2861     }
2862 
2863     RegionScannerImpl(Scan scan) throws IOException {
2864       this(scan, null);
2865     }
2866 
2867     /**
2868      * Reset both the filter and the old filter.
2869      */
2870     protected void resetFilters() {
2871       if (filter != null) {
2872         filter.reset();
2873       }
2874     }
2875 
2876     @Override
2877 	public synchronized boolean next(List<KeyValue> outResults, int limit)
2878         throws IOException {
2879       if (this.filterClosed) {
2880         throw new UnknownScannerException("Scanner was closed (timed out?) " +
2881             "after we renewed it. Could be caused by a very slow scanner " +
2882             "or a lengthy garbage collection");
2883       }
2884       startRegionOperation();
2885       readRequestsCount.increment();
2886       try {
2887 
2888         // This could be a new thread from the last time we called next().
2889         MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
2890 
2891         results.clear();
2892 
2893         boolean returnResult = nextInternal(limit);
2894 
2895         outResults.addAll(results);
2896         resetFilters();
2897         if (isFilterDone()) {
2898           return false;
2899         }
2900         return returnResult;
2901       } finally {
2902         closeRegionOperation();
2903       }
2904     }
2905 
2906     @Override
2907 	public synchronized boolean next(List<KeyValue> outResults)
2908         throws IOException {
2909       // apply the batching limit by default
2910       return next(outResults, batch);
2911     }
2912 
2913     /*
2914      * @return True if a filter rules the scanner is over, done.
2915      */
2916     public synchronized boolean isFilterDone() {
2917       return this.filter != null && this.filter.filterAllRemaining();
2918     }
2919 
2920     private boolean nextInternal(int limit) throws IOException {
2921       while (true) {
2922         byte [] currentRow = peekRow();
2923         if (isStopRow(currentRow)) {
2924           if (filter != null && filter.hasFilterRow()) {
2925             filter.filterRow(results);
2926           }
2927           if (filter != null && filter.filterRow()) {
2928             results.clear();
2929           }
2930 
2931           return false;
2932         } else if (filterRowKey(currentRow)) {
2933           nextRow(currentRow);
2934         } else {
2935           byte [] nextRow;
2936           do {
2937             this.storeHeap.next(results, limit - results.size());
2938             if (limit > 0 && results.size() == limit) {
2939               if (this.filter != null && filter.hasFilterRow()) {
2940                 throw new IncompatibleFilterException(
2941                   "Filter with filterRow(List<KeyValue>) incompatible with scan with limit!");
2942               }
2943               return true; // we are expecting more yes, but also limited to how many we can return.
2944             }
2945           } while (Bytes.equals(currentRow, nextRow = peekRow()));
2946 
2947           final boolean stopRow = isStopRow(nextRow);
2948 
2949           // now that we have an entire row, lets process with a filters:
2950 
2951           // first filter with the filterRow(List)
2952           if (filter != null && filter.hasFilterRow()) {
2953             filter.filterRow(results);
2954           }
2955 
2956           if (results.isEmpty() || filterRow()) {
2957             // this seems like a redundant step - we already consumed the row
2958             // there're no left overs.
2959             // the reasons for calling this method are:
2960             // 1. reset the filters.
2961             // 2. provide a hook to fast forward the row (used by subclasses)
2962             nextRow(currentRow);
2963 
2964             // This row was totally filtered out, if this is NOT the last row,
2965             // we should continue on.
2966 
2967             if (!stopRow) continue;
2968           }
2969           return !stopRow;
2970         }
2971       }
2972     }
2973 
2974     private boolean filterRow() {
2975       return filter != null
2976           && filter.filterRow();
2977     }
2978     private boolean filterRowKey(byte[] row) {
2979       return filter != null
2980           && filter.filterRowKey(row, 0, row.length);
2981     }
2982 
2983     protected void nextRow(byte [] currentRow) throws IOException {
2984       while (Bytes.equals(currentRow, peekRow())) {
2985         this.storeHeap.next(MOCKED_LIST);
2986       }
2987       results.clear();
2988       resetFilters();
2989     }
2990 
2991     private byte[] peekRow() {
2992       KeyValue kv = this.storeHeap.peek();
2993       return kv == null ? null : kv.getRow();
2994     }
2995 
2996     private boolean isStopRow(byte [] currentRow) {
2997       return currentRow == null ||
2998           (stopRow != null &&
2999           comparator.compareRows(stopRow, 0, stopRow.length,
3000               currentRow, 0, currentRow.length) <= isScan);
3001     }
3002 
3003     @Override
3004 	public synchronized void close() {
3005       if (storeHeap != null) {
3006         storeHeap.close();
3007         storeHeap = null;
3008       }
3009       // no need to sychronize here.
3010       scannerReadPoints.remove(this);
3011       this.filterClosed = true;
3012     }
3013 
3014     KeyValueHeap getStoreHeapForTesting() {
3015       return storeHeap;
3016     }
3017   }
3018 
3019   // Utility methods
3020   /**
3021    * A utility method to create new instances of HRegion based on the
3022    * {@link HConstants#REGION_IMPL} configuration property.
3023    * @param tableDir qualified path of directory where region should be located,
3024    * usually the table directory.
3025    * @param log The HLog is the outbound log for any updates to the HRegion
3026    * (There's a single HLog for all the HRegions on a single HRegionServer.)
3027    * The log file is a logfile from the previous execution that's
3028    * custom-computed for this HRegion. The HRegionServer computes and sorts the
3029    * appropriate log info for this HRegion. If there is a previous log file
3030    * (implying that the HRegion has been written-to before), then read it from
3031    * the supplied path.
3032    * @param fs is the filesystem.
3033    * @param conf is global configuration settings.
3034    * @param regionInfo - HRegionInfo that describes the region
3035    * is new), then read them from the supplied path.
3036    * @param htd
3037    * @param rsServices
3038    * @return the new instance
3039    */
3040   public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
3041       Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
3042       RegionServerServices rsServices) {
3043     try {
3044       @SuppressWarnings("unchecked")
3045       Class<? extends HRegion> regionClass =
3046           (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
3047 
3048       Constructor<? extends HRegion> c =
3049           regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
3050               Configuration.class, HRegionInfo.class, HTableDescriptor.class,
3051               RegionServerServices.class);
3052 
3053       return c.newInstance(tableDir, log, fs, conf, regionInfo, htd, rsServices);
3054     } catch (Throwable e) {
3055       // todo: what should I throw here?
3056       throw new IllegalStateException("Could not instantiate a region instance.", e);
3057     }
3058   }
3059 
3060   /**
3061    * Convenience method creating new HRegions. Used by createTable and by the
3062    * bootstrap code in the HMaster constructor.
3063    * Note, this method creates an {@link HLog} for the created region. It
3064    * needs to be closed explicitly.  Use {@link HRegion#getLog()} to get
3065    * access.
3066    * @param info Info for region to create.
3067    * @param rootDir Root directory for HBase instance
3068    * @param conf
3069    * @param hTableDescriptor
3070    * @return new HRegion
3071    *
3072    * @throws IOException
3073    */
3074   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
3075       final Configuration conf, final HTableDescriptor hTableDescriptor)
3076   throws IOException {
3077     return createHRegion(info, rootDir, conf, hTableDescriptor, null);
3078   }
3079 
3080   /**
3081    * Convenience method creating new HRegions. Used by createTable.
3082    * The {@link HLog} for the created region needs to be closed explicitly.
3083    * Use {@link HRegion#getLog()} to get access.
3084    *
3085    * @param info Info for region to create.
3086    * @param rootDir Root directory for HBase instance
3087    * @param conf
3088    * @param hTableDescriptor
3089    * @param hlog shared HLog
3090    * @return new HRegion
3091    *
3092    * @throws IOException
3093    */
3094   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
3095                                       final Configuration conf,
3096                                       final HTableDescriptor hTableDescriptor,
3097                                       final HLog hlog)
3098       throws IOException {
3099     LOG.info("creating HRegion " + info.getTableNameAsString()
3100         + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
3101         " Table name == " + info.getTableNameAsString());
3102 
3103     Path tableDir =
3104         HTableDescriptor.getTableDir(rootDir, info.getTableName());
3105     Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
3106     FileSystem fs = FileSystem.get(conf);
3107     fs.mkdirs(regionDir);
3108     HLog effectiveHLog = hlog;
3109     if (hlog == null) {
3110       effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
3111           new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
3112     }
3113     HRegion region = HRegion.newHRegion(tableDir,
3114         effectiveHLog, fs, conf, info, hTableDescriptor, null);
3115     region.initialize();
3116     return region;
3117   }
3118 
3119   /**
3120    * Open a Region.
3121    * @param info Info for region to be opened.
3122    * @param wal HLog for region to use. This method will call
3123    * HLog#setSequenceNumber(long) passing the result of the call to
3124    * HRegion#getMinSequenceId() to ensure the log id is properly kept
3125    * up.  HRegionStore does this every time it opens a new region.
3126    * @param conf
3127    * @return new HRegion
3128    *
3129    * @throws IOException
3130    */
3131   public static HRegion openHRegion(final HRegionInfo info,
3132       final HTableDescriptor htd, final HLog wal,
3133       final Configuration conf)
3134   throws IOException {
3135     return openHRegion(info, htd, wal, conf, null, null);
3136   }
3137 
3138   /**
3139    * Open a Region.
3140    * @param info Info for region to be opened
3141    * @param htd
3142    * @param wal HLog for region to use. This method will call
3143    * HLog#setSequenceNumber(long) passing the result of the call to
3144    * HRegion#getMinSequenceId() to ensure the log id is properly kept
3145    * up.  HRegionStore does this every time it opens a new region.
3146    * @param conf
3147    * @param rsServices An interface we can request flushes against.
3148    * @param reporter An interface we can report progress against.
3149    * @return new HRegion
3150    *
3151    * @throws IOException
3152    */
3153   public static HRegion openHRegion(final HRegionInfo info,
3154     final HTableDescriptor htd, final HLog wal, final Configuration conf,
3155     final RegionServerServices rsServices,
3156     final CancelableProgressable reporter)
3157   throws IOException {
3158     if (LOG.isDebugEnabled()) {
3159       LOG.debug("Opening region: " + info);
3160     }
3161     if (info == null) {
3162       throw new NullPointerException("Passed region info is null");
3163     }
3164     Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
3165       info.getTableName());
3166     HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
3167       htd, rsServices);
3168     return r.openHRegion(reporter);
3169   }
3170 
3171   public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
3172       final HTableDescriptor htd, final HLog wal, final Configuration conf)
3173   throws IOException {
3174     return openHRegion(tableDir, info, htd, wal, conf, null, null);
3175   }
3176 
3177   /**
3178    * Open a Region.
3179    * @param tableDir Table directory
3180    * @param info Info for region to be opened.
3181    * @param wal HLog for region to use. This method will call
3182    * HLog#setSequenceNumber(long) passing the result of the call to
3183    * HRegion#getMinSequenceId() to ensure the log id is properly kept
3184    * up.  HRegionStore does this every time it opens a new region.
3185    * @param conf
3186    * @param reporter An interface we can report progress against.
3187    * @return new HRegion
3188    *
3189    * @throws IOException
3190    */
3191   public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
3192       final HTableDescriptor htd, final HLog wal, final Configuration conf,
3193       final RegionServerServices rsServices,
3194       final CancelableProgressable reporter)
3195   throws IOException {
3196     if (info == null) throw new NullPointerException("Passed region info is null");
3197     LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
3198     if (LOG.isDebugEnabled()) {
3199       LOG.debug("Opening region: " + info);
3200     }
3201     Path dir = HTableDescriptor.getTableDir(tableDir,
3202         info.getTableName());
3203     HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
3204         htd, rsServices);
3205     return r.openHRegion(reporter);
3206   }
3207 
3208 
3209   /**
3210    * Open HRegion.
3211    * Calls initialize and sets sequenceid.
3212    * @param reporter
3213    * @return Returns <code>this</code>
3214    * @throws IOException
3215    */
3216   protected HRegion openHRegion(final CancelableProgressable reporter)
3217   throws IOException {
3218     checkCompressionCodecs();
3219 
3220     long seqid = initialize(reporter);
3221     if (this.log != null) {
3222       this.log.setSequenceNumber(seqid);
3223     }
3224     return this;
3225   }
3226 
3227   private void checkCompressionCodecs() throws IOException {
3228     for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
3229       CompressionTest.testCompression(fam.getCompression());
3230       CompressionTest.testCompression(fam.getCompactionCompression());
3231     }
3232   }
3233 
3234   /**
3235    * Inserts a new region's meta information into the passed
3236    * <code>meta</code> region. Used by the HMaster bootstrap code adding
3237    * new table to ROOT table.
3238    *
3239    * @param meta META HRegion to be updated
3240    * @param r HRegion to add to <code>meta</code>
3241    *
3242    * @throws IOException
3243    */
3244   public static void addRegionToMETA(HRegion meta, HRegion r)
3245   throws IOException {
3246     meta.checkResources();
3247     // The row key is the region name
3248     byte[] row = r.getRegionName();
3249     Integer lid = meta.obtainRowLock(row);
3250     try {
3251       final long now = EnvironmentEdgeManager.currentTimeMillis();
3252       final List<KeyValue> edits = new ArrayList<KeyValue>(2);
3253       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
3254         HConstants.REGIONINFO_QUALIFIER, now,
3255         Writables.getBytes(r.getRegionInfo())));
3256       // Set into the root table the version of the meta table.
3257       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
3258         HConstants.META_VERSION_QUALIFIER, now,
3259         Bytes.toBytes(HConstants.META_VERSION)));
3260       meta.put(HConstants.CATALOG_FAMILY, edits);
3261     } finally {
3262       meta.releaseRowLock(lid);
3263     }
3264   }
3265 
3266   /**
3267    * Deletes all the files for a HRegion
3268    *
3269    * @param fs the file system object
3270    * @param rootdir qualified path of HBase root directory
3271    * @param info HRegionInfo for region to be deleted
3272    * @throws IOException
3273    */
3274   public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
3275   throws IOException {
3276     deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
3277   }
3278 
3279   private static void deleteRegion(FileSystem fs, Path regiondir)
3280   throws IOException {
3281     if (LOG.isDebugEnabled()) {
3282       LOG.debug("DELETING region " + regiondir.toString());
3283     }
3284     if (!fs.delete(regiondir, true)) {
3285       LOG.warn("Failed delete of " + regiondir);
3286     }
3287   }
3288 
3289   /**
3290    * Computes the Path of the HRegion
3291    *
3292    * @param rootdir qualified path of HBase root directory
3293    * @param info HRegionInfo for the region
3294    * @return qualified path of region directory
3295    */
3296   public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
3297     return new Path(
3298       HTableDescriptor.getTableDir(rootdir, info.getTableName()),
3299                                    info.getEncodedName());
3300   }
3301 
3302   /**
3303    * Determines if the specified row is within the row range specified by the
3304    * specified HRegionInfo
3305    *
3306    * @param info HRegionInfo that specifies the row range
3307    * @param row row to be checked
3308    * @return true if the row is within the range specified by the HRegionInfo
3309    */
3310   public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
3311     return ((info.getStartKey().length == 0) ||
3312         (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
3313         ((info.getEndKey().length == 0) ||
3314             (Bytes.compareTo(info.getEndKey(), row) > 0));
3315   }
3316 
3317   /**
3318    * Make the directories for a specific column family
3319    *
3320    * @param fs the file system
3321    * @param tabledir base directory where region will live (usually the table dir)
3322    * @param hri
3323    * @param colFamily the column family
3324    * @throws IOException
3325    */
3326   public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
3327     final HRegionInfo hri, byte [] colFamily)
3328   throws IOException {
3329     Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
3330     if (!fs.mkdirs(dir)) {
3331       LOG.warn("Failed to create " + dir);
3332     }
3333   }
3334 
3335   /**
3336    * Merge two HRegions.  The regions must be adjacent and must not overlap.
3337    *
3338    * @param srcA
3339    * @param srcB
3340    * @return new merged HRegion
3341    * @throws IOException
3342    */
3343   public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
3344   throws IOException {
3345     HRegion a = srcA;
3346     HRegion b = srcB;
3347 
3348     // Make sure that srcA comes first; important for key-ordering during
3349     // write of the merged file.
3350     if (srcA.getStartKey() == null) {
3351       if (srcB.getStartKey() == null) {
3352         throw new IOException("Cannot merge two regions with null start key");
3353       }
3354       // A's start key is null but B's isn't. Assume A comes before B
3355     } else if ((srcB.getStartKey() == null) ||
3356       (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
3357       a = srcB;
3358       b = srcA;
3359     }
3360 
3361     if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) {
3362       throw new IOException("Cannot merge non-adjacent regions");
3363     }
3364     return merge(a, b);
3365   }
3366 
3367   /**
3368    * Merge two regions whether they are adjacent or not.
3369    *
3370    * @param a region a
3371    * @param b region b
3372    * @return new merged region
3373    * @throws IOException
3374    */
3375   public static HRegion merge(HRegion a, HRegion b)
3376   throws IOException {
3377     if (!a.getRegionInfo().getTableNameAsString().equals(
3378         b.getRegionInfo().getTableNameAsString())) {
3379       throw new IOException("Regions do not belong to the same table");
3380     }
3381 
3382     FileSystem fs = a.getFilesystem();
3383 
3384     // Make sure each region's cache is empty
3385 
3386     a.flushcache();
3387     b.flushcache();
3388 
3389     // Compact each region so we only have one store file per family
3390 
3391     a.compactStores(true);
3392     if (LOG.isDebugEnabled()) {
3393       LOG.debug("Files for region: " + a);
3394       listPaths(fs, a.getRegionDir());
3395     }
3396     b.compactStores(true);
3397     if (LOG.isDebugEnabled()) {
3398       LOG.debug("Files for region: " + b);
3399       listPaths(fs, b.getRegionDir());
3400     }
3401 
3402     Configuration conf = a.getConf();
3403     HTableDescriptor tabledesc = a.getTableDesc();
3404     HLog log = a.getLog();
3405     Path tableDir = a.getTableDir();
3406     // Presume both are of same region type -- i.e. both user or catalog
3407     // table regions.  This way can use comparator.
3408     final byte[] startKey =
3409       (a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
3410            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
3411        || b.comparator.matchingRows(b.getStartKey(), 0,
3412               b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
3413               HConstants.EMPTY_BYTE_ARRAY.length))
3414       ? HConstants.EMPTY_BYTE_ARRAY
3415       : (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
3416              b.getStartKey(), 0, b.getStartKey().length) <= 0
3417          ? a.getStartKey()
3418          : b.getStartKey());
3419     final byte[] endKey =
3420       (a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
3421            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
3422        || a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
3423               HConstants.EMPTY_BYTE_ARRAY, 0,
3424               HConstants.EMPTY_BYTE_ARRAY.length))
3425       ? HConstants.EMPTY_BYTE_ARRAY
3426       : (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
3427              b.getEndKey(), 0, b.getEndKey().length) <= 0
3428          ? b.getEndKey()
3429          : a.getEndKey());
3430 
3431     HRegionInfo newRegionInfo =
3432         new HRegionInfo(tabledesc.getName(), startKey, endKey);
3433     LOG.info("Creating new region " + newRegionInfo.toString());
3434     String encodedName = newRegionInfo.getEncodedName();
3435     Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
3436     if(fs.exists(newRegionDir)) {
3437       throw new IOException("Cannot merge; target file collision at " +
3438           newRegionDir);
3439     }
3440     fs.mkdirs(newRegionDir);
3441 
3442     LOG.info("starting merge of regions: " + a + " and " + b +
3443       " into new region " + newRegionInfo.toString() +
3444         " with start key <" + Bytes.toStringBinary(startKey) + "> and end key <" +
3445         Bytes.toStringBinary(endKey) + ">");
3446 
3447     // Move HStoreFiles under new region directory
3448     Map<byte [], List<StoreFile>> byFamily =
3449       new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
3450     byFamily = filesByFamily(byFamily, a.close());
3451     byFamily = filesByFamily(byFamily, b.close());
3452     for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
3453       byte [] colFamily = es.getKey();
3454       makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily);
3455       // Because we compacted the source regions we should have no more than two
3456       // HStoreFiles per family and there will be no reference store
3457       List<StoreFile> srcFiles = es.getValue();
3458       if (srcFiles.size() == 2) {
3459         long seqA = srcFiles.get(0).getMaxSequenceId();
3460         long seqB = srcFiles.get(1).getMaxSequenceId();
3461         if (seqA == seqB) {
3462           // Can't have same sequenceid since on open of a store, this is what
3463           // distingushes the files (see the map of stores how its keyed by
3464           // sequenceid).
3465           throw new IOException("Files have same sequenceid: " + seqA);
3466         }
3467       }
3468       for (StoreFile hsf: srcFiles) {
3469         StoreFile.rename(fs, hsf.getPath(),
3470           StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir,
3471             newRegionInfo.getEncodedName(), colFamily)));
3472       }
3473     }
3474     if (LOG.isDebugEnabled()) {
3475       LOG.debug("Files for new region");
3476       listPaths(fs, newRegionDir);
3477     }
3478     HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf,
3479         newRegionInfo, a.getTableDesc(), null);
3480     dstRegion.readRequestsCount.set(a.readRequestsCount.get() + b.readRequestsCount.get());
3481     dstRegion.writeRequestsCount.set(a.writeRequestsCount.get() + b.writeRequestsCount.get());
3482     dstRegion.initialize();
3483     dstRegion.compactStores();
3484     if (LOG.isDebugEnabled()) {
3485       LOG.debug("Files for new region");
3486       listPaths(fs, dstRegion.getRegionDir());
3487     }
3488     deleteRegion(fs, a.getRegionDir());
3489     deleteRegion(fs, b.getRegionDir());
3490 
3491     LOG.info("merge completed. New region is " + dstRegion);
3492 
3493     return dstRegion;
3494   }
3495 
3496   /*
3497    * Fills a map with a vector of store files keyed by column family.
3498    * @param byFamily Map to fill.
3499    * @param storeFiles Store files to process.
3500    * @param family
3501    * @return Returns <code>byFamily</code>
3502    */
3503   private static Map<byte [], List<StoreFile>> filesByFamily(
3504       Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
3505     for (StoreFile src: storeFiles) {
3506       byte [] family = src.getFamily();
3507       List<StoreFile> v = byFamily.get(family);
3508       if (v == null) {
3509         v = new ArrayList<StoreFile>();
3510         byFamily.put(family, v);
3511       }
3512       v.add(src);
3513     }
3514     return byFamily;
3515   }
3516 
3517   /**
3518    * @return True if needs a mojor compaction.
3519    * @throws IOException
3520    */
3521   boolean isMajorCompaction() throws IOException {
3522     for (Store store: this.stores.values()) {
3523       if (store.isMajorCompaction()) {
3524         return true;
3525       }
3526     }
3527     return false;
3528   }
3529 
3530   /*
3531    * List the files under the specified directory
3532    *
3533    * @param fs
3534    * @param dir
3535    * @throws IOException
3536    */
3537   private static void listPaths(FileSystem fs, Path dir) throws IOException {
3538     if (LOG.isDebugEnabled()) {
3539       FileStatus[] stats = FSUtils.listStatus(fs, dir, null);
3540       if (stats == null || stats.length == 0) {
3541         return;
3542       }
3543       for (int i = 0; i < stats.length; i++) {
3544         String path = stats[i].getPath().toString();
3545         if (stats[i].isDir()) {
3546           LOG.debug("d " + path);
3547           listPaths(fs, stats[i].getPath());
3548         } else {
3549           LOG.debug("f " + path + " size=" + stats[i].getLen());
3550         }
3551       }
3552     }
3553   }
3554 
3555 
3556   //
3557   // HBASE-880
3558   //
3559   /**
3560    * @param get get object
3561    * @param lockid existing lock id, or null for no previous lock
3562    * @return result
3563    * @throws IOException read exceptions
3564    */
3565   public Result get(final Get get, final Integer lockid) throws IOException {
3566     checkRow(get.getRow(), "Get");
3567     // Verify families are all valid
3568     if (get.hasFamilies()) {
3569       for (byte [] family: get.familySet()) {
3570         checkFamily(family);
3571       }
3572     } else { // Adding all families to scanner
3573       for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
3574         get.addFamily(family);
3575       }
3576     }
3577     List<KeyValue> results = get(get, true);
3578     return new Result(results);
3579   }
3580 
3581   /**
3582    * An optimized version of {@link #get(Get)} that checks MemStore first for
3583    * the specified query.
3584    * <p>
3585    * This is intended for use by increment operations where we have the
3586    * guarantee that versions are never inserted out-of-order so if a value
3587    * exists in MemStore it is the latest value.
3588    * <p>
3589    * It only makes sense to use this method without a TimeRange and maxVersions
3590    * equal to 1.
3591    * @param get
3592    * @return result
3593    * @throws IOException
3594    */
3595   private List<KeyValue> getLastIncrement(final Get get) throws IOException {
3596     InternalScan iscan = new InternalScan(get);
3597 
3598     List<KeyValue> results = new ArrayList<KeyValue>();
3599 
3600     // memstore scan
3601     iscan.checkOnlyMemStore();
3602     RegionScanner scanner = null;
3603     try {
3604       scanner = getScanner(iscan);
3605       scanner.next(results);
3606     } finally {
3607       if (scanner != null)
3608         scanner.close();
3609     }
3610 
3611     // count how many columns we're looking for
3612     int expected = 0;
3613     Map<byte[], NavigableSet<byte[]>> familyMap = get.getFamilyMap();
3614     for (NavigableSet<byte[]> qfs : familyMap.values()) {
3615       expected += qfs.size();
3616     }
3617 
3618     // found everything we were looking for, done
3619     if (results.size() == expected) {
3620       return results;
3621     }
3622 
3623     // still have more columns to find
3624     if (results != null && !results.isEmpty()) {
3625       // subtract what was found in memstore
3626       for (KeyValue kv : results) {
3627         byte [] family = kv.getFamily();
3628         NavigableSet<byte[]> qfs = familyMap.get(family);
3629         qfs.remove(kv.getQualifier());
3630         if (qfs.isEmpty()) familyMap.remove(family);
3631         expected--;
3632       }
3633       // make a new get for just what is left
3634       Get newGet = new Get(get.getRow());
3635       for (Map.Entry<byte[], NavigableSet<byte[]>> f : familyMap.entrySet()) {
3636         byte [] family = f.getKey();
3637         for (byte [] qualifier : f.getValue()) {
3638           newGet.addColumn(family, qualifier);
3639         }
3640       }
3641       newGet.setTimeRange(get.getTimeRange().getMin(),
3642           get.getTimeRange().getMax());
3643       iscan = new InternalScan(newGet);
3644     }
3645 
3646     // check store files for what is left
3647     List<KeyValue> fileResults = new ArrayList<KeyValue>();
3648     iscan.checkOnlyStoreFiles();
3649     scanner = null;
3650     try {
3651       scanner = getScanner(iscan);
3652       scanner.next(fileResults);
3653     } finally {
3654       if (scanner != null)
3655         scanner.close();
3656     }
3657 
3658     // combine and return
3659     results.addAll(fileResults);
3660     Collections.sort(results, KeyValue.COMPARATOR);
3661     return results;
3662   }
3663 
3664   /*
3665    * Do a get based on the get parameter.
3666    * @param withCoprocessor invoke coprocessor or not. We don't want to
3667    * always invoke cp for this private method.
3668    */
3669   private List<KeyValue> get(Get get, boolean withCoprocessor)
3670   throws IOException {
3671     Scan scan = new Scan(get);
3672 
3673     List<KeyValue> results = new ArrayList<KeyValue>();
3674 
3675     // pre-get CP hook
3676     if (withCoprocessor && (coprocessorHost != null)) {
3677        if (coprocessorHost.preGet(get, results)) {
3678          return results;
3679        }
3680     }
3681 
3682     RegionScanner scanner = null;
3683     try {
3684       scanner = getScanner(scan);
3685       scanner.next(results);
3686     } finally {
3687       if (scanner != null)
3688         scanner.close();
3689     }
3690 
3691     // post-get CP hook
3692     if (withCoprocessor && (coprocessorHost != null)) {
3693       coprocessorHost.postGet(get, results);
3694     }
3695 
3696     return results;
3697   }
3698 
3699   /**
3700    *
3701    * Perform one or more increment operations on a row.
3702    * <p>
3703    * Increments performed are done under row lock but reads do not take locks
3704    * out so this can be seen partially complete by gets and scans.
3705    * @param increment
3706    * @param lockid
3707    * @param writeToWAL
3708    * @return new keyvalues after increment
3709    * @throws IOException
3710    */
3711   public Result increment(Increment increment, Integer lockid,
3712       boolean writeToWAL)
3713   throws IOException {
3714     // TODO: Use MVCC to make this set of increments atomic to reads
3715     byte [] row = increment.getRow();
3716     checkRow(row, "increment");
3717     TimeRange tr = increment.getTimeRange();
3718     boolean flush = false;
3719     WALEdit walEdits = null;
3720     List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
3721     List<KeyValue> kvs = new ArrayList<KeyValue>(increment.numColumns());
3722     long now = EnvironmentEdgeManager.currentTimeMillis();
3723     long size = 0;
3724 
3725     // Lock row
3726     startRegionOperation();
3727     this.writeRequestsCount.increment();
3728     try {
3729       Integer lid = getLock(lockid, row, true);
3730       this.updatesLock.readLock().lock();
3731       try {
3732         // Process each family
3733         for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
3734           increment.getFamilyMap().entrySet()) {
3735 
3736           Store store = stores.get(family.getKey());
3737 
3738           // Get previous values for all columns in this family
3739           Get get = new Get(row);
3740           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
3741             get.addColumn(family.getKey(), column.getKey());
3742           }
3743           get.setTimeRange(tr.getMin(), tr.getMax());
3744           List<KeyValue> results = getLastIncrement(get);
3745 
3746           // Iterate the input columns and update existing values if they were
3747           // found, otherwise add new column initialized to the increment amount
3748           int idx = 0;
3749           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
3750             long amount = column.getValue();
3751             if (idx < results.size() &&
3752                 results.get(idx).matchingQualifier(column.getKey())) {
3753               amount += Bytes.toLong(results.get(idx).getValue());
3754               idx++;
3755             }
3756 
3757             // Append new incremented KeyValue to list
3758             KeyValue newKV = new KeyValue(row, family.getKey(), column.getKey(),
3759                 now, Bytes.toBytes(amount));
3760             kvs.add(newKV);
3761 
3762             // Append update to WAL
3763             if (writeToWAL) {
3764               if (walEdits == null) {
3765                 walEdits = new WALEdit();
3766               }
3767               walEdits.add(newKV);
3768             }
3769           }
3770 
3771           // Write the KVs for this family into the store
3772           size += store.upsert(kvs);
3773           allKVs.addAll(kvs);
3774           kvs.clear();
3775         }
3776 
3777         // Actually write to WAL now
3778         if (writeToWAL) {
3779           // Using default cluster id, as this can only happen in the orginating
3780           // cluster. A slave cluster receives the final value (not the delta)
3781           // as a Put.
3782           this.log.append(regionInfo, this.htableDescriptor.getName(),
3783               walEdits, HConstants.DEFAULT_CLUSTER_ID, now,
3784               this.htableDescriptor);
3785         }
3786 
3787         size = this.addAndGetGlobalMemstoreSize(size);
3788         flush = isFlushSize(size);
3789       } finally {
3790         this.updatesLock.readLock().unlock();
3791         releaseRowLock(lid);
3792       }
3793     } finally {
3794       closeRegionOperation();
3795     }
3796 
3797     if (flush) {
3798       // Request a cache flush.  Do it outside update lock.
3799       requestFlush();
3800     }
3801 
3802     return new Result(allKVs);
3803   }
3804 
3805   /**
3806    * @param row
3807    * @param family
3808    * @param qualifier
3809    * @param amount
3810    * @param writeToWAL
3811    * @return The new value.
3812    * @throws IOException
3813    */
3814   public long incrementColumnValue(byte [] row, byte [] family,
3815       byte [] qualifier, long amount, boolean writeToWAL)
3816   throws IOException {
3817     checkRow(row, "increment");
3818     boolean flush = false;
3819     boolean wrongLength = false;
3820     // Lock row
3821     long result = amount;
3822     startRegionOperation();
3823     this.writeRequestsCount.increment();
3824     try {
3825       Integer lid = obtainRowLock(row);
3826       this.updatesLock.readLock().lock();
3827       try {
3828         Store store = stores.get(family);
3829 
3830         // Get the old value:
3831         Get get = new Get(row);
3832         get.addColumn(family, qualifier);
3833 
3834         // we don't want to invoke coprocessor in this case; ICV is wrapped
3835         // in HRegionServer, so we leave getLastIncrement alone
3836         List<KeyValue> results = getLastIncrement(get);
3837 
3838         if (!results.isEmpty()) {
3839           KeyValue kv = results.get(0);
3840           if(kv.getValueLength() == 8){
3841             byte [] buffer = kv.getBuffer();
3842             int valueOffset = kv.getValueOffset();
3843             result += Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG);
3844           }
3845           else{
3846             wrongLength = true;
3847           }
3848         }
3849         if(!wrongLength){
3850           // build the KeyValue now:
3851           KeyValue newKv = new KeyValue(row, family,
3852             qualifier, EnvironmentEdgeManager.currentTimeMillis(),
3853             Bytes.toBytes(result));
3854 
3855           // now log it:
3856           if (writeToWAL) {
3857             long now = EnvironmentEdgeManager.currentTimeMillis();
3858             WALEdit walEdit = new WALEdit();
3859             walEdit.add(newKv);
3860             // Using default cluster id, as this can only happen in the
3861             // orginating cluster. A slave cluster receives the final value (not
3862             // the delta) as a Put.
3863             this.log.append(regionInfo, this.htableDescriptor.getName(),
3864                 walEdit, HConstants.DEFAULT_CLUSTER_ID, now,
3865                 this.htableDescriptor);
3866           }
3867 
3868           // Now request the ICV to the store, this will set the timestamp
3869           // appropriately depending on if there is a value in memcache or not.
3870           // returns the change in the size of the memstore from operation
3871           long size = store.updateColumnValue(row, family, qualifier, result);
3872 
3873           size = this.addAndGetGlobalMemstoreSize(size);
3874           flush = isFlushSize(size);
3875         }
3876       } finally {
3877         this.updatesLock.readLock().unlock();
3878         releaseRowLock(lid);
3879       }
3880     } finally {
3881       closeRegionOperation();
3882     }
3883 
3884     if (flush) {
3885       // Request a cache flush.  Do it outside update lock.
3886       requestFlush();
3887     }
3888     if(wrongLength){
3889     	throw new IOException("Attempted to increment field that isn't 64 bits wide");
3890     }
3891     return result;
3892   }
3893 
3894 
3895   //
3896   // New HBASE-880 Helpers
3897   //
3898 
3899   private void checkFamily(final byte [] family)
3900   throws NoSuchColumnFamilyException {
3901     if (!this.htableDescriptor.hasFamily(family)) {
3902       throw new NoSuchColumnFamilyException("Column family " +
3903           Bytes.toString(family) + " does not exist in region " + this
3904           + " in table " + this.htableDescriptor);
3905   	}
3906   }
3907 
3908   public static final long FIXED_OVERHEAD = ClassSize.align(
3909       ClassSize.OBJECT +
3910       ClassSize.ARRAY +
3911       29 * ClassSize.REFERENCE + Bytes.SIZEOF_INT +
3912       (4 * Bytes.SIZEOF_LONG) +
3913       Bytes.SIZEOF_BOOLEAN);
3914 
3915   public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
3916       ClassSize.OBJECT + // closeLock
3917       (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
3918       ClassSize.ATOMIC_LONG + // memStoreSize
3919       ClassSize.ATOMIC_INTEGER + // lockIdGenerator
3920       (3 * ClassSize.CONCURRENT_HASHMAP) +  // lockedRows, lockIds, scannerReadPoints
3921       WriteState.HEAP_SIZE + // writestate
3922       ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
3923       (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
3924       ClassSize.ARRAYLIST + // recentFlushes
3925       MultiVersionConsistencyControl.FIXED_SIZE // mvcc
3926       ;
3927 
3928   @Override
3929   public long heapSize() {
3930     long heapSize = DEEP_OVERHEAD;
3931     for(Store store : this.stores.values()) {
3932       heapSize += store.heapSize();
3933     }
3934     // this does not take into account row locks, recent flushes, mvcc entries
3935     return heapSize;
3936   }
3937 
3938   /*
3939    * This method calls System.exit.
3940    * @param message Message to print out.  May be null.
3941    */
3942   private static void printUsageAndExit(final String message) {
3943     if (message != null && message.length() > 0) System.out.println(message);
3944     System.out.println("Usage: HRegion CATLALOG_TABLE_DIR [major_compact]");
3945     System.out.println("Options:");
3946     System.out.println(" major_compact  Pass this option to major compact " +
3947       "passed region.");
3948     System.out.println("Default outputs scan of passed region.");
3949     System.exit(1);
3950   }
3951 
3952   /**
3953    * Registers a new CoprocessorProtocol subclass and instance to
3954    * be available for handling {@link HRegion#exec(Exec)} calls.
3955    *
3956    * <p>
3957    * Only a single protocol type/handler combination may be registered per
3958    * region.
3959    * After the first registration, subsequent calls with the same protocol type
3960    * will fail with a return value of {@code false}.
3961    * </p>
3962    * @param protocol a {@code CoprocessorProtocol} subinterface defining the
3963    * protocol methods
3964    * @param handler an instance implementing the interface
3965    * @param <T> the protocol type
3966    * @return {@code true} if the registration was successful, {@code false}
3967    * otherwise
3968    */
3969   public <T extends CoprocessorProtocol> boolean registerProtocol(
3970       Class<T> protocol, T handler) {
3971 
3972     /* No stacking of protocol handlers is currently allowed.  The
3973      * first to claim wins!
3974      */
3975     if (protocolHandlers.containsKey(protocol)) {
3976       LOG.error("Protocol "+protocol.getName()+
3977           " already registered, rejecting request from "+
3978           handler
3979       );
3980       return false;
3981     }
3982 
3983     protocolHandlers.putInstance(protocol, handler);
3984     if (LOG.isDebugEnabled()) {
3985       LOG.debug("Registered protocol handler: region="+
3986           Bytes.toStringBinary(getRegionName())+" protocol="+protocol.getName());
3987     }
3988     return true;
3989   }
3990 
3991   /**
3992    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
3993    * method using the registered protocol handlers.
3994    * {@link CoprocessorProtocol} implementations must be registered via the
3995    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
3996    * method before they are available.
3997    *
3998    * @param call an {@code Exec} instance identifying the protocol, method name,
3999    *     and parameters for the method invocation
4000    * @return an {@code ExecResult} instance containing the region name of the
4001    *     invocation and the return value
4002    * @throws IOException if no registered protocol handler is found or an error
4003    *     occurs during the invocation
4004    * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
4005    */
4006   public ExecResult exec(Exec call)
4007       throws IOException {
4008     Class<? extends CoprocessorProtocol> protocol = call.getProtocol();
4009     if (!protocolHandlers.containsKey(protocol)) {
4010       throw new HBaseRPC.UnknownProtocolException(protocol,
4011           "No matching handler for protocol "+protocol.getName()+
4012           " in region "+Bytes.toStringBinary(getRegionName()));
4013     }
4014 
4015     CoprocessorProtocol handler = protocolHandlers.getInstance(protocol);
4016     Object value;
4017 
4018     try {
4019       Method method = protocol.getMethod(
4020           call.getMethodName(), call.getParameterClasses());
4021       method.setAccessible(true);
4022 
4023       value = method.invoke(handler, call.getParameters());
4024     } catch (InvocationTargetException e) {
4025       Throwable target = e.getTargetException();
4026       if (target instanceof IOException) {
4027         throw (IOException)target;
4028       }
4029       IOException ioe = new IOException(target.toString());
4030       ioe.setStackTrace(target.getStackTrace());
4031       throw ioe;
4032     } catch (Throwable e) {
4033       if (!(e instanceof IOException)) {
4034         LOG.error("Unexpected throwable object ", e);
4035       }
4036       IOException ioe = new IOException(e.toString());
4037       ioe.setStackTrace(e.getStackTrace());
4038       throw ioe;
4039     }
4040 
4041     return new ExecResult(getRegionName(), value);
4042   }
4043 
4044   /*
4045    * Process table.
4046    * Do major compaction or list content.
4047    * @param fs
4048    * @param p
4049    * @param log
4050    * @param c
4051    * @param majorCompact
4052    * @throws IOException
4053    */
4054   private static void processTable(final FileSystem fs, final Path p,
4055       final HLog log, final Configuration c,
4056       final boolean majorCompact)
4057   throws IOException {
4058     HRegion region = null;
4059     String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
4060     String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
4061     // Currently expects tables have one region only.
4062     if (p.getName().startsWith(rootStr)) {
4063       region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
4064         HTableDescriptor.ROOT_TABLEDESC, null);
4065     } else if (p.getName().startsWith(metaStr)) {
4066       region = HRegion.newHRegion(p, log, fs, c,
4067         HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
4068     } else {
4069       throw new IOException("Not a known catalog table: " + p.toString());
4070     }
4071     try {
4072       region.initialize();
4073       if (majorCompact) {
4074         region.compactStores(true);
4075       } else {
4076         // Default behavior
4077         Scan scan = new Scan();
4078         // scan.addFamily(HConstants.CATALOG_FAMILY);
4079         RegionScanner scanner = region.getScanner(scan);
4080         try {
4081           List<KeyValue> kvs = new ArrayList<KeyValue>();
4082           boolean done = false;
4083           do {
4084             kvs.clear();
4085             done = scanner.next(kvs);
4086             if (kvs.size() > 0) LOG.info(kvs);
4087           } while (done);
4088         } finally {
4089           scanner.close();
4090         }
4091       }
4092     } finally {
4093       region.close();
4094     }
4095   }
4096 
4097   boolean shouldForceSplit() {
4098     return this.splitRequest;
4099   }
4100 
4101   byte[] getExplicitSplitPoint() {
4102     return this.explicitSplitPoint;
4103   }
4104 
4105   void forceSplit(byte[] sp) {
4106     // NOTE : this HRegion will go away after the forced split is successfull
4107     //        therefore, no reason to clear this value
4108     this.splitRequest = true;
4109     if (sp != null) {
4110       this.explicitSplitPoint = sp;
4111     }
4112   }
4113 
4114   void clearSplit_TESTS_ONLY() {
4115     this.splitRequest = false;
4116   }
4117 
4118   /**
4119    * Give the region a chance to prepare before it is split.
4120    */
4121   protected void prepareToSplit() {
4122     // nothing
4123   }
4124 
4125   /**
4126    * Return the splitpoint. null indicates the region isn't splittable
4127    * If the splitpoint isn't explicitly specified, it will go over the stores
4128    * to find the best splitpoint. Currently the criteria of best splitpoint
4129    * is based on the size of the store.
4130    */
4131   public byte[] checkSplit() {
4132     // Can't split META
4133     if (getRegionInfo().isMetaRegion()) {
4134       if (shouldForceSplit()) {
4135         LOG.warn("Cannot split meta regions in HBase 0.20 and above");
4136       }
4137       return null;
4138     }
4139 
4140     if (this.explicitSplitPoint != null) {
4141       return this.explicitSplitPoint;
4142     }
4143 
4144     if (!splitPolicy.shouldSplit()) {
4145       return null;
4146     }
4147 
4148     byte[] ret = splitPolicy.getSplitPoint();
4149 
4150     if (ret != null) {
4151       try {
4152         checkRow(ret, "calculated split");
4153       } catch (IOException e) {
4154         LOG.error("Ignoring invalid split", e);
4155         return null;
4156       }
4157     }
4158     return ret;
4159   }
4160 
4161   /**
4162    * @return The priority that this region should have in the compaction queue
4163    */
4164   public int getCompactPriority() {
4165     int count = Integer.MAX_VALUE;
4166     for(Store store : stores.values()) {
4167       count = Math.min(count, store.getCompactPriority());
4168     }
4169     return count;
4170   }
4171 
4172   /**
4173    * Checks every store to see if one has too many
4174    * store files
4175    * @return true if any store has too many store files
4176    */
4177   public boolean needsCompaction() {
4178     for(Store store : stores.values()) {
4179       if(store.needsCompaction()) {
4180         return true;
4181       }
4182     }
4183     return false;
4184   }
4185 
4186   /** @return the coprocessor host */
4187   public RegionCoprocessorHost getCoprocessorHost() {
4188     return coprocessorHost;
4189   }
4190 
4191   /** @param coprocessorHost the new coprocessor host */
4192   public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
4193     this.coprocessorHost = coprocessorHost;
4194   }
4195 
4196   /**
4197    * This method needs to be called before any public call that reads or
4198    * modifies data. It has to be called just before a try.
4199    * #closeRegionOperation needs to be called in the try's finally block
4200    * Acquires a read lock and checks if the region is closing or closed.
4201    * @throws NotServingRegionException when the region is closing or closed
4202    */
4203   private void startRegionOperation() throws NotServingRegionException {
4204     if (this.closing.get()) {
4205       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4206           " is closing");
4207     }
4208     lock.readLock().lock();
4209     if (this.closed.get()) {
4210       lock.readLock().unlock();
4211       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4212           " is closed");
4213     }
4214   }
4215 
4216   /**
4217    * Closes the lock. This needs to be called in the finally block corresponding
4218    * to the try block of #startRegionOperation
4219    */
4220   private void closeRegionOperation(){
4221     lock.readLock().unlock();
4222   }
4223 
4224   /**
4225    * This method needs to be called before any public call that reads or
4226    * modifies stores in bulk. It has to be called just before a try.
4227    * #closeBulkRegionOperation needs to be called in the try's finally block
4228    * Acquires a writelock and checks if the region is closing or closed.
4229    * @throws NotServingRegionException when the region is closing or closed
4230    */
4231   private void startBulkRegionOperation(boolean writeLockNeeded)
4232   throws NotServingRegionException {
4233     if (this.closing.get()) {
4234       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4235           " is closing");
4236     }
4237     if (writeLockNeeded) lock.writeLock().lock();
4238     else lock.readLock().lock();
4239     if (this.closed.get()) {
4240       if (writeLockNeeded) lock.writeLock().unlock();
4241       else lock.readLock().unlock();
4242       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4243           " is closed");
4244     }
4245   }
4246 
4247   /**
4248    * Closes the lock. This needs to be called in the finally block corresponding
4249    * to the try block of #startRegionOperation
4250    */
4251   private void closeBulkRegionOperation(){
4252     if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
4253     else lock.readLock().unlock();
4254   }
4255 
4256   /**
4257    * A mocked list implementaion - discards all updates.
4258    */
4259   private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
4260 
4261     @Override
4262     public void add(int index, KeyValue element) {
4263       // do nothing
4264     }
4265 
4266     @Override
4267     public boolean addAll(int index, Collection<? extends KeyValue> c) {
4268       return false; // this list is never changed as a result of an update
4269     }
4270 
4271     @Override
4272     public KeyValue get(int index) {
4273       throw new UnsupportedOperationException();
4274     }
4275 
4276     @Override
4277     public int size() {
4278       return 0;
4279     }
4280   };
4281 
4282 
4283   /**
4284    * Facility for dumping and compacting catalog tables.
4285    * Only does catalog tables since these are only tables we for sure know
4286    * schema on.  For usage run:
4287    * <pre>
4288    *   ./bin/hbase org.apache.hadoop.hbase.regionserver.HRegion
4289    * </pre>
4290    * @param args
4291    * @throws IOException
4292    */
4293   public static void main(String[] args) throws IOException {
4294     if (args.length < 1) {
4295       printUsageAndExit(null);
4296     }
4297     boolean majorCompact = false;
4298     if (args.length > 1) {
4299       if (!args[1].toLowerCase().startsWith("major")) {
4300         printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
4301       }
4302       majorCompact = true;
4303     }
4304     final Path tableDir = new Path(args[0]);
4305     final Configuration c = HBaseConfiguration.create();
4306     final FileSystem fs = FileSystem.get(c);
4307     final Path logdir = new Path(c.get("hbase.tmp.dir"),
4308         "hlog" + tableDir.getName()
4309         + EnvironmentEdgeManager.currentTimeMillis());
4310     final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
4311         HConstants.HREGION_OLDLOGDIR_NAME);
4312     final HLog log = new HLog(fs, logdir, oldLogDir, c);
4313     try {
4314       processTable(fs, tableDir, log, c, majorCompact);
4315      } finally {
4316        log.close();
4317        // TODO: is this still right?
4318        BlockCache bc = new CacheConfig(c).getBlockCache();
4319        if (bc != null) bc.shutdown();
4320      }
4321   }
4322 }