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