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