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