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