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