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.lang.Thread.UncaughtExceptionHandler;
24  import java.lang.management.ManagementFactory;
25  import java.lang.management.MemoryUsage;
26  import java.lang.management.RuntimeMXBean;
27  import java.lang.reflect.Constructor;
28  import java.net.BindException;
29  import java.net.InetAddress;
30  import java.net.InetSocketAddress;
31  import java.util.ArrayList;
32  import java.util.Collection;
33  import java.util.Collections;
34  import java.util.Comparator;
35  import java.util.HashSet;
36  import java.util.Iterator;
37  import java.util.LinkedList;
38  import java.util.List;
39  import java.util.Map;
40  import java.util.Random;
41  import java.util.Set;
42  import java.util.SortedMap;
43  import java.util.SortedSet;
44  import java.util.TreeMap;
45  import java.util.TreeSet;
46  import java.util.concurrent.BlockingQueue;
47  import java.util.concurrent.ConcurrentHashMap;
48  import java.util.concurrent.LinkedBlockingQueue;
49  import java.util.concurrent.TimeUnit;
50  import java.util.concurrent.atomic.AtomicBoolean;
51  import java.util.concurrent.atomic.AtomicInteger;
52  import java.util.concurrent.locks.ReentrantReadWriteLock;
53  import java.security.PrivilegedExceptionAction;
54  import java.security.PrivilegedAction;
55  
56  import com.google.common.base.Function;
57  import org.apache.commons.logging.Log;
58  import org.apache.commons.logging.LogFactory;
59  import org.apache.hadoop.conf.Configuration;
60  import org.apache.hadoop.fs.FileSystem;
61  import org.apache.hadoop.fs.Path;
62  import org.apache.hadoop.hbase.Chore;
63  import org.apache.hadoop.hbase.HBaseConfiguration;
64  import org.apache.hadoop.hbase.HConstants;
65  import org.apache.hadoop.hbase.HMsg;
66  import org.apache.hadoop.hbase.HRegionInfo;
67  import org.apache.hadoop.hbase.HRegionLocation;
68  import org.apache.hadoop.hbase.HServerAddress;
69  import org.apache.hadoop.hbase.HServerInfo;
70  import org.apache.hadoop.hbase.HServerLoad;
71  import org.apache.hadoop.hbase.HTableDescriptor;
72  import org.apache.hadoop.hbase.KeyValue;
73  import org.apache.hadoop.hbase.LeaseListener;
74  import org.apache.hadoop.hbase.Leases;
75  import org.apache.hadoop.hbase.LocalHBaseCluster;
76  import org.apache.hadoop.hbase.NotServingRegionException;
77  import org.apache.hadoop.hbase.RemoteExceptionHandler;
78  import org.apache.hadoop.hbase.UnknownRowLockException;
79  import org.apache.hadoop.hbase.UnknownScannerException;
80  import org.apache.hadoop.hbase.YouAreDeadException;
81  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
82  import org.apache.hadoop.hbase.HMsg.Type;
83  import org.apache.hadoop.hbase.Leases.LeaseStillHeldException;
84  import org.apache.hadoop.hbase.client.Delete;
85  import org.apache.hadoop.hbase.client.Get;
86  import org.apache.hadoop.hbase.client.MultiPut;
87  import org.apache.hadoop.hbase.client.MultiPutResponse;
88  import org.apache.hadoop.hbase.client.Put;
89  import org.apache.hadoop.hbase.client.Result;
90  import org.apache.hadoop.hbase.client.Scan;
91  import org.apache.hadoop.hbase.client.ServerConnection;
92  import org.apache.hadoop.hbase.client.ServerConnectionManager;
93  import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
94  import org.apache.hadoop.hbase.ipc.HBaseRPC;
95  import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
96  import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
97  import org.apache.hadoop.hbase.ipc.HBaseServer;
98  import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
99  import org.apache.hadoop.hbase.ipc.HRegionInterface;
100 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
101 import org.apache.hadoop.hbase.regionserver.wal.HLog;
102 import org.apache.hadoop.hbase.replication.regionserver.Replication;
103 import org.apache.hadoop.hbase.util.Bytes;
104 import org.apache.hadoop.hbase.util.FSUtils;
105 import org.apache.hadoop.hbase.util.InfoServer;
106 import org.apache.hadoop.hbase.util.Pair;
107 import org.apache.hadoop.hbase.util.Sleeper;
108 import org.apache.hadoop.hbase.util.Threads;
109 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
110 import org.apache.hadoop.io.MapWritable;
111 import org.apache.hadoop.io.Writable;
112 import org.apache.hadoop.net.DNS;
113 import org.apache.hadoop.security.SecurityUtil;
114 import org.apache.hadoop.security.UserGroupInformation;
115 import org.apache.hadoop.util.Progressable;
116 import org.apache.hadoop.util.StringUtils;
117 import org.apache.zookeeper.WatchedEvent;
118 import org.apache.zookeeper.Watcher;
119 import org.apache.zookeeper.Watcher.Event.EventType;
120 import org.apache.zookeeper.Watcher.Event.KeeperState;
121 
122 /**
123  * HRegionServer makes a set of HRegions available to clients.  It checks in with
124  * the HMaster. There are many HRegionServers in a single HBase deployment.
125  */
126 public class HRegionServer implements HRegionInterface,
127     HBaseRPCErrorHandler, Runnable, Watcher, Stoppable, OnlineRegions {
128   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
129   private static final HMsg REPORT_EXITING = new HMsg(Type.MSG_REPORT_EXITING);
130   private static final HMsg REPORT_QUIESCED = new HMsg(Type.MSG_REPORT_QUIESCED);
131   private static final HMsg [] EMPTY_HMSG_ARRAY = new HMsg [] {};
132 
133   // Set when a report to the master comes back with a message asking us to
134   // shutdown.  Also set by call to stop when debugging or running unit tests
135   // of HRegionServer in isolation. We use AtomicBoolean rather than
136   // plain boolean so we can pass a reference to Chore threads.  Otherwise,
137   // Chore threads need to know about the hosting class.
138   protected final AtomicBoolean stopRequested = new AtomicBoolean(false);
139 
140   protected final AtomicBoolean quiesced = new AtomicBoolean(false);
141 
142   // Go down hard.  Used if file system becomes unavailable and also in
143   // debugging and unit tests.
144   protected volatile boolean abortRequested;
145 
146   private volatile boolean killed = false;
147 
148   // If false, the file system has become unavailable
149   protected volatile boolean fsOk;
150 
151   protected HServerInfo serverInfo;
152   protected final Configuration conf;
153 
154   private final ServerConnection connection;
155   protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
156   private FileSystem fs;
157   private Path rootDir;
158   private final Random rand = new Random();
159 
160   // Key is Bytes.hashCode of region name byte array and the value is HRegion
161   // in both of the maps below.  Use Bytes.mapKey(byte []) generating key for
162   // below maps.
163   protected final Map<Integer, HRegion> onlineRegions =
164     new ConcurrentHashMap<Integer, HRegion>();
165 
166   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
167   private final LinkedBlockingQueue<HMsg> outboundMsgs =
168     new LinkedBlockingQueue<HMsg>();
169 
170   final int numRetries;
171   protected final int threadWakeFrequency;
172   private final int msgInterval;
173 
174   protected final int numRegionsToReport;
175 
176   private final long maxScannerResultSize;
177 
178   // Remote HMaster
179   private HMasterRegionInterface hbaseMaster;
180 
181   // Server to handle client requests.  Default access so can be accessed by
182   // unit tests.
183   HBaseServer server;
184 
185   // Leases
186   private Leases leases;
187 
188   // Request counter
189   private volatile AtomicInteger requestCount = new AtomicInteger();
190 
191   // Info server.  Default access so can be used by unit tests.  REGIONSERVER
192   // is name of the webapp and the attribute name used stuffing this instance
193   // into web context.
194   InfoServer infoServer;
195 
196   /** region server process name */
197   public static final String REGIONSERVER = "regionserver";
198 
199   /*
200    * Space is reserved in HRS constructor and then released when aborting
201    * to recover from an OOME. See HBASE-706.  TODO: Make this percentage of the
202    * heap or a minimum.
203    */
204   private final LinkedList<byte[]> reservedSpace = new LinkedList<byte []>();
205 
206   private RegionServerMetrics metrics;
207 
208   // Compactions
209   CompactSplitThread compactSplitThread;
210 
211   // Cache flushing
212   MemStoreFlusher cacheFlusher;
213 
214   /* Check for major compactions.
215    */
216   Chore majorCompactionChecker;
217 
218   // HLog and HLog roller.  log is protected rather than private to avoid
219   // eclipse warning when accessed by inner classes
220   protected volatile HLog hlog;
221   LogRoller hlogRoller;
222 
223   // flag set after we're done setting up server threads (used for testing)
224   protected volatile boolean isOnline;
225 
226   final Map<String, InternalScanner> scanners =
227     new ConcurrentHashMap<String, InternalScanner>();
228 
229   private ZooKeeperWrapper zooKeeperWrapper;
230 
231   // A sleeper that sleeps for msgInterval.
232   private final Sleeper sleeper;
233 
234   private final long rpcTimeout;
235 
236   // Address passed in to constructor.  This is not always the address we run
237   // with.  For example, if passed port is 0, then we are to pick a port.  The
238   // actual address we run with is in the #serverInfo data member.
239   private final HServerAddress address;
240 
241   // The main region server thread.
242   private Thread regionServerThread;
243 
244   private final String machineName;
245 
246   // Replication-related attributes
247   private Replication replicationHandler;
248   // End of replication
249 
250   private final UserGroupInformation serverUser;
251 
252   /**
253    * Starts a HRegionServer at the default location
254    * @param conf
255    * @throws IOException
256    */
257   public HRegionServer(Configuration conf) throws IOException {
258     serverUser = UserGroupInformation.getCurrentUser();
259 
260     machineName = DNS.getDefaultHost(
261         conf.get("hbase.regionserver.dns.interface","default"),
262         conf.get("hbase.regionserver.dns.nameserver","default"));
263     String addressStr = machineName + ":" +
264       conf.get(HConstants.REGIONSERVER_PORT,
265           Integer.toString(HConstants.DEFAULT_REGIONSERVER_PORT));
266     // This is not necessarily the address we will run with.  The address we
267     // use will be in #serverInfo data member.  For example, we may have been
268     // passed a port of 0 which means we should pick some ephemeral port to bind
269     // to.
270     address = new HServerAddress(addressStr);
271     LOG.info("My address is " + address);
272 
273     this.abortRequested = false;
274     this.fsOk = true;
275     this.conf = conf;
276     this.connection = ServerConnectionManager.getConnection(conf);
277 
278     this.isOnline = false;
279 
280     // Config'ed params
281     this.numRetries =  conf.getInt("hbase.client.retries.number", 2);
282     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
283         10 * 1000);
284     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 1 * 1000);
285 
286     sleeper = new Sleeper(this.msgInterval, this.stopRequested);
287 
288     this.maxScannerResultSize = conf.getLong(
289             HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
290             HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
291 
292     // Task thread to process requests from Master
293     this.worker = new Worker();
294 
295     this.numRegionsToReport =
296       conf.getInt("hbase.regionserver.numregionstoreport", 10);
297 
298     this.rpcTimeout =
299       conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
300           HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
301 
302     reinitialize();
303   }
304 
305   private static final int NORMAL_QOS = 0;
306   private static final int QOS_THRESHOLD = 10;  // the line between low and high qos
307   private static final int HIGH_QOS = 100;
308 
309   class QosFunction implements Function<Writable,Integer> {
310     public boolean isMetaRegion(byte[] regionName) {
311       HRegion region;
312       try {
313         region = getRegion(regionName);
314       } catch (NotServingRegionException ignored) {
315         return false;
316       }
317       return region.getRegionInfo().isMetaRegion();
318     }
319 
320     @Override
321     public Integer apply(Writable from) {
322       if (from instanceof HBaseRPC.Invocation) {
323         HBaseRPC.Invocation inv = (HBaseRPC.Invocation) from;
324 
325         String methodName = inv.getMethodName();
326 
327         // scanner methods...
328         if (methodName.equals("next") || methodName.equals("close")) {
329           // translate!
330           Long scannerId;
331           try {
332             scannerId = (Long) inv.getParameters()[0];
333           } catch (ClassCastException ignored) {
334             //LOG.debug("Low priority: " + from);
335             return NORMAL_QOS; // doh.
336           }
337           String scannerIdString = Long.toString(scannerId);
338           InternalScanner scanner = scanners.get(scannerIdString);
339           if (scanner instanceof HRegion.RegionScanner) {
340             HRegion.RegionScanner rs = (HRegion.RegionScanner) scanner;
341             HRegionInfo regionName = rs.getRegionName();
342             if (regionName.isMetaRegion()) {
343               //LOG.debug("High priority scanner request: " + scannerId);
344               return HIGH_QOS;
345             }
346           }
347         }
348         else if (methodName.equals("getHServerInfo") ||
349             methodName.equals("getRegionsAssignment") ||
350             methodName.equals("unlockRow") ||
351             methodName.equals("getProtocolVersion") ||
352             methodName.equals("getClosestRowBefore")) {
353           //LOG.debug("High priority method: " + methodName);
354           return HIGH_QOS;
355         }
356         else if (inv.getParameterClasses()[0] == byte[].class) {
357           // first arg is byte array, so assume this is a regionname:
358           if (isMetaRegion((byte[]) inv.getParameters()[0])) {
359             //LOG.debug("High priority with method: " + methodName + " and region: "
360             //    + Bytes.toString((byte[]) inv.getParameters()[0]));
361             return HIGH_QOS;
362           }
363         }
364       }
365       //LOG.debug("Low priority: " + from.toString());
366       return NORMAL_QOS;
367     }
368   }
369 
370   /**
371    * Creates all of the state that needs to be reconstructed in case we are
372    * doing a restart. This is shared between the constructor and restart().
373    * Both call it.
374    * @throws IOException
375    */
376   private void reinitialize() throws IOException {
377     this.abortRequested = false;
378     this.stopRequested.set(false);
379 
380     // Server to handle client requests
381     this.server = HBaseRPC.getServer(this,
382         new Class<?>[]{HRegionInterface.class, HBaseRPCErrorHandler.class,
383         OnlineRegions.class},
384         address.getBindAddress(),
385         address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
386         conf.getInt("hbase.regionserver.metahandler.count", 10),
387         false, conf, QOS_THRESHOLD);
388     this.server.setErrorHandler(this);
389     this.server.setQosFunction(new QosFunction());
390 
391     // Address is giving a default IP for the moment. Will be changed after
392     // calling the master.
393     this.serverInfo = new HServerInfo(new HServerAddress(
394       new InetSocketAddress(address.getBindAddress(),
395       this.server.getListenerAddress().getPort())), System.currentTimeMillis(),
396       this.conf.getInt("hbase.regionserver.info.port", 60030), machineName);
397     if (this.serverInfo.getServerAddress() == null) {
398       throw new NullPointerException("Server address cannot be null; " +
399         "hbase-958 debugging");
400     }
401 
402     reinitializeThreads();
403     reinitializeZooKeeper();
404     int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4);
405     for(int i = 0; i < nbBlocks; i++)  {
406       reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]);
407     }
408   }
409 
410   private void reinitializeZooKeeper() throws IOException {
411     zooKeeperWrapper =
412         ZooKeeperWrapper.createInstance(conf, serverInfo.getServerName());
413     zooKeeperWrapper.registerListener(this);
414     watchMasterAddress();
415   }
416 
417   private void reinitializeThreads() {
418     this.workerThread = new Thread(worker);
419 
420     // Cache flushing thread.
421     this.cacheFlusher = new MemStoreFlusher(conf, this);
422 
423     // Compaction thread
424     this.compactSplitThread = new CompactSplitThread(this);
425 
426     // Log rolling thread
427     this.hlogRoller = new LogRoller(this);
428 
429     // Background thread to check for major compactions; needed if region
430     // has not gotten updates in a while.  Make it run at a lesser frequency.
431     int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY +
432         ".multiplier", 1000);
433     this.majorCompactionChecker = new MajorCompactionChecker(this,
434       this.threadWakeFrequency * multiplier,  this.stopRequested);
435 
436     this.leases = new Leases(
437         (int) conf.getLong(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
438             HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD),
439         this.threadWakeFrequency);
440   }
441 
442   /**
443    * We register ourselves as a watcher on the master address ZNode. This is
444    * called by ZooKeeper when we get an event on that ZNode. When this method
445    * is called it means either our master has died, or a new one has come up.
446    * Either way we need to update our knowledge of the master.
447    * @param event WatchedEvent from ZooKeeper.
448    */
449   public void process(WatchedEvent event) {
450     EventType type = event.getType();
451     KeeperState state = event.getState();
452     LOG.info("Got ZooKeeper event, state: " + state + ", type: " +
453       type + ", path: " + event.getPath());
454 
455     // Ignore events if we're shutting down.
456     if (this.stopRequested.get()) {
457       LOG.debug("Ignoring ZooKeeper event while shutting down");
458       return;
459     }
460 
461     if (state == KeeperState.Expired) {
462       LOG.error("ZooKeeper session expired");
463       boolean restart =
464         this.conf.getBoolean("hbase.regionserver.restart.on.zk.expire", false);
465       if (restart) {
466         restart();
467       } else {
468         abort("ZooKeeper session expired");
469       }
470     } else if (type == EventType.NodeDeleted) {
471       watchMasterAddress();
472     } else if (type == EventType.NodeCreated) {
473       getMaster();
474 
475       // ZooKeeper watches are one time only, so we need to re-register our watch.
476       watchMasterAddress();
477     }
478   }
479 
480   private void watchMasterAddress() {
481     while (!stopRequested.get() && !zooKeeperWrapper.watchMasterAddress(this)) {
482       LOG.warn("Unable to set watcher on ZooKeeper master address. Retrying.");
483       sleeper.sleep();
484     }
485   }
486 
487   private void restart() {
488     abort("Restarting region server");
489     Threads.shutdown(regionServerThread);
490     boolean done = false;
491     while (!done) {
492       try {
493         reinitialize();
494         done = true;
495       } catch (IOException e) {
496         LOG.debug("Error trying to reinitialize ZooKeeper", e);
497       }
498     }
499     Thread t = new Thread(this);
500     String name = regionServerThread.getName();
501     t.setName(name);
502     t.start();
503   }
504 
505   /** @return ZooKeeperWrapper used by RegionServer. */
506   public ZooKeeperWrapper getZooKeeperWrapper() {
507     return zooKeeperWrapper;
508   }
509 
510   /**
511    * The HRegionServer sticks in this loop until closed. It repeatedly checks
512    * in with the HMaster, sending heartbeats & reports, and receiving HRegion
513    * load/unload instructions.
514    */
515   public void run() {
516     serverUser.doAs(new PrivilegedAction<Void>() {
517         public Void run() {
518           doRun();
519           return null;
520         }
521       });
522   }
523 
524   private void doRun() {
525     regionServerThread = Thread.currentThread();
526     boolean quiesceRequested = false;
527     try {
528       MapWritable w = null;
529       while (!stopRequested.get()) {
530         w = reportForDuty();
531         if (w != null) {
532           init(w);
533           break;
534         }
535         sleeper.sleep();
536         LOG.warn("No response from master on reportForDuty. Sleeping and " +
537           "then trying again.");
538       }
539       List<HMsg> outboundMessages = new ArrayList<HMsg>();
540       long lastMsg = 0;
541       // Now ask master what it wants us to do and tell it what we have done
542       for (int tries = 0; !stopRequested.get() && isHealthy();) {
543         // Try to get the root region location from the master.
544         if (!haveRootRegion.get()) {
545           HServerAddress rootServer = zooKeeperWrapper.readRootRegionLocation();
546           if (rootServer != null) {
547             // By setting the root region location, we bypass the wait imposed on
548             // HTable for all regions being assigned.
549             this.connection.setRootRegionLocation(
550                 new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, rootServer));
551             haveRootRegion.set(true);
552           }
553         }
554         long now = System.currentTimeMillis();
555         // Drop into the send loop if msgInterval has elapsed or if something
556         // to send.  If we fail talking to the master, then we'll sleep below
557         // on poll of the outboundMsgs blockingqueue.
558         if ((now - lastMsg) >= msgInterval || !outboundMessages.isEmpty()) {
559           try {
560             doMetrics();
561             MemoryUsage memory =
562               ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
563             HServerLoad hsl = new HServerLoad(requestCount.get(),
564               (int)(memory.getUsed()/1024/1024),
565               (int)(memory.getMax()/1024/1024));
566             for (HRegion r: onlineRegions.values()) {
567               hsl.addRegionInfo(createRegionLoad(r));
568             }
569             this.serverInfo.setLoad(hsl);
570             this.requestCount.set(0);
571             addOutboundMsgs(outboundMessages);
572             HMsg msgs[] = this.hbaseMaster.regionServerReport(
573               serverInfo, outboundMessages.toArray(EMPTY_HMSG_ARRAY),
574               getMostLoadedRegions());
575             lastMsg = System.currentTimeMillis();
576             updateOutboundMsgs(outboundMessages);
577             outboundMessages.clear();
578             if (this.quiesced.get() && onlineRegions.size() == 0) {
579               // We've just told the master we're exiting because we aren't
580               // serving any regions. So set the stop bit and exit.
581               LOG.info("Server quiesced and not serving any regions. " +
582                 "Starting shutdown");
583               stopRequested.set(true);
584               this.outboundMsgs.clear();
585               continue;
586             }
587 
588             // Queue up the HMaster's instruction stream for processing
589             boolean restart = false;
590             for(int i = 0;
591                 !restart && !stopRequested.get() && i < msgs.length;
592                 i++) {
593               LOG.info(msgs[i].toString());
594               this.connection.unsetRootRegionLocation();
595               switch(msgs[i].getType()) {
596 
597               case MSG_REGIONSERVER_STOP:
598                 stopRequested.set(true);
599                 break;
600 
601               case MSG_REGIONSERVER_QUIESCE:
602                 if (!quiesceRequested) {
603                   try {
604                     toDo.put(new ToDoEntry(msgs[i]));
605                   } catch (InterruptedException e) {
606                     throw new RuntimeException("Putting into msgQueue was " +
607                         "interrupted.", e);
608                   }
609                   quiesceRequested = true;
610                 }
611                 break;
612 
613               default:
614                 if (fsOk) {
615                   try {
616                     toDo.put(new ToDoEntry(msgs[i]));
617                   } catch (InterruptedException e) {
618                     throw new RuntimeException("Putting into msgQueue was " +
619                         "interrupted.", e);
620                   }
621                 }
622               }
623             }
624             // Reset tries count if we had a successful transaction.
625             tries = 0;
626 
627             if (restart || this.stopRequested.get()) {
628               toDo.clear();
629               continue;
630             }
631           } catch (Exception e) { // FindBugs REC_CATCH_EXCEPTION
632             // Two special exceptions could be printed out here,
633             // PleaseHoldException and YouAreDeadException
634             if (e instanceof IOException) {
635               e = RemoteExceptionHandler.checkIOException((IOException) e);
636             }
637             if (e instanceof YouAreDeadException) {
638               // This will be caught and handled as a fatal error below
639               throw e;
640             }
641             tries++;
642             if (tries > 0 && (tries % this.numRetries) == 0) {
643               // Check filesystem every so often.
644               checkFileSystem();
645             }
646             if (this.stopRequested.get()) {
647               LOG.info("Stop requested, clearing toDo despite exception");
648               toDo.clear();
649               continue;
650             }
651             LOG.warn("Attempt=" + tries, e);
652             // No point retrying immediately; this is probably connection to
653             // master issue.  Doing below will cause us to sleep.
654             lastMsg = System.currentTimeMillis();
655           }
656         }
657         now = System.currentTimeMillis();
658         HMsg msg = this.outboundMsgs.poll((msgInterval - (now - lastMsg)),
659           TimeUnit.MILLISECONDS);
660         // If we got something, add it to list of things to send.
661         if (msg != null) outboundMessages.add(msg);
662         // Do some housekeeping before going back around
663         housekeeping();
664       } // for
665     } catch (Throwable t) {
666       if (!checkOOME(t)) {
667         abort("Unhandled exception", t);
668       }
669     }
670     this.leases.closeAfterLeasesExpire();
671     this.worker.stop();
672     this.server.stop();
673     if (this.infoServer != null) {
674       LOG.info("Stopping infoServer");
675       try {
676         this.infoServer.stop();
677       } catch (Exception e) {
678         e.printStackTrace();
679       }
680     }
681     // Send cache a shutdown.
682     LruBlockCache c = (LruBlockCache)StoreFile.getBlockCache(this.conf);
683     if (c != null) c.shutdown();
684 
685     // Send interrupts to wake up threads if sleeping so they notice shutdown.
686     // TODO: Should we check they are alive?  If OOME could have exited already
687     cacheFlusher.interruptIfNecessary();
688     compactSplitThread.interruptIfNecessary();
689     hlogRoller.interruptIfNecessary();
690     this.majorCompactionChecker.interrupt();
691 
692     if (killed) {
693       // Just skip out w/o closing regions.
694     } else if (abortRequested) {
695       if (this.fsOk) {
696         // Only try to clean up if the file system is available
697         try {
698           if (this.hlog != null) {
699             this.hlog.close();
700             LOG.info("On abort, closed hlog");
701           }
702         } catch (Throwable e) {
703           LOG.error("Unable to close log in abort",
704             RemoteExceptionHandler.checkThrowable(e));
705         }
706         closeAllRegions(); // Don't leave any open file handles
707       }
708       LOG.info("aborting server at: " + this.serverInfo.getServerName());
709     } else {
710       ArrayList<HRegion> closedRegions = closeAllRegions();
711       try {
712         if (this.hlog != null) {
713           hlog.closeAndDelete();
714         }
715       } catch (Throwable e) {
716         LOG.error("Close and delete failed",
717           RemoteExceptionHandler.checkThrowable(e));
718       }
719       try {
720         HMsg[] exitMsg = new HMsg[closedRegions.size() + 1];
721         exitMsg[0] = REPORT_EXITING;
722         // Tell the master what regions we are/were serving
723         int i = 1;
724         for (HRegion region: closedRegions) {
725           exitMsg[i++] = new HMsg(HMsg.Type.MSG_REPORT_CLOSE,
726               region.getRegionInfo());
727         }
728 
729         LOG.info("telling master that region server is shutting down at: " +
730             serverInfo.getServerName());
731         hbaseMaster.regionServerReport(serverInfo, exitMsg, (HRegionInfo[])null);
732       } catch (Throwable e) {
733         LOG.warn("Failed to send exiting message to master: ",
734           RemoteExceptionHandler.checkThrowable(e));
735       }
736       LOG.info("stopping server at: " + this.serverInfo.getServerName());
737     }
738 
739     // Make sure the proxy is down.
740     if (this.hbaseMaster != null) {
741       HBaseRPC.stopProxy(this.hbaseMaster);
742       this.hbaseMaster = null;
743     }
744 
745     if (!killed) {
746       this.zooKeeperWrapper.close();
747       join();
748     }
749     LOG.info(Thread.currentThread().getName() + " exiting");
750   }
751 
752   /*
753    * Add to the passed <code>msgs</code> messages to pass to the master.
754    * @param msgs Current outboundMsgs array; we'll add messages to this List.
755    */
756   private void addOutboundMsgs(final List<HMsg> msgs) {
757     if (msgs.isEmpty()) {
758       this.outboundMsgs.drainTo(msgs);
759       return;
760     }
761     OUTER: for (HMsg m: this.outboundMsgs) {
762       for (HMsg mm: msgs) {
763         // Be careful don't add duplicates.
764         if (mm.equals(m)) {
765           continue OUTER;
766         }
767       }
768       msgs.add(m);
769     }
770   }
771 
772   /*
773    * Remove from this.outboundMsgs those messsages we sent the master.
774    * @param msgs Messages we sent the master.
775    */
776   private void updateOutboundMsgs(final List<HMsg> msgs) {
777     if (msgs.isEmpty()) return;
778     for (HMsg m: this.outboundMsgs) {
779       for (HMsg mm: msgs) {
780         if (mm.equals(m)) {
781           this.outboundMsgs.remove(m);
782           break;
783         }
784       }
785     }
786   }
787 
788   /*
789    * Run init. Sets up hlog and starts up all server threads.
790    * @param c Extra configuration.
791    */
792   protected void init(final MapWritable c) throws IOException {
793     try {
794       for (Map.Entry<Writable, Writable> e: c.entrySet()) {
795         String key = e.getKey().toString();
796         String value = e.getValue().toString();
797         if (LOG.isDebugEnabled()) {
798           LOG.debug("Config from master: " + key + "=" + value);
799         }
800         this.conf.set(key, value);
801       }
802       // Master may have sent us a new address with the other configs.
803       // Update our address in this case. See HBASE-719
804       String hra = conf.get("hbase.regionserver.address");
805       // TODO: The below used to be this.address != null.  Was broken by what
806       // looks like a mistake in:
807       //
808       // HBASE-1215 migration; metautils scan of meta region was broken; wouldn't see first row
809       // ------------------------------------------------------------------------
810       // r796326 | stack | 2009-07-21 07:40:34 -0700 (Tue, 21 Jul 2009) | 38 lines
811       if (hra != null) {
812         HServerAddress hsa = new HServerAddress (hra,
813           this.serverInfo.getServerAddress().getPort());
814         LOG.info("Master passed us address to use. Was=" +
815           this.serverInfo.getServerAddress() + ", Now=" + hra);
816         this.serverInfo.setServerAddress(hsa);
817       }
818 
819       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
820       // config param for task trackers, but we can piggyback off of it.
821       if (this.conf.get("mapred.task.id") == null) {
822         this.conf.set("mapred.task.id", 
823             "hb_rs_" + this.serverInfo.getServerName() + "_" +
824             System.currentTimeMillis());
825       }
826 
827       // Master sent us hbase.rootdir to use. Should be fully qualified
828       // path with file system specification included.  Set 'fs.defaultFS'
829       // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
830       // accessors will be going against wrong filesystem (unless all is set
831       // to defaults).
832       this.conf.set("fs.defaultFS", this.conf.get("hbase.rootdir"));
833       // Get fs instance used by this RS
834       this.fs = FileSystem.get(this.conf);
835       this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
836       this.hlog = setupHLog();
837       // Init in here rather than in constructor after thread name has been set
838       this.metrics = new RegionServerMetrics();
839       startServiceThreads();
840       isOnline = true;
841     } catch (Throwable e) {
842       this.isOnline = false;
843       this.stopRequested.set(true);
844       throw convertThrowableToIOE(cleanup(e, "Failed init"),
845         "Region server startup failed");
846     }
847   }
848 
849   /*
850    * @param r Region to get RegionLoad for.
851    * @return RegionLoad instance.
852    * @throws IOException
853    */
854   private HServerLoad.RegionLoad createRegionLoad(final HRegion r) {
855     byte[] name = r.getRegionName();
856     int stores = 0;
857     int storefiles = 0;
858     int storefileSizeMB = 0;
859     int memstoreSizeMB = (int)(r.memstoreSize.get()/1024/1024);
860     int storefileIndexSizeMB = 0;
861     synchronized (r.stores) {
862       stores += r.stores.size();
863       for (Store store: r.stores.values()) {
864         storefiles += store.getStorefilesCount();
865         storefileSizeMB +=
866           (int)(store.getStorefilesSize()/1024/1024);
867         storefileIndexSizeMB +=
868           (int)(store.getStorefilesIndexSize()/1024/1024);
869       }
870     }
871     return new HServerLoad.RegionLoad(name, stores, storefiles,
872       storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB);
873   }
874 
875   /**
876    * @param regionName
877    * @return An instance of RegionLoad.
878    * @throws IOException
879    */
880   public HServerLoad.RegionLoad createRegionLoad(final byte [] regionName) {
881     return createRegionLoad(this.onlineRegions.get(Bytes.mapKey(regionName)));
882   }
883 
884   /*
885    * Cleanup after Throwable caught invoking method.  Converts <code>t</code>
886    * to IOE if it isn't already.
887    * @param t Throwable
888    * @return Throwable converted to an IOE; methods can only let out IOEs.
889    */
890   private Throwable cleanup(final Throwable t) {
891     return cleanup(t, null);
892   }
893 
894   /*
895    * Cleanup after Throwable caught invoking method.  Converts <code>t</code>
896    * to IOE if it isn't already.
897    * @param t Throwable
898    * @param msg Message to log in error.  Can be null.
899    * @return Throwable converted to an IOE; methods can only let out IOEs.
900    */
901   private Throwable cleanup(final Throwable t, final String msg) {
902     // Don't log as error if NSRE; NSRE is 'normal' operation.
903     if (t instanceof NotServingRegionException) {
904       LOG.debug("NotServingRegionException; " +  t.getMessage());
905       return t;
906     }
907     if (msg == null) {
908       LOG.error("", RemoteExceptionHandler.checkThrowable(t));
909     } else {
910       LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
911     }
912     if (!checkOOME(t)) {
913       checkFileSystem();
914     }
915     return t;
916   }
917 
918   /*
919    * @param t
920    * @return Make <code>t</code> an IOE if it isn't already.
921    */
922   private IOException convertThrowableToIOE(final Throwable t) {
923     return convertThrowableToIOE(t, null);
924   }
925 
926   /*
927    * @param t
928    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
929    * @return Make <code>t</code> an IOE if it isn't already.
930    */
931   private IOException convertThrowableToIOE(final Throwable t,
932       final String msg) {
933     return (t instanceof IOException? (IOException)t:
934       msg == null || msg.length() == 0?
935         new IOException(t): new IOException(msg, t));
936   }
937   /*
938    * Check if an OOME and if so, call abort.
939    * @param e
940    * @return True if we OOME'd and are aborting.
941    */
942   public boolean checkOOME(final Throwable e) {
943     boolean stop = false;
944     if (e instanceof OutOfMemoryError ||
945       (e.getCause() != null && e.getCause() instanceof OutOfMemoryError) ||
946       (e.getMessage() != null &&
947         e.getMessage().contains("java.lang.OutOfMemoryError"))) {
948       abort("OutOfMemoryError, aborting", e);
949       stop = true;
950     }
951     return stop;
952   }
953 
954 
955   /**
956    * Checks to see if the file system is still accessible.
957    * If not, sets abortRequested and stopRequested
958    *
959    * @return false if file system is not available
960    */
961   protected boolean checkFileSystem() {
962     if (this.fsOk && this.fs != null) {
963       try {
964         FSUtils.checkFileSystemAvailable(this.fs);
965       } catch (IOException e) {
966         abort("File System not available", e);
967         this.fsOk = false;
968       }
969     }
970     return this.fsOk;
971   }
972 
973   /*
974    * Inner class that runs on a long period checking if regions need major
975    * compaction.
976    */
977   private static class MajorCompactionChecker extends Chore {
978     private final HRegionServer instance;
979 
980     MajorCompactionChecker(final HRegionServer h,
981         final int sleepTime, final AtomicBoolean stopper) {
982       super("MajorCompactionChecker", sleepTime, stopper);
983       this.instance = h;
984       LOG.info("Runs every " + sleepTime + "ms");
985     }
986 
987     @Override
988     protected void chore() {
989       Set<Integer> keys = this.instance.onlineRegions.keySet();
990       for (Integer i: keys) {
991         HRegion r = this.instance.onlineRegions.get(i);
992         try {
993           if (r != null && r.isMajorCompaction()) {
994             // Queue a compaction.  Will recognize if major is needed.
995             this.instance.compactSplitThread.
996               compactionRequested(r, getName() + " requests major compaction");
997           }
998         } catch (IOException e) {
999           LOG.warn("Failed major compaction check on " + r, e);
1000         }
1001       }
1002     }
1003   }
1004 
1005   /**
1006    * Report the status of the server. A server is online once all the startup
1007    * is completed (setting up filesystem, starting service threads, etc.). This
1008    * method is designed mostly to be useful in tests.
1009    * @return true if online, false if not.
1010    */
1011   public boolean isOnline() {
1012     return isOnline;
1013   }
1014 
1015   private HLog setupHLog() throws IOException {
1016     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1017     Path logdir = new Path(rootDir, HLog.getHLogDirectoryName(this.serverInfo));
1018     if (LOG.isDebugEnabled()) {
1019       LOG.debug("Log dir " + logdir);
1020     }
1021     if (fs.exists(logdir)) {
1022       throw new RegionServerRunningException("region server already " +
1023         "running at " + this.serverInfo.getServerName() +
1024         " because logdir " + logdir.toString() + " exists");
1025     }
1026     this.replicationHandler = new Replication(this.conf,this.serverInfo,
1027         this.fs, logdir, oldLogDir, stopRequested);
1028     HLog log = instantiateHLog(logdir, oldLogDir);
1029     this.replicationHandler.addLogEntryVisitor(log);
1030     return log;
1031   }
1032 
1033   // instantiate
1034   protected HLog instantiateHLog(Path logdir, Path oldLogDir) throws IOException {
1035     return new HLog(this.fs, logdir, oldLogDir, this.conf, this.hlogRoller,
1036       this.replicationHandler.getReplicationManager(),
1037         this.serverInfo.getServerAddress().toString());
1038   }
1039 
1040 
1041   protected LogRoller getLogRoller() {
1042     return hlogRoller;
1043   }
1044 
1045   /*
1046    * @param interval Interval since last time metrics were called.
1047    */
1048   protected void doMetrics() {
1049     try {
1050       metrics();
1051     } catch (Throwable e) {
1052       LOG.warn("Failed metrics", e);
1053     }
1054   }
1055 
1056   protected void metrics() {
1057     this.metrics.regions.set(this.onlineRegions.size());
1058     this.metrics.incrementRequests(this.requestCount.get());
1059     // Is this too expensive every three seconds getting a lock on onlineRegions
1060     // and then per store carried?  Can I make metrics be sloppier and avoid
1061     // the synchronizations?
1062     int stores = 0;
1063     int storefiles = 0;
1064     long memstoreSize = 0;
1065     long storefileIndexSize = 0;
1066     synchronized (this.onlineRegions) {
1067       for (Map.Entry<Integer, HRegion> e: this.onlineRegions.entrySet()) {
1068         HRegion r = e.getValue();
1069         memstoreSize += r.memstoreSize.get();
1070         synchronized (r.stores) {
1071           stores += r.stores.size();
1072           for(Map.Entry<byte [], Store> ee: r.stores.entrySet()) {
1073             Store store = ee.getValue();
1074             storefiles += store.getStorefilesCount();
1075             storefileIndexSize += store.getStorefilesIndexSize();
1076           }
1077         }
1078       }
1079     }
1080     this.metrics.stores.set(stores);
1081     this.metrics.storefiles.set(storefiles);
1082     this.metrics.memstoreSizeMB.set((int)(memstoreSize/(1024*1024)));
1083     this.metrics.storefileIndexSizeMB.set((int)(storefileIndexSize/(1024*1024)));
1084     this.metrics.compactionQueueSize.set(compactSplitThread.
1085       getCompactionQueueSize());
1086 
1087     LruBlockCache lruBlockCache = (LruBlockCache)StoreFile.getBlockCache(conf);
1088     if (lruBlockCache != null) {
1089       this.metrics.blockCacheCount.set(lruBlockCache.size());
1090       this.metrics.blockCacheFree.set(lruBlockCache.getFreeSize());
1091       this.metrics.blockCacheSize.set(lruBlockCache.getCurrentSize());
1092       double ratio = lruBlockCache.getStats().getHitRatio();
1093       int percent = (int) (ratio * 100);
1094       this.metrics.blockCacheHitRatio.set(percent);
1095     }
1096   }
1097 
1098   /**
1099    * @return Region server metrics instance.
1100    */
1101   public RegionServerMetrics getMetrics() {
1102     return this.metrics;
1103   }
1104 
1105   /*
1106    * Start maintanence Threads, Server, Worker and lease checker threads.
1107    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1108    * get an unhandled exception.  We cannot set the handler on all threads.
1109    * Server's internal Listener thread is off limits.  For Server, if an OOME,
1110    * it waits a while then retries.  Meantime, a flush or a compaction that
1111    * tries to run should trigger same critical condition and the shutdown will
1112    * run.  On its way out, this server will shut down Server.  Leases are sort
1113    * of inbetween. It has an internal thread that while it inherits from
1114    * Chore, it keeps its own internal stop mechanism so needs to be stopped
1115    * by this hosting server.  Worker logs the exception and exits.
1116    */
1117   private void startServiceThreads() throws IOException {
1118     String n = Thread.currentThread().getName();
1119     UncaughtExceptionHandler handler = new UncaughtExceptionHandler() {
1120       public void uncaughtException(Thread t, Throwable e) {
1121         abort("Uncaught exception in service thread " + t.getName(), e);
1122       }
1123     };
1124     Threads.setDaemonThreadRunning(this.hlogRoller, n + ".logRoller",
1125         handler);
1126     Threads.setDaemonThreadRunning(this.cacheFlusher, n + ".cacheFlusher",
1127       handler);
1128     Threads.setDaemonThreadRunning(this.compactSplitThread, n + ".compactor",
1129         handler);
1130     Threads.setDaemonThreadRunning(this.workerThread, n + ".worker", handler);
1131     Threads.setDaemonThreadRunning(this.majorCompactionChecker,
1132         n + ".majorCompactionChecker", handler);
1133 
1134     // Leases is not a Thread. Internally it runs a daemon thread.  If it gets
1135     // an unhandled exception, it will just exit.
1136     this.leases.setName(n + ".leaseChecker");
1137     this.leases.start();
1138     // Put up info server.
1139     int port = this.conf.getInt("hbase.regionserver.info.port", 60030);
1140     // -1 is for disabling info server
1141     if (port >= 0) {
1142       String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1143       // check if auto port bind enabled
1144       boolean auto = this.conf.getBoolean("hbase.regionserver.info.port.auto",
1145           false);
1146       while (true) {
1147         try {
1148           this.infoServer = new InfoServer("regionserver", addr, port, false);
1149           this.infoServer.setAttribute("regionserver", this);
1150           this.infoServer.start();
1151           break;
1152         } catch (BindException e) {
1153           if (!auto){
1154             // auto bind disabled throw BindException
1155             throw e;
1156           }
1157           // auto bind enabled, try to use another port
1158           LOG.info("Failed binding http info server to port: " + port);
1159           port++;
1160           // update HRS server info port.
1161           this.serverInfo = new HServerInfo(this.serverInfo.getServerAddress(),
1162             this.serverInfo.getStartCode(), port,
1163             this.serverInfo.getHostname());
1164         }
1165       }
1166     }
1167 
1168     this.replicationHandler.startReplicationServices();
1169 
1170     // Start Server.  This service is like leases in that it internally runs
1171     // a thread.
1172     this.server.start();
1173     LOG.info("HRegionServer started at: " +
1174       this.serverInfo.getServerAddress().toString());
1175   }
1176 
1177   /*
1178    * Verify that server is healthy
1179    */
1180   private boolean isHealthy() {
1181     if (!fsOk) {
1182       // File system problem
1183       return false;
1184     }
1185     // Verify that all threads are alive
1186     if (!(leases.isAlive() && compactSplitThread.isAlive() &&
1187         cacheFlusher.isAlive() && hlogRoller.isAlive() &&
1188         workerThread.isAlive() && this.majorCompactionChecker.isAlive())) {
1189       // One or more threads are no longer alive - shut down
1190       stop();
1191       return false;
1192     }
1193     return true;
1194   }
1195 
1196   /*
1197    * Run some housekeeping tasks.
1198    */
1199   private void housekeeping() {
1200     // If the todo list has > 0 messages, iterate looking for open region
1201     // messages. Send the master a message that we're working on its
1202     // processing so it doesn't assign the region elsewhere.
1203     if (this.toDo.isEmpty()) {
1204       return;
1205     }
1206     // This iterator isn't safe if elements are gone and HRS.Worker could
1207     // remove them (it already checks for null there). Goes from oldest.
1208     for (ToDoEntry e: this.toDo) {
1209       if(e == null) {
1210         LOG.warn("toDo gave a null entry during iteration");
1211         break;
1212       }
1213       HMsg msg = e.msg;
1214       if (msg != null) {
1215         if (msg.isType(HMsg.Type.MSG_REGION_OPEN)) {
1216           addProcessingMessage(msg.getRegionInfo());
1217         }
1218       } else {
1219         LOG.warn("Message is empty: " + e);
1220       }
1221     }
1222   }
1223 
1224   /** @return the HLog */
1225   public HLog getLog() {
1226     return this.hlog;
1227   }
1228 
1229   /**
1230    * Sets a flag that will cause all the HRegionServer threads to shut down
1231    * in an orderly fashion.  Used by unit tests.
1232    */
1233   public void stop() {
1234     this.stopRequested.set(true);
1235     synchronized(this) {
1236       // Wakes run() if it is sleeping
1237       notifyAll(); // FindBugs NN_NAKED_NOTIFY
1238     }
1239   }
1240 
1241   /**
1242    * Cause the server to exit without closing the regions it is serving, the
1243    * log it is using and without notifying the master.
1244    * Used unit testing and on catastrophic events such as HDFS is yanked out
1245    * from under hbase or we OOME.
1246    * @param reason the reason we are aborting
1247    * @param cause the exception that caused the abort, or null
1248    */
1249   public void abort(String reason, Throwable cause) {
1250     if (cause != null) {
1251       LOG.fatal("Aborting region server " + this + ": " + reason, cause);
1252     } else {
1253       LOG.fatal("Aborting region server " + this + ": " + reason);
1254     }
1255     this.abortRequested = true;
1256     this.reservedSpace.clear();
1257     if (this.metrics != null) {
1258       LOG.info("Dump of metrics: " + this.metrics);
1259     }
1260     stop();
1261   }
1262   
1263   /**
1264    * @see HRegionServer#abort(String, Throwable)
1265    */
1266   public void abort(String reason) {
1267     abort(reason, null);
1268   }
1269 
1270   /*
1271    * Simulate a kill -9 of this server.
1272    * Exits w/o closing regions or cleaninup logs but it does close socket in
1273    * case want to bring up server on old hostname+port immediately.
1274    */
1275   protected void kill() {
1276     this.killed = true;
1277     abort("Simulated kill");
1278   }
1279 
1280   /**
1281    * Wait on all threads to finish.
1282    * Presumption is that all closes and stops have already been called.
1283    */
1284   protected void join() {
1285     Threads.shutdown(this.majorCompactionChecker);
1286     Threads.shutdown(this.workerThread);
1287     Threads.shutdown(this.cacheFlusher);
1288     Threads.shutdown(this.compactSplitThread);
1289     Threads.shutdown(this.hlogRoller);
1290     this.replicationHandler.join();
1291   }
1292 
1293   private boolean getMaster() {
1294     HServerAddress masterAddress = null;
1295     while (masterAddress == null) {
1296       if (stopRequested.get()) {
1297         return false;
1298       }
1299       try {
1300         masterAddress = zooKeeperWrapper.readMasterAddressOrThrow();
1301       } catch (IOException e) {
1302         LOG.warn("Unable to read master address from ZooKeeper. Retrying." +
1303                  " Error was:", e);
1304         sleeper.sleep();
1305       }
1306     }
1307 
1308     LOG.info("Telling master at " + masterAddress + " that we are up");
1309     HMasterRegionInterface master = null;
1310     while (!stopRequested.get() && master == null) {
1311       try {
1312         // Do initial RPC setup.  The final argument indicates that the RPC
1313         // should retry indefinitely.
1314         master = (HMasterRegionInterface)HBaseRPC.waitForProxy(
1315           HMasterRegionInterface.class, HBaseRPCProtocolVersion.versionID,
1316           masterAddress.getInetSocketAddress(), this.conf, -1, this.rpcTimeout);
1317       } catch (IOException e) {
1318         LOG.warn("Unable to connect to master. Retrying. Error was:", e);
1319         sleeper.sleep();
1320       }
1321     }
1322     this.hbaseMaster = master;
1323     return true;
1324   }
1325 
1326   /*
1327    * Let the master know we're here
1328    * Run initialization using parameters passed us by the master.
1329    */
1330   private MapWritable reportForDuty() {
1331     while (!stopRequested.get() && !getMaster()) {
1332       sleeper.sleep();
1333       LOG.warn("Unable to get master for initialization");
1334     }
1335 
1336     MapWritable result = null;
1337     long lastMsg = 0;
1338     while(!stopRequested.get()) {
1339       try {
1340         this.requestCount.set(0);
1341         MemoryUsage memory =
1342           ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
1343         HServerLoad hsl = new HServerLoad(0, (int)memory.getUsed()/1024/1024,
1344           (int)memory.getMax()/1024/1024);
1345         this.serverInfo.setLoad(hsl);
1346         if (LOG.isDebugEnabled())
1347           LOG.debug("sending initial server load: " + hsl);
1348         lastMsg = System.currentTimeMillis();
1349         boolean startCodeOk = false;
1350         while(!startCodeOk) {
1351           this.serverInfo = createServerInfoWithNewStartCode(this.serverInfo);
1352           startCodeOk = zooKeeperWrapper.writeRSLocation(this.serverInfo);
1353           if(!startCodeOk) {
1354            LOG.debug("Start code already taken, trying another one");
1355           }
1356         }
1357         result = this.hbaseMaster.regionServerStartup(this.serverInfo);
1358         break;
1359       } catch (IOException e) {
1360         LOG.warn("error telling master we are up", e);
1361       }
1362       sleeper.sleep(lastMsg);
1363     }
1364     return result;
1365   }
1366 
1367   private HServerInfo createServerInfoWithNewStartCode(final HServerInfo hsi) {
1368     return new HServerInfo(hsi.getServerAddress(), hsi.getInfoPort(),
1369       hsi.getHostname());
1370   }
1371 
1372   /* Add to the outbound message buffer */
1373   private void reportOpen(HRegionInfo region) {
1374     this.outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, region));
1375   }
1376 
1377   /* Add to the outbound message buffer */
1378   private void reportClose(HRegionInfo region) {
1379     reportClose(region, null);
1380   }
1381 
1382   /* Add to the outbound message buffer */
1383   private void reportClose(final HRegionInfo region, final byte[] message) {
1384     this.outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_CLOSE, region, message));
1385   }
1386 
1387   /**
1388    * Add to the outbound message buffer
1389    *
1390    * When a region splits, we need to tell the master that there are two new
1391    * regions that need to be assigned.
1392    *
1393    * We do not need to inform the master about the old region, because we've
1394    * updated the meta or root regions, and the master will pick that up on its
1395    * next rescan of the root or meta tables.
1396    */
1397   void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA,
1398       HRegionInfo newRegionB) {
1399     this.outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT_INCLUDES_DAUGHTERS,
1400       oldRegion, newRegionA, newRegionB,
1401       Bytes.toBytes("Daughters; " +
1402           newRegionA.getRegionNameAsString() + ", " +
1403           newRegionB.getRegionNameAsString())));
1404   }
1405 
1406   //////////////////////////////////////////////////////////////////////////////
1407   // HMaster-given operations
1408   //////////////////////////////////////////////////////////////////////////////
1409 
1410   /*
1411    * Data structure to hold a HMsg and retries count.
1412    */
1413   private static final class ToDoEntry {
1414     protected final AtomicInteger tries = new AtomicInteger(0);
1415     protected final HMsg msg;
1416 
1417     ToDoEntry(final HMsg msg) {
1418       this.msg = msg;
1419     }
1420   }
1421 
1422   final BlockingQueue<ToDoEntry> toDo = new LinkedBlockingQueue<ToDoEntry>();
1423   private Worker worker;
1424   private Thread workerThread;
1425 
1426   /** Thread that performs long running requests from the master */
1427   class Worker implements Runnable {
1428     void stop() {
1429       synchronized(toDo) {
1430         toDo.notifyAll();
1431       }
1432     }
1433 
1434     public void run() {
1435       try {
1436         while(!stopRequested.get()) {
1437           ToDoEntry e = null;
1438           try {
1439             e = toDo.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
1440             if(e == null || stopRequested.get()) {
1441               continue;
1442             }
1443             LOG.info("Worker: " + e.msg);
1444             HRegion region = null;
1445             HRegionInfo info = e.msg.getRegionInfo();
1446             switch(e.msg.getType()) {
1447 
1448             case MSG_REGIONSERVER_QUIESCE:
1449               closeUserRegions();
1450               break;
1451 
1452             case MSG_REGION_OPEN:
1453               // Open a region
1454               if (!haveRootRegion.get() && !info.isRootRegion()) {
1455                 // root region is not online yet. requeue this task
1456                 LOG.info("putting region open request back into queue because" +
1457                     " root region is not yet available");
1458                 try {
1459                   toDo.put(e);
1460                 } catch (InterruptedException ex) {
1461                   LOG.warn("insertion into toDo queue was interrupted", ex);
1462                   break;
1463                 }
1464               }
1465               openRegion(info);
1466               break;
1467 
1468             case MSG_REGION_CLOSE:
1469               // Close a region
1470               closeRegion(e.msg.getRegionInfo(), true);
1471               break;
1472 
1473             case MSG_REGION_CLOSE_WITHOUT_REPORT:
1474               // Close a region, don't reply
1475               closeRegion(e.msg.getRegionInfo(), false);
1476               break;
1477 
1478             case MSG_REGION_SPLIT:
1479               region = getRegion(info.getRegionName());
1480               region.flushcache();
1481               region.shouldSplit(true);
1482               // force a compaction; split will be side-effect.
1483               compactSplitThread.compactionRequested(region,
1484                 e.msg.getType().name());
1485               break;
1486 
1487             case MSG_REGION_MAJOR_COMPACT:
1488             case MSG_REGION_COMPACT:
1489               // Compact a region
1490               region = getRegion(info.getRegionName());
1491               compactSplitThread.compactionRequested(region,
1492                 e.msg.isType(Type.MSG_REGION_MAJOR_COMPACT),
1493                 e.msg.getType().name());
1494               break;
1495 
1496             case MSG_REGION_FLUSH:
1497               region = getRegion(info.getRegionName());
1498               region.flushcache();
1499               break;
1500 
1501             case TESTING_MSG_BLOCK_RS:
1502               while (!stopRequested.get()) {
1503                 Threads.sleep(1000);
1504                 LOG.info("Regionserver blocked by " +
1505                   HMsg.Type.TESTING_MSG_BLOCK_RS + "; " + stopRequested.get());
1506               }
1507               break;
1508 
1509             default:
1510               throw new AssertionError(
1511                   "Impossible state during msg processing.  Instruction: "
1512                   + e.msg.toString());
1513             }
1514           } catch (InterruptedException ex) {
1515             LOG.warn("Processing Worker queue", ex);
1516           } catch (Exception ex) {
1517             if (ex instanceof IOException) {
1518               ex = RemoteExceptionHandler.checkIOException((IOException) ex);
1519             }
1520             if(e != null && e.tries.get() < numRetries) {
1521               LOG.warn(ex);
1522               e.tries.incrementAndGet();
1523               try {
1524                 toDo.put(e);
1525               } catch (InterruptedException ie) {
1526                 throw new RuntimeException("Putting into msgQueue was " +
1527                     "interrupted.", ex);
1528               }
1529             } else {
1530               LOG.error("unable to process message" +
1531                   (e != null ? (": " + e.msg.toString()) : ""), ex);
1532               if (!checkFileSystem()) {
1533                 break;
1534               }
1535             }
1536           }
1537         }
1538       } catch(Throwable t) {
1539         if (!checkOOME(t)) {
1540           LOG.fatal("Unhandled exception", t);
1541         }
1542       } finally {
1543         LOG.info("worker thread exiting");
1544       }
1545     }
1546   }
1547 
1548   void openRegion(final HRegionInfo regionInfo) {
1549     Integer mapKey = Bytes.mapKey(regionInfo.getRegionName());
1550     HRegion region = this.onlineRegions.get(mapKey);
1551     if (region == null) {
1552       try {
1553         region = instantiateRegion(regionInfo, this.hlog);
1554         // Startup a compaction early if one is needed, if region has references
1555         // or if a store has too many store files
1556         if (region.hasReferences() || region.hasTooManyStoreFiles()) {
1557           this.compactSplitThread.compactionRequested(region,
1558             region.hasReferences() ? "Region has references on open" :
1559                                      "Region has too many store files");
1560         }
1561       } catch (Throwable e) {
1562         Throwable t = cleanup(e,
1563           "Error opening " + regionInfo.getRegionNameAsString());
1564         // TODO: add an extra field in HRegionInfo to indicate that there is
1565         // an error. We can't do that now because that would be an incompatible
1566         // change that would require a migration
1567         reportClose(regionInfo, StringUtils.stringifyException(t).getBytes());
1568         return;
1569       }
1570       addToOnlineRegions(region);
1571     }
1572     reportOpen(regionInfo);
1573   }
1574 
1575   /*
1576    * @param regionInfo RegionInfo for the Region we're to instantiate and
1577    * initialize.
1578    * @param wal Set into here the regions' seqid.
1579    * @return
1580    * @throws IOException
1581    */
1582   protected HRegion instantiateRegion(final HRegionInfo regionInfo, final HLog wal)
1583   throws IOException {
1584     Path dir =
1585       HTableDescriptor.getTableDir(rootDir, regionInfo.getTableDesc().getName());
1586     HRegion r = HRegion.newHRegion(dir, this.hlog, this.fs, conf, regionInfo,
1587       this.cacheFlusher);
1588     long seqid = r.initialize(new Progressable() {
1589       public void progress() {
1590         addProcessingMessage(regionInfo);
1591       }
1592     });
1593     // If seqid  > current wal seqid, the wal seqid is updated.
1594     if (wal != null) wal.setSequenceNumber(seqid);
1595     return r;
1596   }
1597 
1598   /**
1599    * Add a MSG_REPORT_PROCESS_OPEN to the outbound queue.
1600    * This method is called while region is in the queue of regions to process
1601    * and then while the region is being opened, it is called from the Worker
1602    * thread that is running the region open.
1603    * @param hri Region to add the message for
1604    */
1605   public void addProcessingMessage(final HRegionInfo hri) {
1606     getOutboundMsgs().add(new HMsg(HMsg.Type.MSG_REPORT_PROCESS_OPEN, hri));
1607   }
1608 
1609   protected void closeRegion(final HRegionInfo hri, final boolean reportWhenCompleted)
1610   throws IOException {
1611     HRegion region = this.removeFromOnlineRegions(hri);
1612     if (region != null) {
1613       region.close();
1614       if(reportWhenCompleted) {
1615         reportClose(hri);
1616       }
1617     }
1618   }
1619 
1620   /** Called either when the master tells us to restart or from stop() */
1621   ArrayList<HRegion> closeAllRegions() {
1622     ArrayList<HRegion> regionsToClose = new ArrayList<HRegion>();
1623     this.lock.writeLock().lock();
1624     try {
1625       regionsToClose.addAll(onlineRegions.values());
1626       onlineRegions.clear();
1627     } finally {
1628       this.lock.writeLock().unlock();
1629     }
1630     // Close any outstanding scanners.  Means they'll get an UnknownScanner
1631     // exception next time they come in.
1632     for (Map.Entry<String, InternalScanner> e: this.scanners.entrySet()) {
1633       try {
1634         e.getValue().close();
1635       } catch (IOException ioe) {
1636         LOG.warn("Closing scanner " + e.getKey(), ioe);
1637       }
1638     }
1639     for (HRegion region: regionsToClose) {
1640       if (LOG.isDebugEnabled()) {
1641         LOG.debug("closing region " + Bytes.toString(region.getRegionName()));
1642       }
1643       try {
1644         region.close(abortRequested);
1645       } catch (Throwable e) {
1646         cleanup(e, "Error closing " + Bytes.toString(region.getRegionName()));
1647       }
1648     }
1649     return regionsToClose;
1650   }
1651 
1652   /*
1653    * Thread to run close of a region.
1654    */
1655   private static class RegionCloserThread extends Thread {
1656     private final HRegion r;
1657 
1658     protected RegionCloserThread(final HRegion r) {
1659       super(Thread.currentThread().getName() + ".regionCloser." + r.toString());
1660       this.r = r;
1661     }
1662 
1663     @Override
1664     public void run() {
1665       try {
1666         if (LOG.isDebugEnabled()) {
1667           LOG.debug("Closing region " + r.toString());
1668         }
1669         r.close();
1670       } catch (Throwable e) {
1671         LOG.error("Error closing region " + r.toString(),
1672           RemoteExceptionHandler.checkThrowable(e));
1673       }
1674     }
1675   }
1676 
1677   /** Called as the first stage of cluster shutdown. */
1678   void closeUserRegions() {
1679     ArrayList<HRegion> regionsToClose = new ArrayList<HRegion>();
1680     this.lock.writeLock().lock();
1681     try {
1682       synchronized (onlineRegions) {
1683         for (Iterator<Map.Entry<Integer, HRegion>> i =
1684             onlineRegions.entrySet().iterator(); i.hasNext();) {
1685           Map.Entry<Integer, HRegion> e = i.next();
1686           HRegion r = e.getValue();
1687           if (!r.getRegionInfo().isMetaRegion()) {
1688             regionsToClose.add(r);
1689             i.remove();
1690           }
1691         }
1692       }
1693     } finally {
1694       this.lock.writeLock().unlock();
1695     }
1696     // Run region closes in parallel.
1697     Set<Thread> threads = new HashSet<Thread>();
1698     try {
1699       for (final HRegion r : regionsToClose) {
1700         RegionCloserThread t = new RegionCloserThread(r);
1701         t.start();
1702         threads.add(t);
1703       }
1704     } finally {
1705       for (Thread t : threads) {
1706         while (t.isAlive()) {
1707           try {
1708             t.join();
1709           } catch (InterruptedException e) {
1710             e.printStackTrace();
1711           }
1712         }
1713       }
1714     }
1715     this.quiesced.set(true);
1716     if (onlineRegions.size() == 0) {
1717       outboundMsgs.add(REPORT_EXITING);
1718     } else {
1719       outboundMsgs.add(REPORT_QUIESCED);
1720     }
1721   }
1722 
1723   //
1724   // HRegionInterface
1725   //
1726 
1727   public HRegionInfo getRegionInfo(final byte [] regionName)
1728   throws NotServingRegionException {
1729     requestCount.incrementAndGet();
1730     return getRegion(regionName).getRegionInfo();
1731   }
1732 
1733 
1734   public Result getClosestRowBefore(final byte [] regionName,
1735     final byte [] row, final byte [] family)
1736   throws IOException {
1737     checkOpen();
1738     requestCount.incrementAndGet();
1739     try {
1740       // locate the region we're operating on
1741       HRegion region = getRegion(regionName);
1742       // ask the region for all the data
1743 
1744       Result r = region.getClosestRowBefore(row, family);
1745       return r;
1746     } catch (Throwable t) {
1747       throw convertThrowableToIOE(cleanup(t));
1748     }
1749   }
1750 
1751   /** {@inheritDoc} */
1752   public Result get(byte [] regionName, Get get) throws IOException {
1753     checkOpen();
1754     requestCount.incrementAndGet();
1755     try {
1756       HRegion region = getRegion(regionName);
1757       return region.get(get, getLockFromId(get.getLockId()));
1758     } catch(Throwable t) {
1759       throw convertThrowableToIOE(cleanup(t));
1760     }
1761   }
1762 
1763   public boolean exists(byte [] regionName, Get get) throws IOException {
1764     checkOpen();
1765     requestCount.incrementAndGet();
1766     try {
1767       HRegion region = getRegion(regionName);
1768       Result r = region.get(get, getLockFromId(get.getLockId()));
1769       return r != null && !r.isEmpty();
1770     } catch(Throwable t) {
1771       throw convertThrowableToIOE(cleanup(t));
1772     }
1773   }
1774 
1775   public void put(final byte [] regionName, final Put put)
1776   throws IOException {
1777     if (put.getRow() == null)
1778       throw new IllegalArgumentException("update has null row");
1779 
1780     checkOpen();
1781     this.requestCount.incrementAndGet();
1782     HRegion region = getRegion(regionName);
1783     try {
1784       if (!region.getRegionInfo().isMetaTable()) {
1785         this.cacheFlusher.reclaimMemStoreMemory();
1786       }
1787       boolean writeToWAL = put.getWriteToWAL();
1788       region.put(put, getLockFromId(put.getLockId()), writeToWAL);
1789     } catch (Throwable t) {
1790       throw convertThrowableToIOE(cleanup(t));
1791     }
1792   }
1793 
1794   public int put(final byte[] regionName, final List<Put> puts)
1795   throws IOException {
1796     checkOpen();
1797     HRegion region = null;
1798     try {
1799       region = getRegion(regionName);
1800       if (!region.getRegionInfo().isMetaTable()) {
1801         this.cacheFlusher.reclaimMemStoreMemory();
1802       }
1803       
1804       @SuppressWarnings("unchecked")
1805       Pair<Put, Integer>[] putsWithLocks = new Pair[puts.size()];
1806       
1807       int i = 0;
1808       for (Put p : puts) {
1809         Integer lock = getLockFromId(p.getLockId());
1810         putsWithLocks[i++] = new Pair<Put, Integer>(p, lock);
1811       }
1812       
1813       this.requestCount.addAndGet(puts.size());
1814       OperationStatusCode[] codes = region.put(putsWithLocks);
1815       for (i = 0; i < codes.length; i++) {
1816         if (codes[i] != OperationStatusCode.SUCCESS)
1817           return i;
1818       }
1819       return -1;
1820     } catch (Throwable t) {
1821       throw convertThrowableToIOE(cleanup(t));
1822     }
1823   }
1824 
1825   private boolean checkAndMutate(final byte[] regionName, final byte [] row,
1826       final byte [] family, final byte [] qualifier, final byte [] value,
1827       final Writable w, Integer lock) throws IOException {
1828     checkOpen();
1829     this.requestCount.incrementAndGet();
1830     HRegion region = getRegion(regionName);
1831     try {
1832       if (!region.getRegionInfo().isMetaTable()) {
1833         this.cacheFlusher.reclaimMemStoreMemory();
1834       }
1835       return region.checkAndMutate(row, family, qualifier, value, w, lock,
1836           true);
1837     } catch (Throwable t) {
1838       throw convertThrowableToIOE(cleanup(t));
1839     }
1840   }
1841 
1842 
1843   /**
1844    *
1845    * @param regionName
1846    * @param row
1847    * @param family
1848    * @param qualifier
1849    * @param value the expected value
1850    * @param put
1851    * @throws IOException
1852    * @return true if the new put was execute, false otherwise
1853    */
1854   public boolean checkAndPut(final byte[] regionName, final byte [] row,
1855       final byte [] family, final byte [] qualifier, final byte [] value,
1856       final Put put) throws IOException{
1857     return checkAndMutate(regionName, row, family, qualifier, value, put,
1858         getLockFromId(put.getLockId()));
1859   }
1860 
1861   /**
1862    *
1863    * @param regionName
1864    * @param row
1865    * @param family
1866    * @param qualifier
1867    * @param value the expected value
1868    * @param delete
1869    * @throws IOException
1870    * @return true if the new put was execute, false otherwise
1871    */
1872   public boolean checkAndDelete(final byte[] regionName, final byte [] row,
1873       final byte [] family, final byte [] qualifier, final byte [] value,
1874       final Delete delete) throws IOException{
1875     return checkAndMutate(regionName, row, family, qualifier, value, delete,
1876         getLockFromId(delete.getLockId()));
1877   }
1878 
1879   //
1880   // remote scanner interface
1881   //
1882 
1883   public long openScanner(byte [] regionName, Scan scan)
1884   throws IOException {
1885     checkOpen();
1886     NullPointerException npe = null;
1887     if (regionName == null) {
1888       npe = new NullPointerException("regionName is null");
1889     } else if (scan == null) {
1890       npe = new NullPointerException("scan is null");
1891     }
1892     if (npe != null) {
1893       throw new IOException("Invalid arguments to openScanner", npe);
1894     }
1895     requestCount.incrementAndGet();
1896     try {
1897       HRegion r = getRegion(regionName);
1898       return addScanner(r.getScanner(scan));
1899     } catch (Throwable t) {
1900       throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
1901     }
1902   }
1903 
1904   protected long addScanner(InternalScanner s) throws LeaseStillHeldException {
1905     long scannerId = -1L;
1906     scannerId = rand.nextLong();
1907     String scannerName = String.valueOf(scannerId);
1908     scanners.put(scannerName, s);
1909     this.leases.
1910       createLease(scannerName, new ScannerListener(scannerName));
1911     return scannerId;
1912   }
1913 
1914   public Result next(final long scannerId) throws IOException {
1915     Result [] res = next(scannerId, 1);
1916     if(res == null || res.length == 0) {
1917       return null;
1918     }
1919     return res[0];
1920   }
1921 
1922   public Result [] next(final long scannerId, int nbRows) throws IOException {
1923     try {
1924       String scannerName = String.valueOf(scannerId);
1925       InternalScanner s = this.scanners.get(scannerName);
1926       if (s == null) {
1927         throw new UnknownScannerException("Name: " + scannerName);
1928       }
1929       try {
1930         checkOpen();
1931       } catch (IOException e) {
1932         // If checkOpen failed, server not running or filesystem gone,
1933         // cancel this lease; filesystem is gone or we're closing or something.
1934         this.leases.cancelLease(scannerName);
1935         throw e;
1936       }
1937       this.leases.renewLease(scannerName);
1938       List<Result> results = new ArrayList<Result>(nbRows);
1939       long currentScanResultSize = 0;
1940       List<KeyValue> values = new ArrayList<KeyValue>();
1941       for (int i = 0; i < nbRows && currentScanResultSize < maxScannerResultSize; i++) {
1942         requestCount.incrementAndGet();
1943         // Collect values to be returned here
1944         boolean moreRows = s.next(values);
1945         if (!values.isEmpty()) {
1946           for (KeyValue kv : values) {
1947             currentScanResultSize += kv.heapSize();
1948           }
1949           results.add(new Result(values));
1950         }
1951         if (!moreRows) {
1952           break;
1953         }
1954         values.clear();
1955       }
1956       // Below is an ugly hack where we cast the InternalScanner to be a
1957       // HRegion.RegionScanner.  The alternative is to change InternalScanner
1958       // interface but its used everywhere whereas we just need a bit of info
1959       // from HRegion.RegionScanner, IF its filter if any is done with the scan
1960       // and wants to tell the client to stop the scan.  This is done by passing
1961       // a null result.
1962       return ((HRegion.RegionScanner)s).isFilterDone() && results.isEmpty()?
1963         null: results.toArray(new Result[0]);
1964     } catch (Throwable t) {
1965       if (t instanceof NotServingRegionException) {
1966         String scannerName = String.valueOf(scannerId);
1967         this.scanners.remove(scannerName);
1968       }
1969       throw convertThrowableToIOE(cleanup(t));
1970     }
1971   }
1972 
1973   public void close(final long scannerId) throws IOException {
1974     try {
1975       checkOpen();
1976       requestCount.incrementAndGet();
1977       String scannerName = String.valueOf(scannerId);
1978       InternalScanner s = scanners.remove(scannerName);
1979       if (s != null) {
1980         s.close();
1981         this.leases.cancelLease(scannerName);
1982       }
1983     } catch (Throwable t) {
1984       throw convertThrowableToIOE(cleanup(t));
1985     }
1986   }
1987 
1988   /**
1989    * Instantiated as a scanner lease.
1990    * If the lease times out, the scanner is closed
1991    */
1992   private class ScannerListener implements LeaseListener {
1993     private final String scannerName;
1994 
1995     ScannerListener(final String n) {
1996       this.scannerName = n;
1997     }
1998 
1999     public void leaseExpired() {
2000       LOG.info("Scanner " + this.scannerName + " lease expired");
2001       InternalScanner s = scanners.remove(this.scannerName);
2002       if (s != null) {
2003         try {
2004           s.close();
2005         } catch (IOException e) {
2006           LOG.error("Closing scanner", e);
2007         }
2008       }
2009     }
2010   }
2011 
2012   //
2013   // Methods that do the actual work for the remote API
2014   //
2015   public void delete(final byte [] regionName, final Delete delete)
2016   throws IOException {
2017     checkOpen();
2018     try {
2019       boolean writeToWAL = true;
2020       this.requestCount.incrementAndGet();
2021       HRegion region = getRegion(regionName);
2022       if (!region.getRegionInfo().isMetaTable()) {
2023         this.cacheFlusher.reclaimMemStoreMemory();
2024       }
2025       Integer lid = getLockFromId(delete.getLockId());
2026       region.delete(delete, lid, writeToWAL);
2027     } catch (Throwable t) {
2028       throw convertThrowableToIOE(cleanup(t));
2029     }
2030   }
2031 
2032   public int delete(final byte[] regionName, final List<Delete> deletes)
2033   throws IOException {
2034     // Count of Deletes processed.
2035     int i = 0;
2036     checkOpen();
2037     HRegion region = null;
2038     try {
2039       boolean writeToWAL = true;
2040       region = getRegion(regionName);
2041       if (!region.getRegionInfo().isMetaTable()) {
2042         this.cacheFlusher.reclaimMemStoreMemory();
2043       }
2044       int size = deletes.size();
2045       Integer[] locks = new Integer[size];
2046       for (Delete delete: deletes) {
2047         this.requestCount.incrementAndGet();
2048         locks[i] = getLockFromId(delete.getLockId());
2049         region.delete(delete, locks[i], writeToWAL);
2050         i++;
2051       }
2052     } catch (WrongRegionException ex) {
2053       LOG.debug("Batch deletes: " + i, ex);
2054       return i;
2055     } catch (NotServingRegionException ex) {
2056       return i;
2057     } catch (Throwable t) {
2058       throw convertThrowableToIOE(cleanup(t));
2059     }
2060     return -1;
2061   }
2062 
2063   public long lockRow(byte [] regionName, byte [] row)
2064   throws IOException {
2065     checkOpen();
2066     NullPointerException npe = null;
2067     if(regionName == null) {
2068       npe = new NullPointerException("regionName is null");
2069     } else if(row == null) {
2070       npe = new NullPointerException("row to lock is null");
2071     }
2072     if(npe != null) {
2073       IOException io = new IOException("Invalid arguments to lockRow");
2074       io.initCause(npe);
2075       throw io;
2076     }
2077     requestCount.incrementAndGet();
2078     try {
2079       HRegion region = getRegion(regionName);
2080       Integer r = region.obtainRowLock(row);
2081       long lockId = addRowLock(r,region);
2082       LOG.debug("Row lock " + lockId + " explicitly acquired by client");
2083       return lockId;
2084     } catch (Throwable t) {
2085       throw convertThrowableToIOE(cleanup(t,
2086         "Error obtaining row lock (fsOk: " + this.fsOk + ")"));
2087     }
2088   }
2089 
2090   protected long addRowLock(Integer r, HRegion region) throws LeaseStillHeldException {
2091     long lockId = -1L;
2092     lockId = rand.nextLong();
2093     String lockName = String.valueOf(lockId);
2094     rowlocks.put(lockName, r);
2095     this.leases.
2096       createLease(lockName, new RowLockListener(lockName, region));
2097     return lockId;
2098   }
2099 
2100   /**
2101    * Method to get the Integer lock identifier used internally
2102    * from the long lock identifier used by the client.
2103    * @param lockId long row lock identifier from client
2104    * @return intId Integer row lock used internally in HRegion
2105    * @throws IOException Thrown if this is not a valid client lock id.
2106    */
2107   Integer getLockFromId(long lockId)
2108   throws IOException {
2109     if (lockId == -1L) {
2110       return null;
2111     }
2112     String lockName = String.valueOf(lockId);
2113     Integer rl = rowlocks.get(lockName);
2114     if (rl == null) {
2115       throw new IOException("Invalid row lock");
2116     }
2117     this.leases.renewLease(lockName);
2118     return rl;
2119   }
2120 
2121   public void unlockRow(byte [] regionName, long lockId)
2122   throws IOException {
2123     checkOpen();
2124     NullPointerException npe = null;
2125     if(regionName == null) {
2126       npe = new NullPointerException("regionName is null");
2127     } else if(lockId == -1L) {
2128       npe = new NullPointerException("lockId is null");
2129     }
2130     if(npe != null) {
2131       IOException io = new IOException("Invalid arguments to unlockRow");
2132       io.initCause(npe);
2133       throw io;
2134     }
2135     requestCount.incrementAndGet();
2136     try {
2137       HRegion region = getRegion(regionName);
2138       String lockName = String.valueOf(lockId);
2139       Integer r = rowlocks.remove(lockName);
2140       if(r == null) {
2141         throw new UnknownRowLockException(lockName);
2142       }
2143       region.releaseRowLock(r);
2144       this.leases.cancelLease(lockName);
2145       LOG.debug("Row lock " + lockId + " has been explicitly released by client");
2146     } catch (Throwable t) {
2147       throw convertThrowableToIOE(cleanup(t));
2148     }
2149   }
2150 
2151   @Override
2152   public void bulkLoadHFile(
2153       String hfilePath, byte[] regionName, byte[] familyName)
2154   throws IOException {
2155     HRegion region = getRegion(regionName);
2156     region.bulkLoadHFile(hfilePath, familyName);
2157   }
2158 
2159   Map<String, Integer> rowlocks =
2160     new ConcurrentHashMap<String, Integer>();
2161 
2162   /**
2163    * Instantiated as a row lock lease.
2164    * If the lease times out, the row lock is released
2165    */
2166   private class RowLockListener implements LeaseListener {
2167     private final String lockName;
2168     private final HRegion region;
2169 
2170     RowLockListener(final String lockName, final HRegion region) {
2171       this.lockName = lockName;
2172       this.region = region;
2173     }
2174 
2175     public void leaseExpired() {
2176       LOG.info("Row Lock " + this.lockName + " lease expired");
2177       Integer r = rowlocks.remove(this.lockName);
2178       if(r != null) {
2179         region.releaseRowLock(r);
2180       }
2181     }
2182   }
2183 
2184   /** @return the info server */
2185   public InfoServer getInfoServer() {
2186     return infoServer;
2187   }
2188 
2189   /**
2190    * @return true if a stop has been requested.
2191    */
2192   public boolean isStopRequested() {
2193     return this.stopRequested.get();
2194   }
2195 
2196   /**
2197    *
2198    * @return the configuration
2199    */
2200   public Configuration getConfiguration() {
2201     return conf;
2202   }
2203 
2204   /** @return the write lock for the server */
2205   ReentrantReadWriteLock.WriteLock getWriteLock() {
2206     return lock.writeLock();
2207   }
2208 
2209   /**
2210    * @return Immutable list of this servers regions.
2211    */
2212   public Collection<HRegion> getOnlineRegions() {
2213     return Collections.unmodifiableCollection(onlineRegions.values());
2214   }
2215 
2216   public HRegion [] getOnlineRegionsAsArray() {
2217     return getOnlineRegions().toArray(new HRegion[0]);
2218   }
2219 
2220   public int getNumberOfOnlineRegions() {
2221     return onlineRegions.size();
2222   }
2223 
2224   /**
2225    * @return The HRegionInfos from online regions sorted
2226    */
2227   public SortedSet<HRegionInfo> getSortedOnlineRegionInfos() {
2228     SortedSet<HRegionInfo> result = new TreeSet<HRegionInfo>();
2229     synchronized(this.onlineRegions) {
2230       for (HRegion r: this.onlineRegions.values()) {
2231         result.add(r.getRegionInfo());
2232       }
2233     }
2234     return result;
2235   }
2236 
2237   public void addToOnlineRegions(final HRegion r) {
2238     this.lock.writeLock().lock();
2239     try {
2240       this.onlineRegions.put(Bytes.mapKey(r.getRegionInfo().getRegionName()), r);
2241     } finally {
2242       this.lock.writeLock().unlock();
2243     }
2244   }
2245 
2246   public HRegion removeFromOnlineRegions(HRegionInfo hri) {
2247     this.lock.writeLock().lock();
2248     HRegion toReturn = null;
2249     try {
2250       toReturn = onlineRegions.remove(Bytes.mapKey(hri.getRegionName()));
2251     } finally {
2252       this.lock.writeLock().unlock();
2253     }
2254     return toReturn;
2255   }
2256 
2257   /**
2258    * @return A new Map of online regions sorted by region size with the first
2259    * entry being the biggest.
2260    */
2261   public SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
2262     // we'll sort the regions in reverse
2263     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
2264         new Comparator<Long>() {
2265           public int compare(Long a, Long b) {
2266             return -1 * a.compareTo(b);
2267           }
2268         });
2269     // Copy over all regions. Regions are sorted by size with biggest first.
2270     synchronized (this.onlineRegions) {
2271       for (HRegion region : this.onlineRegions.values()) {
2272         sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
2273       }
2274     }
2275     return sortedRegions;
2276   }
2277 
2278   /**
2279    * @param regionName
2280    * @return HRegion for the passed <code>regionName</code> or null if named
2281    * region is not member of the online regions.
2282    */
2283   public HRegion getOnlineRegion(final byte [] regionName) {
2284     return onlineRegions.get(Bytes.mapKey(regionName));
2285   }
2286 
2287   /** @return the request count */
2288   public AtomicInteger getRequestCount() {
2289     return this.requestCount;
2290   }
2291 
2292   /** @return reference to FlushRequester */
2293   public FlushRequester getFlushRequester() {
2294     return this.cacheFlusher;
2295   }
2296 
2297   /**
2298    * Protected utility method for safely obtaining an HRegion handle.
2299    * @param regionName Name of online {@link HRegion} to return
2300    * @return {@link HRegion} for <code>regionName</code>
2301    * @throws NotServingRegionException
2302    */
2303   protected HRegion getRegion(final byte [] regionName)
2304   throws NotServingRegionException {
2305     HRegion region = null;
2306     this.lock.readLock().lock();
2307     try {
2308       region = onlineRegions.get(Integer.valueOf(Bytes.hashCode(regionName)));
2309       if (region == null) {
2310         throw new NotServingRegionException(regionName);
2311       }
2312       return region;
2313     } finally {
2314       this.lock.readLock().unlock();
2315     }
2316   }
2317 
2318   /**
2319    * Get the top N most loaded regions this server is serving so we can
2320    * tell the master which regions it can reallocate if we're overloaded.
2321    * TODO: actually calculate which regions are most loaded. (Right now, we're
2322    * just grabbing the first N regions being served regardless of load.)
2323    */
2324   protected HRegionInfo[] getMostLoadedRegions() {
2325     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2326     synchronized (onlineRegions) {
2327       for (HRegion r : onlineRegions.values()) {
2328         if (r.isClosed() || r.isClosing()) {
2329           continue;
2330         }
2331         if (regions.size() < numRegionsToReport) {
2332           regions.add(r.getRegionInfo());
2333         } else {
2334           break;
2335         }
2336       }
2337     }
2338     return regions.toArray(new HRegionInfo[regions.size()]);
2339   }
2340 
2341   /**
2342    * Called to verify that this server is up and running.
2343    *
2344    * @throws IOException
2345    */
2346   protected void checkOpen() throws IOException {
2347     if (this.stopRequested.get() || this.abortRequested) {
2348       throw new IOException("Server not running" +
2349         (this.abortRequested? ", aborting": ""));
2350     }
2351     if (!fsOk) {
2352       throw new IOException("File system not available");
2353     }
2354   }
2355 
2356   /**
2357    * @return Returns list of non-closed regions hosted on this server.  If no
2358    * regions to check, returns an empty list.
2359    */
2360   protected Set<HRegion> getRegionsToCheck() {
2361     HashSet<HRegion> regionsToCheck = new HashSet<HRegion>();
2362     //TODO: is this locking necessary?
2363     lock.readLock().lock();
2364     try {
2365       regionsToCheck.addAll(this.onlineRegions.values());
2366     } finally {
2367       lock.readLock().unlock();
2368     }
2369     // Purge closed regions.
2370     for (final Iterator<HRegion> i = regionsToCheck.iterator(); i.hasNext();) {
2371       HRegion r = i.next();
2372       if (r.isClosed()) {
2373         i.remove();
2374       }
2375     }
2376     return regionsToCheck;
2377   }
2378 
2379   public long getProtocolVersion(final String protocol,
2380       final long clientVersion)
2381   throws IOException {
2382     if (protocol.equals(HRegionInterface.class.getName())) {
2383       return HBaseRPCProtocolVersion.versionID;
2384     }
2385     throw new IOException("Unknown protocol to name node: " + protocol);
2386   }
2387 
2388   /**
2389    * @return Queue to which you can add outbound messages.
2390    */
2391   protected LinkedBlockingQueue<HMsg> getOutboundMsgs() {
2392     return this.outboundMsgs;
2393   }
2394 
2395   /**
2396    * Return the total size of all memstores in every region.
2397    * @return memstore size in bytes
2398    */
2399   public long getGlobalMemStoreSize() {
2400     long total = 0;
2401     synchronized (onlineRegions) {
2402       for (HRegion region : onlineRegions.values()) {
2403         total += region.memstoreSize.get();
2404       }
2405     }
2406     return total;
2407   }
2408 
2409   /**
2410    * @return Return the leases.
2411    */
2412   protected Leases getLeases() {
2413     return leases;
2414   }
2415 
2416   /**
2417    * @return Return the rootDir.
2418    */
2419   protected Path getRootDir() {
2420     return rootDir;
2421   }
2422 
2423   /**
2424    * @return Return the fs.
2425    */
2426   protected FileSystem getFileSystem() {
2427     return fs;
2428   }
2429 
2430   /**
2431    * @return Info on port this server has bound to, etc.
2432    */
2433   public HServerInfo getServerInfo() { return this.serverInfo; }
2434 
2435   /** {@inheritDoc} */
2436   public long incrementColumnValue(byte [] regionName, byte [] row,
2437       byte [] family, byte [] qualifier, long amount, boolean writeToWAL)
2438   throws IOException {
2439     checkOpen();
2440 
2441     if (regionName == null) {
2442       throw new IOException("Invalid arguments to incrementColumnValue " +
2443       "regionName is null");
2444     }
2445     requestCount.incrementAndGet();
2446     try {
2447       HRegion region = getRegion(regionName);
2448       long retval = region.incrementColumnValue(row, family, qualifier, amount,
2449           writeToWAL);
2450 
2451       return retval;
2452     } catch (IOException e) {
2453       checkFileSystem();
2454       throw e;
2455     }
2456   }
2457 
2458   /** {@inheritDoc} */
2459   public HRegionInfo[] getRegionsAssignment() throws IOException {
2460     HRegionInfo[] regions = new HRegionInfo[onlineRegions.size()];
2461     Iterator<HRegion> ite = onlineRegions.values().iterator();
2462     for(int i = 0; ite.hasNext(); i++) {
2463       regions[i] = ite.next().getRegionInfo();
2464     }
2465     return regions;
2466   }
2467 
2468   /** {@inheritDoc} */
2469   public HServerInfo getHServerInfo() throws IOException {
2470     return serverInfo;
2471   }
2472 
2473   @Override
2474   public MultiPutResponse multiPut(MultiPut puts) throws IOException {
2475     MultiPutResponse resp = new MultiPutResponse();
2476 
2477     // do each region as it's own.
2478     for( Map.Entry<byte[], List<Put>> e: puts.puts.entrySet()) {
2479       int result = put(e.getKey(), e.getValue());
2480       resp.addResult(e.getKey(), result);
2481 
2482       e.getValue().clear(); // clear some RAM
2483     }
2484 
2485     return resp;
2486   }
2487 
2488   public String toString() {
2489     return this.serverInfo.toString();
2490   }
2491 
2492   /**
2493    * Interval at which threads should run
2494    * @return the interval
2495    */
2496   public int getThreadWakeFrequency() {
2497     return threadWakeFrequency;
2498   }
2499 
2500   //
2501   // Main program and support routines
2502   //
2503 
2504   /**
2505    * @param hrs
2506    * @return Thread the RegionServer is running in correctly named.
2507    * @throws IOException
2508    */
2509   public static Thread startRegionServer(final HRegionServer hrs)
2510   throws IOException {
2511     return startRegionServer(hrs,
2512       "regionserver" + hrs.getServerInfo().getServerAddress().getPort());
2513   }
2514 
2515   /**
2516    * @param hrs
2517    * @param name
2518    * @return Thread the RegionServer is running in correctly named.
2519    * @throws IOException
2520    */
2521   public static Thread startRegionServer(final HRegionServer hrs,
2522       final String name)
2523   throws IOException {
2524     Thread t = new Thread(hrs);
2525     t.setName(name);
2526 
2527     t.start();
2528     // Install shutdown hook that will catch signals and run an orderly shutdown
2529     // of the hrs.
2530     ShutdownHook.install(hrs.getConfiguration(),
2531       FileSystem.get(hrs.getConfiguration()), hrs, t);
2532     return t;
2533   }
2534 
2535   /**
2536    * Utility for constructing an instance of the passed HRegionServer class.
2537    * @param regionServerClass
2538    * @param conf2
2539    * @return HRegionServer instance.
2540    */
2541   public static HRegionServer constructRegionServer(Class<? extends HRegionServer> regionServerClass,
2542       final Configuration conf2)  {
2543     try {
2544       final Constructor<? extends HRegionServer> c =
2545         regionServerClass.getConstructor(Configuration.class);
2546       UserGroupInformation ugi = loginFromKeytab(conf2);
2547       return ugi.doAs(new PrivilegedExceptionAction<HRegionServer>() {
2548           public HRegionServer run() throws Exception {
2549             return c.newInstance(conf2);
2550           }
2551         });
2552     } catch (Exception e) {
2553       throw new RuntimeException("Failed construction of " +
2554         "Master: " + regionServerClass.toString(), e);
2555     }
2556   }
2557 
2558   /**
2559    * TODO collapse with HMaster loginFromKeytab
2560    * TODO also shouldn't really be public - JVMClusterUtil is painful.
2561    * These will be resolved in trunk patch
2562    */
2563   public static UserGroupInformation loginFromKeytab(Configuration conf)
2564     throws IOException {
2565     String keytabFileKey = "hbase.regionserver.keytab.file";
2566     String userNameKey = "hbase.regionserver.kerberos.principal";
2567     
2568     String keytabFilename = conf.get(keytabFileKey);
2569     if (keytabFilename == null) {
2570       if (UserGroupInformation.isSecurityEnabled()) {
2571         throw new IOException("No keytab file '" + keytabFileKey + "' configured.");
2572       }
2573       return UserGroupInformation.getLoginUser();
2574     }
2575 
2576     String principalConfig = conf.get(userNameKey, System
2577         .getProperty("user.name"));
2578     String principalName = SecurityUtil.getServerPrincipal(principalConfig,
2579         InetAddress.getLocalHost().getCanonicalHostName());
2580 
2581     return UserGroupInformation.loginUserFromKeytabAndReturnUGI(
2582       principalName, keytabFilename);
2583   }
2584 
2585 
2586   @Override
2587   public void replicateLogEntries(HLog.Entry[] entries) throws IOException {
2588     this.replicationHandler.replicateLogEntries(entries);
2589   }
2590 
2591   /**
2592    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
2593    */
2594   public static void main(String[] args) throws Exception {
2595     Configuration conf = HBaseConfiguration.create();
2596     @SuppressWarnings("unchecked")
2597     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
2598         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
2599 
2600     new HRegionServerCommandLine(regionServerClass).doMain(args);
2601   }
2602 }