View Javadoc

1   /**
2    * Copyright 2010 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.IOException;
23  import java.io.StringWriter;
24  import java.lang.Thread.UncaughtExceptionHandler;
25  import java.lang.annotation.Retention;
26  import java.lang.annotation.RetentionPolicy;
27  import java.lang.management.ManagementFactory;
28  import java.lang.management.MemoryUsage;
29  import java.lang.reflect.Constructor;
30  import java.lang.reflect.Method;
31  import java.net.BindException;
32  import java.net.InetSocketAddress;
33  import java.util.ArrayList;
34  import java.util.Collection;
35  import java.util.Collections;
36  import java.util.Comparator;
37  import java.util.HashMap;
38  import java.util.LinkedList;
39  import java.util.List;
40  import java.util.Map;
41  import java.util.Random;
42  import java.util.Set;
43  import java.util.SortedMap;
44  import java.util.TreeMap;
45  import java.util.concurrent.ConcurrentHashMap;
46  import java.util.concurrent.ConcurrentSkipListMap;
47  import java.util.concurrent.atomic.AtomicBoolean;
48  import java.util.concurrent.atomic.AtomicInteger;
49  import java.util.concurrent.locks.ReentrantReadWriteLock;
50  
51  import org.apache.commons.logging.Log;
52  import org.apache.commons.logging.LogFactory;
53  import org.apache.hadoop.conf.Configuration;
54  import org.apache.hadoop.fs.FileSystem;
55  import org.apache.hadoop.fs.Path;
56  import org.apache.hadoop.hbase.Chore;
57  import org.apache.hadoop.hbase.ClockOutOfSyncException;
58  import org.apache.hadoop.hbase.DoNotRetryIOException;
59  import org.apache.hadoop.hbase.HBaseConfiguration;
60  import org.apache.hadoop.hbase.HConstants;
61  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
62  import org.apache.hadoop.hbase.HRegionInfo;
63  import org.apache.hadoop.hbase.HServerAddress;
64  import org.apache.hadoop.hbase.HServerInfo;
65  import org.apache.hadoop.hbase.HServerLoad;
66  import org.apache.hadoop.hbase.HTableDescriptor;
67  import org.apache.hadoop.hbase.KeyValue;
68  import org.apache.hadoop.hbase.MasterAddressTracker;
69  import org.apache.hadoop.hbase.NotServingRegionException;
70  import org.apache.hadoop.hbase.RemoteExceptionHandler;
71  import org.apache.hadoop.hbase.ServerName;
72  import org.apache.hadoop.hbase.Stoppable;
73  import org.apache.hadoop.hbase.TableDescriptors;
74  import org.apache.hadoop.hbase.UnknownRowLockException;
75  import org.apache.hadoop.hbase.UnknownScannerException;
76  import org.apache.hadoop.hbase.YouAreDeadException;
77  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
78  import org.apache.hadoop.hbase.catalog.CatalogTracker;
79  import org.apache.hadoop.hbase.catalog.MetaEditor;
80  import org.apache.hadoop.hbase.catalog.RootLocationEditor;
81  import org.apache.hadoop.hbase.client.Action;
82  import org.apache.hadoop.hbase.client.Delete;
83  import org.apache.hadoop.hbase.client.Get;
84  import org.apache.hadoop.hbase.client.HConnectionManager;
85  import org.apache.hadoop.hbase.client.Increment;
86  import org.apache.hadoop.hbase.client.MultiAction;
87  import org.apache.hadoop.hbase.client.MultiResponse;
88  import org.apache.hadoop.hbase.client.Put;
89  import org.apache.hadoop.hbase.client.Result;
90  import org.apache.hadoop.hbase.client.Row;
91  import org.apache.hadoop.hbase.client.Scan;
92  import org.apache.hadoop.hbase.client.coprocessor.Exec;
93  import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
94  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
95  import org.apache.hadoop.hbase.executor.ExecutorService;
96  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
97  import org.apache.hadoop.hbase.filter.BinaryComparator;
98  import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
99  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
100 import org.apache.hadoop.hbase.io.hfile.BlockCache;
101 import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
102 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
103 import org.apache.hadoop.hbase.io.hfile.CacheStats;
104 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
105 import org.apache.hadoop.hbase.ipc.HBaseRPC;
106 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
107 import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
108 import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
109 import org.apache.hadoop.hbase.ipc.HRegionInterface;
110 import org.apache.hadoop.hbase.ipc.Invocation;
111 import org.apache.hadoop.hbase.ipc.ProtocolSignature;
112 import org.apache.hadoop.hbase.ipc.RpcServer;
113 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
114 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
115 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
116 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
117 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
118 import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler;
119 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
120 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
121 import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
122 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
123 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
124 import org.apache.hadoop.hbase.regionserver.wal.HLog;
125 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
126 import org.apache.hadoop.hbase.replication.regionserver.Replication;
127 import org.apache.hadoop.hbase.security.User;
128 import org.apache.hadoop.hbase.util.Bytes;
129 import org.apache.hadoop.hbase.util.CompressionTest;
130 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
131 import org.apache.hadoop.hbase.util.FSTableDescriptors;
132 import org.apache.hadoop.hbase.util.FSUtils;
133 import org.apache.hadoop.hbase.util.InfoServer;
134 import org.apache.hadoop.hbase.util.Pair;
135 import org.apache.hadoop.hbase.util.Sleeper;
136 import org.apache.hadoop.hbase.util.Threads;
137 import org.apache.hadoop.hbase.util.VersionInfo;
138 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
139 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
140 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
141 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
142 import org.apache.hadoop.io.MapWritable;
143 import org.apache.hadoop.io.Writable;
144 import org.apache.hadoop.ipc.RemoteException;
145 import org.apache.hadoop.net.DNS;
146 import org.apache.hadoop.util.StringUtils;
147 import org.apache.zookeeper.KeeperException;
148 import org.codehaus.jackson.map.ObjectMapper;
149 
150 import com.google.common.base.Function;
151 import com.google.common.collect.Lists;
152 
153 /**
154  * HRegionServer makes a set of HRegions available to clients. It checks in with
155  * the HMaster. There are many HRegionServers in a single HBase deployment.
156  */
157 public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
158     Runnable, RegionServerServices {
159 
160   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
161 
162   // Set when a report to the master comes back with a message asking us to
163   // shutdown. Also set by call to stop when debugging or running unit tests
164   // of HRegionServer in isolation.
165   protected volatile boolean stopped = false;
166 
167   // A state before we go into stopped state.  At this stage we're closing user
168   // space regions.
169   private boolean stopping = false;
170 
171   // Go down hard. Used if file system becomes unavailable and also in
172   // debugging and unit tests.
173   protected volatile boolean abortRequested;
174 
175   private volatile boolean killed = false;
176 
177   // If false, the file system has become unavailable
178   protected volatile boolean fsOk;
179 
180   protected final Configuration conf;
181 
182   protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
183   private FileSystem fs;
184   private Path rootDir;
185   private final Random rand = new Random();
186 
187   //RegionName vs current action in progress
188   //true - if open region action in progress
189   //false - if close region action in progress
190   private final ConcurrentSkipListMap<byte[], Boolean> regionsInTransitionInRS =
191       new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
192 
193   /**
194    * Map of regions currently being served by this region server. Key is the
195    * encoded region name.  All access should be synchronized.
196    */
197   protected final Map<String, HRegion> onlineRegions =
198     new ConcurrentHashMap<String, HRegion>();
199 
200   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
201 
202   final int numRetries;
203   protected final int threadWakeFrequency;
204   private final int msgInterval;
205 
206   protected final int numRegionsToReport;
207 
208   private final long maxScannerResultSize;
209 
210   // Remote HMaster
211   private HMasterRegionInterface hbaseMaster;
212 
213   // Server to handle client requests. Default access so can be accessed by
214   // unit tests.
215   RpcServer rpcServer;
216 
217   private final InetSocketAddress isa;
218 
219   // Leases
220   private Leases leases;
221 
222   // Request counter.
223   // Do we need this?  Can't we just sum region counters?  St.Ack 20110412
224   private AtomicInteger requestCount = new AtomicInteger();
225 
226   // Info server. Default access so can be used by unit tests. REGIONSERVER
227   // is name of the webapp and the attribute name used stuffing this instance
228   // into web context.
229   InfoServer infoServer;
230 
231   /** region server process name */
232   public static final String REGIONSERVER = "regionserver";
233 
234   /*
235    * Space is reserved in HRS constructor and then released when aborting to
236    * recover from an OOME. See HBASE-706. TODO: Make this percentage of the heap
237    * or a minimum.
238    */
239   private final LinkedList<byte[]> reservedSpace = new LinkedList<byte[]>();
240 
241   private RegionServerMetrics metrics;
242 
243   // Compactions
244   public CompactSplitThread compactSplitThread;
245 
246   // Cache flushing
247   MemStoreFlusher cacheFlusher;
248 
249   /*
250    * Check for compactions requests.
251    */
252   Chore compactionChecker;
253 
254   // HLog and HLog roller. log is protected rather than private to avoid
255   // eclipse warning when accessed by inner classes
256   protected volatile HLog hlog;
257   LogRoller hlogRoller;
258 
259   // flag set after we're done setting up server threads (used for testing)
260   protected volatile boolean isOnline;
261 
262   final Map<String, RegionScanner> scanners =
263     new ConcurrentHashMap<String, RegionScanner>();
264 
265   // zookeeper connection and watcher
266   private ZooKeeperWatcher zooKeeper;
267 
268   // master address manager and watcher
269   private MasterAddressTracker masterAddressManager;
270 
271   // catalog tracker
272   private CatalogTracker catalogTracker;
273 
274   // Cluster Status Tracker
275   private ClusterStatusTracker clusterStatusTracker;
276 
277   // Log Splitting Worker
278   private SplitLogWorker splitLogWorker;
279 
280   // A sleeper that sleeps for msgInterval.
281   private final Sleeper sleeper;
282 
283   private final int rpcTimeout;
284 
285   // Instance of the hbase executor service.
286   private ExecutorService service;
287 
288   // Replication services. If no replication, this handler will be null.
289   private Replication replicationHandler;
290 
291   private final RegionServerAccounting regionServerAccounting;
292 
293   // Cache configuration and block cache reference
294   private final CacheConfig cacheConfig;
295 
296   /**
297    * The server name the Master sees us as.  Its made from the hostname the
298    * master passes us, port, and server startcode. Gets set after registration
299    * against  Master.  The hostname can differ from the hostname in {@link #isa}
300    * but usually doesn't if both servers resolve .
301    */
302   private ServerName serverNameFromMasterPOV;
303 
304   // Port we put up the webui on.
305   private int webuiport = -1;
306 
307   /**
308    * This servers startcode.
309    */
310   private final long startcode;
311 
312   /**
313    * Go here to get table descriptors.
314    */
315   private TableDescriptors tableDescriptors;
316 
317   /*
318    * Strings to be used in forming the exception message for
319    * RegionsAlreadyInTransitionException.
320    */
321   private static final String OPEN = "OPEN";
322   private static final String CLOSE = "CLOSE";
323 
324   /**
325    * Starts a HRegionServer at the default location
326    *
327    * @param conf
328    * @throws IOException
329    * @throws InterruptedException
330    */
331   public HRegionServer(Configuration conf)
332   throws IOException, InterruptedException {
333     this.fsOk = true;
334     this.conf = conf;
335     // Set how many times to retry talking to another server over HConnection.
336     HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
337     this.isOnline = false;
338     checkCodecs(this.conf);
339 
340     // Config'ed params
341     this.numRetries = conf.getInt("hbase.client.retries.number", 10);
342     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
343       10 * 1000);
344     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
345 
346     this.sleeper = new Sleeper(this.msgInterval, this);
347 
348     this.maxScannerResultSize = conf.getLong(
349       HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
350       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
351 
352     this.numRegionsToReport = conf.getInt(
353       "hbase.regionserver.numregionstoreport", 10);
354 
355     this.rpcTimeout = conf.getInt(
356       HConstants.HBASE_RPC_TIMEOUT_KEY,
357       HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
358 
359     this.abortRequested = false;
360     this.stopped = false;
361 
362     // Server to handle client requests.
363     String hostname = DNS.getDefaultHost(
364       conf.get("hbase.regionserver.dns.interface", "default"),
365       conf.get("hbase.regionserver.dns.nameserver", "default"));
366     int port = conf.getInt(HConstants.REGIONSERVER_PORT,
367       HConstants.DEFAULT_REGIONSERVER_PORT);
368     // Creation of a HSA will force a resolve.
369     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
370     if (initialIsa.getAddress() == null) {
371       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
372     }
373     this.rpcServer = HBaseRPC.getServer(this,
374       new Class<?>[]{HRegionInterface.class, HBaseRPCErrorHandler.class,
375         OnlineRegions.class},
376         initialIsa.getHostName(), // BindAddress is IP we got for this server.
377         initialIsa.getPort(),
378         conf.getInt("hbase.regionserver.handler.count", 10),
379         conf.getInt("hbase.regionserver.metahandler.count", 10),
380         conf.getBoolean("hbase.rpc.verbose", false),
381         conf, QOS_THRESHOLD);
382     // Set our address.
383     this.isa = this.rpcServer.getListenerAddress();
384 
385     this.rpcServer.setErrorHandler(this);
386     this.rpcServer.setQosFunction(new QosFunction());
387     this.startcode = System.currentTimeMillis();
388 
389     // login the server principal (if using secure Hadoop)
390     User.login(this.conf, "hbase.regionserver.keytab.file",
391       "hbase.regionserver.kerberos.principal", this.isa.getHostName());
392     regionServerAccounting = new RegionServerAccounting();
393     cacheConfig = new CacheConfig(conf);
394   }
395 
396   /**
397    * Run test on configured codecs to make sure supporting libs are in place.
398    * @param c
399    * @throws IOException
400    */
401   private static void checkCodecs(final Configuration c) throws IOException {
402     // check to see if the codec list is available:
403     String [] codecs = c.getStrings("hbase.regionserver.codecs", (String[])null);
404     if (codecs == null) return;
405     for (String codec : codecs) {
406       if (!CompressionTest.testCompression(codec)) {
407         throw new IOException("Compression codec " + codec +
408           " not supported, aborting RS construction");
409       }
410     }
411   }
412 
413   private static final int NORMAL_QOS = 0;
414   private static final int QOS_THRESHOLD = 10;  // the line between low and high qos
415   private static final int HIGH_QOS = 100;
416 
417   @Retention(RetentionPolicy.RUNTIME)
418   private @interface QosPriority {
419     int priority() default 0;
420   }
421 
422   /**
423    * Utility used ensuring higher quality of service for priority rpcs; e.g.
424    * rpcs to .META. and -ROOT-, etc.
425    */
426   class QosFunction implements Function<Writable,Integer> {
427     private final Map<String, Integer> annotatedQos;
428 
429     public QosFunction() {
430       Map<String, Integer> qosMap = new HashMap<String, Integer>();
431       for (Method m : HRegionServer.class.getMethods()) {
432         QosPriority p = m.getAnnotation(QosPriority.class);
433         if (p != null) {
434           qosMap.put(m.getName(), p.priority());
435         }
436       }
437 
438       annotatedQos = qosMap;
439     }
440 
441     public boolean isMetaRegion(byte[] regionName) {
442       HRegion region;
443       try {
444         region = getRegion(regionName);
445       } catch (NotServingRegionException ignored) {
446         return false;
447       }
448       return region.getRegionInfo().isMetaRegion();
449     }
450 
451     @Override
452     public Integer apply(Writable from) {
453       if (!(from instanceof Invocation)) return NORMAL_QOS;
454 
455       Invocation inv = (Invocation) from;
456       String methodName = inv.getMethodName();
457 
458       Integer priorityByAnnotation = annotatedQos.get(methodName);
459       if (priorityByAnnotation != null) {
460         return priorityByAnnotation;
461       }
462 
463       // scanner methods...
464       if (methodName.equals("next") || methodName.equals("close")) {
465         // translate!
466         Long scannerId;
467         try {
468           scannerId = (Long) inv.getParameters()[0];
469         } catch (ClassCastException ignored) {
470           // LOG.debug("Low priority: " + from);
471           return NORMAL_QOS; // doh.
472         }
473         String scannerIdString = Long.toString(scannerId);
474         RegionScanner scanner = scanners.get(scannerIdString);
475         if (scanner != null && scanner.getRegionInfo().isMetaRegion()) {
476           // LOG.debug("High priority scanner request: " + scannerId);
477           return HIGH_QOS;
478         }
479       } else if (inv.getParameterClasses().length == 0) {
480        // Just let it through.  This is getOnlineRegions, etc.
481       } else if (inv.getParameterClasses()[0] == byte[].class) {
482         // first arg is byte array, so assume this is a regionname:
483         if (isMetaRegion((byte[]) inv.getParameters()[0])) {
484           // LOG.debug("High priority with method: " + methodName +
485           // " and region: "
486           // + Bytes.toString((byte[]) inv.getParameters()[0]));
487           return HIGH_QOS;
488         }
489       } else if (inv.getParameterClasses()[0] == MultiAction.class) {
490         MultiAction<?> ma = (MultiAction<?>) inv.getParameters()[0];
491         Set<byte[]> regions = ma.getRegions();
492         // ok this sucks, but if any single of the actions touches a meta, the
493         // whole
494         // thing gets pingged high priority. This is a dangerous hack because
495         // people
496         // can get their multi action tagged high QOS by tossing a Get(.META.)
497         // AND this
498         // regionserver hosts META/-ROOT-
499         for (byte[] region : regions) {
500           if (isMetaRegion(region)) {
501             // LOG.debug("High priority multi with region: " +
502             // Bytes.toString(region));
503             return HIGH_QOS; // short circuit for the win.
504           }
505         }
506       }
507       // LOG.debug("Low priority: " + from.toString());
508       return NORMAL_QOS;
509     }
510   }
511 
512   /**
513    * All initialization needed before we go register with Master.
514    *
515    * @throws IOException
516    * @throws InterruptedException
517    */
518   private void preRegistrationInitialization(){
519     try {
520       initializeZooKeeper();
521       initializeThreads();
522       int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4);
523       for (int i = 0; i < nbBlocks; i++) {
524         reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]);
525       }
526     } catch (Throwable t) {
527       // Call stop if error or process will stick around for ever since server
528       // puts up non-daemon threads.
529       this.rpcServer.stop();
530       abort("Initialization of RS failed.  Hence aborting RS.", t);
531     }
532   }
533 
534   /**
535    * Bring up connection to zk ensemble and then wait until a master for this
536    * cluster and then after that, wait until cluster 'up' flag has been set.
537    * This is the order in which master does things.
538    * Finally put up a catalog tracker.
539    * @throws IOException
540    * @throws InterruptedException
541    */
542   private void initializeZooKeeper() throws IOException, InterruptedException {
543     // Open connection to zookeeper and set primary watcher
544     this.zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER + ":" +
545       this.isa.getPort(), this);
546 
547     // Create the master address manager, register with zk, and start it.  Then
548     // block until a master is available.  No point in starting up if no master
549     // running.
550     this.masterAddressManager = new MasterAddressTracker(this.zooKeeper, this);
551     this.masterAddressManager.start();
552     blockAndCheckIfStopped(this.masterAddressManager);
553 
554     // Wait on cluster being up.  Master will set this flag up in zookeeper
555     // when ready.
556     this.clusterStatusTracker = new ClusterStatusTracker(this.zooKeeper, this);
557     this.clusterStatusTracker.start();
558     blockAndCheckIfStopped(this.clusterStatusTracker);
559 
560     // Create the catalog tracker and start it;
561     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf,
562       this, this.conf.getInt("hbase.regionserver.catalog.timeout", Integer.MAX_VALUE));
563     catalogTracker.start();
564   }
565 
566   /**
567    * Utilty method to wait indefinitely on a znode availability while checking
568    * if the region server is shut down
569    * @param tracker znode tracker to use
570    * @throws IOException any IO exception, plus if the RS is stopped
571    * @throws InterruptedException
572    */
573   private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
574       throws IOException, InterruptedException {
575     if (false == tracker.checkIfBaseNodeAvailable()) {
576       String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. "
577           + "There could be a mismatch with the one configured in the master.";
578       LOG.error(errorMsg);
579       abort(errorMsg);
580     }
581     while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
582       if (this.stopped) {
583         throw new IOException("Received the shutdown message while waiting.");
584       }
585     }
586   }
587 
588   /**
589    * @return False if cluster shutdown in progress
590    */
591   private boolean isClusterUp() {
592     return this.clusterStatusTracker.isClusterUp();
593   }
594 
595   private void initializeThreads() throws IOException {
596     // Cache flushing thread.
597     this.cacheFlusher = new MemStoreFlusher(conf, this);
598 
599     // Compaction thread
600     this.compactSplitThread = new CompactSplitThread(this);
601 
602     // Background thread to check for compactions; needed if region
603     // has not gotten updates in a while. Make it run at a lesser frequency.
604     int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY +
605       ".multiplier", 1000);
606     this.compactionChecker = new CompactionChecker(this,
607       this.threadWakeFrequency * multiplier, this);
608 
609     this.leases = new Leases((int) conf.getLong(
610         HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
611         HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD),
612         this.threadWakeFrequency);
613   }
614 
615   /**
616    * The HRegionServer sticks in this loop until closed.
617    */
618   public void run() {
619     try {
620       // Do pre-registration initializations; zookeeper, lease threads, etc.
621       preRegistrationInitialization();
622     } catch (Throwable e) {
623       abort("Fatal exception during initialization", e);
624     }
625 
626     try {
627       // Try and register with the Master; tell it we are here.  Break if
628       // server is stopped or the clusterup flag is down or hdfs went wacky.
629       while (keepLooping()) {
630         MapWritable w = reportForDuty();
631         if (w == null) {
632           LOG.warn("reportForDuty failed; sleeping and then retrying.");
633           this.sleeper.sleep();
634         } else {
635           handleReportForDutyResponse(w);
636           break;
637         }
638       }
639 
640       // We registered with the Master.  Go into run mode.
641       long lastMsg = 0;
642       long oldRequestCount = -1;
643       // The main run loop.
644       while (!this.stopped && isHealthy()) {
645         if (!isClusterUp()) {
646           if (isOnlineRegionsEmpty()) {
647             stop("Exiting; cluster shutdown set and not carrying any regions");
648           } else if (!this.stopping) {
649             this.stopping = true;
650             LOG.info("Closing user regions");
651             closeUserRegions(this.abortRequested);
652           } else if (this.stopping) {
653             boolean allUserRegionsOffline = areAllUserRegionsOffline();
654             if (allUserRegionsOffline) {
655               // Set stopped if no requests since last time we went around the loop.
656               // The remaining meta regions will be closed on our way out.
657               if (oldRequestCount == this.requestCount.get()) {
658                 stop("Stopped; only catalog regions remaining online");
659                 break;
660               }
661               oldRequestCount = this.requestCount.get();
662             } else {
663               // Make sure all regions have been closed -- some regions may
664               // have not got it because we were splitting at the time of
665               // the call to closeUserRegions.
666               closeUserRegions(this.abortRequested);
667             }
668             LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
669           }
670         }
671         long now = System.currentTimeMillis();
672         if ((now - lastMsg) >= msgInterval) {
673           doMetrics();
674           tryRegionServerReport();
675           lastMsg = System.currentTimeMillis();
676         }
677         if (!this.stopped) this.sleeper.sleep();
678       } // for
679     } catch (Throwable t) {
680       if (!checkOOME(t)) {
681         abort("Unhandled exception: " + t.getMessage(), t);
682       }
683     }
684     // Run shutdown.
685     this.leases.closeAfterLeasesExpire();
686     this.rpcServer.stop();
687     if (this.splitLogWorker != null) {
688       splitLogWorker.stop();
689     }
690     if (this.infoServer != null) {
691       LOG.info("Stopping infoServer");
692       try {
693         this.infoServer.stop();
694       } catch (Exception e) {
695         e.printStackTrace();
696       }
697     }
698     // Send cache a shutdown.
699     if (cacheConfig.isBlockCacheEnabled()) {
700       cacheConfig.getBlockCache().shutdown();
701     }
702 
703     // Send interrupts to wake up threads if sleeping so they notice shutdown.
704     // TODO: Should we check they are alive? If OOME could have exited already
705     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
706     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
707     if (this.hlogRoller != null) this.hlogRoller.interruptIfNecessary();
708     if (this.compactionChecker != null)
709       this.compactionChecker.interrupt();
710 
711     if (this.killed) {
712       // Just skip out w/o closing regions.  Used when testing.
713     } else if (abortRequested) {
714       if (this.fsOk) {
715         closeAllRegions(abortRequested); // Don't leave any open file handles
716       }
717       LOG.info("aborting server " + this.serverNameFromMasterPOV);
718     } else {
719       closeAllRegions(abortRequested);
720       closeAllScanners();
721       LOG.info("stopping server " + this.serverNameFromMasterPOV);
722     }
723     // Interrupt catalog tracker here in case any regions being opened out in
724     // handlers are stuck waiting on meta or root.
725     if (this.catalogTracker != null) this.catalogTracker.stop();
726     if (this.fsOk) {
727       waitOnAllRegionsToClose(abortRequested);
728       LOG.info("stopping server " + this.serverNameFromMasterPOV +
729         "; all regions closed.");
730     }
731     
732     //fsOk flag may be changed when closing regions throws exception. 
733     if (!this.killed && this.fsOk) {
734       closeWAL(abortRequested ? false : true);
735     }
736     
737     // Make sure the proxy is down.
738     if (this.hbaseMaster != null) {
739       HBaseRPC.stopProxy(this.hbaseMaster);
740       this.hbaseMaster = null;
741     }
742     this.leases.close();
743     try {
744       deleteMyEphemeralNode();
745     } catch (KeeperException e) {
746       LOG.warn("Failed deleting my ephemeral node", e);
747     }
748     this.zooKeeper.close();
749     LOG.info("stopping server " + this.serverNameFromMasterPOV +
750       "; zookeeper connection closed.");
751 
752     if (!killed) {
753       join();
754     }
755     LOG.info(Thread.currentThread().getName() + " exiting");
756   }
757 
758   private boolean areAllUserRegionsOffline() {
759     if (getNumberOfOnlineRegions() > 2) return false;
760     boolean allUserRegionsOffline = true;
761     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
762       if (!e.getValue().getRegionInfo().isMetaRegion()) {
763         allUserRegionsOffline = false;
764         break;
765       }
766     }
767     return allUserRegionsOffline;
768   }
769 
770   void tryRegionServerReport()
771   throws IOException {
772     HServerLoad hsl = buildServerLoad();
773     // Why we do this?
774     this.requestCount.set(0);
775     try {
776       this.hbaseMaster.regionServerReport(this.serverNameFromMasterPOV.getVersionedBytes(), hsl);
777     } catch (IOException ioe) {
778       if (ioe instanceof RemoteException) {
779         ioe = ((RemoteException)ioe).unwrapRemoteException();
780       }
781       if (ioe instanceof YouAreDeadException) {
782         // This will be caught and handled as a fatal error in run()
783         throw ioe;
784       }
785       // Couldn't connect to the master, get location from zk and reconnect
786       // Method blocks until new master is found or we are stopped
787       getMaster();
788     }
789   }
790 
791   HServerLoad buildServerLoad() {
792     Collection<HRegion> regions = getOnlineRegionsLocalContext();
793     TreeMap<byte [], HServerLoad.RegionLoad> regionLoads =
794       new TreeMap<byte [], HServerLoad.RegionLoad>(Bytes.BYTES_COMPARATOR);
795     for (HRegion region: regions) {
796       regionLoads.put(region.getRegionName(), createRegionLoad(region));
797     }
798     MemoryUsage memory =
799       ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
800     return new HServerLoad(requestCount.get(),(int)metrics.getRequests(),
801       (int)(memory.getUsed() / 1024 / 1024),
802       (int) (memory.getMax() / 1024 / 1024), regionLoads,
803       this.hlog.getCoprocessorHost().getCoprocessors());
804   }
805 
806   String getOnlineRegionsAsPrintableString() {
807     StringBuilder sb = new StringBuilder();
808     for (HRegion r: this.onlineRegions.values()) {
809       if (sb.length() > 0) sb.append(", ");
810       sb.append(r.getRegionInfo().getEncodedName());
811     }
812     return sb.toString();
813   }
814 
815   /**
816    * Wait on regions close.
817    */
818   private void waitOnAllRegionsToClose(final boolean abort) {
819     // Wait till all regions are closed before going out.
820     int lastCount = -1;
821     while (!isOnlineRegionsEmpty()) {
822       int count = getNumberOfOnlineRegions();
823       // Only print a message if the count of regions has changed.
824       if (count != lastCount) {
825         lastCount = count;
826         LOG.info("Waiting on " + count + " regions to close");
827         // Only print out regions still closing if a small number else will
828         // swamp the log.
829         if (count < 10 && LOG.isDebugEnabled()) {
830           LOG.debug(this.onlineRegions);
831         }
832       }
833       // Ensure all user regions have been sent a close. Use this to
834       // protect against the case where an open comes in after we start the
835       // iterator of onlineRegions to close all user regions.
836       for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
837         HRegionInfo hri = e.getValue().getRegionInfo();
838         if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())) {
839           // Don't update zk with this close transition; pass false.
840           closeRegion(hri, abort, false);
841         }
842       }
843       Threads.sleep(1000);
844     }
845   }
846 
847   private void closeWAL(final boolean delete) {
848     try {
849       if (this.hlog != null) {
850         if (delete) {
851           hlog.closeAndDelete();
852         } else {
853           hlog.close();
854         }
855       }
856     } catch (Throwable e) {
857       LOG.error("Close and delete failed", RemoteExceptionHandler.checkThrowable(e));
858     }
859   }
860 
861   private void closeAllScanners() {
862     // Close any outstanding scanners. Means they'll get an UnknownScanner
863     // exception next time they come in.
864     for (Map.Entry<String, RegionScanner> e : this.scanners.entrySet()) {
865       try {
866         e.getValue().close();
867       } catch (IOException ioe) {
868         LOG.warn("Closing scanner " + e.getKey(), ioe);
869       }
870     }
871   }
872 
873   /*
874    * Run init. Sets up hlog and starts up all server threads.
875    *
876    * @param c Extra configuration.
877    */
878   protected void handleReportForDutyResponse(final MapWritable c)
879   throws IOException {
880     try {
881       for (Map.Entry<Writable, Writable> e :c.entrySet()) {
882         String key = e.getKey().toString();
883         // The hostname the master sees us as.
884         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
885           String hostnameFromMasterPOV = e.getValue().toString();
886           this.serverNameFromMasterPOV = new ServerName(hostnameFromMasterPOV,
887             this.isa.getPort(), this.startcode);
888           LOG.info("Master passed us hostname to use. Was=" +
889             this.isa.getHostName() + ", Now=" +
890             this.serverNameFromMasterPOV.getHostname());
891           continue;
892         }
893         String value = e.getValue().toString();
894         if (LOG.isDebugEnabled()) {
895           LOG.debug("Config from master: " + key + "=" + value);
896         }
897         this.conf.set(key, value);
898       }
899 
900       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
901       // config param for task trackers, but we can piggyback off of it.
902       if (this.conf.get("mapred.task.id") == null) {
903         this.conf.set("mapred.task.id", "hb_rs_" +
904           this.serverNameFromMasterPOV.toString());
905       }
906       // Set our ephemeral znode up in zookeeper now we have a name.
907       createMyEphemeralNode();
908 
909       // Master sent us hbase.rootdir to use. Should be fully qualified
910       // path with file system specification included. Set 'fs.defaultFS'
911       // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
912       // accessors will be going against wrong filesystem (unless all is set
913       // to defaults).
914       this.conf.set("fs.defaultFS", this.conf.get("hbase.rootdir"));
915       // Get fs instance used by this RS
916       this.fs = FileSystem.get(this.conf);
917       this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
918       this.tableDescriptors = new FSTableDescriptors(this.fs, this.rootDir, true);
919       this.hlog = setupWALAndReplication();
920       // Init in here rather than in constructor after thread name has been set
921       this.metrics = new RegionServerMetrics();
922       startServiceThreads();
923       LOG.info("Serving as " + this.serverNameFromMasterPOV +
924         ", RPC listening on " + this.isa +
925         ", sessionid=0x" +
926         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
927       isOnline = true;
928     } catch (Throwable e) {
929       this.isOnline = false;
930       stop("Failed initialization");
931       throw convertThrowableToIOE(cleanup(e, "Failed init"),
932           "Region server startup failed");
933     }
934   }
935 
936   private String getMyEphemeralNodePath() {
937     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
938   }
939 
940   private void createMyEphemeralNode() throws KeeperException {
941     ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper, getMyEphemeralNodePath(),
942       HConstants.EMPTY_BYTE_ARRAY);
943   }
944 
945   private void deleteMyEphemeralNode() throws KeeperException {
946     ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
947   }
948 
949   public RegionServerAccounting getRegionServerAccounting() {
950     return regionServerAccounting;
951   }
952 
953   /*
954    * @param r Region to get RegionLoad for.
955    *
956    * @return RegionLoad instance.
957    *
958    * @throws IOException
959    */
960   private HServerLoad.RegionLoad createRegionLoad(final HRegion r) {
961     byte[] name = r.getRegionName();
962     int stores = 0;
963     int storefiles = 0;
964     int storeUncompressedSizeMB = 0;
965     int storefileSizeMB = 0;
966     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
967     int storefileIndexSizeMB = 0;
968     int rootIndexSizeKB = 0;
969     int totalStaticIndexSizeKB = 0;
970     int totalStaticBloomSizeKB = 0;
971     long totalCompactingKVs = 0;
972     long currentCompactedKVs = 0;
973     synchronized (r.stores) {
974       stores += r.stores.size();
975       for (Store store : r.stores.values()) {
976         storefiles += store.getStorefilesCount();
977         storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
978             / 1024 / 1024);
979         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
980         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
981         CompactionProgress progress = store.getCompactionProgress();
982         if (progress != null) {
983           totalCompactingKVs += progress.totalCompactingKVs;
984           currentCompactedKVs += progress.currentCompactedKVs;
985         }
986 
987         rootIndexSizeKB +=
988             (int) (store.getStorefilesIndexSize() / 1024);
989 
990         totalStaticIndexSizeKB +=
991           (int) (store.getTotalStaticIndexSize() / 1024);
992 
993         totalStaticBloomSizeKB +=
994           (int) (store.getTotalStaticBloomSize() / 1024);
995       }
996     }
997     return new HServerLoad.RegionLoad(name, stores, storefiles,
998         storeUncompressedSizeMB,
999         storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, rootIndexSizeKB,
1000         totalStaticIndexSizeKB, totalStaticBloomSizeKB,
1001         (int) r.readRequestsCount.get(), (int) r.writeRequestsCount.get(),
1002         totalCompactingKVs, currentCompactedKVs,
1003         r.getCoprocessorHost().getCoprocessors());
1004   }
1005 
1006   /**
1007    * @param encodedRegionName
1008    * @return An instance of RegionLoad.
1009    */
1010   public HServerLoad.RegionLoad createRegionLoad(final String encodedRegionName) {
1011     HRegion r = null;
1012     r = this.onlineRegions.get(encodedRegionName);
1013     return r != null ? createRegionLoad(r) : null;
1014   }
1015 
1016   /*
1017    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
1018    * IOE if it isn't already.
1019    *
1020    * @param t Throwable
1021    *
1022    * @return Throwable converted to an IOE; methods can only let out IOEs.
1023    */
1024   private Throwable cleanup(final Throwable t) {
1025     return cleanup(t, null);
1026   }
1027 
1028   /*
1029    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
1030    * IOE if it isn't already.
1031    *
1032    * @param t Throwable
1033    *
1034    * @param msg Message to log in error. Can be null.
1035    *
1036    * @return Throwable converted to an IOE; methods can only let out IOEs.
1037    */
1038   private Throwable cleanup(final Throwable t, final String msg) {
1039     // Don't log as error if NSRE; NSRE is 'normal' operation.
1040     if (t instanceof NotServingRegionException) {
1041       LOG.debug("NotServingRegionException; " +  t.getMessage());
1042       return t;
1043     }
1044     if (msg == null) {
1045       LOG.error("", RemoteExceptionHandler.checkThrowable(t));
1046     } else {
1047       LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
1048     }
1049     if (!checkOOME(t)) {
1050       checkFileSystem();
1051     }
1052     return t;
1053   }
1054 
1055   /*
1056    * @param t
1057    *
1058    * @return Make <code>t</code> an IOE if it isn't already.
1059    */
1060   private IOException convertThrowableToIOE(final Throwable t) {
1061     return convertThrowableToIOE(t, null);
1062   }
1063 
1064   /*
1065    * @param t
1066    *
1067    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
1068    *
1069    * @return Make <code>t</code> an IOE if it isn't already.
1070    */
1071   private IOException convertThrowableToIOE(final Throwable t, final String msg) {
1072     return (t instanceof IOException ? (IOException) t : msg == null
1073         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
1074   }
1075 
1076   /*
1077    * Check if an OOME and if so, call abort.
1078    *
1079    * @param e
1080    *
1081    * @return True if we OOME'd and are aborting.
1082    */
1083   public boolean checkOOME(final Throwable e) {
1084     boolean stop = false;
1085     if (e instanceof OutOfMemoryError
1086         || (e.getCause() != null && e.getCause() instanceof OutOfMemoryError)
1087         || (e.getMessage() != null && e.getMessage().contains(
1088             "java.lang.OutOfMemoryError"))) {
1089       abort("OutOfMemoryError, aborting", e);
1090       stop = true;
1091     }
1092     return stop;
1093   }
1094 
1095   /**
1096    * Checks to see if the file system is still accessible. If not, sets
1097    * abortRequested and stopRequested
1098    *
1099    * @return false if file system is not available
1100    */
1101   public boolean checkFileSystem() {
1102     if (this.fsOk && this.fs != null) {
1103       try {
1104         FSUtils.checkFileSystemAvailable(this.fs);
1105       } catch (IOException e) {
1106         abort("File System not available", e);
1107         this.fsOk = false;
1108       }
1109     }
1110     return this.fsOk;
1111   }
1112 
1113   /*
1114    * Inner class that runs on a long period checking if regions need compaction.
1115    */
1116   private static class CompactionChecker extends Chore {
1117     private final HRegionServer instance;
1118     private final int majorCompactPriority;
1119     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1120 
1121     CompactionChecker(final HRegionServer h, final int sleepTime,
1122         final Stoppable stopper) {
1123       super("CompactionChecker", sleepTime, h);
1124       this.instance = h;
1125       LOG.info("Runs every " + StringUtils.formatTime(sleepTime));
1126 
1127       /* MajorCompactPriority is configurable.
1128        * If not set, the compaction will use default priority.
1129        */
1130       this.majorCompactPriority = this.instance.conf.
1131         getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1132         DEFAULT_PRIORITY);
1133     }
1134 
1135     @Override
1136     protected void chore() {
1137       for (HRegion r : this.instance.onlineRegions.values()) {
1138         if (r == null)
1139           continue;
1140         for (Store s : r.getStores().values()) {
1141           try {
1142             if (s.needsCompaction()) {
1143               // Queue a compaction. Will recognize if major is needed.
1144               this.instance.compactSplitThread.requestCompaction(r, s,
1145                 getName() + " requests compaction");
1146             } else if (s.isMajorCompaction()) {
1147               if (majorCompactPriority == DEFAULT_PRIORITY ||
1148                   majorCompactPriority > r.getCompactPriority()) {
1149                 this.instance.compactSplitThread.requestCompaction(r, s,
1150                     getName() + " requests major compaction; use default priority");
1151               } else {
1152                this.instance.compactSplitThread.requestCompaction(r, s,
1153                   getName() + " requests major compaction; use configured priority",
1154                   this.majorCompactPriority);
1155               }
1156             }
1157           } catch (IOException e) {
1158             LOG.warn("Failed major compaction check on " + r, e);
1159           }
1160         }
1161       }
1162     }
1163   }
1164 
1165   /**
1166    * Report the status of the server. A server is online once all the startup is
1167    * completed (setting up filesystem, starting service threads, etc.). This
1168    * method is designed mostly to be useful in tests.
1169    *
1170    * @return true if online, false if not.
1171    */
1172   public boolean isOnline() {
1173     return isOnline;
1174   }
1175 
1176   /**
1177    * Setup WAL log and replication if enabled.
1178    * Replication setup is done in here because it wants to be hooked up to WAL.
1179    * @return A WAL instance.
1180    * @throws IOException
1181    */
1182   private HLog setupWALAndReplication() throws IOException {
1183     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1184     Path logdir = new Path(rootDir,
1185       HLog.getHLogDirectoryName(this.serverNameFromMasterPOV.toString()));
1186     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1187     if (this.fs.exists(logdir)) {
1188       throw new RegionServerRunningException("Region server has already " +
1189         "created directory at " + this.serverNameFromMasterPOV.toString());
1190     }
1191 
1192     // Instantiate replication manager if replication enabled.  Pass it the
1193     // log directories.
1194     try {
1195       this.replicationHandler = Replication.isReplication(this.conf)?
1196         new Replication(this, this.fs, logdir, oldLogDir): null;
1197     } catch (KeeperException e) {
1198       throw new IOException("Failed replication handler create", e);
1199     }
1200     return instantiateHLog(logdir, oldLogDir);
1201   }
1202 
1203   /**
1204    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1205    * @param logdir
1206    * @param oldLogDir
1207    * @return WAL instance.
1208    * @throws IOException
1209    */
1210   protected HLog instantiateHLog(Path logdir, Path oldLogDir) throws IOException {
1211     return new HLog(this.fs, logdir, oldLogDir, this.conf,
1212       getWALActionListeners(), this.serverNameFromMasterPOV.toString());
1213   }
1214 
1215   /**
1216    * Called by {@link #instantiateHLog(Path, Path)} setting up WAL instance.
1217    * Add any {@link WALActionsListener}s you want inserted before WAL startup.
1218    * @return List of WALActionsListener that will be passed in to
1219    * {@link HLog} on construction.
1220    */
1221   protected List<WALActionsListener> getWALActionListeners() {
1222     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1223     // Log roller.
1224     this.hlogRoller = new LogRoller(this, this);
1225     listeners.add(this.hlogRoller);
1226     if (this.replicationHandler != null) {
1227       // Replication handler is an implementation of WALActionsListener.
1228       listeners.add(this.replicationHandler);
1229     }
1230     return listeners;
1231   }
1232 
1233   protected LogRoller getLogRoller() {
1234     return hlogRoller;
1235   }
1236 
1237   /*
1238    * @param interval Interval since last time metrics were called.
1239    */
1240   protected void doMetrics() {
1241     try {
1242       metrics();
1243     } catch (Throwable e) {
1244       LOG.warn("Failed metrics", e);
1245     }
1246   }
1247 
1248   protected void metrics() {
1249     this.metrics.regions.set(this.onlineRegions.size());
1250     this.metrics.incrementRequests(this.requestCount.get());
1251     this.metrics.requests.intervalHeartBeat();
1252     // Is this too expensive every three seconds getting a lock on onlineRegions
1253     // and then per store carried? Can I make metrics be sloppier and avoid
1254     // the synchronizations?
1255     int stores = 0;
1256     int storefiles = 0;
1257     long memstoreSize = 0;
1258     int readRequestsCount = 0;
1259     int writeRequestsCount = 0;
1260     long storefileIndexSize = 0;
1261     HDFSBlocksDistribution hdfsBlocksDistribution =
1262       new HDFSBlocksDistribution();
1263     long totalStaticIndexSize = 0;
1264     long totalStaticBloomSize = 0;
1265     for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1266         HRegion r = e.getValue();
1267         memstoreSize += r.memstoreSize.get();
1268         readRequestsCount += r.readRequestsCount.get();
1269         writeRequestsCount += r.writeRequestsCount.get();
1270         synchronized (r.stores) {
1271           stores += r.stores.size();
1272           for (Map.Entry<byte[], Store> ee : r.stores.entrySet()) {
1273             Store store = ee.getValue();
1274             storefiles += store.getStorefilesCount();
1275             storefileIndexSize += store.getStorefilesIndexSize();
1276             totalStaticIndexSize += store.getTotalStaticIndexSize();
1277             totalStaticBloomSize += store.getTotalStaticBloomSize();
1278           }
1279         }
1280 
1281         hdfsBlocksDistribution.add(r.getHDFSBlocksDistribution());
1282       }
1283     this.metrics.stores.set(stores);
1284     this.metrics.storefiles.set(storefiles);
1285     this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024)));
1286     this.metrics.storefileIndexSizeMB.set(
1287         (int) (storefileIndexSize / (1024 * 1024)));
1288     this.metrics.rootIndexSizeKB.set(
1289         (int) (storefileIndexSize / 1024));
1290     this.metrics.totalStaticIndexSizeKB.set(
1291         (int) (totalStaticIndexSize / 1024));
1292     this.metrics.totalStaticBloomSizeKB.set(
1293         (int) (totalStaticBloomSize / 1024));
1294     this.metrics.readRequestsCount.set(readRequestsCount);
1295     this.metrics.writeRequestsCount.set(writeRequestsCount);
1296 
1297     BlockCache blockCache = cacheConfig.getBlockCache();
1298     if (blockCache != null) {
1299       this.metrics.blockCacheCount.set(blockCache.size());
1300       this.metrics.blockCacheFree.set(blockCache.getFreeSize());
1301       this.metrics.blockCacheSize.set(blockCache.getCurrentSize());
1302       CacheStats cacheStats = blockCache.getStats();
1303       this.metrics.blockCacheHitCount.set(cacheStats.getHitCount());
1304       this.metrics.blockCacheMissCount.set(cacheStats.getMissCount());
1305       this.metrics.blockCacheEvictedCount.set(blockCache.getEvictedCount());
1306       double ratio = blockCache.getStats().getHitRatio();
1307       int percent = (int) (ratio * 100);
1308       this.metrics.blockCacheHitRatio.set(percent);
1309       ratio = blockCache.getStats().getHitCachingRatio();
1310       percent = (int) (ratio * 100);
1311       this.metrics.blockCacheHitCachingRatio.set(percent);
1312     }
1313     float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
1314       getServerName().getHostname());
1315     int percent = (int) (localityIndex * 100);
1316     this.metrics.hdfsBlocksLocalityIndex.set(percent);
1317 
1318   }
1319 
1320   /**
1321    * @return Region server metrics instance.
1322    */
1323   public RegionServerMetrics getMetrics() {
1324     return this.metrics;
1325   }
1326 
1327   /*
1328    * Start maintanence Threads, Server, Worker and lease checker threads.
1329    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1330    * get an unhandled exception. We cannot set the handler on all threads.
1331    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1332    * waits a while then retries. Meantime, a flush or a compaction that tries to
1333    * run should trigger same critical condition and the shutdown will run. On
1334    * its way out, this server will shut down Server. Leases are sort of
1335    * inbetween. It has an internal thread that while it inherits from Chore, it
1336    * keeps its own internal stop mechanism so needs to be stopped by this
1337    * hosting server. Worker logs the exception and exits.
1338    */
1339   private void startServiceThreads() throws IOException {
1340     String n = Thread.currentThread().getName();
1341     UncaughtExceptionHandler handler = new UncaughtExceptionHandler() {
1342       public void uncaughtException(Thread t, Throwable e) {
1343         abort("Uncaught exception in service thread " + t.getName(), e);
1344       }
1345     };
1346 
1347     // Start executor services
1348     this.service = new ExecutorService(getServerName().toString());
1349     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1350       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1351     this.service.startExecutorService(ExecutorType.RS_OPEN_ROOT,
1352       conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
1353     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1354       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1355     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1356       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1357     this.service.startExecutorService(ExecutorType.RS_CLOSE_ROOT,
1358       conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
1359     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1360       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1361 
1362     Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), n + ".logRoller", handler);
1363     Threads.setDaemonThreadRunning(this.cacheFlusher.getThread(), n + ".cacheFlusher",
1364       handler);
1365     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), n +
1366       ".compactionChecker", handler);
1367 
1368     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1369     // an unhandled exception, it will just exit.
1370     this.leases.setName(n + ".leaseChecker");
1371     this.leases.start();
1372 
1373     // Put up the webui.  Webui may come up on port other than configured if
1374     // that port is occupied. Adjust serverInfo if this is the case.
1375     this.webuiport = putUpWebUI();
1376 
1377     if (this.replicationHandler != null) {
1378       this.replicationHandler.startReplicationServices();
1379     }
1380 
1381     // Start Server.  This service is like leases in that it internally runs
1382     // a thread.
1383     this.rpcServer.start();
1384 
1385     // Create the log splitting worker and start it
1386     this.splitLogWorker = new SplitLogWorker(this.zooKeeper,
1387         this.getConfiguration(), this.getServerName().toString());
1388     splitLogWorker.start();
1389   }
1390 
1391   /**
1392    * Puts up the webui.
1393    * @return Returns final port -- maybe different from what we started with.
1394    * @throws IOException
1395    */
1396   private int putUpWebUI() throws IOException {
1397     int port = this.conf.getInt("hbase.regionserver.info.port", 60030);
1398     // -1 is for disabling info server
1399     if (port < 0) return port;
1400     String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1401     // check if auto port bind enabled
1402     boolean auto = this.conf.getBoolean("hbase.regionserver.info.port.auto", false);
1403     while (true) {
1404       try {
1405         this.infoServer = new InfoServer("regionserver", addr, port, false, this.conf);
1406         this.infoServer.addServlet("status", "/rs-status", RSStatusServlet.class);
1407         this.infoServer.addServlet("dump", "/dump", RSDumpServlet.class);
1408         this.infoServer.setAttribute(REGIONSERVER, this);
1409         this.infoServer.start();
1410         break;
1411       } catch (BindException e) {
1412         if (!auto) {
1413           // auto bind disabled throw BindException
1414           throw e;
1415         }
1416         // auto bind enabled, try to use another port
1417         LOG.info("Failed binding http info server to port: " + port);
1418         port++;
1419       }
1420     }
1421     return port;
1422   }
1423 
1424   /*
1425    * Verify that server is healthy
1426    */
1427   private boolean isHealthy() {
1428     if (!fsOk) {
1429       // File system problem
1430       return false;
1431     }
1432     // Verify that all threads are alive
1433     if (!(leases.isAlive()
1434         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1435         && this.compactionChecker.isAlive())) {
1436       stop("One or more threads are no longer alive -- stop");
1437       return false;
1438     }
1439     return true;
1440   }
1441 
1442   @Override
1443   public HLog getWAL() {
1444     return this.hlog;
1445   }
1446 
1447   @Override
1448   public CatalogTracker getCatalogTracker() {
1449     return this.catalogTracker;
1450   }
1451 
1452   @Override
1453   public void stop(final String msg) {
1454     this.stopped = true;
1455     LOG.info("STOPPED: " + msg);
1456     synchronized (this) {
1457       // Wakes run() if it is sleeping
1458       notifyAll(); // FindBugs NN_NAKED_NOTIFY
1459     }
1460   }
1461 
1462   @Override
1463   public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct,
1464       final boolean daughter)
1465   throws KeeperException, IOException {
1466     LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString() +
1467       ", daughter=" + daughter);
1468     // Do checks to see if we need to compact (references or too many files)
1469     for (Store s : r.getStores().values()) {
1470       if (s.hasReferences() || s.needsCompaction()) {
1471         getCompactionRequester().requestCompaction(r, s, "Opening Region");
1472       }
1473     }
1474 
1475     // Add to online regions if all above was successful.
1476     addToOnlineRegions(r);
1477     // Update ZK, ROOT or META
1478     if (r.getRegionInfo().isRootRegion()) {
1479       RootLocationEditor.setRootLocation(getZooKeeper(),
1480        this.serverNameFromMasterPOV);
1481     } else if (r.getRegionInfo().isMetaRegion()) {
1482       MetaEditor.updateMetaLocation(ct, r.getRegionInfo(),
1483         this.serverNameFromMasterPOV);
1484     } else {
1485       if (daughter) {
1486         // If daughter of a split, update whole row, not just location.
1487         MetaEditor.addDaughter(ct, r.getRegionInfo(),
1488           this.serverNameFromMasterPOV);
1489       } else {
1490         MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
1491           this.serverNameFromMasterPOV);
1492       }
1493     }
1494     LOG.info("Done with post open deploy task for region=" +
1495       r.getRegionNameAsString() + ", daughter=" + daughter);
1496 
1497   }
1498 
1499   /**
1500    * Return a reference to the metrics instance used for counting RPC calls.
1501    * @return Metrics instance.
1502    */
1503   public HBaseRpcMetrics getRpcMetrics() {
1504     return rpcServer.getRpcMetrics();
1505   }
1506 
1507   @Override
1508   public RpcServer getRpcServer() {
1509     return rpcServer;
1510   }
1511 
1512   /**
1513    * Cause the server to exit without closing the regions it is serving, the log
1514    * it is using and without notifying the master. Used unit testing and on
1515    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1516    *
1517    * @param reason
1518    *          the reason we are aborting
1519    * @param cause
1520    *          the exception that caused the abort, or null
1521    */
1522   public void abort(String reason, Throwable cause) {
1523     String msg = "ABORTING region server " + this + ": " + reason;
1524     if (cause != null) {
1525       LOG.fatal(msg, cause);
1526     } else {
1527       LOG.fatal(msg);
1528     }
1529     this.abortRequested = true;
1530     this.reservedSpace.clear();
1531     // HBASE-4014: show list of coprocessors that were loaded to help debug
1532     // regionserver crashes.Note that we're implicitly using
1533     // java.util.HashSet's toString() method to print the coprocessor names.
1534     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1535         CoprocessorHost.getLoadedCoprocessors());
1536     if (this.metrics != null) {
1537       LOG.info("Dump of metrics: " + this.metrics);
1538     }
1539     // Do our best to report our abort to the master, but this may not work
1540     try {
1541       if (cause != null) {
1542         msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1543       }
1544       if (hbaseMaster != null) {
1545         hbaseMaster.reportRSFatalError(
1546             this.serverNameFromMasterPOV.getVersionedBytes(), msg);
1547       }
1548     } catch (Throwable t) {
1549       LOG.warn("Unable to report fatal error to master", t);
1550     }
1551     stop(reason);
1552   }
1553 
1554   /**
1555    * @see HRegionServer#abort(String, Throwable)
1556    */
1557   public void abort(String reason) {
1558     abort(reason, null);
1559   }
1560 
1561   public boolean isAborted() {
1562     return this.abortRequested;
1563   }
1564   
1565   /*
1566    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1567    * logs but it does close socket in case want to bring up server on old
1568    * hostname+port immediately.
1569    */
1570   protected void kill() {
1571     this.killed = true;
1572     abort("Simulated kill");
1573   }
1574 
1575   /**
1576    * Wait on all threads to finish. Presumption is that all closes and stops
1577    * have already been called.
1578    */
1579   protected void join() {
1580     Threads.shutdown(this.compactionChecker.getThread());
1581     Threads.shutdown(this.cacheFlusher.getThread());
1582     Threads.shutdown(this.hlogRoller.getThread());
1583     if (this.compactSplitThread != null) {
1584       this.compactSplitThread.join();
1585     }
1586     if (this.service != null) this.service.shutdown();
1587     if (this.replicationHandler != null) {
1588       this.replicationHandler.join();
1589     }
1590   }
1591 
1592   /**
1593    * Get the current master from ZooKeeper and open the RPC connection to it.
1594    *
1595    * Method will block until a master is available. You can break from this
1596    * block by requesting the server stop.
1597    *
1598    * @return master + port, or null if server has been stopped
1599    */
1600   private ServerName getMaster() {
1601     ServerName masterServerName = null;
1602     while ((masterServerName = this.masterAddressManager.getMasterAddress()) == null) {
1603       if (!keepLooping()) return null;
1604       LOG.debug("No master found; retry");
1605       sleeper.sleep();
1606     }
1607     InetSocketAddress isa =
1608       new InetSocketAddress(masterServerName.getHostname(), masterServerName.getPort());
1609     HMasterRegionInterface master = null;
1610     while (keepLooping() && master == null) {
1611       LOG.info("Attempting connect to Master server at " +
1612         this.masterAddressManager.getMasterAddress());
1613       try {
1614         // Do initial RPC setup. The final argument indicates that the RPC
1615         // should retry indefinitely.
1616         master = (HMasterRegionInterface) HBaseRPC.waitForProxy(
1617             HMasterRegionInterface.class, HMasterRegionInterface.VERSION,
1618             isa, this.conf, -1,
1619             this.rpcTimeout, this.rpcTimeout);
1620       } catch (IOException e) {
1621         e = e instanceof RemoteException ?
1622             ((RemoteException)e).unwrapRemoteException() : e;
1623         if (e instanceof ServerNotRunningYetException) {
1624           LOG.info("Master isn't available yet, retrying");
1625         } else {
1626           LOG.warn("Unable to connect to master. Retrying. Error was:", e);
1627         }
1628         sleeper.sleep();
1629       }
1630     }
1631     LOG.info("Connected to master at " + isa);
1632     this.hbaseMaster = master;
1633     return masterServerName;
1634   }
1635 
1636   /**
1637    * @return True if we should break loop because cluster is going down or
1638    * this server has been stopped or hdfs has gone bad.
1639    */
1640   private boolean keepLooping() {
1641     return !this.stopped && isClusterUp();
1642   }
1643 
1644   /*
1645    * Let the master know we're here Run initialization using parameters passed
1646    * us by the master.
1647    * @return A Map of key/value configurations we got from the Master else
1648    * null if we failed to register.
1649    * @throws IOException
1650    */
1651   private MapWritable reportForDuty() throws IOException {
1652     MapWritable result = null;
1653     ServerName masterServerName = getMaster();
1654     if (masterServerName == null) return result;
1655     try {
1656       this.requestCount.set(0);
1657       LOG.info("Telling master at " + masterServerName + " that we are up " +
1658         "with port=" + this.isa.getPort() + ", startcode=" + this.startcode);
1659       long now = EnvironmentEdgeManager.currentTimeMillis();
1660       int port = this.isa.getPort();
1661       result = this.hbaseMaster.regionServerStartup(port, this.startcode, now);
1662     } catch (RemoteException e) {
1663       IOException ioe = e.unwrapRemoteException();
1664       if (ioe instanceof ClockOutOfSyncException) {
1665         LOG.fatal("Master rejected startup because clock is out of sync", ioe);
1666         // Re-throw IOE will cause RS to abort
1667         throw ioe;
1668       } else {
1669         LOG.warn("remote error telling master we are up", e);
1670       }
1671     } catch (IOException e) {
1672       LOG.warn("error telling master we are up", e);
1673     }
1674     return result;
1675   }
1676 
1677   /**
1678    * Closes all regions.  Called on our way out.
1679    * Assumes that its not possible for new regions to be added to onlineRegions
1680    * while this method runs.
1681    */
1682   protected void closeAllRegions(final boolean abort) {
1683     closeUserRegions(abort);
1684     // Only root and meta should remain.  Are we carrying root or meta?
1685     HRegion meta = null;
1686     HRegion root = null;
1687     this.lock.writeLock().lock();
1688     try {
1689       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
1690         HRegionInfo hri = e.getValue().getRegionInfo();
1691         if (hri.isRootRegion()) {
1692           root = e.getValue();
1693         } else if (hri.isMetaRegion()) {
1694           meta = e.getValue();
1695         }
1696         if (meta != null && root != null) break;
1697       }
1698     } finally {
1699       this.lock.writeLock().unlock();
1700     }
1701     if (meta != null) closeRegion(meta.getRegionInfo(), abort, false);
1702     if (root != null) closeRegion(root.getRegionInfo(), abort, false);
1703   }
1704 
1705   /**
1706    * Schedule closes on all user regions.
1707    * Should be safe calling multiple times because it wont' close regions
1708    * that are already closed or that are closing.
1709    * @param abort Whether we're running an abort.
1710    */
1711   void closeUserRegions(final boolean abort) {
1712     this.lock.writeLock().lock();
1713     try {
1714       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1715         HRegion r = e.getValue();
1716         if (!r.getRegionInfo().isMetaRegion()) {
1717           if (r.isClosed() || r.isClosing()) continue;
1718           // Don't update zk with this close transition; pass false.
1719           closeRegion(r.getRegionInfo(), abort, false);
1720         }
1721       }
1722     } finally {
1723       this.lock.writeLock().unlock();
1724     }
1725   }
1726 
1727   @Override
1728   @QosPriority(priority=HIGH_QOS)
1729   public HRegionInfo getRegionInfo(final byte[] regionName)
1730   throws NotServingRegionException, IOException {
1731     checkOpen();
1732     requestCount.incrementAndGet();
1733     return getRegion(regionName).getRegionInfo();
1734   }
1735 
1736   public Result getClosestRowBefore(final byte[] regionName, final byte[] row,
1737       final byte[] family) throws IOException {
1738     checkOpen();
1739     requestCount.incrementAndGet();
1740     try {
1741       // locate the region we're operating on
1742       HRegion region = getRegion(regionName);
1743       // ask the region for all the data
1744 
1745       Result r = region.getClosestRowBefore(row, family);
1746       return r;
1747     } catch (Throwable t) {
1748       throw convertThrowableToIOE(cleanup(t));
1749     }
1750   }
1751 
1752   /** {@inheritDoc} */
1753   public Result get(byte[] regionName, Get get) throws IOException {
1754     checkOpen();
1755     requestCount.incrementAndGet();
1756     try {
1757       HRegion region = getRegion(regionName);
1758       return region.get(get, getLockFromId(get.getLockId()));
1759     } catch (Throwable t) {
1760       throw convertThrowableToIOE(cleanup(t));
1761     }
1762   }
1763 
1764   public boolean exists(byte[] regionName, Get get) throws IOException {
1765     checkOpen();
1766     requestCount.incrementAndGet();
1767     try {
1768       HRegion region = getRegion(regionName);
1769       Integer lock = getLockFromId(get.getLockId());
1770       if (region.getCoprocessorHost() != null) {
1771         Boolean result = region.getCoprocessorHost().preExists(get);
1772         if (result != null) {
1773           return result.booleanValue();
1774         }
1775       }
1776       Result r = region.get(get, lock);
1777       boolean result = r != null && !r.isEmpty();
1778       if (region.getCoprocessorHost() != null) {
1779         result = region.getCoprocessorHost().postExists(get, result);
1780       }
1781       return result;
1782     } catch (Throwable t) {
1783       throw convertThrowableToIOE(cleanup(t));
1784     }
1785   }
1786 
1787   public void put(final byte[] regionName, final Put put) throws IOException {
1788     if (put.getRow() == null) {
1789       throw new IllegalArgumentException("update has null row");
1790     }
1791 
1792     checkOpen();
1793     this.requestCount.incrementAndGet();
1794     HRegion region = getRegion(regionName);
1795     try {
1796       if (!region.getRegionInfo().isMetaTable()) {
1797         this.cacheFlusher.reclaimMemStoreMemory();
1798       }
1799       boolean writeToWAL = put.getWriteToWAL();
1800       region.put(put, getLockFromId(put.getLockId()), writeToWAL);
1801     } catch (Throwable t) {
1802       throw convertThrowableToIOE(cleanup(t));
1803     }
1804   }
1805 
1806   public int put(final byte[] regionName, final List<Put> puts)
1807       throws IOException {
1808     checkOpen();
1809     HRegion region = null;
1810     try {
1811       region = getRegion(regionName);
1812       if (!region.getRegionInfo().isMetaTable()) {
1813         this.cacheFlusher.reclaimMemStoreMemory();
1814       }
1815 
1816       @SuppressWarnings("unchecked")
1817       Pair<Put, Integer>[] putsWithLocks = new Pair[puts.size()];
1818 
1819       int i = 0;
1820       for (Put p : puts) {
1821         Integer lock = getLockFromId(p.getLockId());
1822         putsWithLocks[i++] = new Pair<Put, Integer>(p, lock);
1823       }
1824 
1825       this.requestCount.addAndGet(puts.size());
1826       OperationStatus codes[] = region.put(putsWithLocks);
1827       for (i = 0; i < codes.length; i++) {
1828         if (codes[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
1829           return i;
1830         }
1831       }
1832       return -1;
1833     } catch (Throwable t) {
1834       throw convertThrowableToIOE(cleanup(t));
1835     }
1836   }
1837 
1838   private boolean checkAndMutate(final byte[] regionName, final byte[] row,
1839       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
1840       final WritableByteArrayComparable comparator, final Writable w,
1841       Integer lock) throws IOException {
1842     checkOpen();
1843     this.requestCount.incrementAndGet();
1844     HRegion region = getRegion(regionName);
1845     try {
1846       if (!region.getRegionInfo().isMetaTable()) {
1847         this.cacheFlusher.reclaimMemStoreMemory();
1848       }
1849       return region.checkAndMutate(row, family, qualifier, compareOp,
1850         comparator, w, lock, true);
1851     } catch (Throwable t) {
1852       throw convertThrowableToIOE(cleanup(t));
1853     }
1854   }
1855 
1856   /**
1857    *
1858    * @param regionName
1859    * @param row
1860    * @param family
1861    * @param qualifier
1862    * @param value
1863    *          the expected value
1864    * @param put
1865    * @throws IOException
1866    * @return true if the new put was execute, false otherwise
1867    */
1868   public boolean checkAndPut(final byte[] regionName, final byte[] row,
1869       final byte[] family, final byte[] qualifier, final byte[] value,
1870       final Put put) throws IOException {
1871     checkOpen();
1872     if (regionName == null) {
1873       throw new IOException("Invalid arguments to checkAndPut "
1874           + "regionName is null");
1875     }
1876     HRegion region = getRegion(regionName);
1877     Integer lock = getLockFromId(put.getLockId());
1878     WritableByteArrayComparable comparator = new BinaryComparator(value);
1879     if (region.getCoprocessorHost() != null) {
1880       Boolean result = region.getCoprocessorHost()
1881         .preCheckAndPut(row, family, qualifier, CompareOp.EQUAL, comparator,
1882           put);
1883       if (result != null) {
1884         return result.booleanValue();
1885       }
1886     }
1887     boolean result = checkAndMutate(regionName, row, family, qualifier,
1888       CompareOp.EQUAL, new BinaryComparator(value), put,
1889       lock);
1890     if (region.getCoprocessorHost() != null) {
1891       result = region.getCoprocessorHost().postCheckAndPut(row, family,
1892         qualifier, CompareOp.EQUAL, comparator, put, result);
1893     }
1894     return result;
1895   }
1896 
1897   /**
1898    *
1899    * @param regionName
1900    * @param row
1901    * @param family
1902    * @param qualifier
1903    * @param compareOp
1904    * @param comparator
1905    * @param put
1906    * @throws IOException
1907    * @return true if the new put was execute, false otherwise
1908    */
1909   public boolean checkAndPut(final byte[] regionName, final byte[] row,
1910       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
1911       final WritableByteArrayComparable comparator, final Put put)
1912        throws IOException {
1913     checkOpen();
1914     if (regionName == null) {
1915       throw new IOException("Invalid arguments to checkAndPut "
1916           + "regionName is null");
1917     }
1918     HRegion region = getRegion(regionName);
1919     Integer lock = getLockFromId(put.getLockId());
1920     if (region.getCoprocessorHost() != null) {
1921       Boolean result = region.getCoprocessorHost()
1922         .preCheckAndPut(row, family, qualifier, compareOp, comparator, put);
1923       if (result != null) {
1924         return result.booleanValue();
1925       }
1926     }
1927     boolean result = checkAndMutate(regionName, row, family, qualifier,
1928       compareOp, comparator, put, lock);
1929     if (region.getCoprocessorHost() != null) {
1930       result = region.getCoprocessorHost().postCheckAndPut(row, family,
1931         qualifier, compareOp, comparator, put, result);
1932     }
1933     return result;
1934   }
1935 
1936   /**
1937    *
1938    * @param regionName
1939    * @param row
1940    * @param family
1941    * @param qualifier
1942    * @param value
1943    *          the expected value
1944    * @param delete
1945    * @throws IOException
1946    * @return true if the new put was execute, false otherwise
1947    */
1948   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
1949       final byte[] family, final byte[] qualifier, final byte[] value,
1950       final Delete delete) throws IOException {
1951     checkOpen();
1952 
1953     if (regionName == null) {
1954       throw new IOException("Invalid arguments to checkAndDelete "
1955           + "regionName is null");
1956     }
1957     HRegion region = getRegion(regionName);
1958     Integer lock = getLockFromId(delete.getLockId());
1959     WritableByteArrayComparable comparator = new BinaryComparator(value);
1960     if (region.getCoprocessorHost() != null) {
1961       Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
1962         family, qualifier, CompareOp.EQUAL, comparator, delete);
1963       if (result != null) {
1964         return result.booleanValue();
1965       }
1966     }
1967     boolean result = checkAndMutate(regionName, row, family, qualifier,
1968       CompareOp.EQUAL, comparator, delete, lock);
1969     if (region.getCoprocessorHost() != null) {
1970       result = region.getCoprocessorHost().postCheckAndDelete(row, family,
1971         qualifier, CompareOp.EQUAL, comparator, delete, result);
1972     }
1973     return result;
1974   }
1975 
1976   /**
1977    *
1978    * @param regionName
1979    * @param row
1980    * @param family
1981    * @param qualifier
1982    * @param compareOp
1983    * @param comparator
1984    * @param delete
1985    * @throws IOException
1986    * @return true if the new put was execute, false otherwise
1987    */
1988   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
1989       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
1990       final WritableByteArrayComparable comparator, final Delete delete)
1991       throws IOException {
1992     checkOpen();
1993 
1994     if (regionName == null) {
1995       throw new IOException("Invalid arguments to checkAndDelete "
1996         + "regionName is null");
1997     }
1998     HRegion region = getRegion(regionName);
1999     Integer lock = getLockFromId(delete.getLockId());
2000     if (region.getCoprocessorHost() != null) {
2001       Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
2002         family, qualifier, compareOp, comparator, delete);
2003      if (result != null) {
2004        return result.booleanValue();
2005      }
2006     }
2007     boolean result = checkAndMutate(regionName, row, family, qualifier,
2008       compareOp, comparator, delete, lock);
2009    if (region.getCoprocessorHost() != null) {
2010      result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2011        qualifier, compareOp, comparator, delete, result);
2012    }
2013    return result;
2014  }
2015 
2016   //
2017   // remote scanner interface
2018   //
2019 
2020   public long openScanner(byte[] regionName, Scan scan) throws IOException {
2021     checkOpen();
2022     NullPointerException npe = null;
2023     if (regionName == null) {
2024       npe = new NullPointerException("regionName is null");
2025     } else if (scan == null) {
2026       npe = new NullPointerException("scan is null");
2027     }
2028     if (npe != null) {
2029       throw new IOException("Invalid arguments to openScanner", npe);
2030     }
2031     requestCount.incrementAndGet();
2032     try {
2033       HRegion r = getRegion(regionName);
2034       r.checkRow(scan.getStartRow(), "Scan");
2035       r.prepareScanner(scan);
2036       RegionScanner s = null;
2037       if (r.getCoprocessorHost() != null) {
2038         s = r.getCoprocessorHost().preScannerOpen(scan);
2039       }
2040       if (s == null) {
2041         s = r.getScanner(scan);
2042       }
2043       if (r.getCoprocessorHost() != null) {
2044         s = r.getCoprocessorHost().postScannerOpen(scan, s);
2045       }
2046       return addScanner(s);
2047     } catch (Throwable t) {
2048       throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
2049     }
2050   }
2051 
2052   protected long addScanner(RegionScanner s) throws LeaseStillHeldException {
2053     long scannerId = -1L;
2054     scannerId = rand.nextLong();
2055     String scannerName = String.valueOf(scannerId);
2056     scanners.put(scannerName, s);
2057     this.leases.createLease(scannerName, new ScannerListener(scannerName));
2058     return scannerId;
2059   }
2060 
2061   public Result next(final long scannerId) throws IOException {
2062     Result[] res = next(scannerId, 1);
2063     if (res == null || res.length == 0) {
2064       return null;
2065     }
2066     return res[0];
2067   }
2068 
2069   public Result[] next(final long scannerId, int nbRows) throws IOException {
2070     String scannerName = String.valueOf(scannerId);
2071     RegionScanner s = this.scanners.get(scannerName);
2072     if (s == null) throw new UnknownScannerException("Name: " + scannerName);
2073     try {
2074       checkOpen();
2075     } catch (IOException e) {
2076       // If checkOpen failed, server not running or filesystem gone,
2077       // cancel this lease; filesystem is gone or we're closing or something.
2078       try {
2079         this.leases.cancelLease(scannerName);
2080       } catch (LeaseException le) {
2081         LOG.info("Server shutting down and client tried to access missing scanner " +
2082           scannerName);
2083       }
2084       throw e;
2085     }
2086     Leases.Lease lease = null;
2087     try {
2088       // Remove lease while its being processed in server; protects against case
2089       // where processing of request takes > lease expiration time.
2090       lease = this.leases.removeLease(scannerName);
2091       List<Result> results = new ArrayList<Result>(nbRows);
2092       long currentScanResultSize = 0;
2093       List<KeyValue> values = new ArrayList<KeyValue>();
2094 
2095       // Call coprocessor. Get region info from scanner.
2096       HRegion region = getRegion(s.getRegionInfo().getRegionName());
2097       if (region != null && region.getCoprocessorHost() != null) {
2098         Boolean bypass = region.getCoprocessorHost().preScannerNext(s,
2099             results, nbRows);
2100         if (!results.isEmpty()) {
2101           for (Result r : results) {
2102             for (KeyValue kv : r.raw()) {
2103               currentScanResultSize += kv.heapSize();
2104             }
2105           }
2106         }
2107         if (bypass != null) {
2108           return s.isFilterDone() && results.isEmpty() ? null
2109               : results.toArray(new Result[0]);
2110         }
2111       }
2112 
2113       for (int i = 0; i < nbRows
2114           && currentScanResultSize < maxScannerResultSize; i++) {
2115         requestCount.incrementAndGet();
2116         // Collect values to be returned here
2117         boolean moreRows = s.next(values);
2118         if (!values.isEmpty()) {
2119           for (KeyValue kv : values) {
2120             currentScanResultSize += kv.heapSize();
2121           }
2122           results.add(new Result(values));
2123         }
2124         if (!moreRows) {
2125           break;
2126         }
2127         values.clear();
2128       }
2129 
2130       // coprocessor postNext hook
2131       if (region != null && region.getCoprocessorHost() != null) {
2132         region.getCoprocessorHost().postScannerNext(s, results, nbRows, true);
2133       }
2134 
2135       // If the scanner's filter - if any - is done with the scan
2136       // and wants to tell the client to stop the scan. This is done by passing
2137       // a null result.
2138       return s.isFilterDone() && results.isEmpty() ? null
2139           : results.toArray(new Result[0]);
2140     } catch (Throwable t) {
2141       if (t instanceof NotServingRegionException) {
2142         this.scanners.remove(scannerName);
2143       }
2144       throw convertThrowableToIOE(cleanup(t));
2145     } finally {
2146       // We're done. On way out readd the above removed lease.  Adding resets
2147       // expiration time on lease.
2148       if (this.scanners.containsKey(scannerName)) {
2149         if (lease != null) this.leases.addLease(lease);
2150       }
2151     }
2152   }
2153 
2154   public void close(final long scannerId) throws IOException {
2155     try {
2156       checkOpen();
2157       requestCount.incrementAndGet();
2158       String scannerName = String.valueOf(scannerId);
2159       RegionScanner s = scanners.get(scannerName);
2160 
2161       HRegion region = null;
2162       if (s != null) {
2163         // call coprocessor.
2164         region = getRegion(s.getRegionInfo().getRegionName());
2165         if (region != null && region.getCoprocessorHost() != null) {
2166           if (region.getCoprocessorHost().preScannerClose(s)) {
2167             return; // bypass
2168           }
2169         }
2170       }
2171 
2172       s = scanners.remove(scannerName);
2173       if (s != null) {
2174         s.close();
2175         this.leases.cancelLease(scannerName);
2176 
2177         if (region != null && region.getCoprocessorHost() != null) {
2178           region.getCoprocessorHost().postScannerClose(s);
2179         }
2180       }
2181     } catch (Throwable t) {
2182       throw convertThrowableToIOE(cleanup(t));
2183     }
2184   }
2185 
2186   /**
2187    * Instantiated as a scanner lease. If the lease times out, the scanner is
2188    * closed
2189    */
2190   private class ScannerListener implements LeaseListener {
2191     private final String scannerName;
2192 
2193     ScannerListener(final String n) {
2194       this.scannerName = n;
2195     }
2196 
2197     public void leaseExpired() {
2198       LOG.info("Scanner " + this.scannerName + " lease expired");
2199       RegionScanner s = scanners.remove(this.scannerName);
2200       if (s != null) {
2201         try {
2202           s.close();
2203         } catch (IOException e) {
2204           LOG.error("Closing scanner", e);
2205         }
2206       }
2207     }
2208   }
2209 
2210   //
2211   // Methods that do the actual work for the remote API
2212   //
2213   public void delete(final byte[] regionName, final Delete delete)
2214       throws IOException {
2215     checkOpen();
2216     try {
2217       boolean writeToWAL = delete.getWriteToWAL();
2218       this.requestCount.incrementAndGet();
2219       HRegion region = getRegion(regionName);
2220       if (!region.getRegionInfo().isMetaTable()) {
2221         this.cacheFlusher.reclaimMemStoreMemory();
2222       }
2223       Integer lid = getLockFromId(delete.getLockId());
2224       region.delete(delete, lid, writeToWAL);
2225     } catch (Throwable t) {
2226       throw convertThrowableToIOE(cleanup(t));
2227     }
2228   }
2229 
2230   public int delete(final byte[] regionName, final List<Delete> deletes)
2231       throws IOException {
2232     checkOpen();
2233     // Count of Deletes processed.
2234     int i = 0;
2235     HRegion region = null;
2236     try {
2237       region = getRegion(regionName);
2238       if (!region.getRegionInfo().isMetaTable()) {
2239         this.cacheFlusher.reclaimMemStoreMemory();
2240       }
2241       int size = deletes.size();
2242       Integer[] locks = new Integer[size];
2243       for (Delete delete : deletes) {
2244         this.requestCount.incrementAndGet();
2245         locks[i] = getLockFromId(delete.getLockId());
2246         region.delete(delete, locks[i], delete.getWriteToWAL());
2247         i++;
2248       }
2249     } catch (WrongRegionException ex) {
2250       LOG.debug("Batch deletes: " + i, ex);
2251       return i;
2252     } catch (NotServingRegionException ex) {
2253       return i;
2254     } catch (Throwable t) {
2255       throw convertThrowableToIOE(cleanup(t));
2256     }
2257     return -1;
2258   }
2259 
2260   public long lockRow(byte[] regionName, byte[] row) throws IOException {
2261     checkOpen();
2262     NullPointerException npe = null;
2263     if (regionName == null) {
2264       npe = new NullPointerException("regionName is null");
2265     } else if (row == null) {
2266       npe = new NullPointerException("row to lock is null");
2267     }
2268     if (npe != null) {
2269       IOException io = new IOException("Invalid arguments to lockRow");
2270       io.initCause(npe);
2271       throw io;
2272     }
2273     requestCount.incrementAndGet();
2274     try {
2275       HRegion region = getRegion(regionName);
2276       Integer r = region.obtainRowLock(row);
2277       long lockId = addRowLock(r, region);
2278       LOG.debug("Row lock " + lockId + " explicitly acquired by client");
2279       return lockId;
2280     } catch (Throwable t) {
2281       throw convertThrowableToIOE(cleanup(t, "Error obtaining row lock (fsOk: "
2282           + this.fsOk + ")"));
2283     }
2284   }
2285 
2286   protected long addRowLock(Integer r, HRegion region)
2287       throws LeaseStillHeldException {
2288     long lockId = -1L;
2289     lockId = rand.nextLong();
2290     String lockName = String.valueOf(lockId);
2291     rowlocks.put(lockName, r);
2292     this.leases.createLease(lockName, new RowLockListener(lockName, region));
2293     return lockId;
2294   }
2295 
2296   /**
2297    * Method to get the Integer lock identifier used internally from the long
2298    * lock identifier used by the client.
2299    *
2300    * @param lockId
2301    *          long row lock identifier from client
2302    * @return intId Integer row lock used internally in HRegion
2303    * @throws IOException
2304    *           Thrown if this is not a valid client lock id.
2305    */
2306   Integer getLockFromId(long lockId) throws IOException {
2307     if (lockId == -1L) {
2308       return null;
2309     }
2310     String lockName = String.valueOf(lockId);
2311     Integer rl = rowlocks.get(lockName);
2312     if (rl == null) {
2313       throw new UnknownRowLockException("Invalid row lock");
2314     }
2315     this.leases.renewLease(lockName);
2316     return rl;
2317   }
2318 
2319   @Override
2320   @QosPriority(priority=HIGH_QOS)
2321   public void unlockRow(byte[] regionName, long lockId) throws IOException {
2322     checkOpen();
2323     NullPointerException npe = null;
2324     if (regionName == null) {
2325       npe = new NullPointerException("regionName is null");
2326     } else if (lockId == -1L) {
2327       npe = new NullPointerException("lockId is null");
2328     }
2329     if (npe != null) {
2330       IOException io = new IOException("Invalid arguments to unlockRow");
2331       io.initCause(npe);
2332       throw io;
2333     }
2334     requestCount.incrementAndGet();
2335     try {
2336       HRegion region = getRegion(regionName);
2337       String lockName = String.valueOf(lockId);
2338       Integer r = rowlocks.remove(lockName);
2339       if (r == null) {
2340         throw new UnknownRowLockException(lockName);
2341       }
2342       region.releaseRowLock(r);
2343       this.leases.cancelLease(lockName);
2344       LOG.debug("Row lock " + lockId
2345           + " has been explicitly released by client");
2346     } catch (Throwable t) {
2347       throw convertThrowableToIOE(cleanup(t));
2348     }
2349   }
2350 
2351   /**
2352    * Atomically bulk load several HFiles into an open region
2353    * @return true if successful, false is failed but recoverably (no action)
2354    * @throws IOException if failed unrecoverably
2355    */
2356   @Override
2357   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
2358       byte[] regionName) throws IOException {
2359     checkOpen();
2360     HRegion region = getRegion(regionName);
2361     return region.bulkLoadHFiles(familyPaths);
2362   }
2363 
2364   Map<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
2365 
2366   /**
2367    * Instantiated as a row lock lease. If the lease times out, the row lock is
2368    * released
2369    */
2370   private class RowLockListener implements LeaseListener {
2371     private final String lockName;
2372     private final HRegion region;
2373 
2374     RowLockListener(final String lockName, final HRegion region) {
2375       this.lockName = lockName;
2376       this.region = region;
2377     }
2378 
2379     public void leaseExpired() {
2380       LOG.info("Row Lock " + this.lockName + " lease expired");
2381       Integer r = rowlocks.remove(this.lockName);
2382       if (r != null) {
2383         region.releaseRowLock(r);
2384       }
2385     }
2386   }
2387 
2388   // Region open/close direct RPCs
2389 
2390   @Override
2391   @QosPriority(priority=HIGH_QOS)
2392   public RegionOpeningState openRegion(HRegionInfo region)
2393   throws IOException {
2394     return openRegion(region, -1);
2395   }
2396   @Override
2397   @QosPriority(priority = HIGH_QOS)
2398   public RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode)
2399       throws IOException {
2400     checkOpen();
2401     checkIfRegionInTransition(region, OPEN);
2402     HRegion onlineRegion = this.getFromOnlineRegions(region.getEncodedName());
2403     if (null != onlineRegion) {
2404       LOG.warn("Attempted open of " + region.getEncodedName()
2405           + " but already online on this server");
2406       return RegionOpeningState.ALREADY_OPENED;
2407     }
2408     LOG.info("Received request to open region: " +
2409       region.getRegionNameAsString());
2410     this.regionsInTransitionInRS.putIfAbsent(region.getEncodedNameAsBytes(),
2411         true);
2412     HTableDescriptor htd = this.tableDescriptors.get(region.getTableName());
2413     // Need to pass the expected version in the constructor.
2414     if (region.isRootRegion()) {
2415       this.service.submit(new OpenRootHandler(this, this, region, htd,
2416           versionOfOfflineNode));
2417     } else if (region.isMetaRegion()) {
2418       this.service.submit(new OpenMetaHandler(this, this, region, htd,
2419           versionOfOfflineNode));
2420     } else {
2421       this.service.submit(new OpenRegionHandler(this, this, region, htd,
2422           versionOfOfflineNode));
2423     }
2424     return RegionOpeningState.OPENED;
2425   }
2426   
2427   private void checkIfRegionInTransition(HRegionInfo region,
2428       String currentAction) throws RegionAlreadyInTransitionException {
2429     byte[] encodedName = region.getEncodedNameAsBytes();
2430     if (this.regionsInTransitionInRS.containsKey(encodedName)) {
2431       boolean openAction = this.regionsInTransitionInRS.get(encodedName);
2432       // The below exception message will be used in master.
2433       throw new RegionAlreadyInTransitionException("Received:" + currentAction + 
2434         " for the region:" + region.getRegionNameAsString() +
2435         " ,which we are already trying to " + 
2436         (openAction ? OPEN : CLOSE)+ ".");
2437     }
2438   }
2439 
2440   @Override
2441   @QosPriority(priority=HIGH_QOS)
2442   public void openRegions(List<HRegionInfo> regions)
2443   throws IOException {
2444     checkOpen();
2445     LOG.info("Received request to open " + regions.size() + " region(s)");
2446     for (HRegionInfo region: regions) openRegion(region);
2447   }
2448 
2449   @Override
2450   @QosPriority(priority=HIGH_QOS)
2451   public boolean closeRegion(HRegionInfo region)
2452   throws IOException {
2453     return closeRegion(region, true);
2454   }
2455 
2456   @Override
2457   @QosPriority(priority=HIGH_QOS)
2458   public boolean closeRegion(HRegionInfo region, final boolean zk)
2459   throws IOException {
2460     checkOpen();
2461     LOG.info("Received close region: " + region.getRegionNameAsString());
2462     boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
2463     if (!hasit) {
2464       LOG.warn("Received close for region we are not serving; " +
2465         region.getEncodedName());
2466       throw new NotServingRegionException("Received close for "
2467         + region.getRegionNameAsString() + " but we are not serving it");
2468     }
2469     checkIfRegionInTransition(region, CLOSE);
2470     return closeRegion(region, false, zk);
2471   }
2472   
2473   @Override
2474   @QosPriority(priority=HIGH_QOS)
2475   public boolean closeRegion(byte[] encodedRegionName, boolean zk) throws IOException {
2476     return closeRegion(encodedRegionName, false, zk);
2477   }
2478 
2479   /**
2480    * @param region Region to close
2481    * @param abort True if we are aborting
2482    * @param zk True if we are to update zk about the region close; if the close
2483    * was orchestrated by master, then update zk.  If the close is being run by
2484    * the regionserver because its going down, don't update zk.
2485    * @return True if closed a region.
2486    */
2487   protected boolean closeRegion(HRegionInfo region, final boolean abort,
2488       final boolean zk) {
2489     if (this.regionsInTransitionInRS.containsKey(region.getEncodedNameAsBytes())) {
2490       LOG.warn("Received close for region we are already opening or closing; " +
2491           region.getEncodedName());
2492       return false;
2493     }
2494     this.regionsInTransitionInRS.putIfAbsent(region.getEncodedNameAsBytes(), false);
2495     CloseRegionHandler crh = null;
2496     if (region.isRootRegion()) {
2497       crh = new CloseRootHandler(this, this, region, abort, zk);
2498     } else if (region.isMetaRegion()) {
2499       crh = new CloseMetaHandler(this, this, region, abort, zk);
2500     } else {
2501       crh = new CloseRegionHandler(this, this, region, abort, zk);
2502     }
2503     this.service.submit(crh);
2504     return true;
2505   }
2506   
2507   /**
2508    * @param encodedRegionName
2509    *          encodedregionName to close
2510    * @param abort
2511    *          True if we are aborting
2512    * @param zk
2513    *          True if we are to update zk about the region close; if the close
2514    *          was orchestrated by master, then update zk. If the close is being
2515    *          run by the regionserver because its going down, don't update zk.
2516    * @return True if closed a region.
2517    */
2518   protected boolean closeRegion(byte[] encodedRegionName, final boolean abort,
2519       final boolean zk) throws IOException {
2520     String encodedRegionNameStr = Bytes.toString(encodedRegionName);
2521     HRegion region = this.getFromOnlineRegions(encodedRegionNameStr);
2522     if (null != region) {
2523       return closeRegion(region.getRegionInfo(), abort, zk);
2524     }
2525     LOG.error("The specified region name" + encodedRegionNameStr
2526         + " does not exist to close the region.");
2527     return false;
2528   }
2529 
2530   // Manual remote region administration RPCs
2531 
2532   @Override
2533   @QosPriority(priority=HIGH_QOS)
2534   public void flushRegion(HRegionInfo regionInfo)
2535       throws NotServingRegionException, IOException {
2536     checkOpen();
2537     LOG.info("Flushing " + regionInfo.getRegionNameAsString());
2538     HRegion region = getRegion(regionInfo.getRegionName());
2539     region.flushcache();
2540   }
2541 
2542   @Override
2543   @QosPriority(priority=HIGH_QOS)
2544   public void splitRegion(HRegionInfo regionInfo)
2545       throws NotServingRegionException, IOException {
2546     splitRegion(regionInfo, null);
2547   }
2548 
2549   @Override
2550   public void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
2551       throws NotServingRegionException, IOException {
2552     checkOpen();
2553     HRegion region = getRegion(regionInfo.getRegionName());
2554     region.flushcache();
2555     region.forceSplit(splitPoint);
2556     compactSplitThread.requestSplit(region, region.checkSplit());
2557   }
2558 
2559   @Override
2560   @QosPriority(priority=HIGH_QOS)
2561   public void compactRegion(HRegionInfo regionInfo, boolean major)
2562       throws NotServingRegionException, IOException {
2563     checkOpen();
2564     HRegion region = getRegion(regionInfo.getRegionName());
2565     if (major) {
2566       region.triggerMajorCompaction();
2567     }
2568     compactSplitThread.requestCompaction(region, "User-triggered "
2569         + (major ? "major " : "") + "compaction",
2570         CompactSplitThread.PRIORITY_USER);
2571   }
2572 
2573   /** @return the info server */
2574   public InfoServer getInfoServer() {
2575     return infoServer;
2576   }
2577 
2578   /**
2579    * @return true if a stop has been requested.
2580    */
2581   public boolean isStopped() {
2582     return this.stopped;
2583   }
2584 
2585   @Override
2586   public boolean isStopping() {
2587     return this.stopping;
2588   }
2589 
2590   /**
2591    *
2592    * @return the configuration
2593    */
2594   public Configuration getConfiguration() {
2595     return conf;
2596   }
2597 
2598   /** @return the write lock for the server */
2599   ReentrantReadWriteLock.WriteLock getWriteLock() {
2600     return lock.writeLock();
2601   }
2602 
2603   @Override
2604   @QosPriority(priority=HIGH_QOS)
2605   public List<HRegionInfo> getOnlineRegions() throws IOException {
2606     checkOpen();
2607     List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
2608     for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
2609       list.add(e.getValue().getRegionInfo());
2610     }
2611     Collections.sort(list);
2612     return list;
2613   }
2614 
2615   public int getNumberOfOnlineRegions() {
2616     return this.onlineRegions.size();
2617   }
2618 
2619   boolean isOnlineRegionsEmpty() {
2620     return this.onlineRegions.isEmpty();
2621   }
2622 
2623   /**
2624    * @param encodedRegionName
2625    * @return JSON Map of labels to values for passed in <code>encodedRegionName</code>
2626    * @throws IOException
2627    */
2628   public byte [] getRegionStats(final String encodedRegionName)
2629   throws IOException {
2630     HRegion r = null;
2631     synchronized (this.onlineRegions) {
2632       r = this.onlineRegions.get(encodedRegionName);
2633     }
2634     if (r == null) return null;
2635     ObjectMapper mapper = new ObjectMapper();
2636     int stores = 0;
2637     int storefiles = 0;
2638     int storefileSizeMB = 0;
2639     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
2640     int storefileIndexSizeMB = 0;
2641     long totalCompactingKVs = 0;
2642     long currentCompactedKVs = 0;
2643     synchronized (r.stores) {
2644       stores += r.stores.size();
2645       for (Store store : r.stores.values()) {
2646         storefiles += store.getStorefilesCount();
2647         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
2648         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
2649       }
2650     }
2651     Map<String, Integer> map = new TreeMap<String, Integer>();
2652     map.put("stores", stores);
2653     map.put("storefiles", storefiles);
2654     map.put("storefileSizeMB", storefileIndexSizeMB);
2655     map.put("memstoreSizeMB", memstoreSizeMB);
2656     StringWriter w = new StringWriter();
2657     mapper.writeValue(w, map);
2658     w.close();
2659     return Bytes.toBytes(w.toString());
2660   }
2661 
2662   /**
2663    * For tests and web ui.
2664    * This method will only work if HRegionServer is in the same JVM as client;
2665    * HRegion cannot be serialized to cross an rpc.
2666    * @see #getOnlineRegions()
2667    */
2668   public Collection<HRegion> getOnlineRegionsLocalContext() {
2669     Collection<HRegion> regions = this.onlineRegions.values();
2670     return Collections.unmodifiableCollection(regions);
2671   }
2672 
2673   @Override
2674   public void addToOnlineRegions(HRegion region) {
2675     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2676   }
2677 
2678   @Override
2679   public boolean removeFromOnlineRegions(final String encodedName) {
2680     HRegion toReturn = null;
2681     toReturn = this.onlineRegions.remove(encodedName);
2682     return toReturn != null;
2683   }
2684 
2685   /**
2686    * @return A new Map of online regions sorted by region size with the first
2687    *         entry being the biggest.
2688    */
2689   public SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
2690     // we'll sort the regions in reverse
2691     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
2692         new Comparator<Long>() {
2693           public int compare(Long a, Long b) {
2694             return -1 * a.compareTo(b);
2695           }
2696         });
2697     // Copy over all regions. Regions are sorted by size with biggest first.
2698     for (HRegion region : this.onlineRegions.values()) {
2699       sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
2700     }
2701     return sortedRegions;
2702   }
2703 
2704   @Override
2705   public HRegion getFromOnlineRegions(final String encodedRegionName) {
2706     HRegion r = null;
2707     r = this.onlineRegions.get(encodedRegionName);
2708     return r;
2709   }
2710 
2711   /**
2712    * @param regionName
2713    * @return HRegion for the passed binary <code>regionName</code> or null if
2714    *         named region is not member of the online regions.
2715    */
2716   public HRegion getOnlineRegion(final byte[] regionName) {
2717     return getFromOnlineRegions(HRegionInfo.encodeRegionName(regionName));
2718   }
2719 
2720   /** @return the request count */
2721   public AtomicInteger getRequestCount() {
2722     return this.requestCount;
2723   }
2724 
2725   /** @return reference to FlushRequester */
2726   public FlushRequester getFlushRequester() {
2727     return this.cacheFlusher;
2728   }
2729 
2730   /**
2731    * Protected utility method for safely obtaining an HRegion handle.
2732    *
2733    * @param regionName
2734    *          Name of online {@link HRegion} to return
2735    * @return {@link HRegion} for <code>regionName</code>
2736    * @throws NotServingRegionException
2737    */
2738   protected HRegion getRegion(final byte[] regionName)
2739       throws NotServingRegionException {
2740     HRegion region = null;
2741     region = getOnlineRegion(regionName);
2742     if (region == null) {
2743       throw new NotServingRegionException("Region is not online: " +
2744         Bytes.toStringBinary(regionName));
2745     }
2746     return region;
2747   }
2748 
2749   /**
2750    * Get the top N most loaded regions this server is serving so we can tell the
2751    * master which regions it can reallocate if we're overloaded. TODO: actually
2752    * calculate which regions are most loaded. (Right now, we're just grabbing
2753    * the first N regions being served regardless of load.)
2754    */
2755   protected HRegionInfo[] getMostLoadedRegions() {
2756     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2757     for (HRegion r : onlineRegions.values()) {
2758       if (r.isClosed() || r.isClosing()) {
2759         continue;
2760       }
2761       if (regions.size() < numRegionsToReport) {
2762         regions.add(r.getRegionInfo());
2763       } else {
2764         break;
2765       }
2766     }
2767     return regions.toArray(new HRegionInfo[regions.size()]);
2768   }
2769 
2770   /**
2771    * Called to verify that this server is up and running.
2772    *
2773    * @throws IOException
2774    */
2775   protected void checkOpen() throws IOException {
2776     if (this.stopped || this.abortRequested) {
2777       throw new RegionServerStoppedException("Server " + getServerName() +
2778         " not running" + (this.abortRequested ? ", aborting" : ""));
2779     }
2780     if (!fsOk) {
2781       throw new RegionServerStoppedException("File system not available");
2782     }
2783   }
2784 
2785   @Override
2786   @QosPriority(priority=HIGH_QOS)
2787   public ProtocolSignature getProtocolSignature(
2788       String protocol, long version, int clientMethodsHashCode)
2789   throws IOException {
2790     if (protocol.equals(HRegionInterface.class.getName())) {
2791       return new ProtocolSignature(HRegionInterface.VERSION, null);
2792     }
2793     throw new IOException("Unknown protocol: " + protocol);
2794   }
2795 
2796   @Override
2797   @QosPriority(priority=HIGH_QOS)
2798   public long getProtocolVersion(final String protocol, final long clientVersion)
2799   throws IOException {
2800     if (protocol.equals(HRegionInterface.class.getName())) {
2801       return HRegionInterface.VERSION;
2802     }
2803     throw new IOException("Unknown protocol: " + protocol);
2804   }
2805 
2806   /**
2807    * @return Return the leases.
2808    */
2809   protected Leases getLeases() {
2810     return leases;
2811   }
2812 
2813   /**
2814    * @return Return the rootDir.
2815    */
2816   protected Path getRootDir() {
2817     return rootDir;
2818   }
2819 
2820   /**
2821    * @return Return the fs.
2822    */
2823   protected FileSystem getFileSystem() {
2824     return fs;
2825   }
2826 
2827   /**
2828    * @return This servers {@link HServerInfo}
2829    */
2830   // TODO: Deprecate and do getServerName instead.
2831   public HServerInfo getServerInfo() {
2832     try {
2833       return getHServerInfo();
2834     } catch (IOException e) {
2835       e.printStackTrace();
2836     }
2837     return null;
2838   }
2839 
2840   @Override
2841   public Result increment(byte[] regionName, Increment increment)
2842   throws IOException {
2843     checkOpen();
2844     if (regionName == null) {
2845       throw new IOException("Invalid arguments to increment " +
2846       "regionName is null");
2847     }
2848     requestCount.incrementAndGet();
2849     try {
2850       HRegion region = getRegion(regionName);
2851       Integer lock = getLockFromId(increment.getLockId());
2852       Increment incVal = increment;
2853       Result resVal;
2854       if (region.getCoprocessorHost() != null) {
2855         resVal = region.getCoprocessorHost().preIncrement(incVal);
2856         if (resVal != null) {
2857           return resVal;
2858         }
2859       }
2860       resVal = region.increment(incVal, lock,
2861           increment.getWriteToWAL());
2862       if (region.getCoprocessorHost() != null) {
2863         region.getCoprocessorHost().postIncrement(incVal, resVal);
2864       }
2865       return resVal;
2866     } catch (IOException e) {
2867       checkFileSystem();
2868       throw e;
2869     }
2870   }
2871 
2872   /** {@inheritDoc} */
2873   public long incrementColumnValue(byte[] regionName, byte[] row,
2874       byte[] family, byte[] qualifier, long amount, boolean writeToWAL)
2875       throws IOException {
2876     checkOpen();
2877 
2878     if (regionName == null) {
2879       throw new IOException("Invalid arguments to incrementColumnValue "
2880           + "regionName is null");
2881     }
2882     requestCount.incrementAndGet();
2883     try {
2884       HRegion region = getRegion(regionName);
2885       if (region.getCoprocessorHost() != null) {
2886         Long amountVal = region.getCoprocessorHost().preIncrementColumnValue(row,
2887           family, qualifier, amount, writeToWAL);
2888         if (amountVal != null) {
2889           return amountVal.longValue();
2890         }
2891       }
2892       long retval = region.incrementColumnValue(row, family, qualifier, amount,
2893         writeToWAL);
2894       if (region.getCoprocessorHost() != null) {
2895         retval = region.getCoprocessorHost().postIncrementColumnValue(row,
2896           family, qualifier, amount, writeToWAL, retval);
2897       }
2898       return retval;
2899     } catch (IOException e) {
2900       checkFileSystem();
2901       throw e;
2902     }
2903   }
2904 
2905   /** {@inheritDoc}
2906    * @deprecated Use {@link #getServerName()} instead.
2907    */
2908   @Override
2909   @QosPriority(priority=HIGH_QOS)
2910   public HServerInfo getHServerInfo() throws IOException {
2911     checkOpen();
2912     return new HServerInfo(new HServerAddress(this.isa),
2913       this.startcode, this.webuiport);
2914   }
2915 
2916   @SuppressWarnings("unchecked")
2917   @Override
2918   public <R> MultiResponse multi(MultiAction<R> multi) throws IOException {
2919     checkOpen();
2920     MultiResponse response = new MultiResponse();
2921     for (Map.Entry<byte[], List<Action<R>>> e : multi.actions.entrySet()) {
2922       byte[] regionName = e.getKey();
2923       List<Action<R>> actionsForRegion = e.getValue();
2924       // sort based on the row id - this helps in the case where we reach the
2925       // end of a region, so that we don't have to try the rest of the
2926       // actions in the list.
2927       Collections.sort(actionsForRegion);
2928       Row action;
2929       List<Action<R>> puts = new ArrayList<Action<R>>();
2930       for (Action<R> a : actionsForRegion) {
2931         action = a.getAction();
2932         int originalIndex = a.getOriginalIndex();
2933 
2934         try {
2935           if (action instanceof Delete) {
2936             delete(regionName, (Delete) action);
2937             response.add(regionName, originalIndex, new Result());
2938           } else if (action instanceof Get) {
2939             response.add(regionName, originalIndex, get(regionName, (Get) action));
2940           } else if (action instanceof Put) {
2941             puts.add(a);  // wont throw.
2942           } else if (action instanceof Exec) {
2943             ExecResult result = execCoprocessor(regionName, (Exec)action);
2944             response.add(regionName, new Pair<Integer, Object>(
2945                 a.getOriginalIndex(), result.getValue()
2946             ));
2947           } else {
2948             LOG.debug("Error: invalid Action, row must be a Get, Delete, " +
2949                 "Put or Exec.");
2950             throw new DoNotRetryIOException("Invalid Action, row must be a " +
2951                 "Get, Delete or Put.");
2952           }
2953         } catch (IOException ex) {
2954           response.add(regionName, originalIndex, ex);
2955         }
2956       }
2957 
2958       // We do the puts with result.put so we can get the batching efficiency
2959       // we so need. All this data munging doesn't seem great, but at least
2960       // we arent copying bytes or anything.
2961       if (!puts.isEmpty()) {
2962         try {
2963           HRegion region = getRegion(regionName);
2964 
2965           if (!region.getRegionInfo().isMetaTable()) {
2966             this.cacheFlusher.reclaimMemStoreMemory();
2967           }
2968 
2969           List<Pair<Put,Integer>> putsWithLocks =
2970               Lists.newArrayListWithCapacity(puts.size());
2971           for (Action<R> a : puts) {
2972             Put p = (Put) a.getAction();
2973 
2974             Integer lock;
2975             try {
2976               lock = getLockFromId(p.getLockId());
2977             } catch (UnknownRowLockException ex) {
2978               response.add(regionName, a.getOriginalIndex(), ex);
2979               continue;
2980             }
2981             putsWithLocks.add(new Pair<Put, Integer>(p, lock));
2982           }
2983 
2984           this.requestCount.addAndGet(puts.size());
2985 
2986           OperationStatus[] codes =
2987               region.put(putsWithLocks.toArray(new Pair[]{}));
2988 
2989           for( int i = 0 ; i < codes.length ; i++) {
2990             OperationStatus code = codes[i];
2991 
2992             Action<R> theAction = puts.get(i);
2993             Object result = null;
2994 
2995             if (code.getOperationStatusCode() == OperationStatusCode.SUCCESS) {
2996               result = new Result();
2997             } else if (code.getOperationStatusCode()
2998                 == OperationStatusCode.BAD_FAMILY) {
2999               result = new NoSuchColumnFamilyException(code.getExceptionMsg());
3000             }
3001             // FAILURE && NOT_RUN becomes null, aka: need to run again.
3002 
3003             response.add(regionName, theAction.getOriginalIndex(), result);
3004           }
3005         } catch (IOException ioe) {
3006           // fail all the puts with the ioe in question.
3007           for (Action<R> a: puts) {
3008             response.add(regionName, a.getOriginalIndex(), ioe);
3009           }
3010         }
3011       }
3012     }
3013     return response;
3014   }
3015 
3016   /**
3017    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
3018    * method using the registered protocol handlers.
3019    * {@link CoprocessorProtocol} implementations must be registered per-region
3020    * via the
3021    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
3022    * method before they are available.
3023    *
3024    * @param regionName name of the region against which the invocation is executed
3025    * @param call an {@code Exec} instance identifying the protocol, method name,
3026    *     and parameters for the method invocation
3027    * @return an {@code ExecResult} instance containing the region name of the
3028    *     invocation and the return value
3029    * @throws IOException if no registered protocol handler is found or an error
3030    *     occurs during the invocation
3031    * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
3032    */
3033   @Override
3034   public ExecResult execCoprocessor(byte[] regionName, Exec call)
3035       throws IOException {
3036     checkOpen();
3037     requestCount.incrementAndGet();
3038     try {
3039       HRegion region = getRegion(regionName);
3040       return region.exec(call);
3041     } catch (Throwable t) {
3042       throw convertThrowableToIOE(cleanup(t));
3043     }
3044   }
3045 
3046   public String toString() {
3047     return getServerName().toString();
3048   }
3049 
3050   /**
3051    * Interval at which threads should run
3052    *
3053    * @return the interval
3054    */
3055   public int getThreadWakeFrequency() {
3056     return threadWakeFrequency;
3057   }
3058 
3059   @Override
3060   public ZooKeeperWatcher getZooKeeper() {
3061     return zooKeeper;
3062   }
3063 
3064   @Override
3065   public ServerName getServerName() {
3066     // Our servername could change after we talk to the master.
3067     return this.serverNameFromMasterPOV == null?
3068       new ServerName(this.isa.getHostName(), this.isa.getPort(), this.startcode):
3069         this.serverNameFromMasterPOV;
3070   }
3071 
3072   @Override
3073   public CompactionRequestor getCompactionRequester() {
3074     return this.compactSplitThread;
3075   }
3076 
3077   public ZooKeeperWatcher getZooKeeperWatcher() {
3078     return this.zooKeeper;
3079   }
3080 
3081 
3082   public ConcurrentSkipListMap<byte[], Boolean> getRegionsInTransitionInRS() {
3083     return this.regionsInTransitionInRS;
3084   }
3085   
3086   public ExecutorService getExecutorService() {
3087     return service;
3088   }
3089 
3090   //
3091   // Main program and support routines
3092   //
3093 
3094   /**
3095    * @param hrs
3096    * @return Thread the RegionServer is running in correctly named.
3097    * @throws IOException
3098    */
3099   public static Thread startRegionServer(final HRegionServer hrs)
3100       throws IOException {
3101     return startRegionServer(hrs, "regionserver" + hrs.isa.getPort());
3102   }
3103 
3104   /**
3105    * @param hrs
3106    * @param name
3107    * @return Thread the RegionServer is running in correctly named.
3108    * @throws IOException
3109    */
3110   public static Thread startRegionServer(final HRegionServer hrs,
3111       final String name) throws IOException {
3112     Thread t = new Thread(hrs);
3113     t.setName(name);
3114     t.start();
3115     // Install shutdown hook that will catch signals and run an orderly shutdown
3116     // of the hrs.
3117     ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
3118         .getConfiguration()), hrs, t);
3119     return t;
3120   }
3121 
3122   /**
3123    * Utility for constructing an instance of the passed HRegionServer class.
3124    *
3125    * @param regionServerClass
3126    * @param conf2
3127    * @return HRegionServer instance.
3128    */
3129   public static HRegionServer constructRegionServer(
3130       Class<? extends HRegionServer> regionServerClass,
3131       final Configuration conf2) {
3132     try {
3133       Constructor<? extends HRegionServer> c = regionServerClass
3134           .getConstructor(Configuration.class);
3135       return c.newInstance(conf2);
3136     } catch (Exception e) {
3137       throw new RuntimeException("Failed construction of " + "Regionserver: "
3138           + regionServerClass.toString(), e);
3139     }
3140   }
3141 
3142   @Override
3143   @QosPriority(priority=HIGH_QOS)
3144   public void replicateLogEntries(final HLog.Entry[] entries)
3145   throws IOException {
3146     checkOpen();
3147     if (this.replicationHandler == null) return;
3148     this.replicationHandler.replicateLogEntries(entries);
3149   }
3150 
3151   /**
3152    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
3153    */
3154   public static void main(String[] args) throws Exception {
3155 	VersionInfo.logVersion();
3156     Configuration conf = HBaseConfiguration.create();
3157     @SuppressWarnings("unchecked")
3158     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
3159         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
3160 
3161     new HRegionServerCommandLine(regionServerClass).doMain(args);
3162   }
3163 
3164   @Override
3165   public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries() throws IOException {
3166     BlockCache c = new CacheConfig(this.conf).getBlockCache();
3167     return c.getBlockCacheColumnFamilySummaries(this.conf);
3168   }
3169 
3170   @Override
3171   public byte[][] rollHLogWriter() throws IOException, FailedLogCloseException {
3172     HLog wal = this.getWAL();
3173     return wal.rollWriter(true);
3174   }
3175 
3176   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
3177   public String[] getCoprocessors() {
3178     HServerLoad hsl = buildServerLoad();
3179     return hsl == null? null: hsl.getCoprocessors();
3180   }
3181 }