View Javadoc

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