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