View Javadoc

1   /*
2    * Copyright 2010 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.text.ParseException;
28  import java.util.AbstractList;
29  import java.util.ArrayList;
30  import java.util.Arrays;
31  import java.util.Collection;
32  import java.util.Collections;
33  import java.util.HashMap;
34  import java.util.List;
35  import java.util.Map;
36  import java.util.NavigableMap;
37  import java.util.NavigableSet;
38  import java.util.Random;
39  import java.util.Set;
40  import java.util.TreeMap;
41  import java.util.TreeSet;
42  import java.util.concurrent.ConcurrentSkipListMap;
43  import java.util.concurrent.atomic.AtomicBoolean;
44  import java.util.concurrent.atomic.AtomicLong;
45  import java.util.concurrent.locks.ReentrantReadWriteLock;
46  
47  import org.apache.commons.logging.Log;
48  import org.apache.commons.logging.LogFactory;
49  import org.apache.hadoop.conf.Configuration;
50  import org.apache.hadoop.fs.FSDataOutputStream;
51  import org.apache.hadoop.fs.FileStatus;
52  import org.apache.hadoop.fs.FileSystem;
53  import org.apache.hadoop.fs.Path;
54  import org.apache.hadoop.hbase.DoNotRetryIOException;
55  import org.apache.hadoop.hbase.DroppedSnapshotException;
56  import org.apache.hadoop.hbase.HBaseConfiguration;
57  import org.apache.hadoop.hbase.HColumnDescriptor;
58  import org.apache.hadoop.hbase.HConstants;
59  import org.apache.hadoop.hbase.HRegionInfo;
60  import org.apache.hadoop.hbase.HTableDescriptor;
61  import org.apache.hadoop.hbase.KeyValue;
62  import org.apache.hadoop.hbase.NotServingRegionException;
63  import org.apache.hadoop.hbase.UnknownScannerException;
64  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
65  import org.apache.hadoop.hbase.client.Delete;
66  import org.apache.hadoop.hbase.client.Get;
67  import org.apache.hadoop.hbase.client.Increment;
68  import org.apache.hadoop.hbase.client.Put;
69  import org.apache.hadoop.hbase.client.Result;
70  import org.apache.hadoop.hbase.client.Row;
71  import org.apache.hadoop.hbase.client.RowLock;
72  import org.apache.hadoop.hbase.client.Scan;
73  import org.apache.hadoop.hbase.filter.Filter;
74  import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
75  import org.apache.hadoop.hbase.io.HeapSize;
76  import org.apache.hadoop.hbase.io.TimeRange;
77  import org.apache.hadoop.hbase.io.hfile.BlockCache;
78  import org.apache.hadoop.hbase.regionserver.wal.HLog;
79  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
80  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
81  import org.apache.hadoop.hbase.util.Bytes;
82  import org.apache.hadoop.hbase.util.CancelableProgressable;
83  import org.apache.hadoop.hbase.util.ClassSize;
84  import org.apache.hadoop.hbase.util.CompressionTest;
85  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
86  import org.apache.hadoop.hbase.util.FSUtils;
87  import org.apache.hadoop.hbase.util.Pair;
88  import org.apache.hadoop.hbase.util.Writables;
89  import org.apache.hadoop.io.Writable;
90  import org.apache.hadoop.util.Progressable;
91  import org.apache.hadoop.util.StringUtils;
92  
93  import com.google.common.collect.Lists;
94  
95  /**
96   * HRegion stores data for a certain region of a table.  It stores all columns
97   * for each row. A given table consists of one or more HRegions.
98   *
99   * <p>We maintain multiple HStores for a single HRegion.
100  *
101  * <p>An Store is a set of rows with some column data; together,
102  * they make up all the data for the rows.
103  *
104  * <p>Each HRegion has a 'startKey' and 'endKey'.
105  * <p>The first is inclusive, the second is exclusive (except for
106  * the final region)  The endKey of region 0 is the same as
107  * startKey for region 1 (if it exists).  The startKey for the
108  * first region is null. The endKey for the final region is null.
109  *
110  * <p>Locking at the HRegion level serves only one purpose: preventing the
111  * region from being closed (and consequently split) while other operations
112  * are ongoing. Each row level operation obtains both a row lock and a region
113  * read lock for the duration of the operation. While a scanner is being
114  * constructed, getScanner holds a read lock. If the scanner is successfully
115  * constructed, it holds a read lock until it is closed. A close takes out a
116  * write lock and consequently will block for ongoing operations and will block
117  * new operations from starting while the close is in progress.
118  *
119  * <p>An HRegion is defined by its table and its key extent.
120  *
121  * <p>It consists of at least one Store.  The number of Stores should be
122  * configurable, so that data which is accessed together is stored in the same
123  * Store.  Right now, we approximate that by building a single Store for
124  * each column family.  (This config info will be communicated via the
125  * tabledesc.)
126  *
127  * <p>The HTableDescriptor contains metainfo about the HRegion's table.
128  * regionName is a unique identifier for this HRegion. (startKey, endKey]
129  * defines the keyspace for this HRegion.
130  */
131 public class HRegion implements HeapSize { // , Writable{
132   public static final Log LOG = LogFactory.getLog(HRegion.class);
133   static final String MERGEDIR = "merges";
134 
135   final AtomicBoolean closed = new AtomicBoolean(false);
136   /* Closing can take some time; use the closing flag if there is stuff we don't
137    * want to do while in closing state; e.g. like offer this region up to the
138    * master as a region to close if the carrying regionserver is overloaded.
139    * Once set, it is never cleared.
140    */
141   final AtomicBoolean closing = new AtomicBoolean(false);
142 
143   //////////////////////////////////////////////////////////////////////////////
144   // Members
145   //////////////////////////////////////////////////////////////////////////////
146 
147   private final Set<byte[]> lockedRows =
148     new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
149   private final Map<Integer, byte []> lockIds =
150     new HashMap<Integer, byte []>();
151   private int lockIdGenerator = 1;
152   static private Random rand = new Random();
153 
154   protected final Map<byte [], Store> stores =
155     new ConcurrentSkipListMap<byte [], Store>(Bytes.BYTES_RAWCOMPARATOR);
156 
157   //These variable are just used for getting data out of the region, to test on
158   //client side
159   // private int numStores = 0;
160   // private int [] storeSize = null;
161   // private byte [] name = null;
162 
163   final AtomicLong memstoreSize = new AtomicLong(0);
164 
165   /**
166    * The directory for the table this region is part of.
167    * This directory contains the directory for this region.
168    */
169   final Path tableDir;
170 
171   final HLog log;
172   final FileSystem fs;
173   final Configuration conf;
174   final HRegionInfo regionInfo;
175   final Path regiondir;
176   KeyValue.KVComparator comparator;
177 
178   /*
179    * Set this when scheduling compaction if want the next compaction to be a
180    * major compaction.  Cleared each time through compaction code.
181    */
182   private volatile boolean forceMajorCompaction = false;
183   private Pair<Long,Long> lastCompactInfo = null;
184 
185   // Used to ensure only one thread closes region at a time.
186   private final Object closeLock = new Object();
187 
188   /*
189    * Data structure of write state flags used coordinating flushes,
190    * compactions and closes.
191    */
192   static class WriteState {
193     // Set while a memstore flush is happening.
194     volatile boolean flushing = false;
195     // Set when a flush has been requested.
196     volatile boolean flushRequested = false;
197     // Set while a compaction is running.
198     volatile boolean compacting = false;
199     // Gets set in close. If set, cannot compact or flush again.
200     volatile boolean writesEnabled = true;
201     // Set if region is read-only
202     volatile boolean readOnly = false;
203 
204     /**
205      * Set flags that make this region read-only.
206      *
207      * @param onOff flip value for region r/o setting
208      */
209     synchronized void setReadOnly(final boolean onOff) {
210       this.writesEnabled = !onOff;
211       this.readOnly = onOff;
212     }
213 
214     boolean isReadOnly() {
215       return this.readOnly;
216     }
217 
218     boolean isFlushRequested() {
219       return this.flushRequested;
220     }
221   }
222 
223   final WriteState writestate = new WriteState();
224 
225   final long memstoreFlushSize;
226   private volatile long lastFlushTime;
227   private List<Pair<Long,Long>> recentFlushes = new ArrayList<Pair<Long,Long>>();
228   final FlushRequester flushRequester;
229   private final long blockingMemStoreSize;
230   final long threadWakeFrequency;
231   // Used to guard closes
232   final ReentrantReadWriteLock lock =
233     new ReentrantReadWriteLock();
234 
235   // Stop updates lock
236   private final ReentrantReadWriteLock updatesLock =
237     new ReentrantReadWriteLock();
238   private boolean splitRequest;
239   private byte[] splitPoint = null;
240 
241   private final ReadWriteConsistencyControl rwcc =
242       new ReadWriteConsistencyControl();
243 
244   /**
245    * Name of the region info file that resides just under the region directory.
246    */
247   public final static String REGIONINFO_FILE = ".regioninfo";
248 
249   /**
250    * Should only be used for testing purposes
251    */
252   public HRegion(){
253     this.tableDir = null;
254     this.blockingMemStoreSize = 0L;
255     this.conf = null;
256     this.flushRequester = null;
257     this.fs = null;
258     this.memstoreFlushSize = 0L;
259     this.log = null;
260     this.regiondir = null;
261     this.regionInfo = null;
262     this.threadWakeFrequency = 0L;
263   }
264 
265   /**
266    * HRegion constructor.  his constructor should only be used for testing and
267    * extensions.  Instances of HRegion should be instantiated with the
268    * {@link HRegion#newHRegion(Path, HLog, FileSystem, Configuration, org.apache.hadoop.hbase.HRegionInfo, FlushRequester)} method.
269    *
270    *
271    * @param tableDir qualified path of directory where region should be located,
272    * usually the table directory.
273    * @param log The HLog is the outbound log for any updates to the HRegion
274    * (There's a single HLog for all the HRegions on a single HRegionServer.)
275    * The log file is a logfile from the previous execution that's
276    * custom-computed for this HRegion. The HRegionServer computes and sorts the
277    * appropriate log info for this HRegion. If there is a previous log file
278    * (implying that the HRegion has been written-to before), then read it from
279    * the supplied path.
280    * @param fs is the filesystem.
281    * @param conf is global configuration settings.
282    * @param regionInfo - HRegionInfo that describes the region
283    * is new), then read them from the supplied path.
284    * @param flushRequester an object that implements {@link FlushRequester} or null
285    *
286    * @see HRegion#newHRegion(Path, HLog, FileSystem, Configuration, org.apache.hadoop.hbase.HRegionInfo, FlushRequester)
287    */
288   public HRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf,
289       HRegionInfo regionInfo, FlushRequester flushRequester) {
290     this.tableDir = tableDir;
291     this.comparator = regionInfo.getComparator();
292     this.log = log;
293     this.fs = fs;
294     this.conf = conf;
295     this.regionInfo = regionInfo;
296     this.flushRequester = flushRequester;
297     this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY,
298         10 * 1000);
299     String encodedNameStr = this.regionInfo.getEncodedName();
300     this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
301     long flushSize = regionInfo.getTableDesc().getMemStoreFlushSize();
302     if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
303       flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
304                       HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
305     }
306     this.memstoreFlushSize = flushSize;
307     this.blockingMemStoreSize = this.memstoreFlushSize *
308       conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
309     if (LOG.isDebugEnabled()) {
310       // Write out region name as string and its encoded name.
311       LOG.debug("Instantiated " + this);
312     }
313   }
314 
315   /**
316    * Initialize this region.
317    * @return What the next sequence (edit) id should be.
318    * @throws IOException e
319    */
320   public long initialize() throws IOException {
321     return initialize(null);
322   }
323 
324   /**
325    * Initialize this region.
326    *
327    * @param reporter Tickle every so often if initialize is taking a while.
328    * @return What the next sequence (edit) id should be.
329    * @throws IOException e
330    */
331   public long initialize(final CancelableProgressable reporter)
332   throws IOException {
333     // A region can be reopened if failed a split; reset flags
334     this.closing.set(false);
335     this.closed.set(false);
336 
337     // Write HRI to a file in case we need to recover .META.
338     checkRegioninfoOnFilesystem();
339 
340     // Remove temporary data left over from old regions
341     cleanupTmpDir();
342 
343     // Load in all the HStores.  Get maximum seqid.
344     long maxSeqId = -1;
345     for (HColumnDescriptor c : this.regionInfo.getTableDesc().getFamilies()) {
346       Store store = instantiateHStore(this.tableDir, c);
347       this.stores.put(c.getName(), store);
348       long storeSeqId = store.getMaxSequenceId();
349       if (storeSeqId > maxSeqId) {
350         maxSeqId = storeSeqId;
351       }
352     }
353     // Recover any edits if available.
354     maxSeqId = replayRecoveredEditsIfAny(this.regiondir, maxSeqId, reporter);
355 
356     // Get rid of any splits or merges that were lost in-progress.  Clean out
357     // these directories here on open.  We may be opening a region that was
358     // being split but we crashed in the middle of it all.
359     SplitTransaction.cleanupAnySplitDetritus(this);
360     FSUtils.deleteDirectory(this.fs, new Path(regiondir, MERGEDIR));
361 
362     this.writestate.setReadOnly(this.regionInfo.getTableDesc().isReadOnly());
363 
364     this.writestate.compacting = false;
365     this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
366     // Use maximum of log sequenceid or that which was found in stores
367     // (particularly if no recovered edits, seqid will be -1).
368     long nextSeqid = maxSeqId + 1;
369     LOG.info("Onlined " + this.toString() + "; next sequenceid=" + nextSeqid);
370     return nextSeqid;
371   }
372 
373   /*
374    * Move any passed HStore files into place (if any).  Used to pick up split
375    * files and any merges from splits and merges dirs.
376    * @param initialFiles
377    * @throws IOException
378    */
379   static void moveInitialFilesIntoPlace(final FileSystem fs,
380     final Path initialFiles, final Path regiondir)
381   throws IOException {
382     if (initialFiles != null && fs.exists(initialFiles)) {
383       if (!fs.rename(initialFiles, regiondir)) {
384         LOG.warn("Unable to rename " + initialFiles + " to " + regiondir);
385       }
386     }
387   }
388 
389   /**
390    * @return True if this region has references.
391    */
392   public boolean hasReferences() {
393     for (Store store : this.stores.values()) {
394       for (StoreFile sf : store.getStorefiles()) {
395         // Found a reference, return.
396         if (sf.isReference()) return true;
397       }
398     }
399     return false;
400   }
401 
402   /*
403    * Write out an info file under the region directory.  Useful recovering
404    * mangled regions.
405    * @throws IOException
406    */
407   private void checkRegioninfoOnFilesystem() throws IOException {
408     Path regioninfoPath = new Path(this.regiondir, REGIONINFO_FILE);
409     if (this.fs.exists(regioninfoPath) &&
410         this.fs.getFileStatus(regioninfoPath).getLen() > 0) {
411       return;
412     }
413     // Create in tmpdir and then move into place in case we crash after
414     // create but before close.  If we don't successfully close the file,
415     // subsequent region reopens will fail the below because create is
416     // registered in NN.
417     Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
418     FSDataOutputStream out = this.fs.create(tmpPath, true);
419     try {
420       this.regionInfo.write(out);
421       out.write('\n');
422       out.write('\n');
423       out.write(Bytes.toBytes(this.regionInfo.toString()));
424     } finally {
425       out.close();
426     }
427     if (!fs.rename(tmpPath, regioninfoPath)) {
428       throw new IOException("Unable to rename " + tmpPath + " to " +
429         regioninfoPath);
430     }
431   }
432 
433   /** @return a HRegionInfo object for this region */
434   public HRegionInfo getRegionInfo() {
435     return this.regionInfo;
436   }
437 
438   /** @return true if region is closed */
439   public boolean isClosed() {
440     return this.closed.get();
441   }
442 
443   /**
444    * @return True if closing process has started.
445    */
446   public boolean isClosing() {
447     return this.closing.get();
448   }
449 
450   boolean areWritesEnabled() {
451     synchronized(this.writestate) {
452       return this.writestate.writesEnabled;
453     }
454   }
455 
456    public ReadWriteConsistencyControl getRWCC() {
457      return rwcc;
458    }
459 
460   /**
461    * Close down this HRegion.  Flush the cache, shut down each HStore, don't
462    * service any more calls.
463    *
464    * <p>This method could take some time to execute, so don't call it from a
465    * time-sensitive thread.
466    *
467    * @return Vector of all the storage files that the HRegion's component
468    * HStores make use of.  It's a list of all HStoreFile objects. Returns empty
469    * vector if already closed and null if judged that it should not close.
470    *
471    * @throws IOException e
472    */
473   public List<StoreFile> close() throws IOException {
474     return close(false);
475   }
476 
477   /**
478    * Close down this HRegion.  Flush the cache unless abort parameter is true,
479    * Shut down each HStore, don't service any more calls.
480    *
481    * This method could take some time to execute, so don't call it from a
482    * time-sensitive thread.
483    *
484    * @param abort true if server is aborting (only during testing)
485    * @return Vector of all the storage files that the HRegion's component
486    * HStores make use of.  It's a list of HStoreFile objects.  Can be null if
487    * we are not to close at this time or we are already closed.
488    *
489    * @throws IOException e
490    */
491   public List<StoreFile> close(final boolean abort) throws IOException {
492     // Only allow one thread to close at a time. Serialize them so dual
493     // threads attempting to close will run up against each other.
494     synchronized (closeLock) {
495       return doClose(abort);
496     }
497   }
498 
499   private List<StoreFile> doClose(final boolean abort)
500   throws IOException {
501     if (isClosed()) {
502       LOG.warn("Region " + this + " already closed");
503       return null;
504     }
505     boolean wasFlushing = false;
506     synchronized (writestate) {
507       // Disable compacting and flushing by background threads for this
508       // region.
509       writestate.writesEnabled = false;
510       wasFlushing = writestate.flushing;
511       LOG.debug("Closing " + this + ": disabling compactions & flushes");
512       while (writestate.compacting || writestate.flushing) {
513         LOG.debug("waiting for" +
514           (writestate.compacting ? " compaction" : "") +
515           (writestate.flushing ?
516             (writestate.compacting ? "," : "") + " cache flush" :
517               "") + " to complete for region " + this);
518         try {
519           writestate.wait();
520         } catch (InterruptedException iex) {
521           // continue
522         }
523       }
524     }
525     // If we were not just flushing, is it worth doing a preflush...one
526     // that will clear out of the bulk of the memstore before we put up
527     // the close flag?
528     if (!abort && !wasFlushing && worthPreFlushing()) {
529       LOG.info("Running close preflush of " + this.getRegionNameAsString());
530       internalFlushcache();
531     }
532     this.closing.set(true);
533     lock.writeLock().lock();
534     try {
535       if (this.isClosed()) {
536         // SplitTransaction handles the null
537         return null;
538       }
539       LOG.debug("Updates disabled for region " + this);
540       // Don't flush the cache if we are aborting
541       if (!abort) {
542         internalFlushcache();
543       }
544 
545       List<StoreFile> result = new ArrayList<StoreFile>();
546       for (Store store : stores.values()) {
547         result.addAll(store.close());
548       }
549       this.closed.set(true);
550       LOG.info("Closed " + this);
551       return result;
552     } finally {
553       lock.writeLock().unlock();
554     }
555   }
556 
557    /**
558     * @return True if its worth doing a flush before we put up the close flag.
559     */
560   private boolean worthPreFlushing() {
561     return this.memstoreSize.get() >
562       this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
563   }
564 
565   //////////////////////////////////////////////////////////////////////////////
566   // HRegion accessors
567   //////////////////////////////////////////////////////////////////////////////
568 
569   /** @return start key for region */
570   public byte [] getStartKey() {
571     return this.regionInfo.getStartKey();
572   }
573 
574   /** @return end key for region */
575   public byte [] getEndKey() {
576     return this.regionInfo.getEndKey();
577   }
578 
579   /** @return region id */
580   public long getRegionId() {
581     return this.regionInfo.getRegionId();
582   }
583 
584   /** @return region name */
585   public byte [] getRegionName() {
586     return this.regionInfo.getRegionName();
587   }
588 
589   /** @return region name as string for logging */
590   public String getRegionNameAsString() {
591     return this.regionInfo.getRegionNameAsString();
592   }
593 
594   /** @return HTableDescriptor for this region */
595   public HTableDescriptor getTableDesc() {
596     return this.regionInfo.getTableDesc();
597   }
598 
599   /** @return HLog in use for this region */
600   public HLog getLog() {
601     return this.log;
602   }
603 
604   /** @return Configuration object */
605   public Configuration getConf() {
606     return this.conf;
607   }
608 
609   /** @return region directory Path */
610   public Path getRegionDir() {
611     return this.regiondir;
612   }
613 
614   /**
615    * Computes the Path of the HRegion
616    *
617    * @param tabledir qualified path for table
618    * @param name ENCODED region name
619    * @return Path of HRegion directory
620    */
621   public static Path getRegionDir(final Path tabledir, final String name) {
622     return new Path(tabledir, name);
623   }
624 
625   /** @return FileSystem being used by this region */
626   public FileSystem getFilesystem() {
627     return this.fs;
628   }
629 
630   /** @return info about the last compaction <time, size> */
631   public Pair<Long,Long> getLastCompactInfo() {
632     return this.lastCompactInfo;
633   }
634 
635   /** @return the last time the region was flushed */
636   public long getLastFlushTime() {
637     return this.lastFlushTime;
638   }
639 
640   /** @return info about the last flushes <time, size> */
641   public List<Pair<Long,Long>> getRecentFlushInfo() {
642     this.lock.readLock().lock();
643     List<Pair<Long,Long>> ret = this.recentFlushes;
644     this.recentFlushes = new ArrayList<Pair<Long,Long>>();
645     this.lock.readLock().unlock();
646     return ret;
647   }
648 
649   //////////////////////////////////////////////////////////////////////////////
650   // HRegion maintenance.
651   //
652   // These methods are meant to be called periodically by the HRegionServer for
653   // upkeep.
654   //////////////////////////////////////////////////////////////////////////////
655 
656   /** @return returns size of largest HStore. */
657   public long getLargestHStoreSize() {
658     long size = 0;
659     for (Store h: stores.values()) {
660       long storeSize = h.getSize();
661       if (storeSize > size) {
662         size = storeSize;
663       }
664     }
665     return size;
666   }
667 
668   /*
669    * Do preparation for pending compaction.
670    * @throws IOException
671    */
672   void doRegionCompactionPrep() throws IOException {
673   }
674 
675   /*
676    * Removes the temporary directory for this Store.
677    */
678   private void cleanupTmpDir() throws IOException {
679     FSUtils.deleteDirectory(this.fs, getTmpDir());
680   }
681 
682   /**
683    * Get the temporary diretory for this region. This directory
684    * will have its contents removed when the region is reopened.
685    */
686   Path getTmpDir() {
687     return new Path(getRegionDir(), ".tmp");
688   }
689 
690   void setForceMajorCompaction(final boolean b) {
691     this.forceMajorCompaction = b;
692   }
693 
694   boolean getForceMajorCompaction() {
695     return this.forceMajorCompaction;
696   }
697 
698   /**
699    * Called by compaction thread and after region is opened to compact the
700    * HStores if necessary.
701    *
702    * <p>This operation could block for a long time, so don't call it from a
703    * time-sensitive thread.
704    *
705    * Note that no locking is necessary at this level because compaction only
706    * conflicts with a region split, and that cannot happen because the region
707    * server does them sequentially and not in parallel.
708    *
709    * @return mid key if split is needed
710    * @throws IOException e
711    */
712   public byte [] compactStores() throws IOException {
713     boolean majorCompaction = this.forceMajorCompaction;
714     this.forceMajorCompaction = false;
715     return compactStores(majorCompaction);
716   }
717 
718   /*
719    * Called by compaction thread and after region is opened to compact the
720    * HStores if necessary.
721    *
722    * <p>This operation could block for a long time, so don't call it from a
723    * time-sensitive thread.
724    *
725    * Note that no locking is necessary at this level because compaction only
726    * conflicts with a region split, and that cannot happen because the region
727    * server does them sequentially and not in parallel.
728    *
729    * @param majorCompaction True to force a major compaction regardless of thresholds
730    * @return split row if split is needed
731    * @throws IOException e
732    */
733   byte [] compactStores(final boolean majorCompaction)
734   throws IOException {
735     if (this.closing.get()) {
736       LOG.debug("Skipping compaction on " + this + " because closing");
737       return null;
738     }
739     lock.readLock().lock();
740     this.lastCompactInfo = null;
741     try {
742       if (this.closed.get()) {
743         LOG.debug("Skipping compaction on " + this + " because closed");
744         return null;
745       }
746       byte [] splitRow = null;
747       if (this.closed.get()) {
748         return splitRow;
749       }
750       try {
751         synchronized (writestate) {
752           if (!writestate.compacting && writestate.writesEnabled) {
753             writestate.compacting = true;
754           } else {
755             LOG.info("NOT compacting region " + this +
756                 ": compacting=" + writestate.compacting + ", writesEnabled=" +
757                 writestate.writesEnabled);
758               return splitRow;
759           }
760         }
761         LOG.info("Starting" + (majorCompaction? " major " : " ") +
762             "compaction on region " + this);
763         long startTime = EnvironmentEdgeManager.currentTimeMillis();
764         doRegionCompactionPrep();
765         long lastCompactSize = 0;
766         long maxSize = -1;
767         boolean completed = false;
768         try {
769           for (Store store: stores.values()) {
770             final Store.StoreSize ss = store.compact(majorCompaction);
771             lastCompactSize += store.getLastCompactSize();
772             if (ss != null && ss.getSize() > maxSize) {
773               maxSize = ss.getSize();
774               splitRow = ss.getSplitRow();
775             }
776           }
777           completed = true;
778         } catch (InterruptedIOException iioe) {
779           LOG.info("compaction interrupted by user: ", iioe);
780         } finally {
781           long now = EnvironmentEdgeManager.currentTimeMillis();
782           LOG.info(((completed) ? "completed" : "aborted")
783               + " compaction on region " + this
784               + " after " + StringUtils.formatTimeDiff(now, startTime));
785           if (completed) {
786             this.lastCompactInfo =
787               new Pair<Long,Long>((now - startTime) / 1000, lastCompactSize);
788           }
789         }
790       } finally {
791         synchronized (writestate) {
792           writestate.compacting = false;
793           writestate.notifyAll();
794         }
795       }
796       if (splitRow != null) {
797         assert splitPoint == null || Bytes.equals(splitRow, splitPoint);
798         this.splitPoint = null; // clear the split point (if set)
799       }
800       return splitRow;
801     } finally {
802       lock.readLock().unlock();
803     }
804   }
805 
806   /**
807    * Flush the cache.
808    *
809    * When this method is called the cache will be flushed unless:
810    * <ol>
811    *   <li>the cache is empty</li>
812    *   <li>the region is closed.</li>
813    *   <li>a flush is already in progress</li>
814    *   <li>writes are disabled</li>
815    * </ol>
816    *
817    * <p>This method may block for some time, so it should not be called from a
818    * time-sensitive thread.
819    *
820    * @return true if cache was flushed
821    *
822    * @throws IOException general io exceptions
823    * @throws DroppedSnapshotException Thrown when replay of hlog is required
824    * because a Snapshot was not properly persisted.
825    */
826   public boolean flushcache() throws IOException {
827     // fail-fast instead of waiting on the lock
828     if (this.closing.get()) {
829       LOG.debug("Skipping flush on " + this + " because closing");
830       return false;
831     }
832     lock.readLock().lock();
833     try {
834       if (this.closed.get()) {
835         LOG.debug("Skipping flush on " + this + " because closed");
836         return false;
837       }
838       try {
839         synchronized (writestate) {
840           if (!writestate.flushing && writestate.writesEnabled) {
841             this.writestate.flushing = true;
842           } else {
843             if (LOG.isDebugEnabled()) {
844               LOG.debug("NOT flushing memstore for region " + this +
845                   ", flushing=" +
846                   writestate.flushing + ", writesEnabled=" +
847                   writestate.writesEnabled);
848             }
849             return false;
850           }
851         }
852         return internalFlushcache();
853       } finally {
854         synchronized (writestate) {
855           writestate.flushing = false;
856           this.writestate.flushRequested = false;
857           writestate.notifyAll();
858         }
859       }
860     } finally {
861       lock.readLock().unlock();
862     }
863   }
864 
865   /**
866    * Flush the memstore.
867    *
868    * Flushing the memstore is a little tricky. We have a lot of updates in the
869    * memstore, all of which have also been written to the log. We need to
870    * write those updates in the memstore out to disk, while being able to
871    * process reads/writes as much as possible during the flush operation. Also,
872    * the log has to state clearly the point in time at which the memstore was
873    * flushed. (That way, during recovery, we know when we can rely on the
874    * on-disk flushed structures and when we have to recover the memstore from
875    * the log.)
876    *
877    * <p>So, we have a three-step process:
878    *
879    * <ul><li>A. Flush the memstore to the on-disk stores, noting the current
880    * sequence ID for the log.<li>
881    *
882    * <li>B. Write a FLUSHCACHE-COMPLETE message to the log, using the sequence
883    * ID that was current at the time of memstore-flush.</li>
884    *
885    * <li>C. Get rid of the memstore structures that are now redundant, as
886    * they've been flushed to the on-disk HStores.</li>
887    * </ul>
888    * <p>This method is protected, but can be accessed via several public
889    * routes.
890    *
891    * <p> This method may block for some time.
892    *
893    * @return true if the region needs compacting
894    *
895    * @throws IOException general io exceptions
896    * @throws DroppedSnapshotException Thrown when replay of hlog is required
897    * because a Snapshot was not properly persisted.
898    */
899   protected boolean internalFlushcache() throws IOException {
900     return internalFlushcache(this.log, -1);
901   }
902 
903   /**
904    * @param wal Null if we're NOT to go via hlog/wal.
905    * @param myseqid The seqid to use if <code>wal</code> is null writing out
906    * flush file.
907    * @return true if the region needs compacting
908    * @throws IOException
909    * @see #internalFlushcache()
910    */
911   protected boolean internalFlushcache(final HLog wal, final long myseqid)
912   throws IOException {
913     final long startTime = EnvironmentEdgeManager.currentTimeMillis();
914     // Clear flush flag.
915     // Record latest flush time
916     this.lastFlushTime = startTime;
917     // If nothing to flush, return and avoid logging start/stop flush.
918     if (this.memstoreSize.get() <= 0) {
919       return false;
920     }
921     if (LOG.isDebugEnabled()) {
922       LOG.debug("Started memstore flush for " + this +
923         ", current region memstore size " +
924         StringUtils.humanReadableInt(this.memstoreSize.get()) +
925         ((wal != null)? "": "; wal is null, using passed sequenceid=" + myseqid));
926     }
927 
928     // Stop updates while we snapshot the memstore of all stores. We only have
929     // to do this for a moment.  Its quick.  The subsequent sequence id that
930     // goes into the HLog after we've flushed all these snapshots also goes
931     // into the info file that sits beside the flushed files.
932     // We also set the memstore size to zero here before we allow updates
933     // again so its value will represent the size of the updates received
934     // during the flush
935     long sequenceId = -1L;
936     long completeSequenceId = -1L;
937 
938     // We have to take a write lock during snapshot, or else a write could
939     // end up in both snapshot and memstore (makes it difficult to do atomic
940     // rows then)
941     this.updatesLock.writeLock().lock();
942     final long currentMemStoreSize = this.memstoreSize.get();
943     List<StoreFlusher> storeFlushers = new ArrayList<StoreFlusher>(stores.size());
944     try {
945       sequenceId = (wal == null)? myseqid: wal.startCacheFlush();
946       completeSequenceId = this.getCompleteCacheFlushSequenceId(sequenceId);
947 
948       for (Store s : stores.values()) {
949         storeFlushers.add(s.getStoreFlusher(completeSequenceId));
950       }
951 
952       // prepare flush (take a snapshot)
953       for (StoreFlusher flusher : storeFlushers) {
954         flusher.prepare();
955       }
956     } finally {
957       this.updatesLock.writeLock().unlock();
958     }
959 
960     LOG.debug("Finished snapshotting, commencing flushing stores");
961 
962     // Any failure from here on out will be catastrophic requiring server
963     // restart so hlog content can be replayed and put back into the memstore.
964     // Otherwise, the snapshot content while backed up in the hlog, it will not
965     // be part of the current running servers state.
966     boolean compactionRequested = false;
967     try {
968       // A.  Flush memstore to all the HStores.
969       // Keep running vector of all store files that includes both old and the
970       // just-made new flush store file.
971 
972       for (StoreFlusher flusher : storeFlushers) {
973         flusher.flushCache();
974       }
975       // Switch snapshot (in memstore) -> new hfile (thus causing
976       // all the store scanners to reset/reseek).
977       for (StoreFlusher flusher : storeFlushers) {
978         boolean needsCompaction = flusher.commit();
979         if (needsCompaction) {
980           compactionRequested = true;
981         }
982       }
983       storeFlushers.clear();
984 
985       // Set down the memstore size by amount of flush.
986       this.memstoreSize.addAndGet(-currentMemStoreSize);
987     } catch (Throwable t) {
988       // An exception here means that the snapshot was not persisted.
989       // The hlog needs to be replayed so its content is restored to memstore.
990       // Currently, only a server restart will do this.
991       // We used to only catch IOEs but its possible that we'd get other
992       // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
993       // all and sundry.
994       if (wal != null) wal.abortCacheFlush();
995       DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
996           Bytes.toStringBinary(getRegionName()));
997       dse.initCause(t);
998       throw dse;
999     }
1000 
1001     // If we get to here, the HStores have been written. If we get an
1002     // error in completeCacheFlush it will release the lock it is holding
1003 
1004     // B.  Write a FLUSHCACHE-COMPLETE message to the log.
1005     //     This tells future readers that the HStores were emitted correctly,
1006     //     and that all updates to the log for this regionName that have lower
1007     //     log-sequence-ids can be safely ignored.
1008     if (wal != null) {
1009       wal.completeCacheFlush(this.regionInfo.getEncodedNameAsBytes(),
1010         regionInfo.getTableDesc().getName(), completeSequenceId,
1011         this.getRegionInfo().isMetaRegion());
1012     }
1013 
1014     // C. Finally notify anyone waiting on memstore to clear:
1015     // e.g. checkResources().
1016     synchronized (this) {
1017       notifyAll(); // FindBugs NN_NAKED_NOTIFY
1018     }
1019 
1020     long time = EnvironmentEdgeManager.currentTimeMillis() - startTime;
1021     if (LOG.isDebugEnabled()) {
1022       LOG.info("Finished memstore flush of ~" +
1023         StringUtils.humanReadableInt(currentMemStoreSize) + " for region " +
1024         this + " in " + time + "ms, sequenceid=" + sequenceId +
1025         ", compaction requested=" + compactionRequested +
1026         ((wal == null)? "; wal=null": ""));
1027     }
1028     this.recentFlushes.add(new Pair<Long,Long>(time/1000,currentMemStoreSize));
1029 
1030     return compactionRequested;
1031   }
1032 
1033    /**
1034    * Get the sequence number to be associated with this cache flush. Used by
1035    * TransactionalRegion to not complete pending transactions.
1036    *
1037    *
1038    * @param currentSequenceId
1039    * @return sequence id to complete the cache flush with
1040    */
1041   protected long getCompleteCacheFlushSequenceId(long currentSequenceId) {
1042     return currentSequenceId;
1043   }
1044 
1045   //////////////////////////////////////////////////////////////////////////////
1046   // get() methods for client use.
1047   //////////////////////////////////////////////////////////////////////////////
1048   /**
1049    * Return all the data for the row that matches <i>row</i> exactly,
1050    * or the one that immediately preceeds it, at or immediately before
1051    * <i>ts</i>.
1052    *
1053    * @param row row key
1054    * @return map of values
1055    * @throws IOException
1056    */
1057   Result getClosestRowBefore(final byte [] row)
1058   throws IOException{
1059     return getClosestRowBefore(row, HConstants.CATALOG_FAMILY);
1060   }
1061 
1062   /**
1063    * Return all the data for the row that matches <i>row</i> exactly,
1064    * or the one that immediately preceeds it, at or immediately before
1065    * <i>ts</i>.
1066    *
1067    * @param row row key
1068    * @param family column family to find on
1069    * @return map of values
1070    * @throws IOException read exceptions
1071    */
1072   public Result getClosestRowBefore(final byte [] row, final byte [] family)
1073   throws IOException {
1074     // look across all the HStores for this region and determine what the
1075     // closest key is across all column families, since the data may be sparse
1076     KeyValue key = null;
1077     checkRow(row);
1078     startRegionOperation();
1079     try {
1080       Store store = getStore(family);
1081       KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1082       // get the closest key. (HStore.getRowKeyAtOrBefore can return null)
1083       key = store.getRowKeyAtOrBefore(kv);
1084       if (key == null) {
1085         return null;
1086       }
1087       Get get = new Get(key.getRow());
1088       get.addFamily(family);
1089       return get(get, null);
1090     } finally {
1091       closeRegionOperation();
1092     }
1093   }
1094 
1095   /**
1096    * Return an iterator that scans over the HRegion, returning the indicated
1097    * columns and rows specified by the {@link Scan}.
1098    * <p>
1099    * This Iterator must be closed by the caller.
1100    *
1101    * @param scan configured {@link Scan}
1102    * @return InternalScanner
1103    * @throws IOException read exceptions
1104    */
1105   public InternalScanner getScanner(Scan scan)
1106   throws IOException {
1107    return getScanner(scan, null);
1108   }
1109 
1110   protected InternalScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
1111     startRegionOperation();
1112     try {
1113       // Verify families are all valid
1114       if(scan.hasFamilies()) {
1115         for(byte [] family : scan.getFamilyMap().keySet()) {
1116           checkFamily(family);
1117         }
1118       } else { // Adding all families to scanner
1119         for(byte[] family: regionInfo.getTableDesc().getFamiliesKeys()){
1120           scan.addFamily(family);
1121         }
1122       }
1123       return instantiateInternalScanner(scan, additionalScanners);
1124 
1125     } finally {
1126       closeRegionOperation();
1127     }
1128   }
1129 
1130   protected InternalScanner instantiateInternalScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
1131     return new RegionScanner(scan, additionalScanners);
1132   }
1133 
1134   /*
1135    * @param delete The passed delete is modified by this method. WARNING!
1136    */
1137   private void prepareDelete(Delete delete) throws IOException {
1138     // Check to see if this is a deleteRow insert
1139     if(delete.getFamilyMap().isEmpty()){
1140       for(byte [] family : regionInfo.getTableDesc().getFamiliesKeys()){
1141         // Don't eat the timestamp
1142         delete.deleteFamily(family, delete.getTimeStamp());
1143       }
1144     } else {
1145       for(byte [] family : delete.getFamilyMap().keySet()) {
1146         if(family == null) {
1147           throw new NoSuchColumnFamilyException("Empty family is invalid");
1148         }
1149         checkFamily(family);
1150       }
1151     }
1152   }
1153 
1154   //////////////////////////////////////////////////////////////////////////////
1155   // set() methods for client use.
1156   //////////////////////////////////////////////////////////////////////////////
1157   /**
1158    * @param delete delete object
1159    * @param lockid existing lock id, or null for grab a lock
1160    * @param writeToWAL append to the write ahead lock or not
1161    * @throws IOException read exceptions
1162    */
1163   public void delete(Delete delete, Integer lockid, boolean writeToWAL)
1164   throws IOException {
1165     checkReadOnly();
1166     checkResources();
1167     Integer lid = null;
1168     startRegionOperation();
1169     try {
1170       byte [] row = delete.getRow();
1171       // If we did not pass an existing row lock, obtain a new one
1172       lid = getLock(lockid, row, true);
1173 
1174       // All edits for the given row (across all column families) must happen atomically.
1175       prepareDelete(delete);
1176       delete(delete.getFamilyMap(), writeToWAL);
1177 
1178     } finally {
1179       if(lockid == null) releaseRowLock(lid);
1180       closeRegionOperation();
1181     }
1182   }
1183 
1184 
1185   /**
1186    * @param familyMap map of family to edits for the given family.
1187    * @param writeToWAL
1188    * @throws IOException
1189    */
1190   public void delete(Map<byte[], List<KeyValue>> familyMap, boolean writeToWAL)
1191   throws IOException {
1192     long now = EnvironmentEdgeManager.currentTimeMillis();
1193     byte [] byteNow = Bytes.toBytes(now);
1194     boolean flush = false;
1195 
1196     updatesLock.readLock().lock();
1197 
1198     try {
1199 
1200       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
1201 
1202         byte[] family = e.getKey();
1203         List<KeyValue> kvs = e.getValue();
1204         Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
1205 
1206         for (KeyValue kv: kvs) {
1207           //  Check if time is LATEST, change to time of most recent addition if so
1208           //  This is expensive.
1209           if (kv.isLatestTimestamp() && kv.isDeleteType()) {
1210             byte[] qual = kv.getQualifier();
1211             if (qual == null) qual = HConstants.EMPTY_BYTE_ARRAY;
1212 
1213             Integer count = kvCount.get(qual);
1214             if (count == null) {
1215               kvCount.put(qual, 1);
1216             } else {
1217               kvCount.put(qual, count + 1);
1218             }
1219             count = kvCount.get(qual);
1220 
1221             Get get = new Get(kv.getRow());
1222             get.setMaxVersions(count);
1223             get.addColumn(family, qual);
1224 
1225             List<KeyValue> result = get(get);
1226 
1227             if (result.size() < count) {
1228               // Nothing to delete
1229               kv.updateLatestStamp(byteNow);
1230               continue;
1231             }
1232             if (result.size() > count) {
1233               throw new RuntimeException("Unexpected size: " + result.size());
1234             }
1235             KeyValue getkv = result.get(count - 1);
1236             Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(),
1237                 getkv.getBuffer(), getkv.getTimestampOffset(), Bytes.SIZEOF_LONG);
1238           } else {
1239             kv.updateLatestStamp(byteNow);
1240           }
1241         }
1242       }
1243 
1244       if (writeToWAL) {
1245         // write/sync to WAL should happen before we touch memstore.
1246         //
1247         // If order is reversed, i.e. we write to memstore first, and
1248         // for some reason fail to write/sync to commit log, the memstore
1249         // will contain uncommitted transactions.
1250         //
1251         // bunch up all edits across all column families into a
1252         // single WALEdit.
1253         WALEdit walEdit = new WALEdit();
1254         addFamilyMapToWALEdit(familyMap, walEdit);
1255         this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
1256             walEdit, now);
1257       }
1258 
1259       // Now make changes to the memstore.
1260       long addedSize = applyFamilyMapToMemstore(familyMap);
1261       flush = isFlushSize(memstoreSize.addAndGet(addedSize));
1262     } finally {
1263       this.updatesLock.readLock().unlock();
1264     }
1265 
1266     if (flush) {
1267       // Request a cache flush.  Do it outside update lock.
1268       requestFlush();
1269     }
1270   }
1271 
1272   /**
1273    * @param put
1274    * @throws IOException
1275    */
1276   public void put(Put put) throws IOException {
1277     this.put(put, null, put.getWriteToWAL());
1278   }
1279 
1280   /**
1281    * @param put
1282    * @param writeToWAL
1283    * @throws IOException
1284    */
1285   public void put(Put put, boolean writeToWAL) throws IOException {
1286     this.put(put, null, writeToWAL);
1287   }
1288 
1289   /**
1290    * @param put
1291    * @param lockid
1292    * @throws IOException
1293    */
1294   public void put(Put put, Integer lockid) throws IOException {
1295     this.put(put, lockid, put.getWriteToWAL());
1296   }
1297 
1298   /**
1299    * @param put
1300    * @param lockid
1301    * @param writeToWAL
1302    * @throws IOException
1303    */
1304   public void put(Put put, Integer lockid, boolean writeToWAL)
1305   throws IOException {
1306     checkReadOnly();
1307 
1308     // Do a rough check that we have resources to accept a write.  The check is
1309     // 'rough' in that between the resource check and the call to obtain a
1310     // read lock, resources may run out.  For now, the thought is that this
1311     // will be extremely rare; we'll deal with it when it happens.
1312     checkResources();
1313     startRegionOperation();
1314     try {
1315       // We obtain a per-row lock, so other clients will block while one client
1316       // performs an update. The read lock is released by the client calling
1317       // #commit or #abort or if the HRegionServer lease on the lock expires.
1318       // See HRegionServer#RegionListener for how the expire on HRegionServer
1319       // invokes a HRegion#abort.
1320       byte [] row = put.getRow();
1321       // If we did not pass an existing row lock, obtain a new one
1322       Integer lid = getLock(lockid, row, true);
1323 
1324       try {
1325         // All edits for the given row (across all column families) must happen atomically.
1326         put(put.getFamilyMap(), writeToWAL);
1327       } finally {
1328         if(lockid == null) releaseRowLock(lid);
1329       }
1330     } finally {
1331       closeRegionOperation();
1332     }
1333   }
1334 
1335   /**
1336    * Struct-like class that tracks the progress of a batch operation,
1337    * accumulating status codes and tracking the index at which processing
1338    * is proceeding.
1339    */
1340   private static class BatchOperationInProgress<T> {
1341     T[] operations;
1342     OperationStatusCode[] retCodes;
1343     int nextIndexToProcess = 0;
1344 
1345     public BatchOperationInProgress(T[] operations) {
1346       this.operations = operations;
1347       retCodes = new OperationStatusCode[operations.length];
1348       Arrays.fill(retCodes, OperationStatusCode.NOT_RUN);
1349     }
1350 
1351     public boolean isDone() {
1352       return nextIndexToProcess == operations.length;
1353     }
1354   }
1355 
1356   /**
1357    * Perform a batch put with no pre-specified locks
1358    * @see HRegion#put(Pair[])
1359    */
1360   public OperationStatusCode[] put(Put[] puts) throws IOException {
1361     @SuppressWarnings("unchecked")
1362     Pair<Put, Integer> putsAndLocks[] = new Pair[puts.length];
1363 
1364     for (int i = 0; i < puts.length; i++) {
1365       putsAndLocks[i] = new Pair<Put, Integer>(puts[i], null);
1366     }
1367     return put(putsAndLocks);
1368   }
1369 
1370   /**
1371    * Perform a batch of puts.
1372    * @param putsAndLocks the list of puts paired with their requested lock IDs.
1373    * @throws IOException
1374    */
1375   public OperationStatusCode[] put(Pair<Put, Integer>[] putsAndLocks) throws IOException {
1376     BatchOperationInProgress<Pair<Put, Integer>> batchOp =
1377       new BatchOperationInProgress<Pair<Put,Integer>>(putsAndLocks);
1378 
1379     while (!batchOp.isDone()) {
1380       checkReadOnly();
1381       checkResources();
1382 
1383       long newSize;
1384       startRegionOperation();
1385       try {
1386         long addedSize = doMiniBatchPut(batchOp);
1387         newSize = memstoreSize.addAndGet(addedSize);
1388       } finally {
1389         closeRegionOperation();
1390       }
1391       if (isFlushSize(newSize)) {
1392         requestFlush();
1393       }
1394     }
1395     return batchOp.retCodes;
1396   }
1397 
1398   private long doMiniBatchPut(BatchOperationInProgress<Pair<Put, Integer>> batchOp) throws IOException {
1399     long now = EnvironmentEdgeManager.currentTimeMillis();
1400     byte[] byteNow = Bytes.toBytes(now);
1401     boolean locked = false;
1402 
1403     /** Keep track of the locks we hold so we can release them in finally clause */
1404     List<Integer> acquiredLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
1405     // We try to set up a batch in the range [firstIndex,lastIndexExclusive)
1406     int firstIndex = batchOp.nextIndexToProcess;
1407     int lastIndexExclusive = firstIndex;
1408     boolean success = false;
1409     try {
1410       // ------------------------------------
1411       // STEP 1. Try to acquire as many locks as we can, and ensure
1412       // we acquire at least one.
1413       // ----------------------------------
1414       int numReadyToWrite = 0;
1415       while (lastIndexExclusive < batchOp.operations.length) {
1416         Pair<Put, Integer> nextPair = batchOp.operations[lastIndexExclusive];
1417         Put put = nextPair.getFirst();
1418         Integer providedLockId = nextPair.getSecond();
1419 
1420         // Check the families in the put. If bad, skip this one.
1421         try {
1422           checkFamilies(put.getFamilyMap().keySet());
1423         } catch (NoSuchColumnFamilyException nscf) {
1424           LOG.warn("No such column family in batch put", nscf);
1425           batchOp.retCodes[lastIndexExclusive] = OperationStatusCode.BAD_FAMILY;
1426           lastIndexExclusive++;
1427           continue;
1428         }
1429 
1430         // If we haven't got any rows in our batch, we should block to
1431         // get the next one.
1432         boolean shouldBlock = numReadyToWrite == 0;
1433         Integer acquiredLockId = getLock(providedLockId, put.getRow(), shouldBlock);
1434         if (acquiredLockId == null) {
1435           // We failed to grab another lock
1436           assert !shouldBlock : "Should never fail to get lock when blocking";
1437           break; // stop acquiring more rows for this batch
1438         }
1439         if (providedLockId == null) {
1440           acquiredLocks.add(acquiredLockId);
1441         }
1442         lastIndexExclusive++;
1443         numReadyToWrite++;
1444       }
1445       // Nothing to put -- an exception in the above such as NoSuchColumnFamily?
1446       if (numReadyToWrite <= 0) return 0L;
1447 
1448       // We've now grabbed as many puts off the list as we can
1449 
1450       // ------------------------------------
1451       // STEP 2. Update any LATEST_TIMESTAMP timestamps
1452       // ----------------------------------
1453       for (int i = firstIndex; i < lastIndexExclusive; i++) {
1454         updateKVTimestamps(
1455             batchOp.operations[i].getFirst().getFamilyMap().values(),
1456             byteNow);
1457       }
1458 
1459 
1460       this.updatesLock.readLock().lock();
1461       locked = true;
1462 
1463       // ------------------------------------
1464       // STEP 3. Write to WAL
1465       // ----------------------------------
1466       WALEdit walEdit = new WALEdit();
1467       for (int i = firstIndex; i < lastIndexExclusive; i++) {
1468         // Skip puts that were determined to be invalid during preprocessing
1469         if (batchOp.retCodes[i] != OperationStatusCode.NOT_RUN) continue;
1470 
1471         Put p = batchOp.operations[i].getFirst();
1472         if (!p.getWriteToWAL()) continue;
1473         addFamilyMapToWALEdit(p.getFamilyMap(), walEdit);
1474       }
1475 
1476       // Append the edit to WAL
1477       this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
1478           walEdit, now);
1479 
1480       // ------------------------------------
1481       // STEP 4. Write back to memstore
1482       // ----------------------------------
1483       long addedSize = 0;
1484       for (int i = firstIndex; i < lastIndexExclusive; i++) {
1485         if (batchOp.retCodes[i] != OperationStatusCode.NOT_RUN) continue;
1486 
1487         Put p = batchOp.operations[i].getFirst();
1488         addedSize += applyFamilyMapToMemstore(p.getFamilyMap());
1489         batchOp.retCodes[i] = OperationStatusCode.SUCCESS;
1490       }
1491       success = true;
1492       return addedSize;
1493     } finally {
1494       if (locked)
1495         this.updatesLock.readLock().unlock();
1496 
1497       for (Integer toRelease : acquiredLocks) {
1498         releaseRowLock(toRelease);
1499       }
1500       if (!success) {
1501         for (int i = firstIndex; i < lastIndexExclusive; i++) {
1502           if (batchOp.retCodes[i] == OperationStatusCode.NOT_RUN) {
1503             batchOp.retCodes[i] = OperationStatusCode.FAILURE;
1504           }
1505         }
1506       }
1507       batchOp.nextIndexToProcess = lastIndexExclusive;
1508     }
1509   }
1510 
1511   //TODO, Think that gets/puts and deletes should be refactored a bit so that
1512   //the getting of the lock happens before, so that you would just pass it into
1513   //the methods. So in the case of checkAndMutate you could just do lockRow,
1514   //get, put, unlockRow or something
1515   /**
1516    *
1517    * @param row
1518    * @param family
1519    * @param qualifier
1520    * @param expectedValue
1521    * @param lockId
1522    * @param writeToWAL
1523    * @throws IOException
1524    * @return true if the new put was execute, false otherwise
1525    */
1526   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
1527       byte [] expectedValue, Writable w, Integer lockId, boolean writeToWAL)
1528   throws IOException{
1529     checkReadOnly();
1530     //TODO, add check for value length or maybe even better move this to the
1531     //client if this becomes a global setting
1532     checkResources();
1533     boolean isPut = w instanceof Put;
1534     if (!isPut && !(w instanceof Delete))
1535       throw new DoNotRetryIOException("Action must be Put or Delete");
1536     Row r = (Row)w;
1537     if (Bytes.compareTo(row, r.getRow()) != 0) {
1538       throw new DoNotRetryIOException("Action's getRow must match the passed row");
1539     }
1540 
1541     startRegionOperation();
1542     try {
1543       RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
1544       Get get = new Get(row, lock);
1545       checkFamily(family);
1546       get.addColumn(family, qualifier);
1547 
1548       // Lock row
1549       Integer lid = getLock(lockId, get.getRow(), true);
1550       List<KeyValue> result = new ArrayList<KeyValue>();
1551       try {
1552         result = get(get);
1553 
1554         boolean matches = false;
1555         if (result.size() == 0 &&
1556             (expectedValue == null || expectedValue.length == 0)) {
1557           matches = true;
1558         } else if (result.size() == 1) {
1559           //Compare the expected value with the actual value
1560           byte [] actualValue = result.get(0).getValue();
1561           matches = Bytes.equals(expectedValue, actualValue);
1562         }
1563         //If matches put the new put or delete the new delete
1564         if (matches) {
1565           // All edits for the given row (across all column families) must happen atomically.
1566           if (isPut) {
1567             put(((Put)w).getFamilyMap(), writeToWAL);
1568           } else {
1569             Delete d = (Delete)w;
1570             prepareDelete(d);
1571             delete(d.getFamilyMap(), writeToWAL);
1572           }
1573           return true;
1574         }
1575         return false;
1576       } finally {
1577         if(lockId == null) releaseRowLock(lid);
1578       }
1579     } finally {
1580       closeRegionOperation();
1581     }
1582   }
1583 
1584 
1585   /**
1586    * Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP}
1587    * with the provided current timestamp.
1588    */
1589   private void updateKVTimestamps(
1590       final Iterable<List<KeyValue>> keyLists, final byte[] now) {
1591     for (List<KeyValue> keys: keyLists) {
1592       if (keys == null) continue;
1593       for (KeyValue key : keys) {
1594         key.updateLatestStamp(now);
1595       }
1596     }
1597   }
1598 
1599   /*
1600    * Check if resources to support an update.
1601    *
1602    * Here we synchronize on HRegion, a broad scoped lock.  Its appropriate
1603    * given we're figuring in here whether this region is able to take on
1604    * writes.  This is only method with a synchronize (at time of writing),
1605    * this and the synchronize on 'this' inside in internalFlushCache to send
1606    * the notify.
1607    */
1608   private void checkResources() {
1609 
1610     // If catalog region, do not impose resource constraints or block updates.
1611     if (this.getRegionInfo().isMetaRegion()) return;
1612 
1613     boolean blocked = false;
1614     while (this.memstoreSize.get() > this.blockingMemStoreSize) {
1615       requestFlush();
1616       if (!blocked) {
1617         LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
1618           "' on region " + Bytes.toStringBinary(getRegionName()) +
1619           ": memstore size " +
1620           StringUtils.humanReadableInt(this.memstoreSize.get()) +
1621           " is >= than blocking " +
1622           StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
1623       }
1624       blocked = true;
1625       synchronized(this) {
1626         try {
1627           wait(threadWakeFrequency);
1628         } catch (InterruptedException e) {
1629           // continue;
1630         }
1631       }
1632     }
1633     if (blocked) {
1634       LOG.info("Unblocking updates for region " + this + " '"
1635           + Thread.currentThread().getName() + "'");
1636     }
1637   }
1638 
1639   /**
1640    * @throws IOException Throws exception if region is in read-only mode.
1641    */
1642   protected void checkReadOnly() throws IOException {
1643     if (this.writestate.isReadOnly()) {
1644       throw new IOException("region is read only");
1645     }
1646   }
1647 
1648   /**
1649    * Add updates first to the hlog and then add values to memstore.
1650    * Warning: Assumption is caller has lock on passed in row.
1651    * @param family
1652    * @param edits Cell updates by column
1653    * @praram now
1654    * @throws IOException
1655    */
1656   private void put(final byte [] family, final List<KeyValue> edits)
1657   throws IOException {
1658     Map<byte[], List<KeyValue>> familyMap = new HashMap<byte[], List<KeyValue>>();
1659     familyMap.put(family, edits);
1660     this.put(familyMap, true);
1661   }
1662 
1663   /**
1664    * Add updates first to the hlog (if writeToWal) and then add values to memstore.
1665    * Warning: Assumption is caller has lock on passed in row.
1666    * @param familyMap map of family to edits for the given family.
1667    * @param writeToWAL if true, then we should write to the log
1668    * @throws IOException
1669    */
1670   private void put(final Map<byte [], List<KeyValue>> familyMap,
1671       boolean writeToWAL) throws IOException {
1672     long now = EnvironmentEdgeManager.currentTimeMillis();
1673     byte[] byteNow = Bytes.toBytes(now);
1674     boolean flush = false;
1675     this.updatesLock.readLock().lock();
1676     try {
1677       checkFamilies(familyMap.keySet());
1678       updateKVTimestamps(familyMap.values(), byteNow);
1679       // write/sync to WAL should happen before we touch memstore.
1680       //
1681       // If order is reversed, i.e. we write to memstore first, and
1682       // for some reason fail to write/sync to commit log, the memstore
1683       // will contain uncommitted transactions.
1684       if (writeToWAL) {
1685         WALEdit walEdit = new WALEdit();
1686         addFamilyMapToWALEdit(familyMap, walEdit);
1687         this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
1688            walEdit, now);
1689       }
1690 
1691       long addedSize = applyFamilyMapToMemstore(familyMap);
1692       flush = isFlushSize(memstoreSize.addAndGet(addedSize));
1693     } finally {
1694       this.updatesLock.readLock().unlock();
1695     }
1696     if (flush) {
1697       // Request a cache flush.  Do it outside update lock.
1698       requestFlush();
1699     }
1700   }
1701 
1702   /**
1703    * Atomically apply the given map of family->edits to the memstore.
1704    * This handles the consistency control on its own, but the caller
1705    * should already have locked updatesLock.readLock(). This also does
1706    * <b>not</b> check the families for validity.
1707    *
1708    * @return the additional memory usage of the memstore caused by the
1709    * new entries.
1710    */
1711   private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap) {
1712     ReadWriteConsistencyControl.WriteEntry w = null;
1713     long size = 0;
1714     try {
1715       w = rwcc.beginMemstoreInsert();
1716 
1717       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
1718         byte[] family = e.getKey();
1719         List<KeyValue> edits = e.getValue();
1720 
1721         Store store = getStore(family);
1722         for (KeyValue kv: edits) {
1723           kv.setMemstoreTS(w.getWriteNumber());
1724           size += store.add(kv);
1725         }
1726       }
1727     } finally {
1728       rwcc.completeMemstoreInsert(w);
1729     }
1730     return size;
1731   }
1732 
1733   /**
1734    * Check the collection of families for validity.
1735    * @throws NoSuchColumnFamilyException if a family does not exist.
1736    */
1737   private void checkFamilies(Collection<byte[]> families)
1738   throws NoSuchColumnFamilyException {
1739     for (byte[] family : families) {
1740       checkFamily(family);
1741     }
1742   }
1743 
1744   /**
1745    * Append the given map of family->edits to a WALEdit data structure.
1746    * This does not write to the HLog itself.
1747    * @param familyMap map of family->edits
1748    * @param walEdit the destination entry to append into
1749    */
1750   private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
1751       WALEdit walEdit) {
1752     for (List<KeyValue> edits : familyMap.values()) {
1753       for (KeyValue kv : edits) {
1754         walEdit.add(kv);
1755       }
1756     }
1757   }
1758 
1759   private void requestFlush() {
1760     if (this.flushRequester == null) {
1761       return;
1762     }
1763     synchronized (writestate) {
1764       if (this.writestate.isFlushRequested()) {
1765         return;
1766       }
1767       writestate.flushRequested = true;
1768     }
1769     // Make request outside of synchronize block; HBASE-818.
1770     this.flushRequester.requestFlush(this);
1771     if (LOG.isDebugEnabled()) {
1772       LOG.debug("Flush requested on " + this);
1773     }
1774   }
1775 
1776   /*
1777    * @param size
1778    * @return True if size is over the flush threshold
1779    */
1780   private boolean isFlushSize(final long size) {
1781     return size > this.memstoreFlushSize;
1782   }
1783 
1784   /**
1785    * Read the edits log put under this region by wal log splitting process.  Put
1786    * the recovered edits back up into this region.
1787    *
1788    * <p>We can ignore any log message that has a sequence ID that's equal to or
1789    * lower than minSeqId.  (Because we know such log messages are already
1790    * reflected in the HFiles.)
1791    *
1792    * <p>While this is running we are putting pressure on memory yet we are
1793    * outside of our usual accounting because we are not yet an onlined region
1794    * (this stuff is being run as part of Region initialization).  This means
1795    * that if we're up against global memory limits, we'll not be flagged to flush
1796    * because we are not online. We can't be flushed by usual mechanisms anyways;
1797    * we're not yet online so our relative sequenceids are not yet aligned with
1798    * HLog sequenceids -- not till we come up online, post processing of split
1799    * edits.
1800    *
1801    * <p>But to help relieve memory pressure, at least manage our own heap size
1802    * flushing if are in excess of per-region limits.  Flushing, though, we have
1803    * to be careful and avoid using the regionserver/hlog sequenceid.  Its running
1804    * on a different line to whats going on in here in this region context so if we
1805    * crashed replaying these edits, but in the midst had a flush that used the
1806    * regionserver log with a sequenceid in excess of whats going on in here
1807    * in this region and with its split editlogs, then we could miss edits the
1808    * next time we go to recover. So, we have to flush inline, using seqids that
1809    * make sense in a this single region context only -- until we online.
1810    *
1811    * @param regiondir
1812    * @param minSeqId Any edit found in split editlogs needs to be in excess of
1813    * this minSeqId to be applied, else its skipped.
1814    * @param reporter
1815    * @return the sequence id of the last edit added to this region out of the
1816    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
1817    * @throws UnsupportedEncodingException
1818    * @throws IOException
1819    */
1820   protected long replayRecoveredEditsIfAny(final Path regiondir,
1821       final long minSeqId, final CancelableProgressable reporter)
1822   throws UnsupportedEncodingException, IOException {
1823     long seqid = minSeqId;
1824     NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
1825     if (files == null || files.isEmpty()) return seqid;
1826     for (Path edits: files) {
1827       if (edits == null || !this.fs.exists(edits)) {
1828         LOG.warn("Null or non-existent edits file: " + edits);
1829         continue;
1830       }
1831       if (isZeroLengthThenDelete(this.fs, edits)) continue;
1832       try {
1833         seqid = replayRecoveredEdits(edits, seqid, reporter);
1834       } catch (IOException e) {
1835         boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
1836         if (skipErrors) {
1837           Path p = HLog.moveAsideBadEditsFile(fs, edits);
1838           LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits +
1839             " as " + p, e);
1840         } else {
1841           throw e;
1842         }
1843       }
1844     }
1845     if (seqid > minSeqId) {
1846       // Then we added some edits to memory. Flush and cleanup split edit files.
1847       internalFlushcache(null, seqid);
1848     }
1849     // Now delete the content of recovered edits.  We're done w/ them.
1850     for (Path file: files) {
1851       if (!this.fs.delete(file, false)) {
1852         LOG.error("Failed delete of " + file);
1853       } else {
1854         LOG.debug("Deleted recovered.edits file=" + file);
1855       }
1856     }
1857     return seqid;
1858   }
1859 
1860   /*
1861    * @param edits File of recovered edits.
1862    * @param minSeqId Minimum sequenceid found in a store file.  Edits in log
1863    * must be larger than this to be replayed.
1864    * @param reporter
1865    * @return the sequence id of the last edit added to this region out of the
1866    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
1867    * @throws IOException
1868    */
1869   private long replayRecoveredEdits(final Path edits,
1870       final long minSeqId, final CancelableProgressable reporter)
1871     throws IOException {
1872     LOG.info("Replaying edits from " + edits + "; minSequenceid=" + minSeqId);
1873     HLog.Reader reader = HLog.getReader(this.fs, edits, conf);
1874     try {
1875     long currentEditSeqId = minSeqId;
1876     long firstSeqIdInLog = -1;
1877     long skippedEdits = 0;
1878     long editsCount = 0;
1879     long intervalEdits = 0;
1880     HLog.Entry entry;
1881     Store store = null;
1882 
1883     try {
1884       // How many edits seen before we check elapsed time
1885       int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
1886           2000);
1887       // How often to send a progress report (default 1/2 master timeout)
1888       int period = this.conf.getInt("hbase.hstore.report.period",
1889           this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
1890               180000) / 2);
1891       long lastReport = EnvironmentEdgeManager.currentTimeMillis();
1892 
1893       while ((entry = reader.next()) != null) {
1894         HLogKey key = entry.getKey();
1895         WALEdit val = entry.getEdit();
1896 
1897         if (reporter != null) {
1898           intervalEdits += val.size();
1899           if (intervalEdits >= interval) {
1900             // Number of edits interval reached
1901             intervalEdits = 0;
1902             long cur = EnvironmentEdgeManager.currentTimeMillis();
1903             if (lastReport + period <= cur) {
1904               // Timeout reached
1905               if(!reporter.progress()) {
1906                 String msg = "Progressable reporter failed, stopping replay";
1907                 LOG.warn(msg);
1908                 throw new IOException(msg);
1909               }
1910               lastReport = cur;
1911             }
1912           }
1913         }
1914 
1915         if (firstSeqIdInLog == -1) {
1916           firstSeqIdInLog = key.getLogSeqNum();
1917         }
1918         // Now, figure if we should skip this edit.
1919         if (key.getLogSeqNum() <= currentEditSeqId) {
1920           skippedEdits++;
1921           continue;
1922         }
1923         currentEditSeqId = key.getLogSeqNum();
1924         boolean flush = false;
1925         for (KeyValue kv: val.getKeyValues()) {
1926           // Check this edit is for me. Also, guard against writing the special
1927           // METACOLUMN info such as HBASE::CACHEFLUSH entries
1928           if (kv.matchingFamily(HLog.METAFAMILY) ||
1929               !Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
1930             skippedEdits++;
1931             continue;
1932               }
1933           // Figure which store the edit is meant for.
1934           if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
1935             store = this.stores.get(kv.getFamily());
1936           }
1937           if (store == null) {
1938             // This should never happen.  Perhaps schema was changed between
1939             // crash and redeploy?
1940             LOG.warn("No family for " + kv);
1941             skippedEdits++;
1942             continue;
1943           }
1944           // Once we are over the limit, restoreEdit will keep returning true to
1945           // flush -- but don't flush until we've played all the kvs that make up
1946           // the WALEdit.
1947           flush = restoreEdit(store, kv);
1948           editsCount++;
1949         }
1950         if (flush) internalFlushcache(null, currentEditSeqId);
1951       }
1952     } catch (EOFException eof) {
1953       Path p = HLog.moveAsideBadEditsFile(fs, edits);
1954       LOG.warn("Encountered EOF. Most likely due to Master failure during " +
1955           "log spliting, so we have this data in another edit.  " +
1956           "Continuing, but renaming " + edits + " as " + p, eof);
1957     } catch (IOException ioe) {
1958       // If the IOE resulted from bad file format,
1959       // then this problem is idempotent and retrying won't help
1960       if (ioe.getCause() instanceof ParseException) {
1961         Path p = HLog.moveAsideBadEditsFile(fs, edits);
1962         LOG.warn("File corruption encountered!  " +
1963             "Continuing, but renaming " + edits + " as " + p, ioe);
1964       } else {
1965         // other IO errors may be transient (bad network connection,
1966         // checksum exception on one datanode, etc).  throw & retry
1967         throw ioe;
1968       }
1969     }
1970     if (LOG.isDebugEnabled()) {
1971       LOG.debug("Applied " + editsCount + ", skipped " + skippedEdits +
1972           ", firstSequenceidInLog=" + firstSeqIdInLog +
1973           ", maxSequenceidInLog=" + currentEditSeqId);
1974     }
1975     return currentEditSeqId;
1976     } finally {
1977       reader.close();
1978     }
1979   }
1980 
1981   /**
1982    * Used by tests
1983    * @param s Store to add edit too.
1984    * @param kv KeyValue to add.
1985    * @return True if we should flush.
1986    */
1987   protected boolean restoreEdit(final Store s, final KeyValue kv) {
1988     return isFlushSize(this.memstoreSize.addAndGet(s.add(kv)));
1989   }
1990 
1991   /*
1992    * @param fs
1993    * @param p File to check.
1994    * @return True if file was zero-length (and if so, we'll delete it in here).
1995    * @throws IOException
1996    */
1997   private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
1998   throws IOException {
1999     FileStatus stat = fs.getFileStatus(p);
2000     if (stat.getLen() > 0) return false;
2001     LOG.warn("File " + p + " is zero-length, deleting.");
2002     fs.delete(p, false);
2003     return true;
2004   }
2005 
2006   protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
2007   throws IOException {
2008     return new Store(tableDir, this, c, this.fs, this.conf);
2009   }
2010 
2011   /**
2012    * Return HStore instance.
2013    * Use with caution.  Exposed for use of fixup utilities.
2014    * @param column Name of column family hosted by this region.
2015    * @return Store that goes with the family on passed <code>column</code>.
2016    * TODO: Make this lookup faster.
2017    */
2018   public Store getStore(final byte [] column) {
2019     return this.stores.get(column);
2020   }
2021 
2022   //////////////////////////////////////////////////////////////////////////////
2023   // Support code
2024   //////////////////////////////////////////////////////////////////////////////
2025 
2026   /** Make sure this is a valid row for the HRegion */
2027   private void checkRow(final byte [] row) throws IOException {
2028     if(!rowIsInRange(regionInfo, row)) {
2029       throw new WrongRegionException("Requested row out of range for " +
2030           "HRegion " + this + ", startKey='" +
2031           Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
2032           Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
2033           Bytes.toStringBinary(row) + "'");
2034     }
2035   }
2036 
2037   /**
2038    * Obtain a lock on the given row.  Blocks until success.
2039    *
2040    * I know it's strange to have two mappings:
2041    * <pre>
2042    *   ROWS  ==> LOCKS
2043    * </pre>
2044    * as well as
2045    * <pre>
2046    *   LOCKS ==> ROWS
2047    * </pre>
2048    *
2049    * But it acts as a guard on the client; a miswritten client just can't
2050    * submit the name of a row and start writing to it; it must know the correct
2051    * lockid, which matches the lock list in memory.
2052    *
2053    * <p>It would be more memory-efficient to assume a correctly-written client,
2054    * which maybe we'll do in the future.
2055    *
2056    * @param row Name of row to lock.
2057    * @throws IOException
2058    * @return The id of the held lock.
2059    */
2060   public Integer obtainRowLock(final byte [] row) throws IOException {
2061     startRegionOperation();
2062     try {
2063       return internalObtainRowLock(row, true);
2064     } finally {
2065       closeRegionOperation();
2066     }
2067   }
2068 
2069   /**
2070    * Tries to obtain a row lock on the given row, but does not block if the
2071    * row lock is not available. If the lock is not available, returns false.
2072    * Otherwise behaves the same as the above method.
2073    * @see HRegion#obtainRowLock(byte[])
2074    */
2075   public Integer tryObtainRowLock(final byte[] row) throws IOException {
2076     startRegionOperation();
2077     try {
2078       return internalObtainRowLock(row, false);
2079     } finally {
2080       closeRegionOperation();
2081     }
2082   }
2083 
2084   /**
2085    * Obtains or tries to obtain the given row lock.
2086    * @param waitForLock if true, will block until the lock is available.
2087    *        Otherwise, just tries to obtain the lock and returns
2088    *        null if unavailable.
2089    */
2090   private Integer internalObtainRowLock(final byte[] row, boolean waitForLock)
2091   throws IOException {
2092     checkRow(row);
2093     startRegionOperation();
2094     try {
2095       synchronized (lockedRows) {
2096         while (lockedRows.contains(row)) {
2097           if (!waitForLock) {
2098             return null;
2099           }
2100           try {
2101             lockedRows.wait();
2102           } catch (InterruptedException ie) {
2103             // Empty
2104           }
2105         }
2106         // generate a new lockid. Attempt to insert the new [lockid, row].
2107         // if this lockid already exists in the map then revert and retry
2108         // We could have first done a lockIds.get, and if it does not exist only
2109         // then do a lockIds.put, but the hope is that the lockIds.put will
2110         // mostly return null the first time itself because there won't be
2111         // too many lockId collisions.
2112         byte [] prev = null;
2113         Integer lockId = null;
2114         do {
2115           lockId = new Integer(lockIdGenerator++);
2116           prev = lockIds.put(lockId, row);
2117           if (prev != null) {
2118             lockIds.put(lockId, prev);    // revert old value
2119             lockIdGenerator = rand.nextInt(); // generate new start point
2120           }
2121         } while (prev != null);
2122 
2123         lockedRows.add(row);
2124         lockedRows.notifyAll();
2125         return lockId;
2126       }
2127     } finally {
2128       closeRegionOperation();
2129     }
2130   }
2131 
2132   /**
2133    * Used by unit tests.
2134    * @param lockid
2135    * @return Row that goes with <code>lockid</code>
2136    */
2137   byte [] getRowFromLock(final Integer lockid) {
2138     synchronized (lockedRows) {
2139       return lockIds.get(lockid);
2140     }
2141   }
2142 
2143   /**
2144    * Release the row lock!
2145    * @param lockid  The lock ID to release.
2146    */
2147   void releaseRowLock(final Integer lockid) {
2148     synchronized (lockedRows) {
2149       byte[] row = lockIds.remove(lockid);
2150       lockedRows.remove(row);
2151       lockedRows.notifyAll();
2152     }
2153   }
2154 
2155   /**
2156    * See if row is currently locked.
2157    * @param lockid
2158    * @return boolean
2159    */
2160   boolean isRowLocked(final Integer lockid) {
2161     synchronized (lockedRows) {
2162       if (lockIds.get(lockid) != null) {
2163         return true;
2164       }
2165       return false;
2166     }
2167   }
2168 
2169   /**
2170    * Returns existing row lock if found, otherwise
2171    * obtains a new row lock and returns it.
2172    * @param lockid requested by the user, or null if the user didn't already hold lock
2173    * @param row the row to lock
2174    * @param waitForLock if true, will block until the lock is available, otherwise will
2175    * simply return null if it could not acquire the lock.
2176    * @return lockid or null if waitForLock is false and the lock was unavailable.
2177    */
2178   private Integer getLock(Integer lockid, byte [] row, boolean waitForLock)
2179   throws IOException {
2180     Integer lid = null;
2181     if (lockid == null) {
2182       lid = internalObtainRowLock(row, waitForLock);
2183     } else {
2184       if (!isRowLocked(lockid)) {
2185         throw new IOException("Invalid row lock");
2186       }
2187       lid = lockid;
2188     }
2189     return lid;
2190   }
2191 
2192   public void bulkLoadHFile(String hfilePath, byte[] familyName)
2193   throws IOException {
2194     startRegionOperation();
2195     try {
2196       Store store = getStore(familyName);
2197       if (store == null) {
2198         throw new DoNotRetryIOException(
2199             "No such column family " + Bytes.toStringBinary(familyName));
2200       }
2201       store.bulkLoadHFile(hfilePath);
2202     } finally {
2203       closeRegionOperation();
2204     }
2205 
2206   }
2207 
2208 
2209   @Override
2210   public boolean equals(Object o) {
2211     if (!(o instanceof HRegion)) {
2212       return false;
2213     }
2214     return this.hashCode() == ((HRegion)o).hashCode();
2215   }
2216 
2217   @Override
2218   public int hashCode() {
2219     return Bytes.hashCode(this.regionInfo.getRegionName());
2220   }
2221 
2222   @Override
2223   public String toString() {
2224     return this.regionInfo.getRegionNameAsString();
2225   }
2226 
2227   /** @return Path of region base directory */
2228   public Path getTableDir() {
2229     return this.tableDir;
2230   }
2231 
2232   /**
2233    * RegionScanner is an iterator through a bunch of rows in an HRegion.
2234    * <p>
2235    * It is used to combine scanners from multiple Stores (aka column families).
2236    */
2237   class RegionScanner implements InternalScanner {
2238     // Package local for testability
2239     KeyValueHeap storeHeap = null;
2240     private final byte [] stopRow;
2241     private Filter filter;
2242     private List<KeyValue> results = new ArrayList<KeyValue>();
2243     private int batch;
2244     private int isScan;
2245     private boolean filterClosed = false;
2246     private long readPt;
2247 
2248     public HRegionInfo getRegionName() {
2249       return regionInfo;
2250     }
2251     RegionScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
2252       //DebugPrint.println("HRegionScanner.<init>");
2253       this.filter = scan.getFilter();
2254       this.batch = scan.getBatch();
2255       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
2256         this.stopRow = null;
2257       } else {
2258         this.stopRow = scan.getStopRow();
2259       }
2260       // If we are doing a get, we want to be [startRow,endRow] normally
2261       // it is [startRow,endRow) and if startRow=endRow we get nothing.
2262       this.isScan = scan.isGetScan() ? -1 : 0;
2263 
2264       this.readPt = ReadWriteConsistencyControl.resetThreadReadPoint(rwcc);
2265 
2266       List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
2267       if (additionalScanners != null) {
2268         scanners.addAll(additionalScanners);
2269       }
2270 
2271       for (Map.Entry<byte[], NavigableSet<byte[]>> entry :
2272           scan.getFamilyMap().entrySet()) {
2273         Store store = stores.get(entry.getKey());
2274         scanners.add(store.getScanner(scan, entry.getValue()));
2275       }
2276       this.storeHeap = new KeyValueHeap(scanners, comparator);
2277     }
2278 
2279     RegionScanner(Scan scan) throws IOException {
2280       this(scan, null);
2281     }
2282 
2283     /**
2284      * Reset both the filter and the old filter.
2285      */
2286     protected void resetFilters() {
2287       if (filter != null) {
2288         filter.reset();
2289       }
2290     }
2291 
2292     public synchronized boolean next(List<KeyValue> outResults, int limit)
2293         throws IOException {
2294       if (this.filterClosed) {
2295         throw new UnknownScannerException("Scanner was closed (timed out?) " +
2296             "after we renewed it. Could be caused by a very slow scanner " +
2297             "or a lengthy garbage collection");
2298       }
2299       startRegionOperation();
2300       try {
2301 
2302         // This could be a new thread from the last time we called next().
2303         ReadWriteConsistencyControl.setThreadReadPoint(this.readPt);
2304 
2305         results.clear();
2306         boolean returnResult = nextInternal(limit);
2307 
2308         outResults.addAll(results);
2309         resetFilters();
2310         if (isFilterDone()) {
2311           return false;
2312         }
2313         return returnResult;
2314       } finally {
2315         closeRegionOperation();
2316       }
2317     }
2318 
2319     public synchronized boolean next(List<KeyValue> outResults)
2320         throws IOException {
2321       // apply the batching limit by default
2322       return next(outResults, batch);
2323     }
2324 
2325     /*
2326      * @return True if a filter rules the scanner is over, done.
2327      */
2328     synchronized boolean isFilterDone() {
2329       return this.filter != null && this.filter.filterAllRemaining();
2330     }
2331 
2332     private boolean nextInternal(int limit) throws IOException {
2333       while (true) {
2334         byte [] currentRow = peekRow();
2335         if (isStopRow(currentRow)) {
2336           if (filter != null && filter.hasFilterRow()) {
2337             filter.filterRow(results);
2338           }
2339           if (filter != null && filter.filterRow()) {
2340             results.clear();
2341           }
2342 
2343           return false;
2344         } else if (filterRowKey(currentRow)) {
2345           nextRow(currentRow);
2346         } else {
2347           byte [] nextRow;
2348           do {
2349             this.storeHeap.next(results, limit - results.size());
2350             if (limit > 0 && results.size() == limit) {
2351               if (this.filter != null && filter.hasFilterRow()) throw new IncompatibleFilterException(
2352                   "Filter with filterRow(List<KeyValue>) incompatible with scan with limit!");
2353               return true; // we are expecting more yes, but also limited to how many we can return.
2354             }
2355           } while (Bytes.equals(currentRow, nextRow = peekRow()));
2356 
2357           final boolean stopRow = isStopRow(nextRow);
2358 
2359           // now that we have an entire row, lets process with a filters:
2360 
2361           // first filter with the filterRow(List)
2362           if (filter != null && filter.hasFilterRow()) {
2363             filter.filterRow(results);
2364           }
2365 
2366           if (results.isEmpty() || filterRow()) {
2367             // this seems like a redundant step - we already consumed the row
2368             // there're no left overs.
2369             // the reasons for calling this method are:
2370             // 1. reset the filters.
2371             // 2. provide a hook to fast forward the row (used by subclasses)
2372             nextRow(currentRow);
2373 
2374             // This row was totally filtered out, if this is NOT the last row,
2375             // we should continue on.
2376 
2377             if (!stopRow) continue;
2378           }
2379           return !stopRow;
2380         }
2381       }
2382     }
2383 
2384     private boolean filterRow() {
2385       return filter != null
2386           && filter.filterRow();
2387     }
2388     private boolean filterRowKey(byte[] row) {
2389       return filter != null
2390           && filter.filterRowKey(row, 0, row.length);
2391     }
2392 
2393     protected void nextRow(byte [] currentRow) throws IOException {
2394       while (Bytes.equals(currentRow, peekRow())) {
2395         this.storeHeap.next(MOCKED_LIST);
2396       }
2397       results.clear();
2398       resetFilters();
2399     }
2400 
2401     private byte[] peekRow() {
2402       KeyValue kv = this.storeHeap.peek();
2403       return kv == null ? null : kv.getRow();
2404     }
2405 
2406     private boolean isStopRow(byte [] currentRow) {
2407       return currentRow == null ||
2408           (stopRow != null &&
2409           comparator.compareRows(stopRow, 0, stopRow.length,
2410               currentRow, 0, currentRow.length) <= isScan);
2411     }
2412 
2413     public synchronized void close() {
2414       if (storeHeap != null) {
2415         storeHeap.close();
2416         storeHeap = null;
2417       }
2418       this.filterClosed = true;
2419     }
2420   }
2421 
2422   // Utility methods
2423   /**
2424    * A utility method to create new instances of HRegion based on the
2425    * {@link HConstants#REGION_IMPL} configuration property.
2426    * @param tableDir qualified path of directory where region should be located,
2427    * usually the table directory.
2428    * @param log The HLog is the outbound log for any updates to the HRegion
2429    * (There's a single HLog for all the HRegions on a single HRegionServer.)
2430    * The log file is a logfile from the previous execution that's
2431    * custom-computed for this HRegion. The HRegionServer computes and sorts the
2432    * appropriate log info for this HRegion. If there is a previous log file
2433    * (implying that the HRegion has been written-to before), then read it from
2434    * the supplied path.
2435    * @param fs is the filesystem.
2436    * @param conf is global configuration settings.
2437    * @param regionInfo - HRegionInfo that describes the region
2438    * is new), then read them from the supplied path.
2439    * @param flushListener an object that implements CacheFlushListener or null
2440    * making progress to master -- otherwise master might think region deploy
2441    * failed.  Can be null.
2442    * @return the new instance
2443    */
2444   public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf,
2445                                    HRegionInfo regionInfo, FlushRequester flushListener) {
2446     try {
2447       @SuppressWarnings("unchecked")
2448       Class<? extends HRegion> regionClass =
2449           (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
2450 
2451       Constructor<? extends HRegion> c =
2452           regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
2453               Configuration.class, HRegionInfo.class, FlushRequester.class);
2454 
2455       return c.newInstance(tableDir, log, fs, conf, regionInfo, flushListener);
2456     } catch (Throwable e) {
2457       // todo: what should I throw here?
2458       throw new IllegalStateException("Could not instantiate a region instance.", e);
2459     }
2460   }
2461 
2462   /**
2463    * Convenience method creating new HRegions. Used by createTable and by the
2464    * bootstrap code in the HMaster constructor.
2465    * Note, this method creates an {@link HLog} for the created region. It
2466    * needs to be closed explicitly.  Use {@link HRegion#getLog()} to get
2467    * access.
2468    * @param info Info for region to create.
2469    * @param rootDir Root directory for HBase instance
2470    * @param conf
2471    * @return new HRegion
2472    *
2473    * @throws IOException
2474    */
2475   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
2476     final Configuration conf)
2477   throws IOException {
2478     Path tableDir =
2479       HTableDescriptor.getTableDir(rootDir, info.getTableDesc().getName());
2480     Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
2481     FileSystem fs = FileSystem.get(conf);
2482     fs.mkdirs(regionDir);
2483     HRegion region = HRegion.newHRegion(tableDir,
2484       new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
2485           new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf),
2486       fs, conf, info, null);
2487     region.initialize();
2488     return region;
2489   }
2490 
2491   /**
2492    * Open a Region.
2493    * @param info Info for region to be opened.
2494    * @param wal HLog for region to use. This method will call
2495    * HLog#setSequenceNumber(long) passing the result of the call to
2496    * HRegion#getMinSequenceId() to ensure the log id is properly kept
2497    * up.  HRegionStore does this every time it opens a new region.
2498    * @param conf
2499    * @return new HRegion
2500    *
2501    * @throws IOException
2502    */
2503   public static HRegion openHRegion(final HRegionInfo info, final HLog wal,
2504       final Configuration conf)
2505   throws IOException {
2506     return openHRegion(info, wal, conf, null, null);
2507   }
2508 
2509   /**
2510    * Open a Region.
2511    * @param info Info for region to be opened.
2512    * @param wal HLog for region to use. This method will call
2513    * HLog#setSequenceNumber(long) passing the result of the call to
2514    * HRegion#getMinSequenceId() to ensure the log id is properly kept
2515    * up.  HRegionStore does this every time it opens a new region.
2516    * @param conf
2517    * @param flusher An interface we can request flushes against.
2518    * @param reporter An interface we can report progress against.
2519    * @return new HRegion
2520    *
2521    * @throws IOException
2522    */
2523   public static HRegion openHRegion(final HRegionInfo info, final HLog wal,
2524     final Configuration conf, final FlushRequester flusher,
2525     final CancelableProgressable reporter)
2526   throws IOException {
2527     if (LOG.isDebugEnabled()) {
2528       LOG.debug("Opening region: " + info);
2529     }
2530     if (info == null) {
2531       throw new NullPointerException("Passed region info is null");
2532     }
2533     Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
2534       info.getTableDesc().getName());
2535     HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
2536       flusher);
2537     return r.openHRegion(reporter);
2538   }
2539 
2540   /**
2541    * Open HRegion.
2542    * Calls initialize and sets sequenceid.
2543    * @param reporter
2544    * @return Returns <code>this</code>
2545    * @throws IOException
2546    */
2547   protected HRegion openHRegion(final CancelableProgressable reporter)
2548   throws IOException {
2549     checkCompressionCodecs();
2550 
2551     long seqid = initialize(reporter);
2552     if (this.log != null) {
2553       this.log.setSequenceNumber(seqid);
2554     }
2555     return this;
2556   }
2557 
2558   private void checkCompressionCodecs() throws IOException {
2559     for (HColumnDescriptor fam: regionInfo.getTableDesc().getColumnFamilies()) {
2560       CompressionTest.testCompression(fam.getCompression());
2561       CompressionTest.testCompression(fam.getCompactionCompression());
2562     }
2563   }
2564 
2565   /**
2566    * Inserts a new region's meta information into the passed
2567    * <code>meta</code> region. Used by the HMaster bootstrap code adding
2568    * new table to ROOT table.
2569    *
2570    * @param meta META HRegion to be updated
2571    * @param r HRegion to add to <code>meta</code>
2572    *
2573    * @throws IOException
2574    */
2575   public static void addRegionToMETA(HRegion meta, HRegion r)
2576   throws IOException {
2577     meta.checkResources();
2578     // The row key is the region name
2579     byte[] row = r.getRegionName();
2580     Integer lid = meta.obtainRowLock(row);
2581     try {
2582       final List<KeyValue> edits = new ArrayList<KeyValue>(1);
2583       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
2584           HConstants.REGIONINFO_QUALIFIER,
2585           EnvironmentEdgeManager.currentTimeMillis(),
2586           Writables.getBytes(r.getRegionInfo())));
2587       meta.put(HConstants.CATALOG_FAMILY, edits);
2588     } finally {
2589       meta.releaseRowLock(lid);
2590     }
2591   }
2592 
2593   /**
2594    * Deletes all the files for a HRegion
2595    *
2596    * @param fs the file system object
2597    * @param rootdir qualified path of HBase root directory
2598    * @param info HRegionInfo for region to be deleted
2599    * @throws IOException
2600    */
2601   public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
2602   throws IOException {
2603     deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
2604   }
2605 
2606   private static void deleteRegion(FileSystem fs, Path regiondir)
2607   throws IOException {
2608     if (LOG.isDebugEnabled()) {
2609       LOG.debug("DELETING region " + regiondir.toString());
2610     }
2611     if (!fs.delete(regiondir, true)) {
2612       LOG.warn("Failed delete of " + regiondir);
2613     }
2614   }
2615 
2616   /**
2617    * Computes the Path of the HRegion
2618    *
2619    * @param rootdir qualified path of HBase root directory
2620    * @param info HRegionInfo for the region
2621    * @return qualified path of region directory
2622    */
2623   public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
2624     return new Path(
2625       HTableDescriptor.getTableDir(rootdir, info.getTableDesc().getName()),
2626                                    info.getEncodedName());
2627   }
2628 
2629   /**
2630    * Determines if the specified row is within the row range specified by the
2631    * specified HRegionInfo
2632    *
2633    * @param info HRegionInfo that specifies the row range
2634    * @param row row to be checked
2635    * @return true if the row is within the range specified by the HRegionInfo
2636    */
2637   public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
2638     return ((info.getStartKey().length == 0) ||
2639         (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
2640         ((info.getEndKey().length == 0) ||
2641             (Bytes.compareTo(info.getEndKey(), row) > 0));
2642   }
2643 
2644   /**
2645    * Make the directories for a specific column family
2646    *
2647    * @param fs the file system
2648    * @param tabledir base directory where region will live (usually the table dir)
2649    * @param hri
2650    * @param colFamily the column family
2651    * @throws IOException
2652    */
2653   public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
2654     final HRegionInfo hri, byte [] colFamily)
2655   throws IOException {
2656     Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
2657     if (!fs.mkdirs(dir)) {
2658       LOG.warn("Failed to create " + dir);
2659     }
2660   }
2661 
2662   /**
2663    * Merge two HRegions.  The regions must be adjacent and must not overlap.
2664    *
2665    * @param srcA
2666    * @param srcB
2667    * @return new merged HRegion
2668    * @throws IOException
2669    */
2670   public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
2671   throws IOException {
2672     HRegion a = srcA;
2673     HRegion b = srcB;
2674 
2675     // Make sure that srcA comes first; important for key-ordering during
2676     // write of the merged file.
2677     if (srcA.getStartKey() == null) {
2678       if (srcB.getStartKey() == null) {
2679         throw new IOException("Cannot merge two regions with null start key");
2680       }
2681       // A's start key is null but B's isn't. Assume A comes before B
2682     } else if ((srcB.getStartKey() == null) ||
2683       (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
2684       a = srcB;
2685       b = srcA;
2686     }
2687 
2688     if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) {
2689       throw new IOException("Cannot merge non-adjacent regions");
2690     }
2691     return merge(a, b);
2692   }
2693 
2694   /**
2695    * Merge two regions whether they are adjacent or not.
2696    *
2697    * @param a region a
2698    * @param b region b
2699    * @return new merged region
2700    * @throws IOException
2701    */
2702   public static HRegion merge(HRegion a, HRegion b) throws IOException {
2703     if (!a.getRegionInfo().getTableDesc().getNameAsString().equals(
2704         b.getRegionInfo().getTableDesc().getNameAsString())) {
2705       throw new IOException("Regions do not belong to the same table");
2706     }
2707 
2708     FileSystem fs = a.getFilesystem();
2709 
2710     // Make sure each region's cache is empty
2711 
2712     a.flushcache();
2713     b.flushcache();
2714 
2715     // Compact each region so we only have one store file per family
2716 
2717     a.compactStores(true);
2718     if (LOG.isDebugEnabled()) {
2719       LOG.debug("Files for region: " + a);
2720       listPaths(fs, a.getRegionDir());
2721     }
2722     b.compactStores(true);
2723     if (LOG.isDebugEnabled()) {
2724       LOG.debug("Files for region: " + b);
2725       listPaths(fs, b.getRegionDir());
2726     }
2727 
2728     Configuration conf = a.getConf();
2729     HTableDescriptor tabledesc = a.getTableDesc();
2730     HLog log = a.getLog();
2731     Path tableDir = a.getTableDir();
2732     // Presume both are of same region type -- i.e. both user or catalog
2733     // table regions.  This way can use comparator.
2734     final byte[] startKey =
2735       (a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
2736            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
2737        || b.comparator.matchingRows(b.getStartKey(), 0,
2738               b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
2739               HConstants.EMPTY_BYTE_ARRAY.length))
2740       ? HConstants.EMPTY_BYTE_ARRAY
2741       : (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
2742              b.getStartKey(), 0, b.getStartKey().length) <= 0
2743          ? a.getStartKey()
2744          : b.getStartKey());
2745     final byte[] endKey =
2746       (a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
2747            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
2748        || a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
2749               HConstants.EMPTY_BYTE_ARRAY, 0,
2750               HConstants.EMPTY_BYTE_ARRAY.length))
2751       ? HConstants.EMPTY_BYTE_ARRAY
2752       : (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
2753              b.getEndKey(), 0, b.getEndKey().length) <= 0
2754          ? b.getEndKey()
2755          : a.getEndKey());
2756 
2757     HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
2758     LOG.info("Creating new region " + newRegionInfo.toString());
2759     String encodedName = newRegionInfo.getEncodedName();
2760     Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
2761     if(fs.exists(newRegionDir)) {
2762       throw new IOException("Cannot merge; target file collision at " +
2763           newRegionDir);
2764     }
2765     fs.mkdirs(newRegionDir);
2766 
2767     LOG.info("starting merge of regions: " + a + " and " + b +
2768       " into new region " + newRegionInfo.toString() +
2769         " with start key <" + Bytes.toString(startKey) + "> and end key <" +
2770         Bytes.toString(endKey) + ">");
2771 
2772     // Move HStoreFiles under new region directory
2773     Map<byte [], List<StoreFile>> byFamily =
2774       new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
2775     byFamily = filesByFamily(byFamily, a.close());
2776     byFamily = filesByFamily(byFamily, b.close());
2777     for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
2778       byte [] colFamily = es.getKey();
2779       makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily);
2780       // Because we compacted the source regions we should have no more than two
2781       // HStoreFiles per family and there will be no reference store
2782       List<StoreFile> srcFiles = es.getValue();
2783       if (srcFiles.size() == 2) {
2784         long seqA = srcFiles.get(0).getMaxSequenceId();
2785         long seqB = srcFiles.get(1).getMaxSequenceId();
2786         if (seqA == seqB) {
2787           // Can't have same sequenceid since on open of a store, this is what
2788           // distingushes the files (see the map of stores how its keyed by
2789           // sequenceid).
2790           throw new IOException("Files have same sequenceid: " + seqA);
2791         }
2792       }
2793       for (StoreFile hsf: srcFiles) {
2794         StoreFile.rename(fs, hsf.getPath(),
2795           StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir,
2796             newRegionInfo.getEncodedName(), colFamily)));
2797       }
2798     }
2799     if (LOG.isDebugEnabled()) {
2800       LOG.debug("Files for new region");
2801       listPaths(fs, newRegionDir);
2802     }
2803     HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf, newRegionInfo, null);
2804     dstRegion.initialize();
2805     dstRegion.compactStores();
2806     if (LOG.isDebugEnabled()) {
2807       LOG.debug("Files for new region");
2808       listPaths(fs, dstRegion.getRegionDir());
2809     }
2810     deleteRegion(fs, a.getRegionDir());
2811     deleteRegion(fs, b.getRegionDir());
2812 
2813     LOG.info("merge completed. New region is " + dstRegion);
2814 
2815     return dstRegion;
2816   }
2817 
2818   /*
2819    * Fills a map with a vector of store files keyed by column family.
2820    * @param byFamily Map to fill.
2821    * @param storeFiles Store files to process.
2822    * @param family
2823    * @return Returns <code>byFamily</code>
2824    */
2825   private static Map<byte [], List<StoreFile>> filesByFamily(
2826       Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
2827     for (StoreFile src: storeFiles) {
2828       byte [] family = src.getFamily();
2829       List<StoreFile> v = byFamily.get(family);
2830       if (v == null) {
2831         v = new ArrayList<StoreFile>();
2832         byFamily.put(family, v);
2833       }
2834       v.add(src);
2835     }
2836     return byFamily;
2837   }
2838 
2839   /**
2840    * @return True if needs a mojor compaction.
2841    * @throws IOException
2842    */
2843   boolean isMajorCompaction() throws IOException {
2844     for (Store store: this.stores.values()) {
2845       if (store.isMajorCompaction()) {
2846         return true;
2847       }
2848     }
2849     return false;
2850   }
2851 
2852   /*
2853    * List the files under the specified directory
2854    *
2855    * @param fs
2856    * @param dir
2857    * @throws IOException
2858    */
2859   private static void listPaths(FileSystem fs, Path dir) throws IOException {
2860     if (LOG.isDebugEnabled()) {
2861       FileStatus[] stats = fs.listStatus(dir);
2862       if (stats == null || stats.length == 0) {
2863         return;
2864       }
2865       for (int i = 0; i < stats.length; i++) {
2866         String path = stats[i].getPath().toString();
2867         if (stats[i].isDir()) {
2868           LOG.debug("d " + path);
2869           listPaths(fs, stats[i].getPath());
2870         } else {
2871           LOG.debug("f " + path + " size=" + stats[i].getLen());
2872         }
2873       }
2874     }
2875   }
2876 
2877 
2878   //
2879   // HBASE-880
2880   //
2881   /**
2882    * @param get get object
2883    * @param lockid existing lock id, or null for no previous lock
2884    * @return result
2885    * @throws IOException read exceptions
2886    */
2887   public Result get(final Get get, final Integer lockid) throws IOException {
2888     // Verify families are all valid
2889     if (get.hasFamilies()) {
2890       for (byte [] family: get.familySet()) {
2891         checkFamily(family);
2892       }
2893     } else { // Adding all families to scanner
2894       for (byte[] family: regionInfo.getTableDesc().getFamiliesKeys()) {
2895         get.addFamily(family);
2896       }
2897     }
2898     List<KeyValue> result = get(get);
2899 
2900     return new Result(result);
2901   }
2902 
2903   /**
2904    * An optimized version of {@link #get(Get)} that checks MemStore first for
2905    * the specified query.
2906    * <p>
2907    * This is intended for use by increment operations where we have the
2908    * guarantee that versions are never inserted out-of-order so if a value
2909    * exists in MemStore it is the latest value.
2910    * <p>
2911    * It only makes sense to use this method without a TimeRange and maxVersions
2912    * equal to 1.
2913    * @param get
2914    * @return result
2915    * @throws IOException
2916    */
2917   private List<KeyValue> getLastIncrement(final Get get) throws IOException {
2918     InternalScan iscan = new InternalScan(get);
2919 
2920     List<KeyValue> results = new ArrayList<KeyValue>();
2921 
2922     // memstore scan
2923     iscan.checkOnlyMemStore();
2924     InternalScanner scanner = null;
2925     try {
2926       scanner = getScanner(iscan);
2927       scanner.next(results);
2928     } finally {
2929       if (scanner != null)
2930         scanner.close();
2931     }
2932 
2933     // count how many columns we're looking for
2934     int expected = 0;
2935     Map<byte[], NavigableSet<byte[]>> familyMap = get.getFamilyMap();
2936     for (NavigableSet<byte[]> qfs : familyMap.values()) {
2937       expected += qfs.size();
2938     }
2939 
2940     // found everything we were looking for, done
2941     if (results.size() == expected) {
2942       return results;
2943     }
2944 
2945     // still have more columns to find
2946     if (results != null && !results.isEmpty()) {
2947       // subtract what was found in memstore
2948       for (KeyValue kv : results) {
2949         byte [] family = kv.getFamily();
2950         NavigableSet<byte[]> qfs = familyMap.get(family);
2951         qfs.remove(kv.getQualifier());
2952         if (qfs.isEmpty()) familyMap.remove(family);
2953         expected--;
2954       }
2955       // make a new get for just what is left
2956       Get newGet = new Get(get.getRow());
2957       for (Map.Entry<byte[], NavigableSet<byte[]>> f : familyMap.entrySet()) {
2958         byte [] family = f.getKey();
2959         for (byte [] qualifier : f.getValue()) {
2960           newGet.addColumn(family, qualifier);
2961         }
2962       }
2963       newGet.setTimeRange(get.getTimeRange().getMin(),
2964           get.getTimeRange().getMax());
2965       iscan = new InternalScan(newGet);
2966     }
2967 
2968     // check store files for what is left
2969     List<KeyValue> fileResults = new ArrayList<KeyValue>();
2970     iscan.checkOnlyStoreFiles();
2971     scanner = null;
2972     try {
2973       scanner = getScanner(iscan);
2974       scanner.next(fileResults);
2975     } finally {
2976       if (scanner != null)
2977         scanner.close();
2978     }
2979 
2980     // combine and return
2981     results.addAll(fileResults);
2982     Collections.sort(results, KeyValue.COMPARATOR);
2983     return results;
2984   }
2985 
2986   /*
2987    * Do a get based on the get parameter.
2988    */
2989   private List<KeyValue> get(final Get get) throws IOException {
2990     Scan scan = new Scan(get);
2991 
2992     List<KeyValue> results = new ArrayList<KeyValue>();
2993 
2994     InternalScanner scanner = null;
2995     try {
2996       scanner = getScanner(scan);
2997       scanner.next(results);
2998     } finally {
2999       if (scanner != null)
3000         scanner.close();
3001     }
3002     return results;
3003   }
3004 
3005   /**
3006    * Perform one or more increment operations on a row.
3007    * <p>
3008    * Increments performed are done under row lock but reads do not take locks
3009    * out so this can be seen partially complete by gets and scans.
3010    * @param increment
3011    * @param lockid
3012    * @param writeToWAL
3013    * @return new keyvalues after increment
3014    * @throws IOException
3015    */
3016   public Result increment(Increment increment, Integer lockid,
3017       boolean writeToWAL)
3018   throws IOException {
3019     // TODO: Use RWCC to make this set of increments atomic to reads
3020     byte [] row = increment.getRow();
3021     checkRow(row);
3022     TimeRange tr = increment.getTimeRange();
3023     boolean flush = false;
3024     WALEdit walEdits = null;
3025     List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
3026     List<KeyValue> kvs = new ArrayList<KeyValue>(increment.numColumns());
3027     long now = EnvironmentEdgeManager.currentTimeMillis();
3028     long size = 0;
3029 
3030     // Lock row
3031     startRegionOperation();
3032     try {
3033       Integer lid = getLock(lockid, row, true);
3034       this.updatesLock.readLock().lock();
3035       try {
3036         // Process each family
3037         for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
3038           increment.getFamilyMap().entrySet()) {
3039 
3040           Store store = stores.get(family.getKey());
3041 
3042           // Get previous values for all columns in this family
3043           Get get = new Get(row);
3044           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
3045             get.addColumn(family.getKey(), column.getKey());
3046           }
3047           get.setTimeRange(tr.getMin(), tr.getMax());
3048           List<KeyValue> results = getLastIncrement(get);
3049 
3050           // Iterate the input columns and update existing values if they were
3051           // found, otherwise add new column initialized to the increment amount
3052           int idx = 0;
3053           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
3054             long amount = column.getValue();
3055             if (idx < results.size() &&
3056                 results.get(idx).matchingQualifier(column.getKey())) {
3057               amount += Bytes.toLong(results.get(idx).getValue());
3058               idx++;
3059             }
3060 
3061             // Append new incremented KeyValue to list
3062             KeyValue newKV = new KeyValue(row, family.getKey(), column.getKey(),
3063                 now, Bytes.toBytes(amount));
3064             kvs.add(newKV);
3065 
3066             // Append update to WAL
3067             if (writeToWAL) {
3068               if (walEdits == null) {
3069                 walEdits = new WALEdit();
3070               }
3071               walEdits.add(newKV);
3072             }
3073           }
3074 
3075           // Write the KVs for this family into the store
3076           size += store.upsert(kvs);
3077           allKVs.addAll(kvs);
3078           kvs.clear();
3079         }
3080 
3081         // Actually write to WAL now
3082         if (writeToWAL) {
3083           this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
3084             walEdits, now);
3085         }
3086 
3087         size = this.memstoreSize.addAndGet(size);
3088         flush = isFlushSize(size);
3089       } finally {
3090         this.updatesLock.readLock().unlock();
3091         releaseRowLock(lid);
3092       }
3093     } finally {
3094       closeRegionOperation();
3095     }
3096 
3097     if (flush) {
3098       // Request a cache flush.  Do it outside update lock.
3099       requestFlush();
3100     }
3101 
3102     return new Result(allKVs);
3103   }
3104 
3105   /**
3106    *
3107    * @param row
3108    * @param family
3109    * @param qualifier
3110    * @param amount
3111    * @param writeToWAL
3112    * @return The new value.
3113    * @throws IOException
3114    */
3115   public long incrementColumnValue(byte [] row, byte [] family,
3116       byte [] qualifier, long amount, boolean writeToWAL)
3117   throws IOException {
3118     checkRow(row);
3119     boolean flush = false;
3120     // Lock row
3121     long result = amount;
3122     startRegionOperation();
3123     try {
3124       Integer lid = obtainRowLock(row);
3125       this.updatesLock.readLock().lock();
3126       try {
3127         Store store = stores.get(family);
3128 
3129         // Get the old value:
3130         Get get = new Get(row);
3131         get.addColumn(family, qualifier);
3132 
3133         List<KeyValue> results = getLastIncrement(get);
3134 
3135         if (!results.isEmpty()) {
3136           KeyValue kv = results.get(0);
3137           byte [] buffer = kv.getBuffer();
3138           int valueOffset = kv.getValueOffset();
3139           result += Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG);
3140         }
3141 
3142         // build the KeyValue now:
3143         KeyValue newKv = new KeyValue(row, family,
3144             qualifier, EnvironmentEdgeManager.currentTimeMillis(),
3145             Bytes.toBytes(result));
3146 
3147         // now log it:
3148         if (writeToWAL) {
3149           long now = EnvironmentEdgeManager.currentTimeMillis();
3150           WALEdit walEdit = new WALEdit();
3151           walEdit.add(newKv);
3152           this.log.append(regionInfo, regionInfo.getTableDesc().getName(),
3153             walEdit, now);
3154         }
3155 
3156         // Now request the ICV to the store, this will set the timestamp
3157         // appropriately depending on if there is a value in memcache or not.
3158         // returns the change in the size of the memstore from operation
3159         long size = store.updateColumnValue(row, family, qualifier, result);
3160 
3161         size = this.memstoreSize.addAndGet(size);
3162         flush = isFlushSize(size);
3163       } finally {
3164         this.updatesLock.readLock().unlock();
3165         releaseRowLock(lid);
3166       }
3167     } finally {
3168       closeRegionOperation();
3169     }
3170 
3171     if (flush) {
3172       // Request a cache flush.  Do it outside update lock.
3173       requestFlush();
3174     }
3175 
3176     return result;
3177   }
3178 
3179 
3180   //
3181   // New HBASE-880 Helpers
3182   //
3183 
3184   private void checkFamily(final byte [] family)
3185   throws NoSuchColumnFamilyException {
3186     if(!regionInfo.getTableDesc().hasFamily(family)) {
3187       throw new NoSuchColumnFamilyException("Column family " +
3188           Bytes.toString(family) + " does not exist in region " + this
3189             + " in table " + regionInfo.getTableDesc());
3190     }
3191   }
3192 
3193   public static final long FIXED_OVERHEAD = ClassSize.align(
3194       (4 * Bytes.SIZEOF_LONG) + Bytes.SIZEOF_BOOLEAN + ClassSize.ARRAY +
3195       (22 * ClassSize.REFERENCE) + ClassSize.OBJECT + Bytes.SIZEOF_INT);
3196 
3197   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
3198       (ClassSize.OBJECT * 2) + (2 * ClassSize.ATOMIC_BOOLEAN) +
3199       ClassSize.ATOMIC_LONG + ClassSize.ATOMIC_INTEGER +
3200 
3201       // Using TreeMap for TreeSet
3202       ClassSize.TREEMAP +
3203 
3204       // Using TreeMap for HashMap
3205       ClassSize.TREEMAP +
3206 
3207       ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY +
3208       ClassSize.align(ClassSize.OBJECT +
3209         (5 * Bytes.SIZEOF_BOOLEAN)) +
3210         (3 * ClassSize.REENTRANT_LOCK));
3211 
3212   public long heapSize() {
3213     long heapSize = DEEP_OVERHEAD;
3214     for(Store store : this.stores.values()) {
3215       heapSize += store.heapSize();
3216     }
3217     return heapSize;
3218   }
3219 
3220   /*
3221    * This method calls System.exit.
3222    * @param message Message to print out.  May be null.
3223    */
3224   private static void printUsageAndExit(final String message) {
3225     if (message != null && message.length() > 0) System.out.println(message);
3226     System.out.println("Usage: HRegion CATLALOG_TABLE_DIR [major_compact]");
3227     System.out.println("Options:");
3228     System.out.println(" major_compact  Pass this option to major compact " +
3229       "passed region.");
3230     System.out.println("Default outputs scan of passed region.");
3231     System.exit(1);
3232   }
3233 
3234   /*
3235    * Process table.
3236    * Do major compaction or list content.
3237    * @param fs
3238    * @param p
3239    * @param log
3240    * @param c
3241    * @param majorCompact
3242    * @throws IOException
3243    */
3244   private static void processTable(final FileSystem fs, final Path p,
3245       final HLog log, final Configuration c,
3246       final boolean majorCompact)
3247   throws IOException {
3248     HRegion region = null;
3249     String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
3250     String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
3251     // Currently expects tables have one region only.
3252     if (p.getName().startsWith(rootStr)) {
3253       region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO, null);
3254     } else if (p.getName().startsWith(metaStr)) {
3255       region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.FIRST_META_REGIONINFO,
3256           null);
3257     } else {
3258       throw new IOException("Not a known catalog table: " + p.toString());
3259     }
3260     try {
3261       region.initialize();
3262       if (majorCompact) {
3263         region.compactStores(true);
3264       } else {
3265         // Default behavior
3266         Scan scan = new Scan();
3267         // scan.addFamily(HConstants.CATALOG_FAMILY);
3268         InternalScanner scanner = region.getScanner(scan);
3269         try {
3270           List<KeyValue> kvs = new ArrayList<KeyValue>();
3271           boolean done = false;
3272           do {
3273             kvs.clear();
3274             done = scanner.next(kvs);
3275             if (kvs.size() > 0) LOG.info(kvs);
3276           } while (done);
3277         } finally {
3278           scanner.close();
3279         }
3280         // System.out.println(region.getClosestRowBefore(Bytes.toBytes("GeneratedCSVContent2,E3652782193BC8D66A0BA1629D0FAAAB,9993372036854775807")));
3281       }
3282     } finally {
3283       region.close();
3284     }
3285   }
3286 
3287   boolean shouldForceSplit() {
3288     return this.splitRequest;
3289   }
3290 
3291   byte[] getSplitPoint() {
3292     return this.splitPoint;
3293   }
3294 
3295   void forceSplit(byte[] sp) {
3296     // NOTE : this HRegion will go away after the forced split is successfull
3297     //        therefore, no reason to clear this value
3298     this.splitRequest = true;
3299     if (sp != null) {
3300       this.splitPoint = sp;
3301     }
3302   }
3303 
3304   /**
3305    * Give the region a chance to prepare before it is split.
3306    */
3307   protected void prepareToSplit() {
3308     // nothing
3309   }
3310 
3311   /**
3312    * @return The priority that this region should have in the compaction queue
3313    */
3314   public int getCompactPriority() {
3315     int count = Integer.MAX_VALUE;
3316     for(Store store : stores.values()) {
3317       count = Math.min(count, store.getCompactPriority());
3318     }
3319     return count;
3320   }
3321 
3322   /**
3323    * Checks every store to see if one has too many
3324    * store files
3325    * @return true if any store has too many store files
3326    */
3327   public boolean hasTooManyStoreFiles() {
3328     for(Store store : stores.values()) {
3329       if(store.hasTooManyStoreFiles()) {
3330         return true;
3331       }
3332     }
3333     return false;
3334   }
3335 
3336   /**
3337    * This method needs to be called before any public call that reads or
3338    * modifies data. It has to be called just before a try.
3339    * #closeRegionOperation needs to be called in the try's finally block
3340    * Acquires a read lock and checks if the region is closing or closed.
3341    * @throws NotServingRegionException when the region is closing or closed
3342    */
3343   private void startRegionOperation() throws NotServingRegionException {
3344     if (this.closing.get()) {
3345       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
3346           " is closing");
3347     }
3348     lock.readLock().lock();
3349     if (this.closed.get()) {
3350       lock.readLock().unlock();
3351       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
3352           " is closed");
3353     }
3354   }
3355 
3356   /**
3357    * Closes the lock. This needs to be called in the finally block corresponding
3358    * to the try block of #startRegionOperation
3359    */
3360   private void closeRegionOperation(){
3361     lock.readLock().unlock();
3362   }
3363 
3364   /**
3365    * A mocked list implementaion - discards all updates.
3366    */
3367   private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
3368 
3369     @Override
3370     public void add(int index, KeyValue element) {
3371       // do nothing
3372     }
3373 
3374     @Override
3375     public boolean addAll(int index, Collection<? extends KeyValue> c) {
3376       return false; // this list is never changed as a result of an update
3377     }
3378 
3379     @Override
3380     public KeyValue get(int index) {
3381       throw new UnsupportedOperationException();
3382     }
3383 
3384     @Override
3385     public int size() {
3386       return 0;
3387     }
3388   };
3389 
3390 
3391   /**
3392    * Facility for dumping and compacting catalog tables.
3393    * Only does catalog tables since these are only tables we for sure know
3394    * schema on.  For usage run:
3395    * <pre>
3396    *   ./bin/hbase org.apache.hadoop.hbase.regionserver.HRegion
3397    * </pre>
3398    * @param args
3399    * @throws IOException
3400    */
3401   public static void main(String[] args) throws IOException {
3402     if (args.length < 1) {
3403       printUsageAndExit(null);
3404     }
3405     boolean majorCompact = false;
3406     if (args.length > 1) {
3407       if (!args[1].toLowerCase().startsWith("major")) {
3408         printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
3409       }
3410       majorCompact = true;
3411     }
3412     final Path tableDir = new Path(args[0]);
3413     final Configuration c = HBaseConfiguration.create();
3414     final FileSystem fs = FileSystem.get(c);
3415     final Path logdir = new Path(c.get("hbase.tmp.dir"),
3416         "hlog" + tableDir.getName()
3417         + EnvironmentEdgeManager.currentTimeMillis());
3418     final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
3419         HConstants.HREGION_OLDLOGDIR_NAME);
3420     final HLog log = new HLog(fs, logdir, oldLogDir, c);
3421     try {
3422       processTable(fs, tableDir, log, c, majorCompact);
3423      } finally {
3424        log.close();
3425        BlockCache bc = StoreFile.getBlockCache(c);
3426        if (bc != null) bc.shutdown();
3427      }
3428   }
3429 }