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