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