View Javadoc

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