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 walSyncSuccessful = 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         addedSize += applyFamilyMapToMemstore(familyMaps[i], w);
2413       }
2414 
2415       // ------------------------------------
2416       // STEP 4. Build WAL edit
2417       // ----------------------------------
2418       Durability durability = Durability.USE_DEFAULT;
2419 
2420       for (int i = firstIndex; i < lastIndexExclusive; i++) {
2421         // Skip puts that were determined to be invalid during preprocessing
2422         if (batchOp.retCodeDetails[i].getOperationStatusCode()
2423             != OperationStatusCode.NOT_RUN) {
2424           continue;
2425         }
2426         batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2427 
2428         Mutation m = batchOp.operations[i].getFirst();
2429         Durability tmpDur = m.getDurability(); 
2430         if (tmpDur.ordinal() > durability.ordinal()) {
2431           durability = tmpDur;
2432         }
2433         if (tmpDur == Durability.SKIP_WAL) {
2434           if (m instanceof Put) {
2435             recordPutWithoutWal(m.getFamilyMap());
2436           }
2437           continue;
2438         }
2439 
2440         // Add WAL edits by CP
2441         WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
2442         if (fromCP != null) {
2443           for (KeyValue kv : fromCP.getKeyValues()) {
2444             walEdit.add(kv);
2445           }
2446         }
2447         addFamilyMapToWALEdit(familyMaps[i], walEdit);
2448       }
2449 
2450       // -------------------------
2451       // STEP 5. Append the edit to WAL. Do not sync wal.
2452       // -------------------------
2453       Mutation first = batchOp.operations[firstIndex].getFirst();
2454       walEdit.addClusterIds(first.getClusterIds());
2455       txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
2456                walEdit, first.getClusterId(), now, this.htableDescriptor);
2457 
2458       // -------------------------------
2459       // STEP 6. Release row locks, etc.
2460       // -------------------------------
2461       if (locked) {
2462         this.updatesLock.readLock().unlock();
2463         locked = false;
2464       }
2465       if (acquiredLocks != null) {
2466         for (Integer toRelease : acquiredLocks) {
2467           releaseRowLock(toRelease);
2468         }
2469         acquiredLocks = null;
2470         rowsAlreadyLocked = null;
2471       }
2472       // -------------------------
2473       // STEP 7. Sync wal.
2474       // -------------------------
2475       if (walEdit.size() > 0) {
2476         syncOrDefer(txid, durability);
2477       }
2478       walSyncSuccessful = true;
2479       // calling the post CP hook for batch mutation
2480       if (coprocessorHost != null) {
2481         MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp = 
2482           new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations, 
2483           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
2484         coprocessorHost.postBatchMutate(miniBatchOp);
2485       }
2486       
2487       // ------------------------------------------------------------------
2488       // STEP 8. Advance mvcc. This will make this put visible to scanners and getters.
2489       // ------------------------------------------------------------------
2490       if (w != null) {
2491         mvcc.completeMemstoreInsert(w);
2492         w = null;
2493       }
2494 
2495       // ------------------------------------
2496       // STEP 9. Run coprocessor post hooks. This should be done after the wal is
2497       // synced so that the coprocessor contract is adhered to.
2498       // ------------------------------------
2499       if (coprocessorHost != null) {
2500         for (int i = firstIndex; i < lastIndexExclusive; i++) {
2501           // only for successful puts
2502           if (batchOp.retCodeDetails[i].getOperationStatusCode()
2503               != OperationStatusCode.SUCCESS) {
2504             continue;
2505           }
2506           Mutation m = batchOp.operations[i].getFirst();
2507           if (m instanceof Put) {
2508             coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
2509           } else {
2510             coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
2511           }
2512         }
2513       }
2514       success = true;
2515       return addedSize;
2516     } finally {
2517 
2518       // if the wal sync was unsuccessful, remove keys from memstore
2519       if (!walSyncSuccessful) {
2520         rollbackMemstore(batchOp, familyMaps, firstIndex, lastIndexExclusive);
2521       }
2522       if (w != null) mvcc.completeMemstoreInsert(w);
2523 
2524       if (locked) {
2525         this.updatesLock.readLock().unlock();
2526       }
2527 
2528       if (acquiredLocks != null) {
2529         for (Integer toRelease : acquiredLocks) {
2530           releaseRowLock(toRelease);
2531         }
2532       }
2533 
2534       // do after lock
2535       final long netTimeMs = EnvironmentEdgeManager.currentTimeMillis()- startTimeMs;
2536 
2537       // See if the column families were consistent through the whole thing.
2538       // if they were then keep them. If they were not then pass a null.
2539       // null will be treated as unknown.
2540       // Total time taken might be involving Puts and Deletes.
2541       // Split the time for puts and deletes based on the total number of Puts and Deletes.
2542       long timeTakenForPuts = 0;
2543       if (noOfPuts > 0) {
2544         // There were some Puts in the batch.
2545         double noOfMutations = noOfPuts + noOfDeletes;
2546         timeTakenForPuts = (long) (netTimeMs * (noOfPuts / noOfMutations));
2547         final Set<byte[]> keptCfs = putsCfSetConsistent ? putsCfSet : null;
2548         this.opMetrics.updateMultiPutMetrics(keptCfs, timeTakenForPuts);
2549       }
2550       if (noOfDeletes > 0) {
2551         // There were some Deletes in the batch.
2552         final Set<byte[]> keptCfs = deletesCfSetConsistent ? deletesCfSet : null;
2553         this.opMetrics.updateMultiDeleteMetrics(keptCfs, netTimeMs - timeTakenForPuts);
2554       }
2555       if (!success) {
2556         for (int i = firstIndex; i < lastIndexExclusive; i++) {
2557           if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
2558             batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
2559           }
2560         }
2561       }
2562       batchOp.nextIndexToProcess = lastIndexExclusive;
2563     }
2564   }
2565 
2566   //TODO, Think that gets/puts and deletes should be refactored a bit so that
2567   //the getting of the lock happens before, so that you would just pass it into
2568   //the methods. So in the case of checkAndMutate you could just do lockRow,
2569   //get, put, unlockRow or something
2570  /**
2571   *
2572   * @param row
2573   * @param family
2574   * @param qualifier
2575   * @param compareOp
2576   * @param comparator
2577   * @param writeToWAL
2578   * @throws IOException
2579   * @return true if the new put was execute, false otherwise
2580   */
2581  public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2582      CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2583      boolean writeToWAL)
2584  throws IOException {
2585    return checkAndMutate(row, family, qualifier, compareOp, comparator, w, null, writeToWAL);
2586  }
2587   
2588   /**
2589    *
2590    * @param row
2591    * @param family
2592    * @param qualifier
2593    * @param compareOp
2594    * @param comparator
2595    * @param lockId
2596    * @param writeToWAL
2597    * @throws IOException
2598    * @return true if the new put was execute, false otherwise
2599    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
2600    */
2601   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2602       CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2603       Integer lockId, boolean writeToWAL)
2604   throws IOException{
2605     checkReadOnly();
2606     //TODO, add check for value length or maybe even better move this to the
2607     //client if this becomes a global setting
2608     checkResources();
2609     boolean isPut = w instanceof Put;
2610     if (!isPut && !(w instanceof Delete))
2611       throw new DoNotRetryIOException("Action must be Put or Delete");
2612     Row r = (Row)w;
2613     if (!Bytes.equals(row, r.getRow())) {
2614       throw new DoNotRetryIOException("Action's getRow must match the passed row");
2615     }
2616 
2617     startRegionOperation();
2618     this.writeRequestsCount.increment();
2619     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2620     try {
2621       RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
2622       Get get = new Get(row, lock);
2623       checkFamily(family);
2624       get.addColumn(family, qualifier);
2625 
2626       // Lock row
2627       Integer lid = getLock(lockId, get.getRow(), true);
2628       // wait for all previous transactions to complete (with lock held)
2629       mvcc.completeMemstoreInsert(mvcc.beginMemstoreInsert());
2630       List<KeyValue> result = new ArrayList<KeyValue>();
2631       try {
2632         result = get(get, false);
2633 
2634         boolean valueIsNull = comparator.getValue() == null ||
2635           comparator.getValue().length == 0;
2636         boolean matches = false;
2637         if (result.size() == 0 && valueIsNull) {
2638           matches = true;
2639         } else if (result.size() > 0 && result.get(0).getValue().length == 0 &&
2640             valueIsNull) {
2641           matches = true;
2642         } else if (result.size() == 1 && !valueIsNull) {
2643           KeyValue kv = result.get(0);
2644           int compareResult = comparator.compareTo(kv.getBuffer(),
2645               kv.getValueOffset(), kv.getValueLength());
2646           switch (compareOp) {
2647           case LESS:
2648             matches = compareResult <= 0;
2649             break;
2650           case LESS_OR_EQUAL:
2651             matches = compareResult < 0;
2652             break;
2653           case EQUAL:
2654             matches = compareResult == 0;
2655             break;
2656           case NOT_EQUAL:
2657             matches = compareResult != 0;
2658             break;
2659           case GREATER_OR_EQUAL:
2660             matches = compareResult > 0;
2661             break;
2662           case GREATER:
2663             matches = compareResult >= 0;
2664             break;
2665           default:
2666             throw new RuntimeException("Unknown Compare op " + compareOp.name());
2667           }
2668         }
2669         //If matches put the new put or delete the new delete
2670         if (matches) {
2671           // All edits for the given row (across all column families) must
2672           // happen atomically.
2673           //
2674           // Using default cluster id, as this can only happen in the
2675           // originating cluster. A slave cluster receives the result as a Put
2676           // or Delete
2677           if (isPut) {
2678             internalPut(((Put) w), HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2679           } else {
2680             Delete d = (Delete)w;
2681             prepareDelete(d);
2682             internalDelete(d, HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2683           }
2684           return true;
2685         }
2686         return false;
2687       } finally {
2688         if(lockId == null) releaseRowLock(lid);
2689       }
2690     } finally {
2691       closeRegionOperation();
2692     }
2693   }
2694 
2695 
2696   /**
2697    * Complete taking the snapshot on the region. Writes the region info and adds references to the
2698    * working snapshot directory.
2699    *
2700    * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
2701    * arg.  (In the future other cancellable HRegion methods could eventually add a
2702    * {@link ForeignExceptionSnare}, or we could do something fancier).
2703    *
2704    * @param desc snasphot description object
2705    * @param exnSnare ForeignExceptionSnare that captures external exeptions in case we need to
2706    *   bail out.  This is allowed to be null and will just be ignored in that case.
2707    * @throws IOException if there is an external or internal error causing the snapshot to fail
2708    */
2709   public void addRegionToSnapshot(SnapshotDescription desc,
2710       ForeignExceptionSnare exnSnare) throws IOException {
2711     // This should be "fast" since we don't rewrite store files but instead
2712     // back up the store files by creating a reference
2713     Path rootDir = FSUtils.getRootDir(this.rsServices.getConfiguration());
2714     Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(desc, rootDir,
2715       regionInfo.getEncodedName());
2716 
2717     // 1. dump region meta info into the snapshot directory
2718     LOG.debug("Storing region-info for snapshot.");
2719     checkRegioninfoOnFilesystem(snapshotRegionDir);
2720 
2721     // 2. iterate through all the stores in the region
2722     LOG.debug("Creating references for hfiles");
2723 
2724     // This ensures that we have an atomic view of the directory as long as we have < ls limit
2725     // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files in
2726     // batches and may miss files being added/deleted. This could be more robust (iteratively
2727     // checking to see if we have all the files until we are sure), but the limit is currently 1000
2728     // files/batch, far more than the number of store files under a single column family.
2729     for (Store store : stores.values()) {
2730       // 2.1. build the snapshot reference directory for the store
2731       Path dstStoreDir = TakeSnapshotUtils.getStoreSnapshotDirectory(snapshotRegionDir,
2732         Bytes.toString(store.getFamily().getName()));
2733       List<StoreFile> storeFiles = store.getStorefiles();
2734       if (LOG.isDebugEnabled()) {
2735         LOG.debug("Adding snapshot references for " + storeFiles  + " hfiles");
2736       }
2737 
2738       // 2.2. iterate through all the store's files and create "references".
2739       int sz = storeFiles.size();
2740       for (int i = 0; i < sz; i++) {
2741         if (exnSnare != null) {
2742           exnSnare.rethrowException();
2743         }
2744         StoreFile storeFile = storeFiles.get(i);
2745         Path file = storeFile.getPath();
2746 
2747         LOG.debug("Creating reference for file (" + (i+1) + "/" + sz + ") : " + file);
2748         Path referenceFile = new Path(dstStoreDir, file.getName());
2749         boolean success = true;
2750         if (storeFile.isReference()) {
2751           // write the Reference object to the snapshot
2752           storeFile.getReference().write(fs, referenceFile);
2753         } else {
2754           // create "reference" to this store file.  It is intentionally an empty file -- all
2755           // necessary information is captured by its fs location and filename.  This allows us to
2756           // only figure out what needs to be done via a single nn operation (instead of having to
2757           // open and read the files as well).
2758           success = HBaseFileSystem.createNewFileOnFileSystem(fs, referenceFile);
2759         }
2760         if (!success) {
2761           throw new IOException("Failed to create reference file:" + referenceFile);
2762         }
2763       }
2764     }
2765   }
2766 
2767   /**
2768    * Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP} with the provided current
2769    * timestamp.
2770    */
2771   private void updateKVTimestamps(
2772       final Iterable<List<KeyValue>> keyLists, final byte[] now) {
2773     for (List<KeyValue> keys: keyLists) {
2774       if (keys == null) continue;
2775       for (KeyValue key : keys) {
2776         key.updateLatestStamp(now);
2777       }
2778     }
2779   }
2780 
2781   /*
2782    * Check if resources to support an update.
2783    *
2784    * Here we synchronize on HRegion, a broad scoped lock.  Its appropriate
2785    * given we're figuring in here whether this region is able to take on
2786    * writes.  This is only method with a synchronize (at time of writing),
2787    * this and the synchronize on 'this' inside in internalFlushCache to send
2788    * the notify.
2789    */
2790   private void checkResources()
2791       throws RegionTooBusyException, InterruptedIOException {
2792 
2793     // If catalog region, do not impose resource constraints or block updates.
2794     if (this.getRegionInfo().isMetaRegion()) return;
2795 
2796     boolean blocked = false;
2797     long startTime = 0;
2798     while (this.memstoreSize.get() > this.blockingMemStoreSize) {
2799       requestFlush();
2800       if (!blocked) {
2801         startTime = EnvironmentEdgeManager.currentTimeMillis();
2802         LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
2803           "' on region " + Bytes.toStringBinary(getRegionName()) +
2804           ": memstore size " +
2805           StringUtils.humanReadableInt(this.memstoreSize.get()) +
2806           " is >= than blocking " +
2807           StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
2808       }
2809       long now = EnvironmentEdgeManager.currentTimeMillis();
2810       long timeToWait = startTime + busyWaitDuration - now;
2811       if (timeToWait <= 0L) {
2812         final long totalTime = now - startTime;
2813         this.updatesBlockedMs.add(totalTime);
2814         LOG.info("Failed to unblock updates for region " + this + " '"
2815           + Thread.currentThread().getName() + "' in " + totalTime
2816           + "ms. The region is still busy.");
2817         throw new RegionTooBusyException("region is flushing");
2818       }
2819       blocked = true;
2820       synchronized(this) {
2821         try {
2822           wait(Math.min(timeToWait, threadWakeFrequency));
2823         } catch (InterruptedException ie) {
2824           final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2825           if (totalTime > 0) {
2826             this.updatesBlockedMs.add(totalTime);
2827           }
2828           LOG.info("Interrupted while waiting to unblock updates for region "
2829             + this + " '" + Thread.currentThread().getName() + "'");
2830           InterruptedIOException iie = new InterruptedIOException();
2831           iie.initCause(ie);
2832           throw iie;
2833         }
2834       }
2835     }
2836     if (blocked) {
2837       // Add in the blocked time if appropriate
2838       final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2839       if(totalTime > 0 ){
2840         this.updatesBlockedMs.add(totalTime);
2841       }
2842       LOG.info("Unblocking updates for region " + this + " '"
2843           + Thread.currentThread().getName() + "'");
2844     }
2845   }
2846 
2847   /**
2848    * @throws IOException Throws exception if region is in read-only mode.
2849    */
2850   protected void checkReadOnly() throws IOException {
2851     if (this.writestate.isReadOnly()) {
2852       throw new IOException("region is read only");
2853     }
2854   }
2855 
2856   /**
2857    * Add updates first to the hlog and then add values to memstore.
2858    * Warning: Assumption is caller has lock on passed in row.
2859    * @param family
2860    * @param edits Cell updates by column
2861    * @praram now
2862    * @throws IOException
2863    */
2864   private void put(byte [] family, List<KeyValue> edits)
2865   throws IOException {
2866     Map<byte[], List<KeyValue>> familyMap;
2867     familyMap = new HashMap<byte[], List<KeyValue>>();
2868 
2869     familyMap.put(family, edits);
2870     Put p = new Put();
2871     p.setFamilyMap(familyMap);
2872     p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
2873     p.setWriteToWAL(true);
2874     this.internalPut(p, HConstants.DEFAULT_CLUSTER_ID, true);
2875   }
2876 
2877   /**
2878    * Add updates first to the hlog (if writeToWal) and then add values to memstore.
2879    * Warning: Assumption is caller has lock on passed in row.
2880    * @param put The Put command
2881    * @param clusterId UUID of the originating cluster (for replication).
2882    * @param writeToWAL if true, then we should write to the log
2883    * @throws IOException
2884    */
2885   private void internalPut(Put put, UUID clusterId, boolean writeToWAL) throws IOException {
2886     Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
2887     WALEdit walEdit = new WALEdit();
2888     /* run pre put hook outside of lock to avoid deadlock */
2889     if (coprocessorHost != null) {
2890       if (coprocessorHost.prePut(put, walEdit, writeToWAL)) {
2891         return;
2892       }
2893     }
2894 
2895     long now = EnvironmentEdgeManager.currentTimeMillis();
2896     byte[] byteNow = Bytes.toBytes(now);
2897     boolean flush = false;
2898 
2899     lock(this.updatesLock.readLock());
2900     try {
2901       checkFamilies(familyMap.keySet());
2902       checkTimestamps(familyMap, now);
2903       updateKVTimestamps(familyMap.values(), byteNow);
2904       // write/sync to WAL should happen before we touch memstore.
2905       //
2906       // If order is reversed, i.e. we write to memstore first, and
2907       // for some reason fail to write/sync to commit log, the memstore
2908       // will contain uncommitted transactions.
2909       if (writeToWAL) {
2910         addFamilyMapToWALEdit(familyMap, walEdit);
2911         walEdit.addClusterIds(put.getClusterIds());
2912         this.log.append(regionInfo, this.htableDescriptor.getName(),
2913             walEdit, clusterId, now, this.htableDescriptor);
2914       } else {
2915         recordPutWithoutWal(familyMap);
2916       }
2917 
2918       long addedSize = applyFamilyMapToMemstore(familyMap, null);
2919       flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
2920     } finally {
2921       this.updatesLock.readLock().unlock();
2922     }
2923 
2924     if (coprocessorHost != null) {
2925       coprocessorHost.postPut(put, walEdit, writeToWAL);
2926     }
2927 
2928     // do after lock
2929     final long after = EnvironmentEdgeManager.currentTimeMillis();
2930     this.opMetrics.updatePutMetrics(familyMap.keySet(), after - now);
2931 
2932     if (flush) {
2933       // Request a cache flush.  Do it outside update lock.
2934       requestFlush();
2935     }
2936   }
2937 
2938   /**
2939    * Atomically apply the given map of family->edits to the memstore.
2940    * This handles the consistency control on its own, but the caller
2941    * should already have locked updatesLock.readLock(). This also does
2942    * <b>not</b> check the families for validity.
2943    *
2944    * @param familyMap Map of kvs per family
2945    * @param localizedWriteEntry The WriteEntry of the MVCC for this transaction.
2946    *        If null, then this method internally creates a mvcc transaction.
2947    * @return the additional memory usage of the memstore caused by the
2948    * new entries.
2949    */
2950   private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap,
2951     MultiVersionConsistencyControl.WriteEntry localizedWriteEntry) {
2952     long size = 0;
2953     boolean freemvcc = false;
2954 
2955     try {
2956       if (localizedWriteEntry == null) {
2957         localizedWriteEntry = mvcc.beginMemstoreInsert();
2958         freemvcc = true;
2959       }
2960 
2961       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2962         byte[] family = e.getKey();
2963         List<KeyValue> edits = e.getValue();
2964 
2965         Store store = getStore(family);
2966         for (KeyValue kv: edits) {
2967           kv.setMemstoreTS(localizedWriteEntry.getWriteNumber());
2968           size += store.add(kv);
2969         }
2970       }
2971     } finally {
2972       if (freemvcc) {
2973         mvcc.completeMemstoreInsert(localizedWriteEntry);
2974       }
2975     }
2976 
2977      return size;
2978    }
2979 
2980   /**
2981    * Remove all the keys listed in the map from the memstore. This method is
2982    * called when a Put/Delete has updated memstore but subequently fails to update
2983    * the wal. This method is then invoked to rollback the memstore.
2984    */
2985   private void rollbackMemstore(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp,
2986                                 Map<byte[], List<KeyValue>>[] familyMaps,
2987                                 int start, int end) {
2988     int kvsRolledback = 0;
2989     for (int i = start; i < end; i++) {
2990       // skip over request that never succeeded in the first place.
2991       if (batchOp.retCodeDetails[i].getOperationStatusCode()
2992             != OperationStatusCode.SUCCESS) {
2993         continue;
2994       }
2995 
2996       // Rollback all the kvs for this row.
2997       Map<byte[], List<KeyValue>> familyMap  = familyMaps[i];
2998       for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2999         byte[] family = e.getKey();
3000         List<KeyValue> edits = e.getValue();
3001 
3002         // Remove those keys from the memstore that matches our
3003         // key's (row, cf, cq, timestamp, memstoreTS). The interesting part is
3004         // that even the memstoreTS has to match for keys that will be rolleded-back.
3005         Store store = getStore(family);
3006         for (KeyValue kv: edits) {
3007           store.rollback(kv);
3008           kvsRolledback++;
3009         }
3010       }
3011     }
3012     LOG.debug("rollbackMemstore rolled back " + kvsRolledback +
3013         " keyvalues from start:" + start + " to end:" + end);
3014   }
3015 
3016   /**
3017    * Check the collection of families for validity.
3018    * @throws NoSuchColumnFamilyException if a family does not exist.
3019    */
3020   private void checkFamilies(Collection<byte[]> families)
3021   throws NoSuchColumnFamilyException {
3022     for (byte[] family : families) {
3023       checkFamily(family);
3024     }
3025   }
3026 
3027   private void checkTimestamps(final Map<byte[], List<KeyValue>> familyMap,
3028       long now) throws DoNotRetryIOException {
3029     if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
3030       return;
3031     }
3032     long maxTs = now + timestampSlop;
3033     for (List<KeyValue> kvs : familyMap.values()) {
3034       for (KeyValue kv : kvs) {
3035         // see if the user-side TS is out of range. latest = server-side
3036         if (!kv.isLatestTimestamp() && kv.getTimestamp() > maxTs) {
3037           throw new DoNotRetryIOException("Timestamp for KV out of range "
3038               + kv + " (too.new=" + timestampSlop + ")");
3039         }
3040       }
3041     }
3042   }
3043 
3044   /**
3045    * Append the given map of family->edits to a WALEdit data structure.
3046    * This does not write to the HLog itself.
3047    * @param familyMap map of family->edits
3048    * @param walEdit the destination entry to append into
3049    */
3050   private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
3051       WALEdit walEdit) {
3052     for (List<KeyValue> edits : familyMap.values()) {
3053       for (KeyValue kv : edits) {
3054         walEdit.add(kv);
3055       }
3056     }
3057   }
3058 
3059   private void requestFlush() {
3060     if (this.rsServices == null) {
3061       return;
3062     }
3063     synchronized (writestate) {
3064       if (this.writestate.isFlushRequested()) {
3065         return;
3066       }
3067       writestate.flushRequested = true;
3068     }
3069     // Make request outside of synchronize block; HBASE-818.
3070     this.rsServices.getFlushRequester().requestFlush(this);
3071     if (LOG.isDebugEnabled()) {
3072       LOG.debug("Flush requested on " + this);
3073     }
3074   }
3075 
3076   /*
3077    * @param size
3078    * @return True if size is over the flush threshold
3079    */
3080   private boolean isFlushSize(final long size) {
3081     return size > this.memstoreFlushSize;
3082   }
3083 
3084   /**
3085    * Read the edits log put under this region by wal log splitting process.  Put
3086    * the recovered edits back up into this region.
3087    *
3088    * <p>We can ignore any log message that has a sequence ID that's equal to or
3089    * lower than minSeqId.  (Because we know such log messages are already
3090    * reflected in the HFiles.)
3091    *
3092    * <p>While this is running we are putting pressure on memory yet we are
3093    * outside of our usual accounting because we are not yet an onlined region
3094    * (this stuff is being run as part of Region initialization).  This means
3095    * that if we're up against global memory limits, we'll not be flagged to flush
3096    * because we are not online. We can't be flushed by usual mechanisms anyways;
3097    * we're not yet online so our relative sequenceids are not yet aligned with
3098    * HLog sequenceids -- not till we come up online, post processing of split
3099    * edits.
3100    *
3101    * <p>But to help relieve memory pressure, at least manage our own heap size
3102    * flushing if are in excess of per-region limits.  Flushing, though, we have
3103    * to be careful and avoid using the regionserver/hlog sequenceid.  Its running
3104    * on a different line to whats going on in here in this region context so if we
3105    * crashed replaying these edits, but in the midst had a flush that used the
3106    * regionserver log with a sequenceid in excess of whats going on in here
3107    * in this region and with its split editlogs, then we could miss edits the
3108    * next time we go to recover. So, we have to flush inline, using seqids that
3109    * make sense in a this single region context only -- until we online.
3110    *
3111    * @param regiondir
3112    * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of
3113    * the maxSeqId for the store to be applied, else its skipped.
3114    * @param reporter
3115    * @return the sequence id of the last edit added to this region out of the
3116    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3117    * @throws UnsupportedEncodingException
3118    * @throws IOException
3119    */
3120   protected long replayRecoveredEditsIfAny(final Path regiondir,
3121       Map<byte[], Long> maxSeqIdInStores,
3122       final CancelableProgressable reporter, final MonitoredTask status)
3123       throws UnsupportedEncodingException, IOException {
3124     long minSeqIdForTheRegion = -1;
3125     for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
3126       if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
3127         minSeqIdForTheRegion = maxSeqIdInStore;
3128       }
3129     }
3130     long seqid = minSeqIdForTheRegion;
3131     NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
3132     if (files == null || files.isEmpty()) return seqid;
3133 
3134     for (Path edits: files) {
3135       if (edits == null || !this.fs.exists(edits)) {
3136         LOG.warn("Null or non-existent edits file: " + edits);
3137         continue;
3138       }
3139       if (isZeroLengthThenDelete(this.fs, edits)) continue;
3140 
3141       long maxSeqId = Long.MAX_VALUE;
3142       String fileName = edits.getName();
3143       maxSeqId = Math.abs(Long.parseLong(fileName));
3144       if (maxSeqId <= minSeqIdForTheRegion) {
3145         String msg = "Maximum sequenceid for this log is " + maxSeqId
3146             + " and minimum sequenceid for the region is " + minSeqIdForTheRegion
3147             + ", skipped the whole file, path=" + edits;
3148         LOG.debug(msg);
3149         continue;
3150       }
3151 
3152       try {
3153         seqid = replayRecoveredEdits(edits, maxSeqIdInStores, reporter);
3154       } catch (IOException e) {
3155         boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
3156         if (skipErrors) {
3157           Path p = HLog.moveAsideBadEditsFile(fs, edits);
3158           LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits +
3159             " as " + p, e);
3160         } else {
3161           throw e;
3162         }
3163       }
3164       // The edits size added into rsAccounting during this replaying will not
3165       // be required any more. So just clear it.
3166       if (this.rsAccounting != null) {
3167         this.rsAccounting.clearRegionReplayEditsSize(this.regionInfo.getRegionName());
3168       }
3169     }
3170     if (seqid > minSeqIdForTheRegion) {
3171       // Then we added some edits to memory. Flush and cleanup split edit files.
3172       internalFlushcache(null, seqid, status);
3173     }
3174     // Now delete the content of recovered edits.  We're done w/ them.
3175     for (Path file: files) {
3176       if (!HBaseFileSystem.deleteFileFromFileSystem(fs, file)) {
3177         LOG.error("Failed delete of " + file);
3178       } else {
3179         LOG.debug("Deleted recovered.edits file=" + file);
3180       }
3181     }
3182     return seqid;
3183   }
3184 
3185   /*
3186    * @param edits File of recovered edits.
3187    * @param maxSeqIdInStores Maximum sequenceid found in each store.  Edits in log
3188    * must be larger than this to be replayed for each store.
3189    * @param reporter
3190    * @return the sequence id of the last edit added to this region out of the
3191    * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
3192    * @throws IOException
3193    */
3194   private long replayRecoveredEdits(final Path edits,
3195       Map<byte[], Long> maxSeqIdInStores, final CancelableProgressable reporter)
3196     throws IOException {
3197     String msg = "Replaying edits from " + edits;
3198     LOG.info(msg);
3199     MonitoredTask status = TaskMonitor.get().createStatus(msg);
3200 
3201     status.setStatus("Opening logs");
3202     HLog.Reader reader = null;
3203     try {
3204       reader = HLog.getReader(this.fs, edits, conf);
3205       long currentEditSeqId = -1;
3206       long firstSeqIdInLog = -1;
3207       long skippedEdits = 0;
3208       long editsCount = 0;
3209       long intervalEdits = 0;
3210       HLog.Entry entry;
3211       Store store = null;
3212       boolean reported_once = false;
3213 
3214       try {
3215         // How many edits seen before we check elapsed time
3216         int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
3217             2000);
3218         // How often to send a progress report (default 1/2 master timeout)
3219         int period = this.conf.getInt("hbase.hstore.report.period",
3220             this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
3221                 180000) / 2);
3222         long lastReport = EnvironmentEdgeManager.currentTimeMillis();
3223 
3224         while ((entry = reader.next()) != null) {
3225           HLogKey key = entry.getKey();
3226           WALEdit val = entry.getEdit();
3227 
3228           if (reporter != null) {
3229             intervalEdits += val.size();
3230             if (intervalEdits >= interval) {
3231               // Number of edits interval reached
3232               intervalEdits = 0;
3233               long cur = EnvironmentEdgeManager.currentTimeMillis();
3234               if (lastReport + period <= cur) {
3235                 status.setStatus("Replaying edits..." +
3236                     " skipped=" + skippedEdits +
3237                     " edits=" + editsCount);
3238                 // Timeout reached
3239                 if(!reporter.progress()) {
3240                   msg = "Progressable reporter failed, stopping replay";
3241                   LOG.warn(msg);
3242                   status.abort(msg);
3243                   throw new IOException(msg);
3244                 }
3245                 reported_once = true;
3246                 lastReport = cur;
3247               }
3248             }
3249           }
3250 
3251           // Start coprocessor replay here. The coprocessor is for each WALEdit
3252           // instead of a KeyValue.
3253           if (coprocessorHost != null) {
3254             status.setStatus("Running pre-WAL-restore hook in coprocessors");
3255             if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
3256               // if bypass this log entry, ignore it ...
3257               continue;
3258             }
3259           }
3260 
3261           if (firstSeqIdInLog == -1) {
3262             firstSeqIdInLog = key.getLogSeqNum();
3263           }
3264           boolean flush = false;
3265           for (KeyValue kv: val.getKeyValues()) {
3266             // Check this edit is for me. Also, guard against writing the special
3267             // METACOLUMN info such as HBASE::CACHEFLUSH entries
3268             if (kv.matchingFamily(HLog.METAFAMILY) ||
3269                 !Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
3270               skippedEdits++;
3271               continue;
3272                 }
3273             // Figure which store the edit is meant for.
3274             if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
3275               store = this.stores.get(kv.getFamily());
3276             }
3277             if (store == null) {
3278               // This should never happen.  Perhaps schema was changed between
3279               // crash and redeploy?
3280               LOG.warn("No family for " + kv);
3281               skippedEdits++;
3282               continue;
3283             }
3284             // Now, figure if we should skip this edit.
3285             if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
3286                 .getName())) {
3287               skippedEdits++;
3288               continue;
3289             }
3290             currentEditSeqId = key.getLogSeqNum();
3291             // Once we are over the limit, restoreEdit will keep returning true to
3292             // flush -- but don't flush until we've played all the kvs that make up
3293             // the WALEdit.
3294             flush = restoreEdit(store, kv);
3295             editsCount++;
3296           }
3297           if (flush) internalFlushcache(null, currentEditSeqId, status);
3298 
3299           if (coprocessorHost != null) {
3300             coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
3301           }
3302         }
3303       } catch (EOFException eof) {
3304         Path p = HLog.moveAsideBadEditsFile(fs, edits);
3305         msg = "Encountered EOF. Most likely due to Master failure during " +
3306             "log spliting, so we have this data in another edit.  " +
3307             "Continuing, but renaming " + edits + " as " + p;
3308         LOG.warn(msg, eof);
3309         status.abort(msg);
3310       } catch (IOException ioe) {
3311         // If the IOE resulted from bad file format,
3312         // then this problem is idempotent and retrying won't help
3313         if (ioe.getCause() instanceof ParseException) {
3314           Path p = HLog.moveAsideBadEditsFile(fs, edits);
3315           msg = "File corruption encountered!  " +
3316               "Continuing, but renaming " + edits + " as " + p;
3317           LOG.warn(msg, ioe);
3318           status.setStatus(msg);
3319         } else {
3320           status.abort(StringUtils.stringifyException(ioe));
3321           // other IO errors may be transient (bad network connection,
3322           // checksum exception on one datanode, etc).  throw & retry
3323           throw ioe;
3324         }
3325       }
3326       if (reporter != null && !reported_once) {
3327         reporter.progress();
3328       }
3329       msg = "Applied " + editsCount + ", skipped " + skippedEdits +
3330         ", firstSequenceidInLog=" + firstSeqIdInLog +
3331         ", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
3332       status.markComplete(msg);
3333       LOG.debug(msg);
3334       return currentEditSeqId;
3335     } finally {
3336       status.cleanup();
3337       if (reader != null) {
3338          reader.close();
3339       }
3340     }
3341   }
3342 
3343   /**
3344    * Used by tests
3345    * @param s Store to add edit too.
3346    * @param kv KeyValue to add.
3347    * @return True if we should flush.
3348    */
3349   protected boolean restoreEdit(final Store s, final KeyValue kv) {
3350     long kvSize = s.add(kv);
3351     if (this.rsAccounting != null) {
3352       rsAccounting.addAndGetRegionReplayEditsSize(this.regionInfo.getRegionName(), kvSize);
3353     }
3354     return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
3355   }
3356 
3357   /*
3358    * @param fs
3359    * @param p File to check.
3360    * @return True if file was zero-length (and if so, we'll delete it in here).
3361    * @throws IOException
3362    */
3363   private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
3364       throws IOException {
3365     FileStatus stat = fs.getFileStatus(p);
3366     if (stat.getLen() > 0) return false;
3367     LOG.warn("File " + p + " is zero-length, deleting.");
3368     HBaseFileSystem.deleteFileFromFileSystem(fs, p);
3369     return true;
3370   }
3371 
3372   protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
3373       throws IOException {
3374     return new Store(tableDir, this, c, this.fs, this.conf);
3375   }
3376 
3377   /**
3378    * Return HStore instance.
3379    * Use with caution.  Exposed for use of fixup utilities.
3380    * @param column Name of column family hosted by this region.
3381    * @return Store that goes with the family on passed <code>column</code>.
3382    * TODO: Make this lookup faster.
3383    */
3384   public Store getStore(final byte [] column) {
3385     return this.stores.get(column);
3386   }
3387 
3388   public Map<byte[], Store> getStores() {
3389     return this.stores;
3390   }
3391 
3392   /**
3393    * Return list of storeFiles for the set of CFs.
3394    * Uses closeLock to prevent the race condition where a region closes
3395    * in between the for loop - closing the stores one by one, some stores
3396    * will return 0 files.
3397    * @return List of storeFiles.
3398    */
3399   public List<String> getStoreFileList(final byte [][] columns)
3400     throws IllegalArgumentException {
3401     List<String> storeFileNames = new ArrayList<String>();
3402     synchronized(closeLock) {
3403       for(byte[] column : columns) {
3404         Store store = this.stores.get(column);
3405         if (store == null) {
3406           throw new IllegalArgumentException("No column family : " +
3407               new String(column) + " available");
3408         }
3409         List<StoreFile> storeFiles = store.getStorefiles();
3410         for (StoreFile storeFile: storeFiles) {
3411           storeFileNames.add(storeFile.getPath().toString());
3412         }
3413       }
3414     }
3415     return storeFileNames;
3416   }
3417   //////////////////////////////////////////////////////////////////////////////
3418   // Support code
3419   //////////////////////////////////////////////////////////////////////////////
3420 
3421   /** Make sure this is a valid row for the HRegion */
3422   void checkRow(final byte [] row, String op) throws IOException {
3423     if(!rowIsInRange(regionInfo, row)) {
3424       throw new WrongRegionException("Requested row out of range for " +
3425           op + " on HRegion " + this + ", startKey='" +
3426           Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
3427           Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
3428           Bytes.toStringBinary(row) + "'");
3429     }
3430   }
3431 
3432   /**
3433    * Obtain a lock on the given row.  Blocks until success.
3434    *
3435    * I know it's strange to have two mappings:
3436    * <pre>
3437    *   ROWS  ==> LOCKS
3438    * </pre>
3439    * as well as
3440    * <pre>
3441    *   LOCKS ==> ROWS
3442    * </pre>
3443    *
3444    * But it acts as a guard on the client; a miswritten client just can't
3445    * submit the name of a row and start writing to it; it must know the correct
3446    * lockid, which matches the lock list in memory.
3447    *
3448    * <p>It would be more memory-efficient to assume a correctly-written client,
3449    * which maybe we'll do in the future.
3450    *
3451    * @param row Name of row to lock.
3452    * @throws IOException
3453    * @return The id of the held lock.
3454    */
3455   public Integer obtainRowLock(final byte [] row) throws IOException {
3456     startRegionOperation();
3457     this.writeRequestsCount.increment();
3458     this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3459     try {
3460       return internalObtainRowLock(new HashedBytes(row), true);
3461     } finally {
3462       closeRegionOperation();
3463     }
3464   }
3465 
3466   /**
3467    * Obtains or tries to obtain the given row lock.
3468    * @param waitForLock if true, will block until the lock is available.
3469    *        Otherwise, just tries to obtain the lock and returns
3470    *        null if unavailable.
3471    */
3472   private Integer internalObtainRowLock(final HashedBytes rowKey, boolean waitForLock)
3473       throws IOException {
3474     checkRow(rowKey.getBytes(), "row lock");
3475     startRegionOperation();
3476     try {
3477       CountDownLatch rowLatch = new CountDownLatch(1);
3478 
3479       // loop until we acquire the row lock (unless !waitForLock)
3480       while (true) {
3481         CountDownLatch existingLatch = lockedRows.putIfAbsent(rowKey, rowLatch);
3482         if (existingLatch == null) {
3483           break;
3484         } else {
3485           // row already locked
3486           if (!waitForLock) {
3487             return null;
3488           }
3489           try {
3490             if (!existingLatch.await(this.rowLockWaitDuration,
3491                             TimeUnit.MILLISECONDS)) {
3492               throw new IOException("Timed out on getting lock for row=" + rowKey);
3493             }
3494           } catch (InterruptedException ie) {
3495             // Empty
3496           }
3497         }
3498       }
3499 
3500       // loop until we generate an unused lock id
3501       while (true) {
3502         Integer lockId = lockIdGenerator.incrementAndGet();
3503         HashedBytes existingRowKey = lockIds.putIfAbsent(lockId, rowKey);
3504         if (existingRowKey == null) {
3505           return lockId;
3506         } else {
3507           // lockId already in use, jump generator to a new spot
3508           lockIdGenerator.set(rand.nextInt());
3509         }
3510       }
3511     } finally {
3512       closeRegionOperation();
3513     }
3514   }
3515 
3516   /**
3517    * Used by unit tests.
3518    * @param lockid
3519    * @return Row that goes with <code>lockid</code>
3520    */
3521   byte[] getRowFromLock(final Integer lockid) {
3522     HashedBytes rowKey = lockIds.get(lockid);
3523     return rowKey == null ? null : rowKey.getBytes();
3524   }
3525 
3526   /**
3527    * Release the row lock!
3528    * @param lockId  The lock ID to release.
3529    */
3530   public void releaseRowLock(final Integer lockId) {
3531     if (lockId == null) return; // null lock id, do nothing
3532     HashedBytes rowKey = lockIds.remove(lockId);
3533     if (rowKey == null) {
3534       LOG.warn("Release unknown lockId: " + lockId);
3535       return;
3536     }
3537     CountDownLatch rowLatch = lockedRows.remove(rowKey);
3538     if (rowLatch == null) {
3539       LOG.error("Releases row not locked, lockId: " + lockId + " row: "
3540           + rowKey);
3541       return;
3542     }
3543     rowLatch.countDown();
3544   }
3545 
3546   /**
3547    * See if row is currently locked.
3548    * @param lockid
3549    * @return boolean
3550    */
3551   boolean isRowLocked(final Integer lockId) {
3552     return lockIds.containsKey(lockId);
3553   }
3554 
3555   /**
3556    * Returns existing row lock if found, otherwise
3557    * obtains a new row lock and returns it.
3558    * @param lockid requested by the user, or null if the user didn't already hold lock
3559    * @param row the row to lock
3560    * @param waitForLock if true, will block until the lock is available, otherwise will
3561    * simply return null if it could not acquire the lock.
3562    * @return lockid or null if waitForLock is false and the lock was unavailable.
3563    */
3564   public Integer getLock(Integer lockid, byte [] row, boolean waitForLock)
3565   throws IOException {
3566     return getLock(lockid, new HashedBytes(row), waitForLock);
3567   }
3568 
3569   /**
3570    * Returns existing row lock if found, otherwise
3571    * obtains a new row lock and returns it.
3572    * @param lockid requested by the user, or null if the user didn't already hold lock
3573    * @param row the row to lock
3574    * @param waitForLock if true, will block until the lock is available, otherwise will
3575    * simply return null if it could not acquire the lock.
3576    * @return lockid or null if waitForLock is false and the lock was unavailable.
3577    */
3578   protected Integer getLock(Integer lockid, HashedBytes row, boolean waitForLock)
3579   throws IOException {
3580     Integer lid;
3581     if (lockid == null) {
3582       lid = internalObtainRowLock(row, waitForLock);
3583     } else {
3584       HashedBytes rowFromLock = lockIds.get(lockid);
3585       if (!row.equals(rowFromLock)) {
3586         throw new IOException("Invalid row lock: LockId: " + lockid + " holds the lock for row: " + rowFromLock + " but wanted lock for row: " + row);
3587       }
3588       lid = lockid;
3589     }
3590     return lid;
3591   }
3592 
3593   /**
3594    * Determines whether multiple column families are present
3595    * Precondition: familyPaths is not null
3596    *
3597    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3598    */
3599   private static boolean hasMultipleColumnFamilies(
3600       List<Pair<byte[], String>> familyPaths) {
3601     boolean multipleFamilies = false;
3602     byte[] family = null;
3603     for (Pair<byte[], String> pair : familyPaths) {
3604       byte[] fam = pair.getFirst();
3605       if (family == null) {
3606         family = fam;
3607       } else if (!Bytes.equals(family, fam)) {
3608         multipleFamilies = true;
3609         break;
3610       }
3611     }
3612     return multipleFamilies;
3613   }
3614 
3615   /**
3616    * Attempts to atomically load a group of hfiles.  This is critical for loading
3617    * rows with multiple column families atomically.
3618    *
3619    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3620    * @return true if successful, false if failed recoverably
3621    * @throws IOException if failed unrecoverably.
3622    */
3623   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths) throws IOException {
3624     return bulkLoadHFiles(familyPaths, false);
3625   }
3626 
3627   /**
3628    * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple
3629    * column families atomically.
3630    * @param familyPaths List of Pair<byte[] column family, String hfilePath> * @param assignSeqNum
3631    *          should we assign sequence numbers
3632    * @return true if successful, false if failed recoverably
3633    * @throws IOException if failed unrecoverably.
3634    */
3635   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, boolean assignSeqId)
3636       throws IOException {
3637     return bulkLoadHFiles(familyPaths, null, assignSeqId);
3638   }
3639 
3640   /**
3641    * Attempts to atomically load a group of hfiles.  This is critical for loading
3642    * rows with multiple column families atomically.
3643    *
3644    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3645    * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
3646    * file about to be bulk loaded
3647    * @return true if successful, false if failed recoverably
3648    * @throws IOException if failed unrecoverably.
3649    */
3650   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
3651       BulkLoadListener bulkLoadListener) throws IOException {
3652     return bulkLoadHFiles(familyPaths, bulkLoadListener, false);
3653   }
3654 
3655   /**
3656    * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple
3657    * column families atomically.
3658    * @param familyPaths List of Pair<byte[] column family, String hfilePath>
3659    * @param bulkLoadListener Internal hooks enabling massaging/preparation of a file about to be
3660    *          bulk loaded * @param assignSeqNum should we assign sequence numbers
3661    * @return true if successful, false if failed recoverably
3662    * @throws IOException if failed unrecoverably.
3663    */
3664   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
3665       BulkLoadListener bulkLoadListener, boolean assignSeqId) throws IOException {
3666     Preconditions.checkNotNull(familyPaths);
3667     // we need writeLock for multi-family bulk load
3668     startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
3669     try {
3670       this.writeRequestsCount.increment();
3671       this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3672 
3673       // There possibly was a split that happend between when the split keys
3674       // were gathered and before the HRegion's write lock was taken.  We need
3675       // to validate the HFile region before attempting to bulk load all of them
3676       List<IOException> ioes = new ArrayList<IOException>();
3677       List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
3678       for (Pair<byte[], String> p : familyPaths) {
3679         byte[] familyName = p.getFirst();
3680         String path = p.getSecond();
3681 
3682         Store store = getStore(familyName);
3683         if (store == null) {
3684           IOException ioe = new DoNotRetryIOException(
3685               "No such column family " + Bytes.toStringBinary(familyName));
3686           ioes.add(ioe);
3687         } else {
3688           try {
3689             store.assertBulkLoadHFileOk(new Path(path));
3690           } catch (WrongRegionException wre) {
3691             // recoverable (file doesn't fit in region)
3692             failures.add(p);
3693           } catch (IOException ioe) {
3694             // unrecoverable (hdfs problem)
3695             ioes.add(ioe);
3696           }
3697         }
3698       }
3699 
3700       // validation failed because of some sort of IO problem.
3701       if (ioes.size() != 0) {
3702         IOException e = MultipleIOException.createIOException(ioes);
3703         LOG.error("There were one or more IO errors when checking if the bulk load is ok.", e);
3704         throw e;
3705       }
3706 
3707       // validation failed, bail out before doing anything permanent.
3708       if (failures.size() != 0) {
3709         StringBuilder list = new StringBuilder();
3710         for (Pair<byte[], String> p : failures) {
3711           list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
3712             .append(p.getSecond());
3713         }
3714         // problem when validating
3715         LOG.warn("There was a recoverable bulk load failure likely due to a" +
3716             " split.  These (family, HFile) pairs were not loaded: " + list);
3717         return false;
3718       }
3719 
3720       for (Pair<byte[], String> p : familyPaths) {
3721         byte[] familyName = p.getFirst();
3722         String path = p.getSecond();
3723         Store store = getStore(familyName);
3724         try {
3725           String finalPath = path;
3726           if(bulkLoadListener != null) {
3727             finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
3728           }
3729           store.bulkLoadHFile(finalPath, assignSeqId ? this.log.obtainSeqNum() : -1);
3730           if(bulkLoadListener != null) {
3731             bulkLoadListener.doneBulkLoad(familyName, path);
3732           }
3733         } catch (IOException ioe) {
3734           // a failure here causes an atomicity violation that we currently
3735           // cannot recover from since it is likely a failed hdfs operation.
3736 
3737           // TODO Need a better story for reverting partial failures due to HDFS.
3738           LOG.error("There was a partial failure due to IO when attempting to" +
3739               " load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond());
3740           if(bulkLoadListener != null) {
3741             try {
3742               bulkLoadListener.failedBulkLoad(familyName, path);
3743             } catch (Exception ex) {
3744               LOG.error("Error while calling failedBulkLoad for family "+
3745                   Bytes.toString(familyName)+" with path "+path, ex);
3746             }
3747           }
3748           throw ioe;
3749         }
3750       }
3751       return true;
3752     } finally {
3753       closeBulkRegionOperation();
3754     }
3755   }
3756 
3757   @Override
3758   public boolean equals(Object o) {
3759     if (!(o instanceof HRegion)) {
3760       return false;
3761     }
3762     return Bytes.equals(this.getRegionName(), ((HRegion) o).getRegionName());
3763   }
3764 
3765   @Override
3766   public int hashCode() {
3767     return Bytes.hashCode(this.getRegionName());
3768   }
3769 
3770   @Override
3771   public String toString() {
3772     return this.regionInfo.getRegionNameAsString();
3773   }
3774 
3775   /** @return Path of region base directory */
3776   public Path getTableDir() {
3777     return this.tableDir;
3778   }
3779 
3780   /**
3781    * RegionScannerImpl is used to combine scanners from multiple Stores (aka column families).
3782    */
3783   class RegionScannerImpl implements RegionScanner {
3784     // Package local for testability
3785     KeyValueHeap storeHeap = null;
3786     /** Heap of key-values that are not essential for the provided filters and are thus read
3787      * on demand, if on-demand column family loading is enabled.*/
3788     KeyValueHeap joinedHeap = null;
3789     /**
3790      * If the joined heap data gathering is interrupted due to scan limits, this will
3791      * contain the row for which we are populating the values.*/
3792     private KeyValue joinedContinuationRow = null;
3793     // KeyValue indicating that limit is reached when scanning
3794     private final KeyValue KV_LIMIT = new KeyValue();
3795     private final byte [] stopRow;
3796     private Filter filter;
3797     private int batch;
3798     private int isScan;
3799     private boolean filterClosed = false;
3800     private long readPt;
3801     private HRegion region;
3802 
3803     public HRegionInfo getRegionInfo() {
3804       return regionInfo;
3805     }
3806     
3807     RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
3808         throws IOException {
3809       // DebugPrint.println("HRegionScanner.<init>");
3810       this.region = region;
3811       this.filter = scan.getFilter();
3812       this.batch = scan.getBatch();
3813       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
3814         this.stopRow = null;
3815       } else {
3816         this.stopRow = scan.getStopRow();
3817       }
3818       // If we are doing a get, we want to be [startRow,endRow] normally
3819       // it is [startRow,endRow) and if startRow=endRow we get nothing.
3820       this.isScan = scan.isGetScan() ? -1 : 0;
3821 
3822       // synchronize on scannerReadPoints so that nobody calculates
3823       // getSmallestReadPoint, before scannerReadPoints is updated.
3824       IsolationLevel isolationLevel = scan.getIsolationLevel();
3825       synchronized(scannerReadPoints) {
3826         if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
3827           // This scan can read even uncommitted transactions
3828           this.readPt = Long.MAX_VALUE;
3829           MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
3830         } else {
3831           this.readPt = MultiVersionConsistencyControl.resetThreadReadPoint(mvcc);
3832         }
3833         scannerReadPoints.put(this, this.readPt);
3834       }
3835 
3836       // Here we separate all scanners into two lists - scanner that provide data required
3837       // by the filter to operate (scanners list) and all others (joinedScanners list).
3838       List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
3839       List<KeyValueScanner> joinedScanners = new ArrayList<KeyValueScanner>();
3840       if (additionalScanners != null) {
3841         scanners.addAll(additionalScanners);
3842       }
3843 
3844       for (Map.Entry<byte[], NavigableSet<byte[]>> entry :
3845           scan.getFamilyMap().entrySet()) {
3846         Store store = stores.get(entry.getKey());
3847         KeyValueScanner scanner = store.getScanner(scan, entry.getValue());
3848         if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
3849           || FilterBase.isFamilyEssential(this.filter, entry.getKey())) {
3850           scanners.add(scanner);
3851         } else {
3852           joinedScanners.add(scanner);
3853         }
3854       }
3855       this.storeHeap = new KeyValueHeap(scanners, comparator);
3856       if (!joinedScanners.isEmpty()) {
3857         this.joinedHeap = new KeyValueHeap(joinedScanners, comparator);
3858       }
3859     }
3860 
3861     RegionScannerImpl(Scan scan, HRegion region) throws IOException {
3862       this(scan, null, region);
3863     }
3864 
3865     @Override
3866     public long getMvccReadPoint() {
3867       return this.readPt;
3868     }
3869     /**
3870      * Reset both the filter and the old filter.
3871      */
3872     protected void resetFilters() {
3873       if (filter != null) {
3874         filter.reset();
3875       }
3876     }
3877 
3878     @Override
3879     public boolean next(List<KeyValue> outResults, int limit)
3880         throws IOException {
3881       return next(outResults, limit, null);
3882     }
3883 
3884     @Override
3885     public synchronized boolean next(List<KeyValue> outResults, int limit,
3886         String metric) throws IOException {
3887       if (this.filterClosed) {
3888         throw new UnknownScannerException("Scanner was closed (timed out?) " +
3889             "after we renewed it. Could be caused by a very slow scanner " +
3890             "or a lengthy garbage collection");
3891       }
3892       startRegionOperation();
3893       readRequestsCount.increment();
3894       opMetrics.setReadRequestCountMetrics(readRequestsCount.get());
3895       try {
3896 
3897         // This could be a new thread from the last time we called next().
3898         MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
3899 
3900         return nextRaw(outResults, limit, metric);
3901       } finally {
3902         closeRegionOperation();
3903       }
3904     }
3905 
3906     @Override
3907     public boolean nextRaw(List<KeyValue> outResults, String metric)
3908         throws IOException {
3909       return nextRaw(outResults, batch, metric);
3910     }
3911 
3912     @Override
3913     public boolean nextRaw(List<KeyValue> outResults, int limit,
3914         String metric) throws IOException {
3915       boolean returnResult;
3916       if (outResults.isEmpty()) {
3917         // Usually outResults is empty. This is true when next is called
3918         // to handle scan or get operation.
3919         returnResult = nextInternal(outResults, limit, metric);
3920       } else {
3921         List<KeyValue> tmpList = new ArrayList<KeyValue>();
3922         returnResult = nextInternal(tmpList, limit, metric);
3923         outResults.addAll(tmpList);
3924       }
3925       resetFilters();
3926       if (isFilterDone()) {
3927         return false;
3928       }
3929       return returnResult;
3930     }
3931 
3932     @Override
3933     public boolean next(List<KeyValue> outResults)
3934         throws IOException {
3935       // apply the batching limit by default
3936       return next(outResults, batch, null);
3937     }
3938 
3939     @Override
3940     public boolean next(List<KeyValue> outResults, String metric)
3941         throws IOException {
3942       // apply the batching limit by default
3943       return next(outResults, batch, metric);
3944     }
3945 
3946     private void populateFromJoinedHeap(List<KeyValue> results, int limit, String metric)
3947         throws IOException {
3948       assert joinedContinuationRow != null;
3949       KeyValue kv = populateResult(results, this.joinedHeap, limit,
3950         joinedContinuationRow.getBuffer(), joinedContinuationRow.getRowOffset(),
3951         joinedContinuationRow.getRowLength(), metric);
3952       if (kv != KV_LIMIT) {
3953         // We are done with this row, reset the continuation.
3954         joinedContinuationRow = null;
3955       }
3956       // As the data is obtained from two independent heaps, we need to
3957       // ensure that result list is sorted, because Result relies on that.
3958       Collections.sort(results, comparator);
3959     }
3960 
3961     /**
3962      * Fetches records with this row into result list, until next row or limit (if not -1).
3963      * @param results
3964      * @param heap KeyValueHeap to fetch data from. It must be positioned on correct row before call.
3965      * @param limit Max amount of KVs to place in result list, -1 means no limit.
3966      * @param currentRow Byte array with key we are fetching.
3967      * @param offset offset for currentRow
3968      * @param length length for currentRow
3969      * @param metric Metric key to be passed into KeyValueHeap::next().
3970      * @return true if limit reached, false otherwise.
3971      */
3972     private KeyValue populateResult(List<KeyValue> results, KeyValueHeap heap, int limit,
3973         byte[] currentRow, int offset, short length, String metric) throws IOException {
3974       KeyValue nextKv;
3975       do {
3976         heap.next(results, limit - results.size(), metric);
3977         if (limit > 0 && results.size() == limit) {
3978           return KV_LIMIT;
3979         }
3980         nextKv = heap.peek();
3981       } while (nextKv != null && nextKv.matchingRow(currentRow, offset, length));
3982       return nextKv;
3983     }
3984 
3985     /*
3986      * @return True if a filter rules the scanner is over, done.
3987      */
3988     public synchronized boolean isFilterDone() {
3989       return this.filter != null && this.filter.filterAllRemaining();
3990     }
3991 
3992     private boolean nextInternal(List<KeyValue> results, int limit, String metric)
3993     throws IOException {
3994       if (!results.isEmpty()) {
3995         throw new IllegalArgumentException("First parameter should be an empty list");
3996       }
3997       RpcCallContext rpcCall = HBaseServer.getCurrentCall();
3998       // The loop here is used only when at some point during the next we determine
3999       // that due to effects of filters or otherwise, we have an empty row in the result.
4000       // Then we loop and try again. Otherwise, we must get out on the first iteration via return,
4001       // "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row,
4002       // and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow).
4003       while (true) {
4004         if (rpcCall != null) {
4005           // If a user specifies a too-restrictive or too-slow scanner, the
4006           // client might time out and disconnect while the server side
4007           // is still processing the request. We should abort aggressively
4008           // in that case.
4009           rpcCall.throwExceptionIfCallerDisconnected();
4010         }
4011 
4012         // Let's see what we have in the storeHeap.
4013         KeyValue current = this.storeHeap.peek();
4014 
4015         byte[] currentRow = null;
4016         int offset = 0;
4017         short length = 0;
4018         if (current != null) {
4019           currentRow = current.getBuffer();
4020           offset = current.getRowOffset();
4021           length = current.getRowLength();
4022         }
4023         boolean stopRow = isStopRow(currentRow, offset, length);
4024         // Check if we were getting data from the joinedHeap abd hit the limit.
4025         // If not, then it's main path - getting results from storeHeap.
4026         if (joinedContinuationRow == null) {
4027           // First, check if we are at a stop row. If so, there are no more results.
4028           if (stopRow) {
4029             if (filter != null && filter.hasFilterRow()) {
4030               filter.filterRow(results);
4031             }
4032             if (filter != null && filter.filterRow()) {
4033               results.clear();
4034             }
4035             return false;
4036           }
4037 
4038           // Check if rowkey filter wants to exclude this row. If so, loop to next.
4039           // Techically, if we hit limits before on this row, we don't need this call.
4040           if (filterRowKey(currentRow, offset, length)) {
4041             results.clear();
4042             boolean moreRows = nextRow(currentRow, offset, length);
4043             if (!moreRows) return false;
4044             continue;
4045           }
4046 
4047           // Ok, we are good, let's try to get some results from the main heap.
4048           KeyValue nextKv = populateResult(results, this.storeHeap, limit, currentRow, offset,
4049               length, metric);
4050           if (nextKv == KV_LIMIT) {
4051             if (this.filter != null && filter.hasFilterRow()) {
4052               throw new IncompatibleFilterException(
4053                 "Filter whose hasFilterRow() returns true is incompatible with scan with limit!");
4054             }
4055             return true; // We hit the limit.
4056           }
4057           stopRow = nextKv == null
4058               || isStopRow(nextKv.getBuffer(), nextKv.getRowOffset(), nextKv.getRowLength());
4059           // save that the row was empty before filters applied to it.
4060           final boolean isEmptyRow = results.isEmpty();
4061 
4062           // We have the part of the row necessary for filtering (all of it, usually).
4063           // First filter with the filterRow(List).            
4064           if (filter != null && filter.hasFilterRow()) {
4065             filter.filterRow(results);
4066           }
4067 
4068           if (isEmptyRow || filterRow()) {
4069             results.clear();
4070             boolean moreRows = nextRow(currentRow, offset, length);
4071             if (!moreRows) return false;
4072 
4073             // This row was totally filtered out, if this is NOT the last row,
4074             // we should continue on. Otherwise, nothing else to do.
4075             if (!stopRow) continue;
4076             return false;
4077           }
4078 
4079           // Ok, we are done with storeHeap for this row.
4080           // Now we may need to fetch additional, non-essential data into row.
4081           // These values are not needed for filter to work, so we postpone their
4082           // fetch to (possibly) reduce amount of data loads from disk.
4083           if (this.joinedHeap != null) {
4084             KeyValue nextJoinedKv = joinedHeap.peek();
4085             // If joinedHeap is pointing to some other row, try to seek to a correct one.
4086             boolean mayHaveData =
4087               (nextJoinedKv != null && nextJoinedKv.matchingRow(currentRow, offset, length))
4088                 || (this.joinedHeap.requestSeek(
4089                     KeyValue.createFirstOnRow(currentRow, offset, length), true, true)
4090                   && joinedHeap.peek() != null
4091                   && joinedHeap.peek().matchingRow(currentRow, offset, length));
4092             if (mayHaveData) {
4093               joinedContinuationRow = current;
4094               populateFromJoinedHeap(results, limit, metric);
4095             }
4096           }
4097         } else {
4098           // Populating from the joined map was stopped by limits, populate some more.
4099           populateFromJoinedHeap(results, limit, metric);
4100         }
4101 
4102         // We may have just called populateFromJoinedMap and hit the limits. If that is
4103         // the case, we need to call it again on the next next() invocation.
4104         if (joinedContinuationRow != null) {
4105           return true;
4106         }
4107 
4108         // Finally, we are done with both joinedHeap and storeHeap.
4109         // Double check to prevent empty rows from appearing in result. It could be
4110         // the case when SingleValueExcludeFilter is used.
4111         if (results.isEmpty()) {
4112           boolean moreRows = nextRow(currentRow, offset, length);
4113           if (!moreRows) return false;
4114           if (!stopRow) continue;
4115         }
4116 
4117         // We are done. Return the result.
4118         return !stopRow;
4119       }
4120     }
4121 
4122     private boolean filterRow() {
4123       return filter != null
4124           && filter.filterRow();
4125     }
4126     private boolean filterRowKey(byte[] row, int offset, short length) {
4127       return filter != null
4128           && filter.filterRowKey(row, offset, length);
4129     }
4130 
4131     protected boolean nextRow(byte [] currentRow, int offset, short length) throws IOException {
4132       KeyValue next;
4133       while((next = this.storeHeap.peek()) != null && next.matchingRow(currentRow, offset, length)) {
4134         this.storeHeap.next(MOCKED_LIST);       
4135       }
4136       resetFilters();
4137       // Calling the hook in CP which allows it to do a fast forward
4138       if (this.region.getCoprocessorHost() != null) {
4139         return this.region.getCoprocessorHost().postScannerFilterRow(this, currentRow);
4140       }
4141       return true;
4142     }
4143 
4144     private boolean isStopRow(byte [] currentRow, int offset, short length) {
4145       return currentRow == null ||
4146           (stopRow != null &&
4147           comparator.compareRows(stopRow, 0, stopRow.length,
4148               currentRow, offset, length) <= isScan);
4149     }
4150 
4151     @Override
4152     public synchronized void close() {
4153       if (storeHeap != null) {
4154         storeHeap.close();
4155         storeHeap = null;
4156       }
4157       if (joinedHeap != null) {
4158         joinedHeap.close();
4159         joinedHeap = null;
4160       }
4161       // no need to sychronize here.
4162       scannerReadPoints.remove(this);
4163       this.filterClosed = true;
4164     }
4165 
4166     KeyValueHeap getStoreHeapForTesting() {
4167       return storeHeap;
4168     }
4169 
4170     @Override
4171     public synchronized boolean reseek(byte[] row) throws IOException {
4172       if (row == null) {
4173         throw new IllegalArgumentException("Row cannot be null.");
4174       }
4175       boolean result = false;
4176       startRegionOperation();
4177       try {
4178         // This could be a new thread from the last time we called next().
4179         MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
4180         KeyValue kv = KeyValue.createFirstOnRow(row);
4181         // use request seek to make use of the lazy seek option. See HBASE-5520
4182         result = this.storeHeap.requestSeek(kv, true, true);
4183         if (this.joinedHeap != null) {
4184           result = this.joinedHeap.requestSeek(kv, true, true) || result;
4185         }
4186       } finally {
4187         closeRegionOperation();
4188       }
4189       return result;
4190     }
4191   }
4192 
4193   // Utility methods
4194   /**
4195    * A utility method to create new instances of HRegion based on the
4196    * {@link HConstants#REGION_IMPL} configuration property.
4197    * @param tableDir qualified path of directory where region should be located,
4198    * usually the table directory.
4199    * @param log The HLog is the outbound log for any updates to the HRegion
4200    * (There's a single HLog for all the HRegions on a single HRegionServer.)
4201    * The log file is a logfile from the previous execution that's
4202    * custom-computed for this HRegion. The HRegionServer computes and sorts the
4203    * appropriate log info for this HRegion. If there is a previous log file
4204    * (implying that the HRegion has been written-to before), then read it from
4205    * the supplied path.
4206    * @param fs is the filesystem.
4207    * @param conf is global configuration settings.
4208    * @param regionInfo - HRegionInfo that describes the region
4209    * is new), then read them from the supplied path.
4210    * @param htd
4211    * @param rsServices
4212    * @return the new instance
4213    */
4214   public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
4215       Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
4216       RegionServerServices rsServices) {
4217     try {
4218       @SuppressWarnings("unchecked")
4219       Class<? extends HRegion> regionClass =
4220           (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
4221 
4222       Constructor<? extends HRegion> c =
4223           regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
4224               Configuration.class, HRegionInfo.class, HTableDescriptor.class,
4225               RegionServerServices.class);
4226 
4227       return c.newInstance(tableDir, log, fs, conf, regionInfo, htd, rsServices);
4228     } catch (Throwable e) {
4229       // todo: what should I throw here?
4230       throw new IllegalStateException("Could not instantiate a region instance.", e);
4231     }
4232   }
4233 
4234   /**
4235    * Convenience method creating new HRegions. Used by createTable and by the
4236    * bootstrap code in the HMaster constructor.
4237    * Note, this method creates an {@link HLog} for the created region. It
4238    * needs to be closed explicitly.  Use {@link HRegion#getLog()} to get
4239    * access.  <b>When done with a region created using this method, you will
4240    * need to explicitly close the {@link HLog} it created too; it will not be
4241    * done for you.  Not closing the log will leave at least a daemon thread
4242    * running.</b>  Call {@link #closeHRegion(HRegion)} and it will do
4243    * necessary cleanup for you.
4244    * @param info Info for region to create.
4245    * @param rootDir Root directory for HBase instance
4246    * @param conf
4247    * @param hTableDescriptor
4248    * @return new HRegion
4249    *
4250    * @throws IOException
4251    */
4252   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4253       final Configuration conf, final HTableDescriptor hTableDescriptor)
4254   throws IOException {
4255     return createHRegion(info, rootDir, conf, hTableDescriptor, null);
4256   }
4257 
4258   /**
4259    * This will do the necessary cleanup a call to {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)}
4260    * requires.  This method will close the region and then close its
4261    * associated {@link HLog} file.  You use it if you call the other createHRegion,
4262    * the one that takes an {@link HLog} instance but don't be surprised by the
4263    * call to the {@link HLog#closeAndDelete()} on the {@link HLog} the
4264    * HRegion was carrying.
4265    * @param r
4266    * @throws IOException
4267    */
4268   public static void closeHRegion(final HRegion r) throws IOException {
4269     if (r == null) return;
4270     r.close();
4271     if (r.getLog() == null) return;
4272     r.getLog().closeAndDelete();
4273   }
4274 
4275   /**
4276    * Convenience method creating new HRegions. Used by createTable.
4277    * The {@link HLog} for the created region needs to be closed explicitly.
4278    * Use {@link HRegion#getLog()} to get access.
4279    *
4280    * @param info Info for region to create.
4281    * @param rootDir Root directory for HBase instance
4282    * @param conf
4283    * @param hTableDescriptor
4284    * @param hlog shared HLog
4285    * @param boolean initialize - true to initialize the region
4286    * @return new HRegion
4287    *
4288    * @throws IOException
4289    */
4290   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4291                                       final Configuration conf,
4292                                       final HTableDescriptor hTableDescriptor,
4293                                       final HLog hlog,
4294                                       final boolean initialize)
4295       throws IOException {
4296     return createHRegion(info, rootDir, conf, hTableDescriptor,
4297         hlog, initialize, false);
4298   }
4299 
4300   /**
4301    * Convenience method creating new HRegions. Used by createTable.
4302    * The {@link HLog} for the created region needs to be closed
4303    * explicitly, if it is not null.
4304    * Use {@link HRegion#getLog()} to get access.
4305    *
4306    * @param info Info for region to create.
4307    * @param rootDir Root directory for HBase instance
4308    * @param conf
4309    * @param hTableDescriptor
4310    * @param hlog shared HLog
4311    * @param boolean initialize - true to initialize the region
4312    * @param boolean ignoreHLog
4313       - true to skip generate new hlog if it is null, mostly for createTable
4314    * @return new HRegion
4315    *
4316    * @throws IOException
4317    */
4318   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4319                                       final Configuration conf,
4320                                       final HTableDescriptor hTableDescriptor,
4321                                       final HLog hlog,
4322                                       final boolean initialize, final boolean ignoreHLog)
4323       throws IOException {
4324     LOG.info("creating HRegion " + info.getTableNameAsString()
4325         + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
4326         " Table name == " + info.getTableNameAsString());
4327 
4328     Path tableDir =
4329         HTableDescriptor.getTableDir(rootDir, info.getTableName());
4330     Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
4331     FileSystem fs = FileSystem.get(conf);
4332     HBaseFileSystem.makeDirOnFileSystem(fs, regionDir);
4333     // Write HRI to a file in case we need to recover .META.
4334     writeRegioninfoOnFilesystem(info, regionDir, fs, conf);
4335     HLog effectiveHLog = hlog;
4336     if (hlog == null && !ignoreHLog) {
4337       effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
4338           new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
4339     }
4340     HRegion region = HRegion.newHRegion(tableDir,
4341         effectiveHLog, fs, conf, info, hTableDescriptor, null);
4342     if (initialize) {
4343       region.initialize();
4344     }
4345     return region;
4346   }
4347 
4348   public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4349                                       final Configuration conf,
4350                                       final HTableDescriptor hTableDescriptor,
4351                                       final HLog hlog)
4352     throws IOException {
4353     return createHRegion(info, rootDir, conf, hTableDescriptor, hlog, true);
4354   }
4355 
4356   /**
4357    * Open a Region.
4358    * @param info Info for region to be opened.
4359    * @param wal HLog for region to use. This method will call
4360    * HLog#setSequenceNumber(long) passing the result of the call to
4361    * HRegion#getMinSequenceId() to ensure the log id is properly kept
4362    * up.  HRegionStore does this every time it opens a new region.
4363    * @param conf
4364    * @return new HRegion
4365    *
4366    * @throws IOException
4367    */
4368   public static HRegion openHRegion(final HRegionInfo info,
4369       final HTableDescriptor htd, final HLog wal,
4370       final Configuration conf)
4371   throws IOException {
4372     return openHRegion(info, htd, wal, conf, null, null);
4373   }
4374 
4375   /**
4376    * Open a Region.
4377    * @param info Info for region to be opened
4378    * @param htd
4379    * @param wal HLog for region to use. This method will call
4380    * HLog#setSequenceNumber(long) passing the result of the call to
4381    * HRegion#getMinSequenceId() to ensure the log id is properly kept
4382    * up.  HRegionStore does this every time it opens a new region.
4383    * @param conf
4384    * @param rsServices An interface we can request flushes against.
4385    * @param reporter An interface we can report progress against.
4386    * @return new HRegion
4387    *
4388    * @throws IOException
4389    */
4390   public static HRegion openHRegion(final HRegionInfo info,
4391     final HTableDescriptor htd, final HLog wal, final Configuration conf,
4392     final RegionServerServices rsServices,
4393     final CancelableProgressable reporter)
4394   throws IOException {
4395     if (LOG.isDebugEnabled()) {
4396       LOG.debug("Opening region: " + info);
4397     }
4398     if (info == null) {
4399       throw new NullPointerException("Passed region info is null");
4400     }
4401     Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
4402       info.getTableName());
4403     FileSystem fs = null;
4404     if (rsServices != null) {
4405       fs = rsServices.getFileSystem();
4406     }
4407     if (fs == null) {
4408       fs = FileSystem.get(conf);
4409     }
4410     HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info,
4411       htd, rsServices);
4412     return r.openHRegion(reporter);
4413   }
4414 
4415   public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
4416       final HTableDescriptor htd, final HLog wal, final Configuration conf)
4417   throws IOException {
4418     return openHRegion(tableDir, info, htd, wal, conf, null, null);
4419   }
4420 
4421   /**
4422    * Open a Region.
4423    * @param tableDir Table directory
4424    * @param info Info for region to be opened.
4425    * @param wal HLog for region to use. This method will call
4426    * HLog#setSequenceNumber(long) passing the result of the call to
4427    * HRegion#getMinSequenceId() to ensure the log id is properly kept
4428    * up.  HRegionStore does this every time it opens a new region.
4429    * @param conf
4430    * @param reporter An interface we can report progress against.
4431    * @return new HRegion
4432    *
4433    * @throws IOException
4434    */
4435   public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
4436       final HTableDescriptor htd, final HLog wal, final Configuration conf,
4437       final RegionServerServices rsServices,
4438       final CancelableProgressable reporter)
4439   throws IOException {
4440     if (info == null) throw new NullPointerException("Passed region info is null");
4441     LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
4442     if (LOG.isDebugEnabled()) {
4443       LOG.debug("Opening region: " + info);
4444     }
4445     Path dir = HTableDescriptor.getTableDir(tableDir,
4446         info.getTableName());
4447     HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
4448         htd, rsServices);
4449     return r.openHRegion(reporter);
4450   }
4451 
4452 
4453   /**
4454    * Open HRegion.
4455    * Calls initialize and sets sequenceid.
4456    * @param reporter
4457    * @return Returns <code>this</code>
4458    * @throws IOException
4459    */
4460   protected HRegion openHRegion(final CancelableProgressable reporter)
4461   throws IOException {
4462     checkCompressionCodecs();
4463 
4464     long seqid = initialize(reporter);
4465     if (this.log != null) {
4466       this.log.setSequenceNumber(seqid);
4467     }
4468     return this;
4469   }
4470 
4471   private void checkCompressionCodecs() throws IOException {
4472     for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
4473       CompressionTest.testCompression(fam.getCompression());
4474       CompressionTest.testCompression(fam.getCompactionCompression());
4475     }
4476   }
4477 
4478   /**
4479    * Inserts a new region's meta information into the passed
4480    * <code>meta</code> region. Used by the HMaster bootstrap code adding
4481    * new table to ROOT table.
4482    *
4483    * @param meta META HRegion to be updated
4484    * @param r HRegion to add to <code>meta</code>
4485    *
4486    * @throws IOException
4487    */
4488   public static void addRegionToMETA(HRegion meta, HRegion r)
4489   throws IOException {
4490     meta.checkResources();
4491     // The row key is the region name
4492     byte[] row = r.getRegionName();
4493     Integer lid = meta.obtainRowLock(row);
4494     try {
4495       final long now = EnvironmentEdgeManager.currentTimeMillis();
4496       final List<KeyValue> edits = new ArrayList<KeyValue>(2);
4497       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4498         HConstants.REGIONINFO_QUALIFIER, now,
4499         Writables.getBytes(r.getRegionInfo())));
4500       // Set into the root table the version of the meta table.
4501       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4502         HConstants.META_VERSION_QUALIFIER, now,
4503         Bytes.toBytes(HConstants.META_VERSION)));
4504       meta.put(HConstants.CATALOG_FAMILY, edits);
4505     } finally {
4506       meta.releaseRowLock(lid);
4507     }
4508   }
4509 
4510   /**
4511    * Deletes all the files for a HRegion
4512    *
4513    * @param fs the file system object
4514    * @param rootdir qualified path of HBase root directory
4515    * @param info HRegionInfo for region to be deleted
4516    * @throws IOException
4517    */
4518   public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
4519   throws IOException {
4520     deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
4521   }
4522 
4523   private static void deleteRegion(FileSystem fs, Path regiondir)
4524   throws IOException {
4525     if (LOG.isDebugEnabled()) {
4526       LOG.debug("DELETING region " + regiondir.toString());
4527     }
4528     if (!HBaseFileSystem.deleteDirFromFileSystem(fs, regiondir)) {
4529       LOG.warn("Failed delete of " + regiondir);
4530     }
4531   }
4532 
4533   /**
4534    * Computes the Path of the HRegion
4535    *
4536    * @param rootdir qualified path of HBase root directory
4537    * @param info HRegionInfo for the region
4538    * @return qualified path of region directory
4539    */
4540   public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
4541     return new Path(
4542       HTableDescriptor.getTableDir(rootdir, info.getTableName()),
4543                                    info.getEncodedName());
4544   }
4545 
4546   /**
4547    * Determines if the specified row is within the row range specified by the
4548    * specified HRegionInfo
4549    *
4550    * @param info HRegionInfo that specifies the row range
4551    * @param row row to be checked
4552    * @return true if the row is within the range specified by the HRegionInfo
4553    */
4554   public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
4555     return ((info.getStartKey().length == 0) ||
4556         (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
4557         ((info.getEndKey().length == 0) ||
4558             (Bytes.compareTo(info.getEndKey(), row) > 0));
4559   }
4560 
4561   /**
4562    * Make the directories for a specific column family
4563    *
4564    * @param fs the file system
4565    * @param tabledir base directory where region will live (usually the table dir)
4566    * @param hri
4567    * @param colFamily the column family
4568    * @throws IOException
4569    */
4570   public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
4571     final HRegionInfo hri, byte [] colFamily)
4572   throws IOException {
4573     Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
4574     if (!HBaseFileSystem.makeDirOnFileSystem(fs, dir)) {
4575       LOG.warn("Failed to create " + dir);
4576     }
4577   }
4578 
4579   /**
4580    * Merge two HRegions.  The regions must be adjacent and must not overlap.
4581    *
4582    * @param srcA
4583    * @param srcB
4584    * @return new merged HRegion
4585    * @throws IOException
4586    */
4587   public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
4588   throws IOException {
4589     HRegion a = srcA;
4590     HRegion b = srcB;
4591 
4592     // Make sure that srcA comes first; important for key-ordering during
4593     // write of the merged file.
4594     if (srcA.getStartKey() == null) {
4595       if (srcB.getStartKey() == null) {
4596         throw new IOException("Cannot merge two regions with null start key");
4597       }
4598       // A's start key is null but B's isn't. Assume A comes before B
4599     } else if ((srcB.getStartKey() == null) ||
4600       (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
4601       a = srcB;
4602       b = srcA;
4603     }
4604 
4605     if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) {
4606       throw new IOException("Cannot merge non-adjacent regions");
4607     }
4608     return merge(a, b);
4609   }
4610 
4611   /**
4612    * Merge two regions whether they are adjacent or not.
4613    *
4614    * @param a region a
4615    * @param b region b
4616    * @return new merged region
4617    * @throws IOException
4618    */
4619   public static HRegion merge(HRegion a, HRegion b)
4620   throws IOException {
4621     if (!a.getRegionInfo().getTableNameAsString().equals(
4622         b.getRegionInfo().getTableNameAsString())) {
4623       throw new IOException("Regions do not belong to the same table");
4624     }
4625 
4626     FileSystem fs = a.getFilesystem();
4627 
4628     // Make sure each region's cache is empty
4629 
4630     a.flushcache();
4631     b.flushcache();
4632 
4633     // Compact each region so we only have one store file per family
4634 
4635     a.compactStores(true);
4636     if (LOG.isDebugEnabled()) {
4637       LOG.debug("Files for region: " + a);
4638       listPaths(fs, a.getRegionDir());
4639     }
4640     b.compactStores(true);
4641     if (LOG.isDebugEnabled()) {
4642       LOG.debug("Files for region: " + b);
4643       listPaths(fs, b.getRegionDir());
4644     }
4645 
4646     Configuration conf = a.getBaseConf();
4647     HTableDescriptor tabledesc = a.getTableDesc();
4648     HLog log = a.getLog();
4649     Path tableDir = a.getTableDir();
4650     // Presume both are of same region type -- i.e. both user or catalog
4651     // table regions.  This way can use comparator.
4652     final byte[] startKey =
4653       (a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
4654            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4655        || b.comparator.matchingRows(b.getStartKey(), 0,
4656               b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
4657               HConstants.EMPTY_BYTE_ARRAY.length))
4658       ? HConstants.EMPTY_BYTE_ARRAY
4659       : (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
4660              b.getStartKey(), 0, b.getStartKey().length) <= 0
4661          ? a.getStartKey()
4662          : b.getStartKey());
4663     final byte[] endKey =
4664       (a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
4665            HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4666        || a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
4667               HConstants.EMPTY_BYTE_ARRAY, 0,
4668               HConstants.EMPTY_BYTE_ARRAY.length))
4669       ? HConstants.EMPTY_BYTE_ARRAY
4670       : (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
4671              b.getEndKey(), 0, b.getEndKey().length) <= 0
4672          ? b.getEndKey()
4673          : a.getEndKey());
4674 
4675     HRegionInfo newRegionInfo =
4676         new HRegionInfo(tabledesc.getName(), startKey, endKey);
4677     LOG.info("Creating new region " + newRegionInfo.toString());
4678     String encodedName = newRegionInfo.getEncodedName();
4679     Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
4680     if(fs.exists(newRegionDir)) {
4681       throw new IOException("Cannot merge; target file collision at " +
4682           newRegionDir);
4683     }
4684     HBaseFileSystem.makeDirOnFileSystem(fs, newRegionDir);
4685 
4686     LOG.info("starting merge of regions: " + a + " and " + b +
4687       " into new region " + newRegionInfo.toString() +
4688         " with start key <" + Bytes.toStringBinary(startKey) + "> and end key <" +
4689         Bytes.toStringBinary(endKey) + ">");
4690 
4691     // Move HStoreFiles under new region directory
4692     Map<byte [], List<StoreFile>> byFamily =
4693       new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
4694     byFamily = filesByFamily(byFamily, a.close());
4695     byFamily = filesByFamily(byFamily, b.close());
4696     for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
4697       byte [] colFamily = es.getKey();
4698       makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily);
4699       // Because we compacted the source regions we should have no more than two
4700       // HStoreFiles per family and there will be no reference store
4701       List<StoreFile> srcFiles = es.getValue();
4702       for (StoreFile hsf: srcFiles) {
4703         StoreFile.rename(fs, hsf.getPath(),
4704           StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir,
4705             newRegionInfo.getEncodedName(), colFamily)));
4706       }
4707     }
4708     if (LOG.isDebugEnabled()) {
4709       LOG.debug("Files for new region");
4710       listPaths(fs, newRegionDir);
4711     }
4712     HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf,
4713         newRegionInfo, a.getTableDesc(), null);
4714     long totalReadRequestCount = a.readRequestsCount.get() + b.readRequestsCount.get();
4715     dstRegion.readRequestsCount.set(totalReadRequestCount);
4716     dstRegion.opMetrics.setReadRequestCountMetrics(totalReadRequestCount);
4717     
4718     long totalWriteRequestCount = a.writeRequestsCount.get() + b.writeRequestsCount.get();
4719     dstRegion.writeRequestsCount.set(totalWriteRequestCount);
4720     dstRegion.opMetrics.setWriteRequestCountMetrics(totalWriteRequestCount);
4721     
4722     dstRegion.initialize();
4723     dstRegion.compactStores();
4724     if (LOG.isDebugEnabled()) {
4725       LOG.debug("Files for new region");
4726       listPaths(fs, dstRegion.getRegionDir());
4727     }
4728 
4729     // delete out the 'A' region
4730     HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(a.getConf()),
4731         a.getTableDir(), a.getRegionDir());
4732     // delete out the 'B' region
4733     HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(b.getConf()),
4734         b.getTableDir(), b.getRegionDir());
4735 
4736     LOG.info("merge completed. New region is " + dstRegion);
4737 
4738     return dstRegion;
4739   }
4740 
4741   /*
4742    * Fills a map with a vector of store files keyed by column family.
4743    * @param byFamily Map to fill.
4744    * @param storeFiles Store files to process.
4745    * @param family
4746    * @return Returns <code>byFamily</code>
4747    */
4748   private static Map<byte [], List<StoreFile>> filesByFamily(
4749       Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
4750     for (StoreFile src: storeFiles) {
4751       byte [] family = src.getFamily();
4752       List<StoreFile> v = byFamily.get(family);
4753       if (v == null) {
4754         v = new ArrayList<StoreFile>();
4755         byFamily.put(family, v);
4756       }
4757       v.add(src);
4758     }
4759     return byFamily;
4760   }
4761 
4762   /**
4763    * @return True if needs a mojor compaction.
4764    * @throws IOException
4765    */
4766   boolean isMajorCompaction() throws IOException {
4767     for (Store store: this.stores.values()) {
4768       if (store.isMajorCompaction()) {
4769         return true;
4770       }
4771     }
4772     return false;
4773   }
4774 
4775   /*
4776    * List the files under the specified directory
4777    *
4778    * @param fs
4779    * @param dir
4780    * @throws IOException
4781    */
4782   private static void listPaths(FileSystem fs, Path dir) throws IOException {
4783     if (LOG.isDebugEnabled()) {
4784       FileStatus[] stats = FSUtils.listStatus(fs, dir, null);
4785       if (stats == null || stats.length == 0) {
4786         return;
4787       }
4788       for (int i = 0; i < stats.length; i++) {
4789         String path = stats[i].getPath().toString();
4790         if (stats[i].isDir()) {
4791           LOG.debug("d " + path);
4792           listPaths(fs, stats[i].getPath());
4793         } else {
4794           LOG.debug("f " + path + " size=" + stats[i].getLen());
4795         }
4796       }
4797     }
4798   }
4799 
4800 
4801   //
4802   // HBASE-880
4803   //
4804   /**
4805    * @param get get object
4806    * @return result
4807    * @throws IOException read exceptions
4808    */
4809   public Result get(final Get get) throws IOException {
4810     return get(get, null);
4811   }
4812 
4813   /**
4814    * @param get get object
4815    * @param lockid existing lock id, or null for no previous lock
4816    * @return result
4817    * @throws IOException read exceptions
4818    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
4819    */
4820   public Result get(final Get get, final Integer lockid) throws IOException {
4821     checkRow(get.getRow(), "Get");
4822     // Verify families are all valid
4823     if (get.hasFamilies()) {
4824       for (byte [] family: get.familySet()) {
4825         checkFamily(family);
4826       }
4827     } else { // Adding all families to scanner
4828       for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
4829         get.addFamily(family);
4830       }
4831     }
4832     List<KeyValue> results = get(get, true);
4833     return new Result(results);
4834   }
4835 
4836   /*
4837    * Do a get based on the get parameter.
4838    * @param withCoprocessor invoke coprocessor or not. We don't want to
4839    * always invoke cp for this private method.
4840    */
4841   private List<KeyValue> get(Get get, boolean withCoprocessor)
4842   throws IOException {
4843     long now = EnvironmentEdgeManager.currentTimeMillis();
4844 
4845     List<KeyValue> results = new ArrayList<KeyValue>();
4846 
4847     // pre-get CP hook
4848     if (withCoprocessor && (coprocessorHost != null)) {
4849        if (coprocessorHost.preGet(get, results)) {
4850          return results;
4851        }
4852     }
4853 
4854     Scan scan = new Scan(get);
4855 
4856     RegionScanner scanner = null;
4857     try {
4858       scanner = getScanner(scan);
4859       scanner.next(results, SchemaMetrics.METRIC_GETSIZE);
4860     } finally {
4861       if (scanner != null)
4862         scanner.close();
4863     }
4864 
4865     // post-get CP hook
4866     if (withCoprocessor && (coprocessorHost != null)) {
4867       coprocessorHost.postGet(get, results);
4868     }
4869 
4870     // do after lock
4871     final long after = EnvironmentEdgeManager.currentTimeMillis();
4872     this.opMetrics.updateGetMetrics(get.familySet(), after - now);
4873 
4874     return results;
4875   }
4876 
4877   public void mutateRow(RowMutations rm) throws IOException {
4878     mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
4879   }
4880 
4881   /**
4882    * Perform atomic mutations within the region.
4883    * @param mutations The list of mutations to perform.
4884    * <code>mutations</code> can contain operations for multiple rows.
4885    * Caller has to ensure that all rows are contained in this region.
4886    * @param rowsToLock Rows to lock
4887    * If multiple rows are locked care should be taken that
4888    * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
4889    * @throws IOException
4890    */
4891   public void mutateRowsWithLocks(Collection<Mutation> mutations,
4892       Collection<byte[]> rowsToLock) throws IOException {
4893     boolean flush = false;
4894 
4895     checkReadOnly();
4896     checkResources();
4897 
4898     startRegionOperation();
4899     List<Integer> acquiredLocks = null;
4900     try {
4901       // 1. run all pre-hooks before the atomic operation
4902       // if any pre hook indicates "bypass", bypass the entire operation
4903 
4904       // one WALEdit is used for all edits.
4905       WALEdit walEdit = new WALEdit();
4906       if (coprocessorHost != null) {
4907         for (Mutation m : mutations) {
4908           if (m instanceof Put) {
4909             if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
4910               // by pass everything
4911               return;
4912             }
4913           } else if (m instanceof Delete) {
4914             Delete d = (Delete) m;
4915             prepareDelete(d);
4916             if (coprocessorHost.preDelete(d, walEdit, d.getWriteToWAL())) {
4917               // by pass everything
4918               return;
4919             }
4920           }
4921         }
4922       }
4923 
4924       long txid = 0;
4925       boolean walSyncSuccessful = false;
4926       boolean memstoreUpdated = false;
4927       boolean locked = false;
4928 
4929       // 2. acquire the row lock(s)
4930       acquiredLocks = new ArrayList<Integer>(rowsToLock.size());
4931       for (byte[] row : rowsToLock) {
4932         // attempt to lock all involved rows, fail if one lock times out
4933         Integer lid = getLock(null, row, true);
4934         if (lid == null) {
4935           throw new IOException("Failed to acquire lock on "
4936               + Bytes.toStringBinary(row));
4937         }
4938         acquiredLocks.add(lid);
4939       }
4940 
4941       // 3. acquire the region lock
4942       lock(this.updatesLock.readLock(), acquiredLocks.size());
4943       locked = true;
4944 
4945       // 4. Get a mvcc write number
4946       MultiVersionConsistencyControl.WriteEntry w = mvcc.beginMemstoreInsert();
4947 
4948       long now = EnvironmentEdgeManager.currentTimeMillis();
4949       byte[] byteNow = Bytes.toBytes(now);
4950       Durability durability = Durability.USE_DEFAULT;
4951       try {
4952         // 5. Check mutations and apply edits to a single WALEdit
4953         for (Mutation m : mutations) {
4954           if (m instanceof Put) {
4955             Map<byte[], List<KeyValue>> familyMap = m.getFamilyMap();
4956             checkFamilies(familyMap.keySet());
4957             checkTimestamps(familyMap, now);
4958             updateKVTimestamps(familyMap.values(), byteNow);
4959           } else if (m instanceof Delete) {
4960             Delete d = (Delete) m;
4961             prepareDelete(d);
4962             prepareDeleteTimestamps(d.getFamilyMap(), byteNow);
4963           } else {
4964             throw new DoNotRetryIOException(
4965                 "Action must be Put or Delete. But was: "
4966                     + m.getClass().getName());
4967           }
4968           Durability tmpDur = m.getDurability(); 
4969           if (tmpDur.ordinal() > durability.ordinal()) {
4970             durability = tmpDur;
4971           }
4972           if (tmpDur != Durability.SKIP_WAL) {
4973             addFamilyMapToWALEdit(m.getFamilyMap(), walEdit);
4974           }
4975         }
4976 
4977         // 6. append all edits at once (don't sync)
4978         if (walEdit.size() > 0) {
4979           txid = this.log.appendNoSync(regionInfo,
4980               this.htableDescriptor.getName(), walEdit,
4981               HConstants.DEFAULT_CLUSTER_ID, now, this.htableDescriptor);
4982         }
4983 
4984         // 7. apply to memstore
4985         long addedSize = 0;
4986         memstoreUpdated = true;
4987         for (Mutation m : mutations) {
4988           addedSize += applyFamilyMapToMemstore(m.getFamilyMap(), w);
4989         }
4990         flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
4991 
4992         // 8. release region and row lock(s)
4993         this.updatesLock.readLock().unlock();
4994         locked = false;
4995         if (acquiredLocks != null) {
4996           for (Integer lid : acquiredLocks) {
4997             releaseRowLock(lid);
4998           }
4999           acquiredLocks = null;
5000         }
5001 
5002         // 9. sync WAL if required
5003         if (walEdit.size() > 0) {
5004           syncOrDefer(txid, durability);
5005         }
5006         walSyncSuccessful = true;
5007 
5008         // 10. advance mvcc
5009         mvcc.completeMemstoreInsert(w);
5010         w = null;
5011 
5012         // 11. run coprocessor post host hooks
5013         // after the WAL is sync'ed and all locks are released
5014         // (similar to doMiniBatchPut)
5015         if (coprocessorHost != null) {
5016           for (Mutation m : mutations) {
5017             if (m instanceof Put) {
5018               coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
5019             } else if (m instanceof Delete) {
5020               coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
5021             }
5022           }
5023         }
5024       } finally {
5025         // 12. clean up if needed
5026         if (memstoreUpdated && !walSyncSuccessful) {
5027           int kvsRolledback = 0;
5028           for (Mutation m : mutations) {
5029             for (Map.Entry<byte[], List<KeyValue>> e : m.getFamilyMap()
5030                 .entrySet()) {
5031               List<KeyValue> kvs = e.getValue();
5032               byte[] family = e.getKey();
5033               Store store = getStore(family);
5034               // roll back each kv
5035               for (KeyValue kv : kvs) {
5036                 store.rollback(kv);
5037                 kvsRolledback++;
5038               }
5039             }
5040           }
5041           LOG.info("mutateRowWithLocks: rolled back " + kvsRolledback
5042               + " KeyValues");
5043         }
5044 
5045         if (w != null) {
5046           mvcc.completeMemstoreInsert(w);
5047         }
5048 
5049         if (locked) {
5050           this.updatesLock.readLock().unlock();
5051         }
5052 
5053         if (acquiredLocks != null) {
5054           for (Integer lid : acquiredLocks) {
5055             releaseRowLock(lid);
5056           }
5057         }
5058       }
5059     } finally {
5060       if (flush) {
5061         // 13. Flush cache if needed. Do it outside update lock.
5062         requestFlush();
5063       }
5064       closeRegionOperation();
5065     }
5066   }
5067 
5068   // TODO: There's a lot of boiler plate code identical
5069   // to increment... See how to better unify that.
5070 
5071   /**
5072   *
5073   * Perform one or more append operations on a row.
5074   * <p>
5075   * Appends performed are done under row lock but reads do not take locks out
5076   * so this can be seen partially complete by gets and scans.
5077   *
5078   * @param append
5079   * @param writeToWAL
5080   * @return new keyvalues after increment
5081   * @throws IOException
5082   */
5083  public Result append(Append append, boolean writeToWAL)
5084      throws IOException {
5085    return append(append, null, writeToWAL);
5086  }
5087   /**
5088    *
5089    * Perform one or more append operations on a row.
5090    * <p>
5091    * Appends performed are done under row lock but reads do not take locks out
5092    * so this can be seen partially complete by gets and scans.
5093    *
5094    * @param append
5095    * @param lockid
5096    * @param writeToWAL
5097    * @return new keyvalues after increment
5098    * @throws IOException
5099    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
5100    */
5101   public Result append(Append append, Integer lockid, boolean writeToWAL)
5102       throws IOException {
5103     // TODO: Use MVCC to make this set of appends atomic to reads
5104     byte[] row = append.getRow();
5105     checkRow(row, "append");
5106     boolean flush = false;
5107     WALEdit walEdits = null;
5108     List<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
5109     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
5110     long before = EnvironmentEdgeManager.currentTimeMillis();
5111     long size = 0;
5112     long txid = 0;
5113 
5114     checkReadOnly();
5115     // Lock row
5116     startRegionOperation();
5117     this.writeRequestsCount.increment();
5118     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5119     try {
5120       Integer lid = getLock(lockid, row, true);
5121       lock(this.updatesLock.readLock());
5122       try {
5123         long now = EnvironmentEdgeManager.currentTimeMillis();
5124         // Process each family
5125         for (Map.Entry<byte[], List<KeyValue>> family : append.getFamilyMap()
5126             .entrySet()) {
5127 
5128           Store store = stores.get(family.getKey());
5129           Collections.sort(family.getValue(), store.getComparator());
5130           List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
5131 
5132           // Get previous values for all columns in this family
5133           Get get = new Get(row);
5134           for (KeyValue kv : family.getValue()) {
5135             get.addColumn(family.getKey(), kv.getQualifier());
5136           }
5137           List<KeyValue> results = get(get, false);
5138 
5139           // Iterate the input columns and update existing values if they were
5140           // found, otherwise add new column initialized to the append value
5141 
5142           // Avoid as much copying as possible. Every byte is copied at most
5143           // once.
5144           // Would be nice if KeyValue had scatter/gather logic
5145           int idx = 0;
5146           for (KeyValue kv : family.getValue()) {
5147             KeyValue newKV;
5148             if (idx < results.size()
5149                 && results.get(idx).matchingQualifier(kv.getBuffer(),
5150                     kv.getQualifierOffset(), kv.getQualifierLength())) {
5151               KeyValue oldKv = results.get(idx);
5152               // allocate an empty kv once
5153               newKV = new KeyValue(row.length, kv.getFamilyLength(),
5154                   kv.getQualifierLength(), now, KeyValue.Type.Put,
5155                   oldKv.getValueLength() + kv.getValueLength());
5156               // copy in the value
5157               System.arraycopy(oldKv.getBuffer(), oldKv.getValueOffset(),
5158                   newKV.getBuffer(), newKV.getValueOffset(),
5159                   oldKv.getValueLength());
5160               System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5161                   newKV.getBuffer(),
5162                   newKV.getValueOffset() + oldKv.getValueLength(),
5163                   kv.getValueLength());
5164               idx++;
5165             } else {
5166               // allocate an empty kv once
5167               newKV = new KeyValue(row.length, kv.getFamilyLength(),
5168                   kv.getQualifierLength(), now, KeyValue.Type.Put,
5169                   kv.getValueLength());
5170               // copy in the value
5171               System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5172                   newKV.getBuffer(), newKV.getValueOffset(),
5173                   kv.getValueLength());
5174             }
5175             // copy in row, family, and qualifier
5176             System.arraycopy(kv.getBuffer(), kv.getRowOffset(),
5177                 newKV.getBuffer(), newKV.getRowOffset(), kv.getRowLength());
5178             System.arraycopy(kv.getBuffer(), kv.getFamilyOffset(),
5179                 newKV.getBuffer(), newKV.getFamilyOffset(),
5180                 kv.getFamilyLength());
5181             System.arraycopy(kv.getBuffer(), kv.getQualifierOffset(),
5182                 newKV.getBuffer(), newKV.getQualifierOffset(),
5183                 kv.getQualifierLength());
5184 
5185             kvs.add(newKV);
5186 
5187             // Append update to WAL
5188             if (writeToWAL) {
5189               if (walEdits == null) {
5190                 walEdits = new WALEdit();
5191               }
5192               walEdits.add(newKV);
5193             }
5194           }
5195 
5196           // store the kvs to the temporary memstore before writing HLog
5197           tempMemstore.put(store, kvs);
5198         }
5199 
5200         // Actually write to WAL now
5201         if (writeToWAL) {
5202           // Using default cluster id, as this can only happen in the orginating
5203           // cluster. A slave cluster receives the final value (not the delta)
5204           // as a Put.
5205           txid = this.log.appendNoSync(regionInfo,
5206               this.htableDescriptor.getName(), walEdits,
5207               HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5208               this.htableDescriptor);
5209         }
5210         // Actually write to Memstore now
5211         for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5212           Store store = entry.getKey();
5213           size += store.upsert(entry.getValue());
5214           allKVs.addAll(entry.getValue());
5215         }
5216         size = this.addAndGetGlobalMemstoreSize(size);
5217         flush = isFlushSize(size);
5218       } finally {
5219         this.updatesLock.readLock().unlock();
5220         releaseRowLock(lid);
5221       }
5222       if (writeToWAL) {
5223         // sync the transaction log outside the rowlock
5224         syncOrDefer(txid, append.getDurability());
5225       }
5226     } finally {
5227       closeRegionOperation();
5228     }
5229 
5230 
5231     long after = EnvironmentEdgeManager.currentTimeMillis();
5232     this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
5233 
5234     if (flush) {
5235       // Request a cache flush. Do it outside update lock.
5236       requestFlush();
5237     }
5238 
5239     return append.isReturnResults() ? new Result(allKVs) : null;
5240   }
5241 
5242   /**
5243   *
5244   * Perform one or more increment operations on a row.
5245   * <p>
5246   * Increments performed are done under row lock but reads do not take locks
5247   * out so this can be seen partially complete by gets and scans.
5248   * @param increment
5249   * @param writeToWAL
5250   * @return new keyvalues after increment
5251   * @throws IOException
5252   */
5253   public Result increment(Increment increment, boolean writeToWAL)
5254   throws IOException {
5255     return increment(increment, null, writeToWAL);
5256   }
5257 
5258   /**
5259    *
5260    * Perform one or more increment operations on a row.
5261    * <p>
5262    * Increments performed are done under row lock but reads do not take locks
5263    * out so this can be seen partially complete by gets and scans.
5264    * @param increment
5265    * @param lockid
5266    * @param writeToWAL
5267    * @return new keyvalues after increment
5268    * @throws IOException
5269    * @deprecated row locks (lockId) held outside the extent of the operation are deprecated.
5270 
5271    */
5272   public Result increment(Increment increment, Integer lockid,
5273       boolean writeToWAL)
5274   throws IOException {
5275     // TODO: Use MVCC to make this set of increments atomic to reads
5276     byte [] row = increment.getRow();
5277     checkRow(row, "increment");
5278     TimeRange tr = increment.getTimeRange();
5279     boolean flush = false;
5280     WALEdit walEdits = null;
5281     List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
5282     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
5283     long before = EnvironmentEdgeManager.currentTimeMillis();
5284     long size = 0;
5285     long txid = 0;
5286 
5287     checkReadOnly();
5288     // Lock row
5289     startRegionOperation();
5290     this.writeRequestsCount.increment();
5291     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5292     try {
5293       Integer lid = getLock(lockid, row, true);
5294       lock(this.updatesLock.readLock());
5295       try {
5296         long now = EnvironmentEdgeManager.currentTimeMillis();
5297         // Process each family
5298         for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
5299           increment.getFamilyMap().entrySet()) {
5300 
5301           Store store = stores.get(family.getKey());
5302           List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
5303 
5304           // Get previous values for all columns in this family
5305           Get get = new Get(row);
5306           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5307             get.addColumn(family.getKey(), column.getKey());
5308           }
5309           get.setTimeRange(tr.getMin(), tr.getMax());
5310           List<KeyValue> results = get(get, false);
5311 
5312           // Iterate the input columns and update existing values if they were
5313           // found, otherwise add new column initialized to the increment amount
5314           int idx = 0;
5315           for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5316             long amount = column.getValue();
5317             if (idx < results.size() &&
5318                 results.get(idx).matchingQualifier(column.getKey())) {
5319               KeyValue kv = results.get(idx);
5320               if(kv.getValueLength() == Bytes.SIZEOF_LONG) {
5321                 amount += Bytes.toLong(kv.getBuffer(), kv.getValueOffset(), Bytes.SIZEOF_LONG);
5322               } else {
5323                 // throw DoNotRetryIOException instead of IllegalArgumentException
5324                 throw new DoNotRetryIOException(
5325                     "Attempted to increment field that isn't 64 bits wide");
5326               }
5327               idx++;
5328             }
5329 
5330             // Append new incremented KeyValue to list
5331             KeyValue newKV = new KeyValue(row, family.getKey(), column.getKey(),
5332                 now, Bytes.toBytes(amount));
5333             kvs.add(newKV);
5334 
5335             // Append update to WAL
5336             if (writeToWAL) {
5337               if (walEdits == null) {
5338                 walEdits = new WALEdit();
5339               }
5340               walEdits.add(newKV);
5341             }
5342           }
5343 
5344           //store the kvs to the temporary memstore before writing HLog
5345           tempMemstore.put(store, kvs);
5346         }
5347 
5348         // Actually write to WAL now
5349         if (writeToWAL) {
5350           // Using default cluster id, as this can only happen in the orginating
5351           // cluster. A slave cluster receives the final value (not the delta)
5352           // as a Put.
5353           txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5354               walEdits, HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5355               this.htableDescriptor);
5356         }
5357 
5358         //Actually write to Memstore now
5359         for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5360           Store store = entry.getKey();
5361           size += store.upsert(entry.getValue());
5362           allKVs.addAll(entry.getValue());
5363         }
5364         size = this.addAndGetGlobalMemstoreSize(size);
5365         flush = isFlushSize(size);
5366       } finally {
5367         this.updatesLock.readLock().unlock();
5368         releaseRowLock(lid);
5369       }
5370       if (writeToWAL) {
5371         // sync the transaction log outside the rowlock
5372         syncOrDefer(txid, Durability.USE_DEFAULT);
5373       }
5374     } finally {
5375       closeRegionOperation();
5376       long after = EnvironmentEdgeManager.currentTimeMillis();
5377       this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
5378     }
5379 
5380     if (flush) {
5381       // Request a cache flush.  Do it outside update lock.
5382       requestFlush();
5383     }
5384 
5385     return new Result(allKVs);
5386   }
5387 
5388   /**
5389    * @param row
5390    * @param family
5391    * @param qualifier
5392    * @param amount
5393    * @param writeToWAL
5394    * @return The new value.
5395    * @throws IOException
5396    */
5397   public long incrementColumnValue(byte [] row, byte [] family,
5398       byte [] qualifier, long amount, boolean writeToWAL)
5399   throws IOException {
5400     // to be used for metrics
5401     long before = EnvironmentEdgeManager.currentTimeMillis();
5402 
5403     checkRow(row, "increment");
5404     boolean flush = false;
5405     boolean wrongLength = false;
5406     long txid = 0;
5407     // Lock row
5408     long result = amount;
5409     startRegionOperation();
5410     this.writeRequestsCount.increment();
5411     this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5412     try {
5413       Integer lid = obtainRowLock(row);
5414       lock(this.updatesLock.readLock());
5415       try {
5416         Store store = stores.get(family);
5417 
5418         // Get the old value:
5419         Get get = new Get(row);
5420         get.addColumn(family, qualifier);
5421 
5422         // we don't want to invoke coprocessor in this case; ICV is wrapped
5423         // in HRegionServer, so we leave getLastIncrement alone
5424         List<KeyValue> results = get(get, false);
5425 
5426         if (!results.isEmpty()) {
5427           KeyValue kv = results.get(0);
5428           if(kv.getValueLength() == Bytes.SIZEOF_LONG){
5429             byte [] buffer = kv.getBuffer();
5430             int valueOffset = kv.getValueOffset();
5431             result += Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG);
5432           }
5433           else{
5434             wrongLength = true;
5435           }
5436         }
5437         if(!wrongLength){
5438           // build the KeyValue now:
5439           KeyValue newKv = new KeyValue(row, family,
5440             qualifier, EnvironmentEdgeManager.currentTimeMillis(),
5441             Bytes.toBytes(result));
5442 
5443           // now log it:
5444           if (writeToWAL) {
5445             long now = EnvironmentEdgeManager.currentTimeMillis();
5446             WALEdit walEdit = new WALEdit();
5447             walEdit.add(newKv);
5448             // Using default cluster id, as this can only happen in the
5449             // orginating cluster. A slave cluster receives the final value (not
5450             // the delta) as a Put.
5451             txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5452                 walEdit, HConstants.DEFAULT_CLUSTER_ID, now,
5453                 this.htableDescriptor);
5454           }
5455 
5456           // Now request the ICV to the store, this will set the timestamp
5457           // appropriately depending on if there is a value in memcache or not.
5458           // returns the change in the size of the memstore from operation
5459           long size = store.updateColumnValue(row, family, qualifier, result);
5460 
5461           size = this.addAndGetGlobalMemstoreSize(size);
5462           flush = isFlushSize(size);
5463         }
5464       } finally {
5465         this.updatesLock.readLock().unlock();
5466         releaseRowLock(lid);
5467       }
5468       if (writeToWAL) {
5469         // sync the transaction log outside the rowlock
5470         syncOrDefer(txid, Durability.USE_DEFAULT);
5471       }
5472     } finally {
5473       closeRegionOperation();
5474     }
5475 
5476     // do after lock
5477     long after = EnvironmentEdgeManager.currentTimeMillis();
5478     this.opMetrics.updateIncrementColumnValueMetrics(family, after - before);
5479 
5480     if (flush) {
5481       // Request a cache flush.  Do it outside update lock.
5482       requestFlush();
5483     }
5484     if(wrongLength){
5485       throw new DoNotRetryIOException(
5486           "Attempted to increment field that isn't 64 bits wide");
5487     }
5488     return result;
5489   }
5490 
5491 
5492   //
5493   // New HBASE-880 Helpers
5494   //
5495 
5496   private void checkFamily(final byte [] family)
5497   throws NoSuchColumnFamilyException {
5498     if (!this.htableDescriptor.hasFamily(family)) {
5499       throw new NoSuchColumnFamilyException("Column family " +
5500           Bytes.toString(family) + " does not exist in region " + this
5501           + " in table " + this.htableDescriptor);
5502   	}
5503   }
5504 
5505   public static final long FIXED_OVERHEAD = ClassSize.align(
5506       ClassSize.OBJECT +
5507       ClassSize.ARRAY +
5508       36 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
5509       (8 * Bytes.SIZEOF_LONG) +
5510       Bytes.SIZEOF_BOOLEAN);
5511 
5512   public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
5513       ClassSize.OBJECT + // closeLock
5514       (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
5515       (3 * ClassSize.ATOMIC_LONG) + // memStoreSize, numPutsWithoutWAL, dataInMemoryWithoutWAL
5516       ClassSize.ATOMIC_INTEGER + // lockIdGenerator
5517       (3 * ClassSize.CONCURRENT_HASHMAP) +  // lockedRows, lockIds, scannerReadPoints
5518       WriteState.HEAP_SIZE + // writestate
5519       ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
5520       (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
5521       ClassSize.ARRAYLIST + // recentFlushes
5522       MultiVersionConsistencyControl.FIXED_SIZE // mvcc
5523       ;
5524 
5525   @Override
5526   public long heapSize() {
5527     long heapSize = DEEP_OVERHEAD;
5528     for(Store store : this.stores.values()) {
5529       heapSize += store.heapSize();
5530     }
5531     // this does not take into account row locks, recent flushes, mvcc entries
5532     return heapSize;
5533   }
5534 
5535   /*
5536    * This method calls System.exit.
5537    * @param message Message to print out.  May be null.
5538    */
5539   private static void printUsageAndExit(final String message) {
5540     if (message != null && message.length() > 0) System.out.println(message);
5541     System.out.println("Usage: HRegion CATLALOG_TABLE_DIR [major_compact]");
5542     System.out.println("Options:");
5543     System.out.println(" major_compact  Pass this option to major compact " +
5544       "passed region.");
5545     System.out.println("Default outputs scan of passed region.");
5546     System.exit(1);
5547   }
5548 
5549   /**
5550    * Registers a new CoprocessorProtocol subclass and instance to
5551    * be available for handling {@link HRegion#exec(Exec)} calls.
5552    *
5553    * <p>
5554    * Only a single protocol type/handler combination may be registered per
5555    * region.
5556    * After the first registration, subsequent calls with the same protocol type
5557    * will fail with a return value of {@code false}.
5558    * </p>
5559    * @param protocol a {@code CoprocessorProtocol} subinterface defining the
5560    * protocol methods
5561    * @param handler an instance implementing the interface
5562    * @param <T> the protocol type
5563    * @return {@code true} if the registration was successful, {@code false}
5564    * otherwise
5565    */
5566   public <T extends CoprocessorProtocol> boolean registerProtocol(
5567       Class<T> protocol, T handler) {
5568 
5569     /* No stacking of protocol handlers is currently allowed.  The
5570      * first to claim wins!
5571      */
5572     if (protocolHandlers.containsKey(protocol)) {
5573       LOG.error("Protocol "+protocol.getName()+
5574           " already registered, rejecting request from "+
5575           handler
5576       );
5577       return false;
5578     }
5579 
5580     protocolHandlers.putInstance(protocol, handler);
5581     protocolHandlerNames.put(protocol.getName(), protocol);
5582     if (LOG.isDebugEnabled()) {
5583       LOG.debug("Registered protocol handler: region="+
5584           Bytes.toStringBinary(getRegionName())+" protocol="+protocol.getName());
5585     }
5586     return true;
5587   }
5588 
5589   /**
5590    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
5591    * method using the registered protocol handlers.
5592    * {@link CoprocessorProtocol} implementations must be registered via the
5593    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
5594    * method before they are available.
5595    *
5596    * @param call an {@code Exec} instance identifying the protocol, method name,
5597    *     and parameters for the method invocation
5598    * @return an {@code ExecResult} instance containing the region name of the
5599    *     invocation and the return value
5600    * @throws IOException if no registered protocol handler is found or an error
5601    *     occurs during the invocation
5602    * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
5603    */
5604   public ExecResult exec(Exec call)
5605       throws IOException {
5606     Class<? extends CoprocessorProtocol> protocol = call.getProtocol();
5607     if (protocol == null) {
5608       String protocolName = call.getProtocolName();
5609       if (LOG.isTraceEnabled()) {
5610         LOG.trace("Received dynamic protocol exec call with protocolName " + protocolName);
5611       }
5612       // detect the actual protocol class
5613       protocol  = protocolHandlerNames.get(protocolName);
5614       if (protocol == null) {
5615         throw new HBaseRPC.UnknownProtocolException(protocol,
5616             "No matching handler for protocol "+protocolName+
5617             " in region "+Bytes.toStringBinary(getRegionName()));
5618       }
5619     }
5620     if (!protocolHandlers.containsKey(protocol)) {
5621       throw new HBaseRPC.UnknownProtocolException(protocol,
5622           "No matching handler for protocol "+protocol.getName()+
5623           " in region "+Bytes.toStringBinary(getRegionName()));
5624     }
5625 
5626     CoprocessorProtocol handler = protocolHandlers.getInstance(protocol);
5627     Object value;
5628 
5629     try {
5630       Method method = protocol.getMethod(
5631           call.getMethodName(), call.getParameterClasses());
5632       method.setAccessible(true);
5633 
5634       value = method.invoke(handler, call.getParameters());
5635     } catch (InvocationTargetException e) {
5636       Throwable target = e.getTargetException();
5637       if (target instanceof IOException) {
5638         throw (IOException)target;
5639       }
5640       IOException ioe = new IOException(target.toString());
5641       ioe.setStackTrace(target.getStackTrace());
5642       throw ioe;
5643     } catch (Throwable e) {
5644       if (!(e instanceof IOException)) {
5645         LOG.error("Unexpected throwable object ", e);
5646       }
5647       IOException ioe = new IOException(e.toString());
5648       ioe.setStackTrace(e.getStackTrace());
5649       throw ioe;
5650     }
5651 
5652     return new ExecResult(getRegionName(), value);
5653   }
5654 
5655   /*
5656    * Process table.
5657    * Do major compaction or list content.
5658    * @param fs
5659    * @param p
5660    * @param log
5661    * @param c
5662    * @param majorCompact
5663    * @throws IOException
5664    */
5665   private static void processTable(final FileSystem fs, final Path p,
5666       final HLog log, final Configuration c,
5667       final boolean majorCompact)
5668   throws IOException {
5669     HRegion region = null;
5670     String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
5671     String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
5672     // Currently expects tables have one region only.
5673     if (p.getName().startsWith(rootStr)) {
5674       region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
5675         HTableDescriptor.ROOT_TABLEDESC, null);
5676     } else if (p.getName().startsWith(metaStr)) {
5677       region = HRegion.newHRegion(p, log, fs, c,
5678         HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
5679     } else {
5680       throw new IOException("Not a known catalog table: " + p.toString());
5681     }
5682     try {
5683       region.initialize();
5684       if (majorCompact) {
5685         region.compactStores(true);
5686       } else {
5687         // Default behavior
5688         Scan scan = new Scan();
5689         // scan.addFamily(HConstants.CATALOG_FAMILY);
5690         RegionScanner scanner = region.getScanner(scan);
5691         try {
5692           List<KeyValue> kvs = new ArrayList<KeyValue>();
5693           boolean done = false;
5694           do {
5695             kvs.clear();
5696             done = scanner.next(kvs);
5697             if (kvs.size() > 0) LOG.info(kvs);
5698           } while (done);
5699         } finally {
5700           scanner.close();
5701         }
5702       }
5703     } finally {
5704       region.close();
5705     }
5706   }
5707 
5708   boolean shouldForceSplit() {
5709     return this.splitRequest;
5710   }
5711 
5712   byte[] getExplicitSplitPoint() {
5713     return this.explicitSplitPoint;
5714   }
5715 
5716   void forceSplit(byte[] sp) {
5717     // NOTE : this HRegion will go away after the forced split is successfull
5718     //        therefore, no reason to clear this value
5719     this.splitRequest = true;
5720     if (sp != null) {
5721       this.explicitSplitPoint = sp;
5722     }
5723   }
5724 
5725   void clearSplit_TESTS_ONLY() {
5726     this.splitRequest = false;
5727   }
5728 
5729   /**
5730    * Give the region a chance to prepare before it is split.
5731    */
5732   protected void prepareToSplit() {
5733     // nothing
5734   }
5735 
5736   /**
5737    * Return the splitpoint. null indicates the region isn't splittable
5738    * If the splitpoint isn't explicitly specified, it will go over the stores
5739    * to find the best splitpoint. Currently the criteria of best splitpoint
5740    * is based on the size of the store.
5741    */
5742   public byte[] checkSplit() {
5743     // Can't split ROOT/META
5744     if (this.regionInfo.isMetaTable()) {
5745       if (shouldForceSplit()) {
5746         LOG.warn("Cannot split root/meta regions in HBase 0.20 and above");
5747       }
5748       return null;
5749     }
5750 
5751     if (!splitPolicy.shouldSplit()) {
5752       return null;
5753     }
5754 
5755     byte[] ret = splitPolicy.getSplitPoint();
5756 
5757     if (ret != null) {
5758       try {
5759         checkRow(ret, "calculated split");
5760       } catch (IOException e) {
5761         LOG.error("Ignoring invalid split", e);
5762         return null;
5763       }
5764     }
5765     return ret;
5766   }
5767 
5768   /**
5769    * @return The priority that this region should have in the compaction queue
5770    */
5771   public int getCompactPriority() {
5772     int count = Integer.MAX_VALUE;
5773     for(Store store : stores.values()) {
5774       count = Math.min(count, store.getCompactPriority());
5775     }
5776     return count;
5777   }
5778 
5779   /**
5780    * Checks every store to see if one has too many
5781    * store files
5782    * @return true if any store has too many store files
5783    */
5784   public boolean needsCompaction() {
5785     for(Store store : stores.values()) {
5786       if(store.needsCompaction()) {
5787         return true;
5788       }
5789     }
5790     return false;
5791   }
5792 
5793   /** @return the coprocessor host */
5794   public RegionCoprocessorHost getCoprocessorHost() {
5795     return coprocessorHost;
5796   }
5797 
5798   /*
5799    * Set the read request count defined in opMetrics
5800    * @param value absolute value of read request count
5801    */
5802   public void setOpMetricsReadRequestCount(long value)
5803   {
5804     this.opMetrics.setReadRequestCountMetrics(value);
5805   }
5806   
5807   /*
5808    * Set the write request count defined in opMetrics
5809    * @param value absolute value of write request count
5810    */
5811   public void setOpMetricsWriteRequestCount(long value)
5812   {
5813     this.opMetrics.setWriteRequestCountMetrics(value);
5814   }
5815   
5816   /** @param coprocessorHost the new coprocessor host */
5817   public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
5818     this.coprocessorHost = coprocessorHost;
5819   }
5820 
5821   /**
5822    * This method needs to be called before any public call that reads or
5823    * modifies data. It has to be called just before a try.
5824    * #closeRegionOperation needs to be called in the try's finally block
5825    * Acquires a read lock and checks if the region is closing or closed.
5826    * @throws NotServingRegionException when the region is closing or closed
5827    * @throws RegionTooBusyException if failed to get the lock in time
5828    * @throws InterruptedIOException if interrupted while waiting for a lock
5829    */
5830   public void startRegionOperation()
5831       throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
5832     if (this.closing.get()) {
5833       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5834           " is closing");
5835     }
5836     lock(lock.readLock());
5837     if (this.closed.get()) {
5838       lock.readLock().unlock();
5839       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5840           " is closed");
5841     }
5842   }
5843 
5844   /**
5845    * Closes the lock. This needs to be called in the finally block corresponding
5846    * to the try block of #startRegionOperation
5847    */
5848   public void closeRegionOperation(){
5849     lock.readLock().unlock();
5850   }
5851 
5852   /**
5853    * This method needs to be called before any public call that reads or
5854    * modifies stores in bulk. It has to be called just before a try.
5855    * #closeBulkRegionOperation needs to be called in the try's finally block
5856    * Acquires a writelock and checks if the region is closing or closed.
5857    * @throws NotServingRegionException when the region is closing or closed
5858    * @throws RegionTooBusyException if failed to get the lock in time
5859    * @throws InterruptedIOException if interrupted while waiting for a lock
5860    */
5861   private void startBulkRegionOperation(boolean writeLockNeeded)
5862       throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
5863     if (this.closing.get()) {
5864       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5865           " is closing");
5866     }
5867     if (writeLockNeeded) lock(lock.writeLock());
5868     else lock(lock.readLock());
5869     if (this.closed.get()) {
5870       if (writeLockNeeded) lock.writeLock().unlock();
5871       else lock.readLock().unlock();
5872       throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5873           " is closed");
5874     }
5875   }
5876 
5877   /**
5878    * Closes the lock. This needs to be called in the finally block corresponding
5879    * to the try block of #startRegionOperation
5880    */
5881   private void closeBulkRegionOperation() {
5882     if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
5883     else lock.readLock().unlock();
5884   }
5885 
5886   /**
5887    * Update counters for numer of puts without wal and the size of possible data loss.
5888    * These information are exposed by the region server metrics.
5889    */
5890   private void recordPutWithoutWal(final Map<byte [], List<KeyValue>> familyMap) {
5891     if (numPutsWithoutWAL.getAndIncrement() == 0) {
5892       LOG.info("writing data to region " + this +
5893                " with WAL disabled. Data may be lost in the event of a crash.");
5894     }
5895 
5896     long putSize = 0;
5897     for (List<KeyValue> edits : familyMap.values()) {
5898       for (KeyValue kv : edits) {
5899         putSize += kv.getKeyLength() + kv.getValueLength();
5900       }
5901     }
5902 
5903     dataInMemoryWithoutWAL.addAndGet(putSize);
5904   }
5905 
5906   private void lock(final Lock lock)
5907       throws RegionTooBusyException, InterruptedIOException {
5908     lock(lock, 1);
5909   }
5910 
5911   /**
5912    * Try to acquire a lock.  Throw RegionTooBusyException
5913    * if failed to get the lock in time. Throw InterruptedIOException
5914    * if interrupted while waiting for the lock.
5915    */
5916   private void lock(final Lock lock, final int multiplier)
5917       throws RegionTooBusyException, InterruptedIOException {
5918     try {
5919       final long waitTime = Math.min(maxBusyWaitDuration,
5920         busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
5921       if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
5922         throw new RegionTooBusyException(
5923           "failed to get a lock in " + waitTime + "ms");
5924       }
5925     } catch (InterruptedException ie) {
5926       LOG.info("Interrupted while waiting for a lock");
5927       InterruptedIOException iie = new InterruptedIOException();
5928       iie.initCause(ie);
5929       throw iie;
5930     }
5931   }
5932 
5933   /**
5934    * Calls sync with the given transaction ID if the region's table is not
5935    * deferring it.
5936    * @param txid should sync up to which transaction
5937    * @throws IOException If anything goes wrong with DFS
5938    */
5939   private void syncOrDefer(long txid, Durability durability) throws IOException {
5940     if (this.getRegionInfo().isMetaRegion()) {
5941       this.log.sync(txid);
5942     } else {
5943       switch(durability) {
5944       case USE_DEFAULT:
5945         // do what CF defaults to
5946         if (!isDeferredLogSyncEnabled()) {
5947           this.log.sync(txid);
5948         }
5949         break;
5950       case SKIP_WAL:
5951         // nothing do to
5952         break;
5953       case ASYNC_WAL:
5954         // defer the sync, unless we globally can't
5955         if (this.deferredLogSyncDisabled) {
5956           this.log.sync(txid);
5957         }
5958         break;
5959       case SYNC_WAL:
5960       case FSYNC_WAL:
5961         // sync the WAL edit (SYNC and FSYNC treated the same for now)
5962         this.log.sync(txid);
5963         break;
5964       }
5965     }
5966   }
5967 
5968   /**
5969    * check if current region is deferred sync enabled.
5970    */
5971   private boolean isDeferredLogSyncEnabled() {
5972     return (this.htableDescriptor.isDeferredLogFlush() && !this.deferredLogSyncDisabled);
5973   }
5974 
5975   /**
5976    * A mocked list implementaion - discards all updates.
5977    */
5978   private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
5979 
5980     @Override
5981     public void add(int index, KeyValue element) {
5982       // do nothing
5983     }
5984 
5985     @Override
5986     public boolean addAll(int index, Collection<? extends KeyValue> c) {
5987       return false; // this list is never changed as a result of an update
5988     }
5989 
5990     @Override
5991     public KeyValue get(int index) {
5992       throw new UnsupportedOperationException();
5993     }
5994 
5995     @Override
5996     public int size() {
5997       return 0;
5998     }
5999   };
6000 
6001   /**
6002    * Facility for dumping and compacting catalog tables.
6003    * Only does catalog tables since these are only tables we for sure know
6004    * schema on.  For usage run:
6005    * <pre>
6006    *   ./bin/hbase org.apache.hadoop.hbase.regionserver.HRegion
6007    * </pre>
6008    * @param args
6009    * @throws IOException
6010    */
6011   public static void main(String[] args) throws IOException {
6012     if (args.length < 1) {
6013       printUsageAndExit(null);
6014     }
6015     boolean majorCompact = false;
6016     if (args.length > 1) {
6017       if (!args[1].toLowerCase().startsWith("major")) {
6018         printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
6019       }
6020       majorCompact = true;
6021     }
6022     final Path tableDir = new Path(args[0]);
6023     final Configuration c = HBaseConfiguration.create();
6024     final FileSystem fs = FileSystem.get(c);
6025     final Path logdir = new Path(c.get("hbase.tmp.dir"),
6026         "hlog" + tableDir.getName()
6027         + EnvironmentEdgeManager.currentTimeMillis());
6028     final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
6029         HConstants.HREGION_OLDLOGDIR_NAME);
6030     final HLog log = new HLog(fs, logdir, oldLogDir, c);
6031     try {
6032       processTable(fs, tableDir, log, c, majorCompact);
6033     } finally {
6034        log.close();
6035        // TODO: is this still right?
6036        BlockCache bc = new CacheConfig(c).getBlockCache();
6037        if (bc != null) bc.shutdown();
6038     }
6039   }
6040 
6041   /**
6042    * Listener class to enable callers of
6043    * bulkLoadHFile() to perform any necessary
6044    * pre/post processing of a given bulkload call
6045    */
6046   public static interface BulkLoadListener {
6047 
6048     /**
6049      * Called before an HFile is actually loaded
6050      * @param family family being loaded to
6051      * @param srcPath path of HFile
6052      * @return final path to be used for actual loading
6053      * @throws IOException
6054      */
6055     String prepareBulkLoad(byte[] family, String srcPath) throws IOException;
6056 
6057     /**
6058      * Called after a successful HFile load
6059      * @param family family being loaded to
6060      * @param srcPath path of HFile
6061      * @throws IOException
6062      */
6063     void doneBulkLoad(byte[] family, String srcPath) throws IOException;
6064 
6065     /**
6066      * Called after a failed HFile load
6067      * @param family family being loaded to
6068      * @param srcPath path of HFile
6069      * @throws IOException
6070      */
6071     void failedBulkLoad(byte[] family, String srcPath) throws IOException;
6072 
6073   }
6074 }