View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver;
21  
22  import java.io.IOException;
23  import java.io.StringWriter;
24  import java.lang.Thread.UncaughtExceptionHandler;
25  import java.lang.annotation.Retention;
26  import java.lang.annotation.RetentionPolicy;
27  import java.lang.management.ManagementFactory;
28  import java.lang.management.MemoryUsage;
29  import java.lang.reflect.Constructor;
30  import java.lang.reflect.Method;
31  import java.net.BindException;
32  import java.net.InetSocketAddress;
33  import java.util.ArrayList;
34  import java.util.Collection;
35  import java.util.Collections;
36  import java.util.Comparator;
37  import java.util.HashMap;
38  import java.util.HashSet;
39  import java.util.LinkedList;
40  import java.util.List;
41  import java.util.Map;
42  import java.util.Map.Entry;
43  import java.util.Random;
44  import java.util.Set;
45  import java.util.SortedMap;
46  import java.util.TreeMap;
47  import java.util.TreeSet;
48  import java.util.concurrent.ConcurrentHashMap;
49  import java.util.concurrent.ConcurrentSkipListMap;
50  import java.util.concurrent.atomic.AtomicBoolean;
51  import java.util.concurrent.atomic.AtomicInteger;
52  import java.util.concurrent.locks.ReentrantReadWriteLock;
53  
54  import javax.management.ObjectName;
55  
56  import org.apache.commons.lang.mutable.MutableDouble;
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.ClockOutOfSyncException;
64  import org.apache.hadoop.hbase.DoNotRetryIOException;
65  import org.apache.hadoop.hbase.HBaseConfiguration;
66  import org.apache.hadoop.hbase.HConstants;
67  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
68  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
69  import org.apache.hadoop.hbase.HRegionInfo;
70  import org.apache.hadoop.hbase.HServerAddress;
71  import org.apache.hadoop.hbase.HServerInfo;
72  import org.apache.hadoop.hbase.HServerLoad;
73  import org.apache.hadoop.hbase.HTableDescriptor;
74  import org.apache.hadoop.hbase.HealthCheckChore;
75  import org.apache.hadoop.hbase.KeyValue;
76  import org.apache.hadoop.hbase.MasterAddressTracker;
77  import org.apache.hadoop.hbase.NotServingRegionException;
78  import org.apache.hadoop.hbase.RemoteExceptionHandler;
79  import org.apache.hadoop.hbase.ServerName;
80  import org.apache.hadoop.hbase.Stoppable;
81  import org.apache.hadoop.hbase.TableDescriptors;
82  import org.apache.hadoop.hbase.UnknownRowLockException;
83  import org.apache.hadoop.hbase.UnknownScannerException;
84  import org.apache.hadoop.hbase.YouAreDeadException;
85  import org.apache.hadoop.hbase.catalog.CatalogTracker;
86  import org.apache.hadoop.hbase.catalog.MetaEditor;
87  import org.apache.hadoop.hbase.catalog.MetaReader;
88  import org.apache.hadoop.hbase.catalog.RootLocationEditor;
89  import org.apache.hadoop.hbase.client.Action;
90  import org.apache.hadoop.hbase.client.Append;
91  import org.apache.hadoop.hbase.client.Delete;
92  import org.apache.hadoop.hbase.client.Get;
93  import org.apache.hadoop.hbase.client.HConnectionManager;
94  import org.apache.hadoop.hbase.client.Increment;
95  import org.apache.hadoop.hbase.client.MultiAction;
96  import org.apache.hadoop.hbase.client.MultiResponse;
97  import org.apache.hadoop.hbase.client.Mutation;
98  import org.apache.hadoop.hbase.client.Put;
99  import org.apache.hadoop.hbase.client.Result;
100 import org.apache.hadoop.hbase.client.Row;
101 import org.apache.hadoop.hbase.client.RowLock;
102 import org.apache.hadoop.hbase.client.RowMutations;
103 import org.apache.hadoop.hbase.client.Scan;
104 import org.apache.hadoop.hbase.client.UserProvider;
105 import org.apache.hadoop.hbase.client.coprocessor.Exec;
106 import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
107 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
108 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
109 import org.apache.hadoop.hbase.executor.ExecutorService;
110 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
111 import org.apache.hadoop.hbase.filter.BinaryComparator;
112 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
113 import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
114 import org.apache.hadoop.hbase.fs.HFileSystem;
115 import org.apache.hadoop.hbase.io.hfile.BlockCache;
116 import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
117 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
118 import org.apache.hadoop.hbase.io.hfile.CacheStats;
119 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
120 import org.apache.hadoop.hbase.ipc.HBaseRPC;
121 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
122 import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
123 import org.apache.hadoop.hbase.ipc.HBaseServer;
124 import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
125 import org.apache.hadoop.hbase.ipc.HRegionInterface;
126 import org.apache.hadoop.hbase.ipc.Invocation;
127 import org.apache.hadoop.hbase.ipc.ProtocolSignature;
128 import org.apache.hadoop.hbase.ipc.RpcEngine;
129 import org.apache.hadoop.hbase.ipc.RpcServer;
130 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
131 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
132 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
133 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
134 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
135 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
136 import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler;
137 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
138 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
139 import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
140 import org.apache.hadoop.hbase.regionserver.metrics.RegionMetricsStorage;
141 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerDynamicMetrics;
142 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
143 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
144 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.StoreMetricType;
145 import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
146 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
147 import org.apache.hadoop.hbase.regionserver.wal.HLog;
148 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
149 import org.apache.hadoop.hbase.util.Bytes;
150 import org.apache.hadoop.hbase.util.CompressionTest;
151 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
152 import org.apache.hadoop.hbase.util.FSTableDescriptors;
153 import org.apache.hadoop.hbase.util.FSUtils;
154 import org.apache.hadoop.hbase.util.InfoServer;
155 import org.apache.hadoop.hbase.util.Pair;
156 import org.apache.hadoop.hbase.util.Sleeper;
157 import org.apache.hadoop.hbase.util.Strings;
158 import org.apache.hadoop.hbase.util.Threads;
159 import org.apache.hadoop.hbase.util.VersionInfo;
160 import org.apache.hadoop.hbase.zookeeper.ClusterId;
161 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
162 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
163 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
164 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
165 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
166 import org.apache.hadoop.io.MapWritable;
167 import org.apache.hadoop.io.Writable;
168 import org.apache.hadoop.ipc.RemoteException;
169 import org.apache.hadoop.metrics.util.MBeanUtil;
170 import org.apache.hadoop.net.DNS;
171 import org.apache.hadoop.util.ReflectionUtils;
172 import org.apache.hadoop.util.StringUtils;
173 import org.apache.zookeeper.KeeperException;
174 import org.codehaus.jackson.map.ObjectMapper;
175 
176 import com.google.common.base.Function;
177 import com.google.common.collect.Lists;
178 
179 /**
180  * HRegionServer makes a set of HRegions available to clients. It checks in with
181  * the HMaster. There are many HRegionServers in a single HBase deployment.
182  */
183 public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
184     Runnable, RegionServerServices {
185 
186   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
187 
188   // Set when a report to the master comes back with a message asking us to
189   // shutdown. Also set by call to stop when debugging or running unit tests
190   // of HRegionServer in isolation.
191   protected volatile boolean stopped = false;
192 
193   // A state before we go into stopped state.  At this stage we're closing user
194   // space regions.
195   private boolean stopping = false;
196 
197   // Go down hard. Used if file system becomes unavailable and also in
198   // debugging and unit tests.
199   protected volatile boolean abortRequested;
200 
201   private volatile boolean killed = false;
202 
203   // If false, the file system has become unavailable
204   protected volatile boolean fsOk;
205 
206   protected final Configuration conf;
207 
208   protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
209   private HFileSystem fs;
210   private boolean useHBaseChecksum; // verify hbase checksums?
211   private Path rootDir;
212   private final Random rand;
213 
214   //RegionName vs current action in progress
215   //true - if open region action in progress
216   //false - if close region action in progress
217   private final ConcurrentSkipListMap<byte[], Boolean> regionsInTransitionInRS =
218       new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
219 
220   /**
221    * Map of regions currently being served by this region server. Key is the
222    * encoded region name.  All access should be synchronized.
223    */
224   protected final Map<String, HRegion> onlineRegions =
225     new ConcurrentHashMap<String, HRegion>();
226 
227   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
228 
229   final int numRetries;
230   protected final int threadWakeFrequency;
231   private final int msgInterval;
232 
233   protected final int numRegionsToReport;
234 
235   private final long maxScannerResultSize;
236 
237   // Remote HMaster
238   private HMasterRegionInterface hbaseMaster;
239 
240   // RPC Engine for master connection
241   private RpcEngine rpcEngine;
242 
243   // Server to handle client requests. Default access so can be accessed by
244   // unit tests.
245   RpcServer rpcServer;
246 
247   // Server to handle client requests.
248   private HBaseServer server;  
249 
250   private final InetSocketAddress isa;
251   private UncaughtExceptionHandler uncaughtExceptionHandler;
252 
253   // Leases
254   private Leases leases;
255 
256   // Request counter.
257   // Do we need this?  Can't we just sum region counters?  St.Ack 20110412
258   private AtomicInteger requestCount = new AtomicInteger();
259 
260   // Info server. Default access so can be used by unit tests. REGIONSERVER
261   // is name of the webapp and the attribute name used stuffing this instance
262   // into web context.
263   InfoServer infoServer;
264 
265   /** region server process name */
266   public static final String REGIONSERVER = "regionserver";
267   
268   /** region server configuration name */
269   public static final String REGIONSERVER_CONF = "regionserver_conf";
270 
271   /*
272    * Space is reserved in HRS constructor and then released when aborting to
273    * recover from an OOME. See HBASE-706. TODO: Make this percentage of the heap
274    * or a minimum.
275    */
276   private final LinkedList<byte[]> reservedSpace = new LinkedList<byte[]>();
277 
278   private RegionServerMetrics metrics;
279 
280   private RegionServerDynamicMetrics dynamicMetrics;
281 
282   // Compactions
283   public CompactSplitThread compactSplitThread;
284 
285   // Cache flushing
286   MemStoreFlusher cacheFlusher;
287 
288   /*
289    * Check for compactions requests.
290    */
291   Chore compactionChecker;
292 
293   /*
294    * Check for flushes
295    */
296   Chore periodicFlusher;
297 
298   // HLog and HLog roller. log is protected rather than private to avoid
299   // eclipse warning when accessed by inner classes
300   protected volatile HLog hlog;
301   // The meta updates are written to a different hlog. If this
302   // regionserver holds meta regions, then this field will be non-null.
303   protected volatile HLog hlogForMeta;
304 
305   LogRoller hlogRoller;
306   LogRoller metaHLogRoller;
307 
308   private final boolean separateHLogForMeta;
309 
310   // flag set after we're done setting up server threads (used for testing)
311   protected volatile boolean isOnline;
312 
313   final Map<String, RegionScanner> scanners =
314     new ConcurrentHashMap<String, RegionScanner>();
315 
316   // zookeeper connection and watcher
317   private ZooKeeperWatcher zooKeeper;
318 
319   // master address manager and watcher
320   private MasterAddressTracker masterAddressManager;
321 
322   // catalog tracker
323   private CatalogTracker catalogTracker;
324 
325   // Cluster Status Tracker
326   private ClusterStatusTracker clusterStatusTracker;
327 
328   // Log Splitting Worker
329   private SplitLogWorker splitLogWorker;
330 
331   // A sleeper that sleeps for msgInterval.
332   private final Sleeper sleeper;
333 
334   private final int rpcTimeout;
335 
336   // Instance of the hbase executor service.
337   private ExecutorService service;
338 
339   // Replication services. If no replication, this handler will be null.
340   private ReplicationSourceService replicationSourceHandler;
341   private ReplicationSinkService replicationSinkHandler;
342 
343   private final RegionServerAccounting regionServerAccounting;
344 
345   // Cache configuration and block cache reference
346   private final CacheConfig cacheConfig;
347 
348   // reference to the Thrift Server.
349   volatile private HRegionThriftServer thriftServer;
350 
351   /**
352    * The server name the Master sees us as.  Its made from the hostname the
353    * master passes us, port, and server startcode. Gets set after registration
354    * against  Master.  The hostname can differ from the hostname in {@link #isa}
355    * but usually doesn't if both servers resolve .
356    */
357   private ServerName serverNameFromMasterPOV;
358 
359   // Port we put up the webui on.
360   private int webuiport = -1;
361 
362   /**
363    * This servers startcode.
364    */
365   private final long startcode;
366 
367   /**
368    * Go here to get table descriptors.
369    */
370   private TableDescriptors tableDescriptors;
371 
372   /*
373    * Strings to be used in forming the exception message for
374    * RegionsAlreadyInTransitionException.
375    */
376   private static final String OPEN = "OPEN";
377   private static final String CLOSE = "CLOSE";
378 
379   /**
380    * MX Bean for RegionServerInfo
381    */
382   private ObjectName mxBean = null;
383 
384   /**
385    * ClusterId
386    */
387   private ClusterId clusterId = null;
388 
389   private RegionServerCoprocessorHost rsHost;
390 
391   /** The health check chore. */
392   private HealthCheckChore healthCheckChore;
393 
394   /**
395    * Starts a HRegionServer at the default location
396    *
397    * @param conf
398    * @throws IOException
399    * @throws InterruptedException
400    */
401   public HRegionServer(Configuration conf)
402   throws IOException, InterruptedException {
403     this.fsOk = true;
404     this.conf = conf;
405     // Set how many times to retry talking to another server over HConnection.
406     HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
407     this.isOnline = false;
408     checkCodecs(this.conf);
409 
410     // do we use checksum verfication in the hbase? If hbase checksum verification
411     // is enabled, then we automatically switch off hdfs checksum verification.
412     this.useHBaseChecksum = conf.getBoolean(
413       HConstants.HBASE_CHECKSUM_VERIFICATION, false);
414 
415     // Config'ed params
416     this.separateHLogForMeta = conf.getBoolean(HLog.SEPARATE_HLOG_FOR_META, false);
417     this.numRetries = conf.getInt("hbase.client.retries.number", 10);
418     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
419       10 * 1000);
420     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
421 
422     this.sleeper = new Sleeper(this.msgInterval, this);
423 
424     this.maxScannerResultSize = conf.getLong(
425       HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
426       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
427 
428     this.numRegionsToReport = conf.getInt(
429       "hbase.regionserver.numregionstoreport", 10);
430 
431     this.rpcTimeout = conf.getInt(
432       HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
433       HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
434 
435     this.abortRequested = false;
436     this.stopped = false;
437 
438     // Server to handle client requests.
439     String hostname = conf.get("hbase.regionserver.ipc.address",
440       Strings.domainNamePointerToHostName(DNS.getDefaultHost(
441         conf.get("hbase.regionserver.dns.interface", "default"),
442         conf.get("hbase.regionserver.dns.nameserver", "default"))));
443     int port = conf.getInt(HConstants.REGIONSERVER_PORT,
444       HConstants.DEFAULT_REGIONSERVER_PORT);
445     // Creation of a HSA will force a resolve.
446     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
447     if (initialIsa.getAddress() == null) {
448       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
449     }
450 
451     this.rand = new Random(initialIsa.hashCode());
452     this.rpcServer = HBaseRPC.getServer(this,
453       new Class<?>[]{HRegionInterface.class, HBaseRPCErrorHandler.class,
454         OnlineRegions.class},
455         initialIsa.getHostName(), // BindAddress is IP we got for this server.
456         initialIsa.getPort(),
457         conf.getInt("hbase.regionserver.handler.count", 10),
458         conf.getInt("hbase.regionserver.metahandler.count", 10),
459         conf.getBoolean("hbase.rpc.verbose", false),
460         conf, HConstants.QOS_THRESHOLD);
461     if (rpcServer instanceof HBaseServer) server = (HBaseServer) rpcServer;
462     // Set our address.
463     this.isa = this.rpcServer.getListenerAddress();
464 
465     this.rpcServer.setErrorHandler(this);
466     this.rpcServer.setQosFunction(new QosFunction());
467     this.startcode = System.currentTimeMillis();
468 
469     conf.set("hbase.regionserver.rpc.client.socket.bind.address", this.isa.getHostName());
470 
471     // login the zookeeper client principal (if using security)
472     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
473       "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
474 
475     // login the server principal (if using secure Hadoop)
476     UserProvider provider = UserProvider.instantiate(conf);
477     provider.login("hbase.regionserver.keytab.file",
478       "hbase.regionserver.kerberos.principal", this.isa.getHostName());
479     regionServerAccounting = new RegionServerAccounting();
480     cacheConfig = new CacheConfig(conf);
481     uncaughtExceptionHandler = new UncaughtExceptionHandler() {
482       public void uncaughtException(Thread t, Throwable e) {
483         abort("Uncaught exception in service thread " + t.getName(), e);
484       }
485     };
486   }
487 
488   /** Handle all the snapshot requests to this server */
489   RegionServerSnapshotManager snapshotManager;
490 
491   /**
492    * Run test on configured codecs to make sure supporting libs are in place.
493    * @param c
494    * @throws IOException
495    */
496   private static void checkCodecs(final Configuration c) throws IOException {
497     // check to see if the codec list is available:
498     String [] codecs = c.getStrings("hbase.regionserver.codecs", (String[])null);
499     if (codecs == null) return;
500     for (String codec : codecs) {
501       if (!CompressionTest.testCompression(codec)) {
502         throw new IOException("Compression codec " + codec +
503           " not supported, aborting RS construction");
504       }
505     }
506   }
507 
508 
509   @Retention(RetentionPolicy.RUNTIME)
510   private @interface QosPriority {
511     int priority() default 0;
512   }
513 
514   /**
515    * Utility used ensuring higher quality of service for priority rpcs; e.g.
516    * rpcs to .META. and -ROOT-, etc.
517    */
518   class QosFunction implements Function<Writable,Integer> {
519     private final Map<String, Integer> annotatedQos;
520 
521     public QosFunction() {
522       Map<String, Integer> qosMap = new HashMap<String, Integer>();
523       for (Method m : HRegionServer.class.getMethods()) {
524         QosPriority p = m.getAnnotation(QosPriority.class);
525         if (p != null) {
526           qosMap.put(m.getName(), p.priority());
527         }
528       }
529 
530       annotatedQos = qosMap;
531     }
532 
533     public boolean isMetaTable(byte[] regionName) {
534       HRegion region;
535       try {
536         region = getRegion(regionName);
537       } catch (NotServingRegionException ignored) {
538         return false;
539       }
540       return region.getRegionInfo().isMetaTable();
541     }
542 
543     @Override
544     public Integer apply(Writable from) {
545       if (!(from instanceof Invocation)) return HConstants.NORMAL_QOS;
546 
547       Invocation inv = (Invocation) from;
548       String methodName = inv.getMethodName();
549 
550       Integer priorityByAnnotation = annotatedQos.get(methodName);
551       if (priorityByAnnotation != null) {
552         return priorityByAnnotation;
553       }
554 
555       // scanner methods...
556       if (methodName.equals("next") || methodName.equals("close")) {
557         // translate!
558         Long scannerId;
559         try {
560           scannerId = (Long) inv.getParameters()[0];
561         } catch (ClassCastException ignored) {
562           // LOG.debug("Low priority: " + from);
563           return HConstants.NORMAL_QOS;
564         }
565         String scannerIdString = Long.toString(scannerId);
566         RegionScanner scanner = scanners.get(scannerIdString);
567         if (scanner != null && scanner.getRegionInfo().isMetaTable()) {
568           // LOG.debug("High priority scanner request: " + scannerId);
569           return HConstants.HIGH_QOS;
570         }
571       } else if (inv.getParameterClasses().length == 0) {
572        // Just let it through.  This is getOnlineRegions, etc.
573       } else if (inv.getParameterClasses()[0] == byte[].class) {
574         // first arg is byte array, so assume this is a regionname:
575         if (isMetaTable((byte[]) inv.getParameters()[0])) {
576           // LOG.debug("High priority with method: " + methodName +
577           // " and region: "
578           // + Bytes.toString((byte[]) inv.getParameters()[0]));
579           return HConstants.HIGH_QOS;
580         }
581       } else if (inv.getParameterClasses()[0] == MultiAction.class) {
582         MultiAction<?> ma = (MultiAction<?>) inv.getParameters()[0];
583         Set<byte[]> regions = ma.getRegions();
584         // ok this sucks, but if any single of the actions touches a meta, the
585         // whole
586         // thing gets pingged high priority. This is a dangerous hack because
587         // people
588         // can get their multi action tagged high QOS by tossing a Get(.META.)
589         // AND this
590         // regionserver hosts META/-ROOT-
591         for (byte[] region : regions) {
592           if (isMetaTable(region)) {
593             // LOG.debug("High priority multi with region: " +
594             // Bytes.toString(region));
595             return HConstants.HIGH_QOS; // short circuit for the win.
596           }
597         }
598       }
599       // LOG.debug("Low priority: " + from.toString());
600       return HConstants.NORMAL_QOS;
601     }
602   }
603 
604   /**
605    * All initialization needed before we go register with Master.
606    *
607    * @throws IOException
608    * @throws InterruptedException
609    */
610   private void preRegistrationInitialization(){
611     try {
612       initializeZooKeeper();
613 
614       clusterId = new ClusterId(zooKeeper, this);
615       if(clusterId.hasId()) {
616         conf.set(HConstants.CLUSTER_ID, clusterId.getId());
617       }
618 
619       initializeThreads();
620       int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4);
621       for (int i = 0; i < nbBlocks; i++) {
622         reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]);
623       }
624 
625       this.rpcEngine = HBaseRPC.getProtocolEngine(conf);
626     } catch (Throwable t) {
627       // Call stop if error or process will stick around for ever since server
628       // puts up non-daemon threads.
629       this.rpcServer.stop();
630       abort("Initialization of RS failed.  Hence aborting RS.", t);
631     }
632   }
633 
634   /**
635    * Bring up connection to zk ensemble and then wait until a master for this
636    * cluster and then after that, wait until cluster 'up' flag has been set.
637    * This is the order in which master does things.
638    * Finally put up a catalog tracker.
639    * @throws IOException
640    * @throws InterruptedException
641    */
642   private void initializeZooKeeper() throws IOException, InterruptedException {
643     // Open connection to zookeeper and set primary watcher
644     this.zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER + ":" +
645       this.isa.getPort(), this);
646 
647     // Create the master address manager, register with zk, and start it.  Then
648     // block until a master is available.  No point in starting up if no master
649     // running.
650     this.masterAddressManager = new MasterAddressTracker(this.zooKeeper, this);
651     this.masterAddressManager.start();
652     blockAndCheckIfStopped(this.masterAddressManager);
653 
654     // Wait on cluster being up.  Master will set this flag up in zookeeper
655     // when ready.
656     this.clusterStatusTracker = new ClusterStatusTracker(this.zooKeeper, this);
657     this.clusterStatusTracker.start();
658     blockAndCheckIfStopped(this.clusterStatusTracker);
659 
660     // Create the catalog tracker and start it;
661     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf, this);
662     catalogTracker.start();
663 
664     // watch for snapshots
665     try {
666       this.snapshotManager = new RegionServerSnapshotManager(this);
667     } catch (KeeperException e) {
668       this.abort("Failed to reach zk cluster when creating snapshot handler.");
669     }
670   }
671 
672   /**
673    * Utilty method to wait indefinitely on a znode availability while checking
674    * if the region server is shut down
675    * @param tracker znode tracker to use
676    * @throws IOException any IO exception, plus if the RS is stopped
677    * @throws InterruptedException
678    */
679   private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
680       throws IOException, InterruptedException {
681     while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
682       if (this.stopped) {
683         throw new IOException("Received the shutdown message while waiting.");
684       }
685     }
686   }
687 
688   /**
689    * @return False if cluster shutdown in progress
690    */
691   private boolean isClusterUp() {
692     return this.clusterStatusTracker.isClusterUp();
693   }
694 
695   private void initializeThreads() throws IOException {
696     // Cache flushing thread.
697     this.cacheFlusher = new MemStoreFlusher(conf, this);
698 
699     // Compaction thread
700     this.compactSplitThread = new CompactSplitThread(this);
701 
702     // Background thread to check for compactions; needed if region
703     // has not gotten updates in a while. Make it run at a lesser frequency.
704     int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY +
705       ".multiplier", 1000);
706     this.compactionChecker = new CompactionChecker(this,
707       this.threadWakeFrequency * multiplier, this);
708 
709     this.periodicFlusher = new PeriodicMemstoreFlusher(this.threadWakeFrequency, this);
710 
711     // Health checker thread.
712     int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
713       HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
714     if (isHealthCheckerConfigured()) {
715       healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
716     }
717 
718     this.leases = new Leases((int) conf.getLong(
719         HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
720         HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD),
721         this.threadWakeFrequency);
722 
723     // Create the thread for the ThriftServer.
724     if (conf.getBoolean("hbase.regionserver.export.thrift", false)) {
725       thriftServer = new HRegionThriftServer(this, conf);
726       thriftServer.start();
727       LOG.info("Started Thrift API from Region Server.");
728     }
729   }
730 
731   /**
732    * The HRegionServer sticks in this loop until closed.
733    */
734   @SuppressWarnings("deprecation")
735   public void run() {
736     try {
737       // Do pre-registration initializations; zookeeper, lease threads, etc.
738       preRegistrationInitialization();
739     } catch (Throwable e) {
740       abort("Fatal exception during initialization", e);
741     }
742 
743     try {
744       // Try and register with the Master; tell it we are here.  Break if
745       // server is stopped or the clusterup flag is down or hdfs went wacky.
746       while (keepLooping()) {
747         MapWritable w = reportForDuty();
748         if (w == null) {
749           LOG.warn("reportForDuty failed; sleeping and then retrying.");
750           this.sleeper.sleep();
751         } else {
752           handleReportForDutyResponse(w);
753           break;
754         }
755       }
756       registerMBean();
757 
758       // start the snapshot handler, since the server is ready to run
759       this.snapshotManager.start();
760 
761       // We registered with the Master.  Go into run mode.
762       long lastMsg = 0;
763       long oldRequestCount = -1;
764       // The main run loop.
765       while (!this.stopped && isHealthy()) {
766         if (!isClusterUp()) {
767           if (isOnlineRegionsEmpty()) {
768             stop("Exiting; cluster shutdown set and not carrying any regions");
769           } else if (!this.stopping) {
770             this.stopping = true;
771             LOG.info("Closing user regions");
772             closeUserRegions(this.abortRequested);
773           } else if (this.stopping) {
774             boolean allUserRegionsOffline = areAllUserRegionsOffline();
775             if (allUserRegionsOffline) {
776               // Set stopped if no requests since last time we went around the loop.
777               // The remaining meta regions will be closed on our way out.
778               if (oldRequestCount == this.requestCount.get()) {
779                 stop("Stopped; only catalog regions remaining online");
780                 break;
781               }
782               oldRequestCount = this.requestCount.get();
783             } else {
784               // Make sure all regions have been closed -- some regions may
785               // have not got it because we were splitting at the time of
786               // the call to closeUserRegions.
787               closeUserRegions(this.abortRequested);
788             }
789             LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
790           }
791         }
792         long now = System.currentTimeMillis();
793         if ((now - lastMsg) >= msgInterval) {
794           doMetrics();
795           tryRegionServerReport();
796           lastMsg = System.currentTimeMillis();
797         }
798         if (!this.stopped) this.sleeper.sleep();
799       } // for
800     } catch (Throwable t) {
801       if (!checkOOME(t)) {
802         abort("Unhandled exception: " + t.getMessage(), t);
803       }
804     }
805     // Run shutdown.
806     if (mxBean != null) {
807       MBeanUtil.unregisterMBean(mxBean);
808       mxBean = null;
809     }
810     if (this.thriftServer != null) this.thriftServer.shutdown();
811     this.leases.closeAfterLeasesExpire();
812     this.rpcServer.stop();
813     if (this.splitLogWorker != null) {
814       splitLogWorker.stop();
815     }
816     if (this.infoServer != null) {
817       LOG.info("Stopping infoServer");
818       try {
819         this.infoServer.stop();
820       } catch (Exception e) {
821         e.printStackTrace();
822       }
823     }
824     // Send cache a shutdown.
825     if (cacheConfig.isBlockCacheEnabled()) {
826       cacheConfig.getBlockCache().shutdown();
827     }
828 
829     // Send interrupts to wake up threads if sleeping so they notice shutdown.
830     // TODO: Should we check they are alive? If OOME could have exited already
831     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
832     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
833     if (this.hlogRoller != null) this.hlogRoller.interruptIfNecessary();
834     if (this.metaHLogRoller != null) this.metaHLogRoller.interruptIfNecessary();
835     if (this.compactionChecker != null)
836       this.compactionChecker.interrupt();
837     if (this.healthCheckChore != null) {
838       this.healthCheckChore.interrupt();
839     }
840 
841     // Stop the snapshot handler, forcefully killing all running tasks
842     try {
843       if (snapshotManager != null) snapshotManager.stop(this.abortRequested || this.killed);
844     } catch (IOException e) {
845       LOG.warn("Failed to close snapshot handler cleanly", e);
846     }
847 
848     if (this.killed) {
849       // Just skip out w/o closing regions.  Used when testing.
850     } else if (abortRequested) {
851       if (this.fsOk) {
852         closeUserRegions(abortRequested); // Don't leave any open file handles
853       }
854       LOG.info("aborting server " + this.serverNameFromMasterPOV);
855     } else {
856       closeUserRegions(abortRequested);
857       closeAllScanners();
858       LOG.info("stopping server " + this.serverNameFromMasterPOV);
859     }
860     // Interrupt catalog tracker here in case any regions being opened out in
861     // handlers are stuck waiting on meta or root.
862     if (this.catalogTracker != null) this.catalogTracker.stop();
863 
864     // Closing the compactSplit thread before closing meta regions
865     if (!this.killed && containsMetaTableRegions()) {
866       if (!abortRequested || this.fsOk) {
867         if (this.compactSplitThread != null) {
868           this.compactSplitThread.join();
869           this.compactSplitThread = null;
870         }
871         closeMetaTableRegions(abortRequested);
872       }
873     }
874 
875     if (!this.killed && this.fsOk) {
876       waitOnAllRegionsToClose(abortRequested);
877       LOG.info("stopping server " + this.serverNameFromMasterPOV +
878         "; all regions closed.");
879     }
880 
881     //fsOk flag may be changed when closing regions throws exception.
882     if (!this.killed && this.fsOk) {
883       closeWAL(abortRequested ? false : true);
884     }
885 
886     // Make sure the proxy is down.
887     this.hbaseMaster = null;
888     this.rpcEngine.close();
889     this.leases.close();
890 
891     if (!killed) {
892       join();
893     }
894 
895     try {
896       deleteMyEphemeralNode();
897     } catch (KeeperException e) {
898       LOG.warn("Failed deleting my ephemeral node", e);
899     }
900     this.zooKeeper.close();
901     LOG.info("stopping server " + this.serverNameFromMasterPOV +
902       "; zookeeper connection closed.");
903 
904     LOG.info(Thread.currentThread().getName() + " exiting");
905   }
906 
907   private boolean containsMetaTableRegions() {
908     return onlineRegions.containsKey(HRegionInfo.ROOT_REGIONINFO.getEncodedName())
909         || onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
910   }
911 
912   private boolean areAllUserRegionsOffline() {
913     if (getNumberOfOnlineRegions() > 2) return false;
914     boolean allUserRegionsOffline = true;
915     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
916       if (!e.getValue().getRegionInfo().isMetaTable()) {
917         allUserRegionsOffline = false;
918         break;
919       }
920     }
921     return allUserRegionsOffline;
922   }
923 
924   void tryRegionServerReport()
925   throws IOException {
926     if (!keepLooping() && hbaseMaster == null) {
927       // the current server is stopping
928       return;
929     }
930     HServerLoad hsl = buildServerLoad();
931     // Why we do this?
932     this.requestCount.set(0);
933     try {
934       this.hbaseMaster.regionServerReport(this.serverNameFromMasterPOV.getVersionedBytes(), hsl);
935     } catch (IOException ioe) {
936       if (ioe instanceof RemoteException) {
937         ioe = ((RemoteException)ioe).unwrapRemoteException();
938       }
939       if (ioe instanceof YouAreDeadException) {
940         // This will be caught and handled as a fatal error in run()
941         throw ioe;
942       }
943       // Couldn't connect to the master, get location from zk and reconnect
944       // Method blocks until new master is found or we are stopped
945       getMaster();
946     }
947   }
948 
949   HServerLoad buildServerLoad() {
950     Collection<HRegion> regions = getOnlineRegionsLocalContext();
951     TreeMap<byte [], HServerLoad.RegionLoad> regionLoads =
952       new TreeMap<byte [], HServerLoad.RegionLoad>(Bytes.BYTES_COMPARATOR);
953     for (HRegion region: regions) {
954       regionLoads.put(region.getRegionName(), createRegionLoad(region));
955     }
956     MemoryUsage memory =
957       ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
958     return new HServerLoad(requestCount.get(),(int)metrics.getRequests(),
959       (int)(memory.getUsed() / 1024 / 1024),
960       (int) (memory.getMax() / 1024 / 1024), regionLoads,
961       this.hlog.getCoprocessorHost().getCoprocessors());
962   }
963 
964   String getOnlineRegionsAsPrintableString() {
965     StringBuilder sb = new StringBuilder();
966     for (HRegion r: this.onlineRegions.values()) {
967       if (sb.length() > 0) sb.append(", ");
968       sb.append(r.getRegionInfo().getEncodedName());
969     }
970     return sb.toString();
971   }
972 
973   /**
974    * Wait on regions close.
975    */
976   private void waitOnAllRegionsToClose(final boolean abort) {
977     // Wait till all regions are closed before going out.
978     int lastCount = -1;
979     long previousLogTime = 0;
980     Set<String> closedRegions = new HashSet<String>();
981     while (!isOnlineRegionsEmpty()) {
982       int count = getNumberOfOnlineRegions();
983       // Only print a message if the count of regions has changed.
984       if (count != lastCount) {
985         // Log every second at most
986         if (System.currentTimeMillis() > (previousLogTime + 1000)) {
987           previousLogTime = System.currentTimeMillis();
988           lastCount = count;
989           LOG.info("Waiting on " + count + " regions to close");
990           // Only print out regions still closing if a small number else will
991           // swamp the log.
992           if (count < 10 && LOG.isDebugEnabled()) {
993             LOG.debug(this.onlineRegions);
994           }
995         }
996       }
997       // Ensure all user regions have been sent a close. Use this to
998       // protect against the case where an open comes in after we start the
999       // iterator of onlineRegions to close all user regions.
1000       for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1001         HRegionInfo hri = e.getValue().getRegionInfo();
1002         if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
1003             && !closedRegions.contains(hri.getEncodedName())) {
1004           closedRegions.add(hri.getEncodedName());
1005           // Don't update zk with this close transition; pass false.
1006           closeRegion(hri, abort, false);
1007         }
1008       }
1009       // No regions in RIT, we could stop waiting now.
1010       if (this.regionsInTransitionInRS.isEmpty()) {
1011         if (!isOnlineRegionsEmpty()) {
1012           LOG.info("We were exiting though online regions are not empty, because some regions failed closing");
1013         }
1014         break;
1015       }
1016       Threads.sleep(200);
1017     }
1018   }
1019 
1020   private void closeWAL(final boolean delete) {
1021     if (this.hlogForMeta != null) {
1022       // All hlogs (meta and non-meta) are in the same directory. Don't call
1023       // closeAndDelete here since that would delete all hlogs not just the
1024       // meta ones. We will just 'close' the hlog for meta here, and leave
1025       // the directory cleanup to the follow-on closeAndDelete call.
1026       try { //Part of the patch from HBASE-7982 to do with exception handling 
1027         this.hlogForMeta.close();
1028       } catch (Throwable e) {
1029         LOG.error("Metalog close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1030       }
1031     }
1032     if (this.hlog != null) {
1033       try {
1034         if (delete) {
1035           hlog.closeAndDelete();
1036         } else {
1037           hlog.close();
1038         }
1039       } catch (Throwable e) {
1040         LOG.error("Close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1041       }
1042     }
1043   }
1044 
1045   private void closeAllScanners() {
1046     // Close any outstanding scanners. Means they'll get an UnknownScanner
1047     // exception next time they come in.
1048     for (Map.Entry<String, RegionScanner> e : this.scanners.entrySet()) {
1049       try {
1050         e.getValue().close();
1051       } catch (IOException ioe) {
1052         LOG.warn("Closing scanner " + e.getKey(), ioe);
1053       }
1054     }
1055   }
1056 
1057   /*
1058    * Run init. Sets up hlog and starts up all server threads.
1059    *
1060    * @param c Extra configuration.
1061    */
1062   protected void handleReportForDutyResponse(final MapWritable c)
1063   throws IOException {
1064     try {
1065       for (Map.Entry<Writable, Writable> e :c.entrySet()) {
1066         String key = e.getKey().toString();
1067         // The hostname the master sees us as.
1068         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1069           String hostnameFromMasterPOV = e.getValue().toString();
1070           this.serverNameFromMasterPOV = new ServerName(hostnameFromMasterPOV,
1071             this.isa.getPort(), this.startcode);
1072           LOG.info("Master passed us hostname to use. Was=" +
1073             this.isa.getHostName() + ", Now=" +
1074             this.serverNameFromMasterPOV.getHostname());
1075           continue;
1076         }
1077         String value = e.getValue().toString();
1078         if (LOG.isDebugEnabled()) {
1079           LOG.debug("Config from master: " + key + "=" + value);
1080         }
1081         this.conf.set(key, value);
1082       }
1083 
1084       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
1085       // config param for task trackers, but we can piggyback off of it.
1086       if (this.conf.get("mapred.task.id") == null) {
1087         this.conf.set("mapred.task.id", "hb_rs_" +
1088           this.serverNameFromMasterPOV.toString());
1089       }
1090       // Set our ephemeral znode up in zookeeper now we have a name.
1091       createMyEphemeralNode();
1092 
1093       // Master sent us hbase.rootdir to use. Should be fully qualified
1094       // path with file system specification included. Set 'fs.defaultFS'
1095       // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
1096       // accessors will be going against wrong filesystem (unless all is set
1097       // to defaults).
1098       this.conf.set("fs.defaultFS", this.conf.get("hbase.rootdir"));
1099       // Get fs instance used by this RS
1100       this.fs = new HFileSystem(this.conf, this.useHBaseChecksum);
1101       this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
1102       this.tableDescriptors = new FSTableDescriptors(this.fs, this.rootDir, true);
1103       this.hlog = setupWALAndReplication();
1104       // Init in here rather than in constructor after thread name has been set
1105       this.metrics = new RegionServerMetrics();
1106       this.dynamicMetrics = RegionServerDynamicMetrics.newInstance(this);
1107       this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
1108       startServiceThreads();
1109       LOG.info("Serving as " + this.serverNameFromMasterPOV +
1110         ", RPC listening on " + this.isa +
1111         ", sessionid=0x" +
1112         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1113       isOnline = true;
1114     } catch (Throwable e) {
1115       LOG.warn("Exception in region server : ", e);
1116       this.isOnline = false;
1117       stop("Failed initialization");
1118       throw convertThrowableToIOE(cleanup(e, "Failed init"),
1119           "Region server startup failed");
1120     } finally {
1121       sleeper.skipSleepCycle();
1122     }
1123   }
1124 
1125   private String getMyEphemeralNodePath() {
1126     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
1127   }
1128 
1129   private void createMyEphemeralNode() throws KeeperException {
1130     ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper, getMyEphemeralNodePath(),
1131       HConstants.EMPTY_BYTE_ARRAY);
1132   }
1133 
1134   private void deleteMyEphemeralNode() throws KeeperException {
1135     ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1136   }
1137 
1138   public RegionServerAccounting getRegionServerAccounting() {
1139     return regionServerAccounting;
1140   }
1141 
1142   /*
1143    * @param r Region to get RegionLoad for.
1144    *
1145    * @return RegionLoad instance.
1146    *
1147    * @throws IOException
1148    */
1149   private HServerLoad.RegionLoad createRegionLoad(final HRegion r) {
1150     byte[] name = r.getRegionName();
1151     int stores = 0;
1152     int storefiles = 0;
1153     int storeUncompressedSizeMB = 0;
1154     int storefileSizeMB = 0;
1155     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
1156     int storefileIndexSizeMB = 0;
1157     int rootIndexSizeKB = 0;
1158     int totalStaticIndexSizeKB = 0;
1159     int totalStaticBloomSizeKB = 0;
1160     long totalCompactingKVs = 0;
1161     long currentCompactedKVs = 0;
1162     synchronized (r.stores) {
1163       stores += r.stores.size();
1164       for (Store store : r.stores.values()) {
1165         storefiles += store.getStorefilesCount();
1166         storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
1167             / 1024 / 1024);
1168         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1169         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
1170         CompactionProgress progress = store.getCompactionProgress();
1171         if (progress != null) {
1172           totalCompactingKVs += progress.totalCompactingKVs;
1173           currentCompactedKVs += progress.currentCompactedKVs;
1174         }
1175 
1176         rootIndexSizeKB +=
1177             (int) (store.getStorefilesIndexSize() / 1024);
1178 
1179         totalStaticIndexSizeKB +=
1180           (int) (store.getTotalStaticIndexSize() / 1024);
1181 
1182         totalStaticBloomSizeKB +=
1183           (int) (store.getTotalStaticBloomSize() / 1024);
1184       }
1185     }
1186     return new HServerLoad.RegionLoad(name, stores, storefiles,
1187         storeUncompressedSizeMB,
1188         storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, rootIndexSizeKB,
1189         totalStaticIndexSizeKB, totalStaticBloomSizeKB,
1190         (int) r.readRequestsCount.get(), (int) r.writeRequestsCount.get(),
1191         totalCompactingKVs, currentCompactedKVs);
1192   }
1193 
1194   /**
1195    * @param encodedRegionName
1196    * @return An instance of RegionLoad.
1197    */
1198   public HServerLoad.RegionLoad createRegionLoad(final String encodedRegionName) {
1199     HRegion r = null;
1200     r = this.onlineRegions.get(encodedRegionName);
1201     return r != null ? createRegionLoad(r) : null;
1202   }
1203 
1204   /*
1205    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
1206    * IOE if it isn't already.
1207    *
1208    * @param t Throwable
1209    *
1210    * @return Throwable converted to an IOE; methods can only let out IOEs.
1211    */
1212   private Throwable cleanup(final Throwable t) {
1213     return cleanup(t, null);
1214   }
1215 
1216   /*
1217    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
1218    * IOE if it isn't already.
1219    *
1220    * @param t Throwable
1221    *
1222    * @param msg Message to log in error. Can be null.
1223    *
1224    * @return Throwable converted to an IOE; methods can only let out IOEs.
1225    */
1226   private Throwable cleanup(final Throwable t, final String msg) {
1227     // Don't log as error if NSRE; NSRE is 'normal' operation.
1228     if (t instanceof NotServingRegionException) {
1229       LOG.debug("NotServingRegionException; " +  t.getMessage());
1230       return t;
1231     }
1232     if (msg == null) {
1233       LOG.error("", RemoteExceptionHandler.checkThrowable(t));
1234     } else {
1235       LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
1236     }
1237     if (!checkOOME(t)) {
1238       checkFileSystem();
1239     }
1240     return t;
1241   }
1242 
1243   /*
1244    * @param t
1245    *
1246    * @return Make <code>t</code> an IOE if it isn't already.
1247    */
1248   private IOException convertThrowableToIOE(final Throwable t) {
1249     return convertThrowableToIOE(t, null);
1250   }
1251 
1252   /*
1253    * @param t
1254    *
1255    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
1256    *
1257    * @return Make <code>t</code> an IOE if it isn't already.
1258    */
1259   private IOException convertThrowableToIOE(final Throwable t, final String msg) {
1260     return (t instanceof IOException ? (IOException) t : msg == null
1261         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
1262   }
1263 
1264   /*
1265    * Check if an OOME and, if so, abort immediately to avoid creating more objects.
1266    *
1267    * @param e
1268    *
1269    * @return True if we OOME'd and are aborting.
1270    */
1271   public boolean checkOOME(final Throwable e) {
1272     boolean stop = false;
1273     try {
1274       if (e instanceof OutOfMemoryError
1275           || (e.getCause() != null && e.getCause() instanceof OutOfMemoryError)
1276           || (e.getMessage() != null && e.getMessage().contains(
1277               "java.lang.OutOfMemoryError"))) {
1278         stop = true;
1279         LOG.fatal(
1280           "Run out of memory; HRegionServer will abort itself immediately", e);
1281       }
1282     } finally {
1283       if (stop) {
1284         Runtime.getRuntime().halt(1);
1285       }
1286     }
1287     return stop;
1288   }
1289 
1290   /**
1291    * Checks to see if the file system is still accessible. If not, sets
1292    * abortRequested and stopRequested
1293    *
1294    * @return false if file system is not available
1295    */
1296   public boolean checkFileSystem() {
1297     if (this.fsOk && this.fs != null) {
1298       try {
1299         FSUtils.checkFileSystemAvailable(this.fs);
1300       } catch (IOException e) {
1301         abort("File System not available", e);
1302         this.fsOk = false;
1303       }
1304     }
1305     return this.fsOk;
1306   }
1307 
1308   /*
1309    * Inner class that runs on a long period checking if regions need compaction.
1310    */
1311   private static class CompactionChecker extends Chore {
1312     private final HRegionServer instance;
1313     private final int majorCompactPriority;
1314     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1315 
1316     CompactionChecker(final HRegionServer h, final int sleepTime,
1317         final Stoppable stopper) {
1318       super("CompactionChecker", sleepTime, h);
1319       this.instance = h;
1320       LOG.info("Runs every " + StringUtils.formatTime(sleepTime));
1321 
1322       /* MajorCompactPriority is configurable.
1323        * If not set, the compaction will use default priority.
1324        */
1325       this.majorCompactPriority = this.instance.conf.
1326         getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1327         DEFAULT_PRIORITY);
1328     }
1329 
1330     @Override
1331     protected void chore() {
1332       for (HRegion r : this.instance.onlineRegions.values()) {
1333         if (r == null)
1334           continue;
1335         for (Store s : r.getStores().values()) {
1336           try {
1337             if (s.needsCompaction()) {
1338               // Queue a compaction. Will recognize if major is needed.
1339               this.instance.compactSplitThread.requestCompaction(r, s, getName()
1340                   + " requests compaction", null);
1341             } else if (s.isMajorCompaction()) {
1342               if (majorCompactPriority == DEFAULT_PRIORITY
1343                   || majorCompactPriority > r.getCompactPriority()) {
1344                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1345                     + " requests major compaction; use default priority", null);
1346               } else {
1347                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1348                     + " requests major compaction; use configured priority",
1349                   this.majorCompactPriority, null);
1350               }
1351             }
1352           } catch (IOException e) {
1353             LOG.warn("Failed major compaction check on " + r, e);
1354           }
1355         }
1356       }
1357     }
1358   }
1359 
1360   class PeriodicMemstoreFlusher extends Chore {
1361     final HRegionServer server;
1362     final static int RANGE_OF_DELAY = 20000; //millisec
1363     final static int MIN_DELAY_TIME = 3000; //millisec
1364     public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1365       super(server.getServerName() + "-MemstoreFlusherChore", cacheFlushInterval, server);
1366       this.server = server;
1367     }
1368 
1369     @Override
1370     protected void chore() {
1371       for (HRegion r : this.server.onlineRegions.values()) {
1372         if (r == null)
1373           continue;
1374         if (r.shouldFlush()) {
1375           FlushRequester requester = server.getFlushRequester();
1376           if (requester != null) {
1377             long randomDelay = rand.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME;
1378             LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() + 
1379                 " after a delay of " + randomDelay);
1380             //Throttle the flushes by putting a delay. If we don't throttle, and there
1381             //is a balanced write-load on the regions in a table, we might end up 
1382             //overwhelming the filesystem with too many flushes at once.
1383             requester.requestDelayedFlush(r, randomDelay);
1384           }
1385         }
1386       }
1387     }
1388   }
1389 
1390   /**
1391    * Report the status of the server. A server is online once all the startup is
1392    * completed (setting up filesystem, starting service threads, etc.). This
1393    * method is designed mostly to be useful in tests.
1394    *
1395    * @return true if online, false if not.
1396    */
1397   public boolean isOnline() {
1398     return isOnline;
1399   }
1400 
1401   /**
1402    * Setup WAL log and replication if enabled.
1403    * Replication setup is done in here because it wants to be hooked up to WAL.
1404    * @return A WAL instance.
1405    * @throws IOException
1406    */
1407   private HLog setupWALAndReplication() throws IOException {
1408     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1409     Path logdir = new Path(rootDir,
1410       HLog.getHLogDirectoryName(this.serverNameFromMasterPOV.toString()));
1411     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1412     if (this.fs.exists(logdir)) {
1413       throw new RegionServerRunningException("Region server has already " +
1414         "created directory at " + this.serverNameFromMasterPOV.toString());
1415     }
1416 
1417     // Instantiate replication manager if replication enabled.  Pass it the
1418     // log directories.
1419     createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
1420     return instantiateHLog(logdir, oldLogDir);
1421   }
1422 
1423   // The method is synchronized to guarantee atomic update to hlogForMeta - 
1424   // It is possible that multiple calls could be made to this method almost 
1425   // at the same time, one for _ROOT_ and another for .META. (if they happen
1426   // to be assigned to the same RS). Also, we want to use the same log for both
1427   private synchronized HLog getMetaWAL() throws IOException {
1428     if (this.hlogForMeta == null) {
1429       final String logName
1430       = HLog.getHLogDirectoryName(this.serverNameFromMasterPOV.toString());
1431 
1432       Path logdir = new Path(rootDir, logName);
1433       final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1434       if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1435       this.hlogForMeta = new HLog(this.fs.getBackingFs(), logdir, oldLogDir, this.conf,
1436           getMetaWALActionListeners(), false, this.serverNameFromMasterPOV.toString(), true);
1437     }
1438     return this.hlogForMeta;
1439   }
1440 
1441   /**
1442    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1443    * @param logdir
1444    * @param oldLogDir
1445    * @return WAL instance.
1446    * @throws IOException
1447    */
1448   protected HLog instantiateHLog(Path logdir, Path oldLogDir) throws IOException {
1449     return new HLog(this.fs.getBackingFs(), logdir, oldLogDir, this.conf,
1450       getWALActionListeners(), this.serverNameFromMasterPOV.toString());
1451   }
1452 
1453   /**
1454    * Called by {@link #instantiateHLog(Path, Path)} setting up WAL instance.
1455    * Add any {@link WALActionsListener}s you want inserted before WAL startup.
1456    * @return List of WALActionsListener that will be passed in to
1457    * {@link HLog} on construction.
1458    */
1459   protected List<WALActionsListener> getWALActionListeners() {
1460     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1461     // Log roller.
1462     this.hlogRoller = new LogRoller(this, this);
1463     listeners.add(this.hlogRoller);
1464     if (this.replicationSourceHandler != null &&
1465         this.replicationSourceHandler.getWALActionsListener() != null) {
1466       // Replication handler is an implementation of WALActionsListener.
1467       listeners.add(this.replicationSourceHandler.getWALActionsListener());
1468     }
1469     return listeners;
1470   }
1471 
1472   protected List<WALActionsListener> getMetaWALActionListeners() {
1473     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1474     // Using a tmp log roller to ensure metaLogRoller is alive once it is not
1475     // null (addendum patch on HBASE-7213)
1476     MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this);
1477     String n = Thread.currentThread().getName();
1478     Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
1479         n + "MetaLogRoller", uncaughtExceptionHandler);
1480     this.metaHLogRoller = tmpLogRoller;
1481     tmpLogRoller = null;
1482     listeners.add(this.metaHLogRoller);
1483     return listeners;
1484   }
1485 
1486   protected LogRoller getLogRoller() {
1487     return hlogRoller;
1488   }
1489 
1490   /*
1491    * @param interval Interval since last time metrics were called.
1492    */
1493   protected void doMetrics() {
1494     try {
1495       metrics();
1496     } catch (Throwable e) {
1497       LOG.warn("Failed metrics", e);
1498     }
1499   }
1500 
1501   protected void metrics() {
1502     this.metrics.regions.set(this.onlineRegions.size());
1503     this.metrics.incrementRequests(this.requestCount.get());
1504     this.metrics.requests.intervalHeartBeat();
1505     // Is this too expensive every three seconds getting a lock on onlineRegions
1506     // and then per store carried? Can I make metrics be sloppier and avoid
1507     // the synchronizations?
1508     int stores = 0;
1509     int storefiles = 0;
1510     long memstoreSize = 0;
1511     int readRequestsCount = 0;
1512     int writeRequestsCount = 0;
1513     long storefileIndexSize = 0;
1514     HDFSBlocksDistribution hdfsBlocksDistribution =
1515       new HDFSBlocksDistribution();
1516     long totalStaticIndexSize = 0;
1517     long totalStaticBloomSize = 0;
1518     long numPutsWithoutWAL = 0;
1519     long dataInMemoryWithoutWAL = 0;
1520     long updatesBlockedMs = 0;
1521 
1522     // Note that this is a map of Doubles instead of Longs. This is because we
1523     // do effective integer division, which would perhaps truncate more than it
1524     // should because we do it only on one part of our sum at a time. Rather
1525     // than dividing at the end, where it is difficult to know the proper
1526     // factor, everything is exact then truncated.
1527     final Map<String, MutableDouble> tempVals =
1528         new HashMap<String, MutableDouble>();
1529 
1530     for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1531       HRegion r = e.getValue();
1532       memstoreSize += r.memstoreSize.get();
1533       numPutsWithoutWAL += r.numPutsWithoutWAL.get();
1534       dataInMemoryWithoutWAL += r.dataInMemoryWithoutWAL.get();
1535       readRequestsCount += r.readRequestsCount.get();
1536       writeRequestsCount += r.writeRequestsCount.get();
1537       updatesBlockedMs += r.updatesBlockedMs.get();
1538       synchronized (r.stores) {
1539         stores += r.stores.size();
1540         for (Map.Entry<byte[], Store> ee : r.stores.entrySet()) {
1541             final Store store = ee.getValue();
1542             final SchemaMetrics schemaMetrics = store.getSchemaMetrics();
1543 
1544             {
1545               long tmpStorefiles = store.getStorefilesCount();
1546               schemaMetrics.accumulateStoreMetric(tempVals,
1547                   StoreMetricType.STORE_FILE_COUNT, tmpStorefiles);
1548               storefiles += tmpStorefiles;
1549             }
1550 
1551 
1552             {
1553               long tmpStorefileIndexSize = store.getStorefilesIndexSize();
1554               schemaMetrics.accumulateStoreMetric(tempVals,
1555                   StoreMetricType.STORE_FILE_INDEX_SIZE,
1556                   (long) (tmpStorefileIndexSize / (1024.0 * 1024)));
1557               storefileIndexSize += tmpStorefileIndexSize;
1558             }
1559 
1560             {
1561               long tmpStorefilesSize = store.getStorefilesSize();
1562               schemaMetrics.accumulateStoreMetric(tempVals,
1563                   StoreMetricType.STORE_FILE_SIZE_MB,
1564                   (long) (tmpStorefilesSize / (1024.0 * 1024)));
1565             }
1566 
1567             {
1568               long tmpStaticBloomSize = store.getTotalStaticBloomSize();
1569               schemaMetrics.accumulateStoreMetric(tempVals,
1570                   StoreMetricType.STATIC_BLOOM_SIZE_KB,
1571                   (long) (tmpStaticBloomSize / 1024.0));
1572               totalStaticBloomSize += tmpStaticBloomSize;
1573             }
1574 
1575             {
1576               long tmpStaticIndexSize = store.getTotalStaticIndexSize();
1577               schemaMetrics.accumulateStoreMetric(tempVals,
1578                   StoreMetricType.STATIC_INDEX_SIZE_KB,
1579                   (long) (tmpStaticIndexSize / 1024.0));
1580               totalStaticIndexSize += tmpStaticIndexSize;
1581             }
1582 
1583             schemaMetrics.accumulateStoreMetric(tempVals,
1584                 StoreMetricType.MEMSTORE_SIZE_MB,
1585                 (long) (store.getMemStoreSize() / (1024.0 * 1024)));
1586         }
1587       }
1588 
1589       hdfsBlocksDistribution.add(r.getHDFSBlocksDistribution());
1590     }
1591 
1592     for (Entry<String, MutableDouble> e : tempVals.entrySet()) {
1593       RegionMetricsStorage.setNumericMetric(e.getKey(), e.getValue().longValue());
1594     }
1595 
1596     this.metrics.stores.set(stores);
1597     this.metrics.storefiles.set(storefiles);
1598     this.metrics.hlogFileCount.set(this.hlog.getNumLogFiles());
1599     this.metrics.hlogFileSizeMB.set(this.hlog.getNumLogFileSize() /(1024 * 1024));
1600     this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024)));
1601     this.metrics.mbInMemoryWithoutWAL.set((int) (dataInMemoryWithoutWAL / (1024 * 1024)));
1602     this.metrics.numPutsWithoutWAL.set(numPutsWithoutWAL);
1603     this.metrics.storefileIndexSizeMB.set(
1604         (int) (storefileIndexSize / (1024 * 1024)));
1605     this.metrics.rootIndexSizeKB.set(
1606         (int) (storefileIndexSize / 1024));
1607     this.metrics.totalStaticIndexSizeKB.set(
1608         (int) (totalStaticIndexSize / 1024));
1609     this.metrics.totalStaticBloomSizeKB.set(
1610         (int) (totalStaticBloomSize / 1024));
1611     this.metrics.readRequestsCount.set(readRequestsCount);
1612     this.metrics.writeRequestsCount.set(writeRequestsCount);
1613     this.metrics.compactionQueueSize.set(compactSplitThread
1614         .getCompactionQueueSize());
1615     this.metrics.flushQueueSize.set(cacheFlusher
1616         .getFlushQueueSize());
1617     this.metrics.updatesBlockedSeconds.set(updatesBlockedMs/1000);
1618     final long updatesBlockedMsHigherWater = cacheFlusher.getUpdatesBlockedMsHighWater().get();
1619     this.metrics.updatesBlockedSecondsHighWater.set(updatesBlockedMsHigherWater/1000);
1620 
1621     BlockCache blockCache = cacheConfig.getBlockCache();
1622     if (blockCache != null) {
1623       this.metrics.blockCacheCount.set(blockCache.size());
1624       this.metrics.blockCacheFree.set(blockCache.getFreeSize());
1625       this.metrics.blockCacheSize.set(blockCache.getCurrentSize());
1626       CacheStats cacheStats = blockCache.getStats();
1627       this.metrics.blockCacheHitCount.set(cacheStats.getHitCount());
1628       this.metrics.blockCacheMissCount.set(cacheStats.getMissCount());
1629       this.metrics.blockCacheEvictedCount.set(blockCache.getEvictedCount());
1630       double ratio = blockCache.getStats().getHitRatio();
1631       int percent = (int) (ratio * 100);
1632       this.metrics.blockCacheHitRatio.set(percent);
1633       ratio = blockCache.getStats().getHitCachingRatio();
1634       percent = (int) (ratio * 100);
1635       this.metrics.blockCacheHitCachingRatio.set(percent);
1636       // past N period block cache hit / hit caching ratios
1637       cacheStats.rollMetricsPeriod();
1638       ratio = cacheStats.getHitRatioPastNPeriods();
1639       percent = (int) (ratio * 100);
1640       this.metrics.blockCacheHitRatioPastNPeriods.set(percent);
1641       ratio = cacheStats.getHitCachingRatioPastNPeriods();
1642       percent = (int) (ratio * 100);
1643       this.metrics.blockCacheHitCachingRatioPastNPeriods.set(percent);
1644     }
1645     float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
1646       getServerName().getHostname());
1647     int percent = (int) (localityIndex * 100);
1648     this.metrics.hdfsBlocksLocalityIndex.set(percent);
1649 
1650   }
1651 
1652   /**
1653    * @return Region server metrics instance.
1654    */
1655   public RegionServerMetrics getMetrics() {
1656     return this.metrics;
1657   }
1658 
1659   /**
1660    * @return Master address tracker instance.
1661    */
1662   public MasterAddressTracker getMasterAddressManager() {
1663     return this.masterAddressManager;
1664   }
1665 
1666   /*
1667    * Start maintanence Threads, Server, Worker and lease checker threads.
1668    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1669    * get an unhandled exception. We cannot set the handler on all threads.
1670    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1671    * waits a while then retries. Meantime, a flush or a compaction that tries to
1672    * run should trigger same critical condition and the shutdown will run. On
1673    * its way out, this server will shut down Server. Leases are sort of
1674    * inbetween. It has an internal thread that while it inherits from Chore, it
1675    * keeps its own internal stop mechanism so needs to be stopped by this
1676    * hosting server. Worker logs the exception and exits.
1677    */
1678   private void startServiceThreads() throws IOException {
1679     String n = Thread.currentThread().getName();
1680     // Start executor services
1681     this.service = new ExecutorService(getServerName().toString());
1682     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1683       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1684     this.service.startExecutorService(ExecutorType.RS_OPEN_ROOT,
1685       conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
1686     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1687       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1688     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1689       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1690     this.service.startExecutorService(ExecutorType.RS_CLOSE_ROOT,
1691       conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
1692     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1693       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1694 
1695     Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), n + ".logRoller",
1696         uncaughtExceptionHandler);
1697     Threads.setDaemonThreadRunning(this.cacheFlusher.getThread(), n + ".cacheFlusher",
1698         uncaughtExceptionHandler);
1699     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), n +
1700       ".compactionChecker", uncaughtExceptionHandler);
1701     Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), n +
1702         ".periodicFlusher", uncaughtExceptionHandler);
1703     if (this.healthCheckChore != null) {
1704       Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
1705           uncaughtExceptionHandler);
1706     }
1707 
1708     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1709     // an unhandled exception, it will just exit.
1710     this.leases.setName(n + ".leaseChecker");
1711     this.leases.start();
1712 
1713     // Put up the webui.  Webui may come up on port other than configured if
1714     // that port is occupied. Adjust serverInfo if this is the case.
1715     this.webuiport = putUpWebUI();
1716 
1717     if (this.replicationSourceHandler == this.replicationSinkHandler &&
1718         this.replicationSourceHandler != null) {
1719       this.replicationSourceHandler.startReplicationService();
1720     } else {
1721       if (this.replicationSourceHandler != null) {
1722         this.replicationSourceHandler.startReplicationService();
1723       }
1724       if (this.replicationSinkHandler != null) {
1725         this.replicationSinkHandler.startReplicationService();
1726       }
1727     }
1728 
1729     // Start Server.  This service is like leases in that it internally runs
1730     // a thread.
1731     this.rpcServer.start();
1732 
1733     // Create the log splitting worker and start it
1734     this.splitLogWorker = new SplitLogWorker(this.zooKeeper,
1735         this.getConfiguration(), this.getServerName().toString());
1736     splitLogWorker.start();
1737     
1738   }
1739 
1740   /**
1741    * Puts up the webui.
1742    * @return Returns final port -- maybe different from what we started with.
1743    * @throws IOException
1744    */
1745   private int putUpWebUI() throws IOException {
1746     int port = this.conf.getInt("hbase.regionserver.info.port", 60030);
1747     // -1 is for disabling info server
1748     if (port < 0) return port;
1749     String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1750     // check if auto port bind enabled
1751     boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
1752         false);
1753     while (true) {
1754       try {
1755         this.infoServer = new InfoServer("regionserver", addr, port, false, this.conf);
1756         this.infoServer.addServlet("status", "/rs-status", RSStatusServlet.class);
1757         this.infoServer.addServlet("dump", "/dump", RSDumpServlet.class);
1758         this.infoServer.setAttribute(REGIONSERVER, this);
1759         this.infoServer.setAttribute(REGIONSERVER_CONF, conf);
1760         this.infoServer.start();
1761         break;
1762       } catch (BindException e) {
1763         if (!auto) {
1764           // auto bind disabled throw BindException
1765           throw e;
1766         }
1767         // auto bind enabled, try to use another port
1768         LOG.info("Failed binding http info server to port: " + port);
1769         port++;
1770       }
1771     }
1772     return port;
1773   }
1774 
1775   /*
1776    * Verify that server is healthy
1777    */
1778   private boolean isHealthy() {
1779     if (!fsOk) {
1780       // File system problem
1781       return false;
1782     }
1783     // Verify that all threads are alive
1784     if (!(leases.isAlive()
1785         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1786         && this.compactionChecker.isAlive())
1787         && this.periodicFlusher.isAlive()) {
1788       stop("One or more threads are no longer alive -- stop");
1789       return false;
1790     }
1791     if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
1792       stop("Meta HLog roller thread is no longer alive -- stop");
1793       return false;
1794     }
1795     return true;
1796   }
1797 
1798   public HLog getWAL() {
1799     try {
1800       return getWAL(null);
1801     } catch (IOException e) {
1802       LOG.warn("getWAL threw exception " + e);
1803       return null; 
1804     }
1805   }
1806 
1807   @Override
1808   public HLog getWAL(HRegionInfo regionInfo) throws IOException {
1809     //TODO: at some point this should delegate to the HLogFactory
1810     //currently, we don't care about the region as much as we care about the 
1811     //table.. (hence checking the tablename below)
1812     //_ROOT_ and .META. regions have separate WAL. 
1813     if (this.separateHLogForMeta && 
1814         regionInfo != null && 
1815         regionInfo.isMetaTable()) {
1816       return getMetaWAL();
1817     }
1818     return this.hlog;
1819   }
1820 
1821   @Override
1822   public CatalogTracker getCatalogTracker() {
1823     return this.catalogTracker;
1824   }
1825 
1826   @Override
1827   public void stop(final String msg) {
1828     try {
1829       if (this.rsHost != null) {
1830         this.rsHost.preStop(msg);
1831       }
1832       this.stopped = true;
1833       LOG.info("STOPPED: " + msg);
1834       // Wakes run() if it is sleeping
1835       sleeper.skipSleepCycle();
1836     } catch (IOException exp) {
1837       LOG.warn("The region server did not stop", exp);
1838     }
1839   }
1840 
1841   public void waitForServerOnline(){
1842     while (!isOnline() && !isStopped()){
1843        sleeper.sleep();
1844     }
1845   }
1846 
1847   @Override
1848   public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct,
1849       final boolean daughter)
1850   throws KeeperException, IOException {
1851     checkOpen();
1852     LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString() +
1853       ", daughter=" + daughter);
1854     // Do checks to see if we need to compact (references or too many files)
1855     for (Store s : r.getStores().values()) {
1856       if (s.hasReferences() || s.needsCompaction()) {
1857         getCompactionRequester().requestCompaction(r, s, "Opening Region", null);
1858       }
1859     }
1860     // Update ZK, ROOT or META
1861     if (r.getRegionInfo().isRootRegion()) {
1862       RootLocationEditor.setRootLocation(getZooKeeper(),
1863        this.serverNameFromMasterPOV);
1864     } else if (r.getRegionInfo().isMetaRegion()) {
1865       MetaEditor.updateMetaLocation(ct, r.getRegionInfo(),
1866         this.serverNameFromMasterPOV);
1867     } else {
1868       if (daughter) {
1869         // If daughter of a split, update whole row, not just location.
1870         MetaEditor.addDaughter(ct, r.getRegionInfo(),
1871           this.serverNameFromMasterPOV);
1872       } else {
1873         MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
1874           this.serverNameFromMasterPOV);
1875       }
1876     }
1877     LOG.info("Done with post open deploy task for region=" +
1878       r.getRegionNameAsString() + ", daughter=" + daughter);
1879 
1880   }
1881 
1882   /**
1883    * Return a reference to the metrics instance used for counting RPC calls.
1884    * @return Metrics instance.
1885    */
1886   public HBaseRpcMetrics getRpcMetrics() {
1887     return rpcServer.getRpcMetrics();
1888   }
1889 
1890   @Override
1891   public RpcServer getRpcServer() {
1892     return rpcServer;
1893   }
1894 
1895   /**
1896    * Cause the server to exit without closing the regions it is serving, the log
1897    * it is using and without notifying the master. Used unit testing and on
1898    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1899    *
1900    * @param reason
1901    *          the reason we are aborting
1902    * @param cause
1903    *          the exception that caused the abort, or null
1904    */
1905   public void abort(String reason, Throwable cause) {
1906     String msg = "ABORTING region server " + this + ": " + reason;
1907     if (cause != null) {
1908       LOG.fatal(msg, cause);
1909     } else {
1910       LOG.fatal(msg);
1911     }
1912     this.abortRequested = true;
1913     this.reservedSpace.clear();
1914     // HBASE-4014: show list of coprocessors that were loaded to help debug
1915     // regionserver crashes.Note that we're implicitly using
1916     // java.util.HashSet's toString() method to print the coprocessor names.
1917     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1918         CoprocessorHost.getLoadedCoprocessors());
1919     if (this.metrics != null) {
1920       LOG.info("Dump of metrics: " + this.metrics);
1921     }
1922     // Do our best to report our abort to the master, but this may not work
1923     try {
1924       if (cause != null) {
1925         msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1926       }
1927       if (hbaseMaster != null) {
1928         hbaseMaster.reportRSFatalError(
1929             this.serverNameFromMasterPOV.getVersionedBytes(), msg);
1930       }
1931     } catch (Throwable t) {
1932       LOG.warn("Unable to report fatal error to master", t);
1933     }
1934     stop(reason);
1935   }
1936 
1937   /**
1938    * @see HRegionServer#abort(String, Throwable)
1939    */
1940   public void abort(String reason) {
1941     abort(reason, null);
1942   }
1943 
1944   public boolean isAborted() {
1945     return this.abortRequested;
1946   }
1947 
1948   /*
1949    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1950    * logs but it does close socket in case want to bring up server on old
1951    * hostname+port immediately.
1952    */
1953   protected void kill() {
1954     this.killed = true;
1955     abort("Simulated kill");
1956   }
1957 
1958   /**
1959    * Wait on all threads to finish. Presumption is that all closes and stops
1960    * have already been called.
1961    */
1962   protected void join() {
1963     Threads.shutdown(this.compactionChecker.getThread());
1964     Threads.shutdown(this.periodicFlusher.getThread());
1965     Threads.shutdown(this.cacheFlusher.getThread());
1966     if (this.healthCheckChore != null) {
1967       Threads.shutdown(this.healthCheckChore.getThread());
1968     }
1969     if (this.hlogRoller != null) {
1970       Threads.shutdown(this.hlogRoller.getThread());
1971     }
1972     if (this.metaHLogRoller != null) {
1973       Threads.shutdown(this.metaHLogRoller.getThread());
1974     }
1975     if (this.compactSplitThread != null) {
1976       this.compactSplitThread.join();
1977     }
1978     if (this.service != null) this.service.shutdown();
1979     if (this.replicationSourceHandler != null &&
1980         this.replicationSourceHandler == this.replicationSinkHandler) {
1981       this.replicationSourceHandler.stopReplicationService();
1982     } else {
1983       if (this.replicationSourceHandler != null) {
1984         this.replicationSourceHandler.stopReplicationService();
1985       }
1986       if (this.replicationSinkHandler != null) {
1987         this.replicationSinkHandler.stopReplicationService();
1988       }
1989     }
1990   }
1991 
1992   /**
1993    * @return Return the object that implements the replication
1994    * source service.
1995    */
1996   ReplicationSourceService getReplicationSourceService() {
1997     return replicationSourceHandler;
1998   }
1999 
2000   /**
2001    * @return Return the object that implements the replication
2002    * sink service.
2003    */
2004   ReplicationSinkService getReplicationSinkService() {
2005     return replicationSinkHandler;
2006   }
2007 
2008   /**
2009    * Get the current master from ZooKeeper and open the RPC connection to it.
2010    *
2011    * Method will block until a master is available. You can break from this
2012    * block by requesting the server stop.
2013    *
2014    * @return master + port, or null if server has been stopped
2015    */
2016   private ServerName getMaster() {
2017     ServerName masterServerName = null;
2018     long previousLogTime = 0;
2019     HMasterRegionInterface master = null;
2020     InetSocketAddress masterIsa = null;
2021     while (keepLooping() && master == null) {
2022       masterServerName = this.masterAddressManager.getMasterAddress();
2023       if (masterServerName == null) {
2024         if (!keepLooping()) {
2025           // give up with no connection.
2026           LOG.debug("No master found and cluster is stopped; bailing out");
2027           return null;
2028         }
2029         LOG.debug("No master found; retry");
2030         previousLogTime = System.currentTimeMillis();
2031 
2032         sleeper.sleep();
2033         continue;
2034       }
2035 
2036       masterIsa =
2037         new InetSocketAddress(masterServerName.getHostname(), masterServerName.getPort());
2038 
2039       LOG.info("Attempting connect to Master server at " + masterServerName);
2040       try {
2041         // Do initial RPC setup. The final argument indicates that the RPC
2042         // should retry indefinitely.
2043         master = HBaseRPC.waitForProxy(this.rpcEngine,
2044             HMasterRegionInterface.class, HMasterRegionInterface.VERSION,
2045             masterIsa, this.conf, -1,
2046             this.rpcTimeout, this.rpcTimeout);
2047       } catch (IOException e) {
2048         e = e instanceof RemoteException ?
2049             ((RemoteException)e).unwrapRemoteException() : e;
2050         if (e instanceof ServerNotRunningYetException) {
2051           if (System.currentTimeMillis() > (previousLogTime+1000)){
2052             LOG.info("Master isn't available yet, retrying");
2053             previousLogTime = System.currentTimeMillis();
2054           }
2055         } else {
2056           if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2057             LOG.warn("Unable to connect to master. Retrying. Error was:", e);
2058             previousLogTime = System.currentTimeMillis();
2059           }
2060         }
2061         try {
2062           Thread.sleep(200);
2063         } catch (InterruptedException ignored) {
2064         }
2065       }
2066     }
2067     LOG.info("Connected to master at " + masterIsa);
2068     this.hbaseMaster = master;
2069     return masterServerName;
2070   }
2071 
2072   /**
2073    * @return True if we should break loop because cluster is going down or
2074    * this server has been stopped or hdfs has gone bad.
2075    */
2076   private boolean keepLooping() {
2077     return !this.stopped && isClusterUp();
2078   }
2079 
2080   /*
2081    * Let the master know we're here Run initialization using parameters passed
2082    * us by the master.
2083    * @return A Map of key/value configurations we got from the Master else
2084    * null if we failed to register.
2085    * @throws IOException
2086    */
2087   private MapWritable reportForDuty() throws IOException {
2088     MapWritable result = null;
2089     ServerName masterServerName = getMaster();
2090     if (masterServerName == null) return result;
2091     try {
2092       this.requestCount.set(0);
2093       LOG.info("Telling master at " + masterServerName + " that we are up " +
2094         "with port=" + this.isa.getPort() + ", startcode=" + this.startcode);
2095       long now = EnvironmentEdgeManager.currentTimeMillis();
2096       int port = this.isa.getPort();
2097       result = this.hbaseMaster.regionServerStartup(port, this.startcode, now);
2098     } catch (RemoteException e) {
2099       IOException ioe = e.unwrapRemoteException();
2100       if (ioe instanceof ClockOutOfSyncException) {
2101         LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2102         // Re-throw IOE will cause RS to abort
2103         throw ioe;
2104       } else {
2105         LOG.warn("remote error telling master we are up", e);
2106       }
2107     } catch (IOException e) {
2108       LOG.warn("error telling master we are up", e);
2109     }
2110     return result;
2111   }
2112 
2113   /**
2114    * Closes all regions.  Called on our way out.
2115    * Assumes that its not possible for new regions to be added to onlineRegions
2116    * while this method runs.
2117    */
2118   protected void closeAllRegions(final boolean abort) {
2119     closeUserRegions(abort);
2120     closeMetaTableRegions(abort);
2121   }
2122 
2123   /**
2124    * Close root and meta regions if we carry them
2125    * @param abort Whether we're running an abort.
2126    */
2127   void closeMetaTableRegions(final boolean abort) {
2128     HRegion meta = null;
2129     HRegion root = null;
2130     this.lock.writeLock().lock();
2131     try {
2132       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2133         HRegionInfo hri = e.getValue().getRegionInfo();
2134         if (hri.isRootRegion()) {
2135           root = e.getValue();
2136         } else if (hri.isMetaRegion()) {
2137           meta = e.getValue();
2138         }
2139         if (meta != null && root != null) break;
2140       }
2141     } finally {
2142       this.lock.writeLock().unlock();
2143     }
2144     if (meta != null) closeRegion(meta.getRegionInfo(), abort, false);
2145     if (root != null) closeRegion(root.getRegionInfo(), abort, false);
2146   }
2147 
2148   /**
2149    * Schedule closes on all user regions.
2150    * Should be safe calling multiple times because it wont' close regions
2151    * that are already closed or that are closing.
2152    * @param abort Whether we're running an abort.
2153    */
2154   void closeUserRegions(final boolean abort) {
2155     this.lock.writeLock().lock();
2156     try {
2157       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2158         HRegion r = e.getValue();
2159         if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2160           // Don't update zk with this close transition; pass false.
2161           closeRegion(r.getRegionInfo(), abort, false);
2162         }
2163       }
2164     } finally {
2165       this.lock.writeLock().unlock();
2166     }
2167   }
2168 
2169   @Override
2170   @QosPriority(priority=HConstants.HIGH_QOS)
2171   public HRegionInfo getRegionInfo(final byte[] regionName)
2172   throws NotServingRegionException, IOException {
2173     checkOpen();
2174     requestCount.incrementAndGet();
2175     return getRegion(regionName).getRegionInfo();
2176   }
2177 
2178   public Result getClosestRowBefore(final byte[] regionName, final byte[] row,
2179       final byte[] family) throws IOException {
2180     checkOpen();
2181     requestCount.incrementAndGet();
2182     try {
2183       // locate the region we're operating on
2184       HRegion region = getRegion(regionName);
2185       // ask the region for all the data
2186 
2187       Result r = region.getClosestRowBefore(row, family);
2188       return r;
2189     } catch (Throwable t) {
2190       throw convertThrowableToIOE(cleanup(t));
2191     }
2192   }
2193 
2194   /** {@inheritDoc} */
2195   public Result get(byte[] regionName, Get get) throws IOException {
2196     checkOpen();
2197     requestCount.incrementAndGet();
2198     try {
2199       HRegion region = getRegion(regionName);
2200       return region.get(get, getLockFromId(get.getLockId()));
2201     } catch (Throwable t) {
2202       throw convertThrowableToIOE(cleanup(t));
2203     }
2204   }
2205 
2206   public boolean exists(byte[] regionName, Get get) throws IOException {
2207     checkOpen();
2208     requestCount.incrementAndGet();
2209     try {
2210       HRegion region = getRegion(regionName);
2211       Integer lock = getLockFromId(get.getLockId());
2212       if (region.getCoprocessorHost() != null) {
2213         Boolean result = region.getCoprocessorHost().preExists(get);
2214         if (result != null) {
2215           return result.booleanValue();
2216         }
2217       }
2218       Result r = region.get(get, lock);
2219       boolean result = r != null && !r.isEmpty();
2220       if (region.getCoprocessorHost() != null) {
2221         result = region.getCoprocessorHost().postExists(get, result);
2222       }
2223       return result;
2224     } catch (Throwable t) {
2225       throw convertThrowableToIOE(cleanup(t));
2226     }
2227   }
2228 
2229   public void put(final byte[] regionName, final Put put) throws IOException {
2230     if (put.getRow() == null) {
2231       throw new IllegalArgumentException("update has null row");
2232     }
2233 
2234     checkOpen();
2235     this.requestCount.incrementAndGet();
2236     HRegion region = getRegion(regionName);
2237     try {
2238       if (!region.getRegionInfo().isMetaTable()) {
2239         this.cacheFlusher.reclaimMemStoreMemory();
2240       }
2241       boolean writeToWAL = put.getWriteToWAL();
2242       region.put(put, getLockFromId(put.getLockId()), writeToWAL);
2243     } catch (Throwable t) {
2244       throw convertThrowableToIOE(cleanup(t));
2245     }
2246   }
2247 
2248   public int put(final byte[] regionName, final List<Put> puts)
2249       throws IOException {
2250     checkOpen();
2251     HRegion region = null;
2252     int i = 0;
2253 
2254     try {
2255       region = getRegion(regionName);
2256       if (!region.getRegionInfo().isMetaTable()) {
2257         this.cacheFlusher.reclaimMemStoreMemory();
2258       }
2259 
2260       @SuppressWarnings("unchecked")
2261       Pair<Mutation, Integer>[] putsWithLocks = new Pair[puts.size()];
2262 
2263       for (Put p : puts) {
2264         Integer lock = getLockFromId(p.getLockId());
2265         putsWithLocks[i++] = new Pair<Mutation, Integer>(p, lock);
2266       }
2267 
2268       this.requestCount.addAndGet(puts.size());
2269       OperationStatus codes[] = region.batchMutate(putsWithLocks);
2270       for (i = 0; i < codes.length; i++) {
2271         if (codes[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
2272           return i;
2273         }
2274       }
2275       return -1;
2276     } catch (Throwable t) {
2277       throw convertThrowableToIOE(cleanup(t));
2278     }
2279   }
2280 
2281   private boolean checkAndMutate(final byte[] regionName, final byte[] row,
2282       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2283       final WritableByteArrayComparable comparator, final Writable w,
2284       Integer lock) throws IOException {
2285     checkOpen();
2286     this.requestCount.incrementAndGet();
2287     HRegion region = getRegion(regionName);
2288     try {
2289       if (!region.getRegionInfo().isMetaTable()) {
2290         this.cacheFlusher.reclaimMemStoreMemory();
2291       }
2292       return region.checkAndMutate(row, family, qualifier, compareOp,
2293         comparator, w, lock, true);
2294     } catch (Throwable t) {
2295       throw convertThrowableToIOE(cleanup(t));
2296     }
2297   }
2298 
2299   /**
2300    *
2301    * @param regionName
2302    * @param row
2303    * @param family
2304    * @param qualifier
2305    * @param value
2306    *          the expected value
2307    * @param put
2308    * @throws IOException
2309    * @return true if the new put was execute, false otherwise
2310    */
2311   public boolean checkAndPut(final byte[] regionName, final byte[] row,
2312       final byte[] family, final byte[] qualifier, final byte[] value,
2313       final Put put) throws IOException {
2314     checkOpen();
2315     if (regionName == null) {
2316       throw new IOException("Invalid arguments to checkAndPut "
2317           + "regionName is null");
2318     }
2319     HRegion region = getRegion(regionName);
2320     Integer lock = getLockFromId(put.getLockId());
2321     WritableByteArrayComparable comparator = new BinaryComparator(value);
2322     if (region.getCoprocessorHost() != null) {
2323       Boolean result = region.getCoprocessorHost()
2324         .preCheckAndPut(row, family, qualifier, CompareOp.EQUAL, comparator,
2325           put);
2326       if (result != null) {
2327         return result.booleanValue();
2328       }
2329     }
2330     boolean result = checkAndMutate(regionName, row, family, qualifier,
2331         CompareOp.EQUAL, comparator, put,
2332       lock);
2333     if (region.getCoprocessorHost() != null) {
2334       result = region.getCoprocessorHost().postCheckAndPut(row, family,
2335         qualifier, CompareOp.EQUAL, comparator, put, result);
2336     }
2337     return result;
2338   }
2339 
2340   /**
2341    *
2342    * @param regionName
2343    * @param row
2344    * @param family
2345    * @param qualifier
2346    * @param compareOp
2347    * @param comparator
2348    * @param put
2349    * @throws IOException
2350    * @return true if the new put was execute, false otherwise
2351    */
2352   public boolean checkAndPut(final byte[] regionName, final byte[] row,
2353       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2354       final WritableByteArrayComparable comparator, final Put put)
2355        throws IOException {
2356     checkOpen();
2357     if (regionName == null) {
2358       throw new IOException("Invalid arguments to checkAndPut "
2359           + "regionName is null");
2360     }
2361     HRegion region = getRegion(regionName);
2362     Integer lock = getLockFromId(put.getLockId());
2363     if (region.getCoprocessorHost() != null) {
2364       Boolean result = region.getCoprocessorHost()
2365         .preCheckAndPut(row, family, qualifier, compareOp, comparator, put);
2366       if (result != null) {
2367         return result.booleanValue();
2368       }
2369     }
2370     boolean result = checkAndMutate(regionName, row, family, qualifier,
2371       compareOp, comparator, put, lock);
2372     if (region.getCoprocessorHost() != null) {
2373       result = region.getCoprocessorHost().postCheckAndPut(row, family,
2374         qualifier, compareOp, comparator, put, result);
2375     }
2376     return result;
2377   }
2378 
2379   /**
2380    *
2381    * @param regionName
2382    * @param row
2383    * @param family
2384    * @param qualifier
2385    * @param value
2386    *          the expected value
2387    * @param delete
2388    * @throws IOException
2389    * @return true if the new put was execute, false otherwise
2390    */
2391   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
2392       final byte[] family, final byte[] qualifier, final byte[] value,
2393       final Delete delete) throws IOException {
2394     checkOpen();
2395 
2396     if (regionName == null) {
2397       throw new IOException("Invalid arguments to checkAndDelete "
2398           + "regionName is null");
2399     }
2400     HRegion region = getRegion(regionName);
2401     Integer lock = getLockFromId(delete.getLockId());
2402     WritableByteArrayComparable comparator = new BinaryComparator(value);
2403     if (region.getCoprocessorHost() != null) {
2404       Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
2405         family, qualifier, CompareOp.EQUAL, comparator, delete);
2406       if (result != null) {
2407         return result.booleanValue();
2408       }
2409     }
2410     boolean result = checkAndMutate(regionName, row, family, qualifier,
2411       CompareOp.EQUAL, comparator, delete, lock);
2412     if (region.getCoprocessorHost() != null) {
2413       result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2414         qualifier, CompareOp.EQUAL, comparator, delete, result);
2415     }
2416     return result;
2417   }
2418 
2419   @Override
2420   public List<String> getStoreFileList(byte[] regionName, byte[] columnFamily)
2421     throws IllegalArgumentException {
2422     return getStoreFileList(regionName, new byte[][]{columnFamily});
2423   }
2424 
2425   @Override
2426   public List<String> getStoreFileList(byte[] regionName, byte[][] columnFamilies)
2427     throws IllegalArgumentException {
2428     HRegion region = getOnlineRegion(regionName);
2429     if (region == null) {
2430       throw new IllegalArgumentException("No region: " + new String(regionName)
2431           + " available");
2432     }
2433     return region.getStoreFileList(columnFamilies);
2434   }
2435 
2436   public List<String> getStoreFileList(byte[] regionName)
2437     throws IllegalArgumentException {
2438     HRegion region = getOnlineRegion(regionName);
2439     if (region == null) {
2440       throw new IllegalArgumentException("No region: " + new String(regionName)
2441           + " available");
2442     }
2443     Set<byte[]> columnFamilies = region.getStores().keySet();
2444     int nCF = columnFamilies.size();
2445     return region.getStoreFileList(columnFamilies.toArray(new byte[nCF][]));
2446   }
2447   
2448  /**
2449   * Flushes the given region
2450   */
2451   public void flushRegion(byte[] regionName)
2452     throws IllegalArgumentException, IOException {
2453     HRegion region = getOnlineRegion(regionName);
2454     if (region == null) {
2455       throw new IllegalArgumentException("No region : " + new String(regionName)
2456       + " available");
2457     }
2458     boolean needsCompaction = region.flushcache();
2459     if (needsCompaction) {
2460       this.compactSplitThread.requestCompaction(region, "Compaction through user triggered flush");
2461     }
2462   }
2463 
2464  /**
2465    * Flushes the given region if lastFlushTime < ifOlderThanTS
2466    */
2467    public void flushRegion(byte[] regionName, long ifOlderThanTS)
2468      throws IllegalArgumentException, IOException {
2469      HRegion region = getOnlineRegion(regionName);
2470      if (region == null) {
2471        throw new IllegalArgumentException("No region : " + new String(regionName)
2472        + " available");
2473      }
2474      if (region.getLastFlushTime() < ifOlderThanTS) {
2475       boolean needsCompaction = region.flushcache();
2476       if (needsCompaction) {
2477         this.compactSplitThread
2478             .requestCompaction(region, "Compaction through user triggered flush");
2479       }
2480     }
2481    }
2482 
2483   /**
2484    * Gets last flush time for the given region
2485    * @return the last flush time for a region
2486    */
2487   public long getLastFlushTime(byte[] regionName) {
2488     HRegion region = getOnlineRegion(regionName);
2489     if (region == null) {
2490       throw new IllegalArgumentException("No region : " + new String(regionName)
2491       + " available");
2492     }
2493     return region.getLastFlushTime();
2494   }
2495  
2496   /**
2497    *
2498    * @param regionName
2499    * @param row
2500    * @param family
2501    * @param qualifier
2502    * @param compareOp
2503    * @param comparator
2504    * @param delete
2505    * @throws IOException
2506    * @return true if the new put was execute, false otherwise
2507    */
2508   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
2509       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2510       final WritableByteArrayComparable comparator, final Delete delete)
2511       throws IOException {
2512     checkOpen();
2513 
2514     if (regionName == null) {
2515       throw new IOException("Invalid arguments to checkAndDelete "
2516         + "regionName is null");
2517     }
2518     HRegion region = getRegion(regionName);
2519     Integer lock = getLockFromId(delete.getLockId());
2520     if (region.getCoprocessorHost() != null) {
2521       Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
2522         family, qualifier, compareOp, comparator, delete);
2523      if (result != null) {
2524        return result.booleanValue();
2525      }
2526     }
2527     boolean result = checkAndMutate(regionName, row, family, qualifier,
2528       compareOp, comparator, delete, lock);
2529    if (region.getCoprocessorHost() != null) {
2530      result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2531        qualifier, compareOp, comparator, delete, result);
2532    }
2533    return result;
2534  }
2535 
2536   //
2537   // remote scanner interface
2538   //
2539   
2540   public long openScanner(byte[] regionName, Scan scan) throws IOException {
2541     RegionScanner s = internalOpenScanner(regionName, scan);
2542     long scannerId = addScanner(s);
2543     return scannerId;
2544   }
2545 
2546   private RegionScanner internalOpenScanner(byte[] regionName, Scan scan)
2547       throws IOException {
2548     checkOpen();
2549     NullPointerException npe = null;
2550     if (regionName == null) {
2551       npe = new NullPointerException("regionName is null");
2552     } else if (scan == null) {
2553       npe = new NullPointerException("scan is null");
2554     }
2555     if (npe != null) {
2556       throw new IOException("Invalid arguments to openScanner", npe);
2557     }
2558     requestCount.incrementAndGet();
2559     try {
2560       HRegion r = getRegion(regionName);
2561       r.checkRow(scan.getStartRow(), "Scan");
2562       scan.setLoadColumnFamiliesOnDemand(r.isLoadingCfsOnDemandDefault()
2563           || scan.doLoadColumnFamiliesOnDemand());
2564       r.prepareScanner(scan);
2565       RegionScanner s = null;
2566       if (r.getCoprocessorHost() != null) {
2567         s = r.getCoprocessorHost().preScannerOpen(scan);
2568       }
2569       if (s == null) {
2570         s = r.getScanner(scan);
2571       }
2572       if (r.getCoprocessorHost() != null) {
2573         RegionScanner savedScanner = r.getCoprocessorHost().postScannerOpen(
2574             scan, s);
2575         if (savedScanner == null) {
2576           LOG.warn("PostScannerOpen impl returning null. "
2577               + "Check the RegionObserver implementation.");
2578         } else {
2579           s = savedScanner;
2580         }
2581       }
2582       return s;
2583     } catch (Throwable t) {
2584       throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
2585     }
2586   }
2587 
2588   protected long addScanner(RegionScanner s) throws LeaseStillHeldException {
2589     long scannerId = -1L;
2590     scannerId = rand.nextLong();
2591     String scannerName = String.valueOf(scannerId);
2592     scanners.put(scannerName, s);
2593     this.leases.createLease(scannerName, new ScannerListener(scannerName));
2594     return scannerId;
2595   }
2596 
2597   public Result next(final long scannerId) throws IOException {
2598     Result[] res = next(scannerId, 1);
2599     if (res == null || res.length == 0) {
2600       return null;
2601     }
2602     return res[0];
2603   }
2604 
2605   public Result[] next(final long scannerId, int nbRows) throws IOException {
2606     String scannerName = String.valueOf(scannerId);
2607     RegionScanner s = this.scanners.get(scannerName);
2608     if (s == null) {
2609       LOG.info("Client tried to access missing scanner " + scannerName);
2610       throw new UnknownScannerException("Name: " + scannerName);
2611     }
2612     return internalNext(s, nbRows, scannerName);
2613   }
2614 
2615   private Result[] internalNext(final RegionScanner s, int nbRows,
2616       String scannerName) throws IOException {
2617     try {
2618       checkOpen();
2619     } catch (IOException e) {
2620       // If checkOpen failed, server not running or filesystem gone,
2621       // cancel this lease; filesystem is gone or we're closing or something.
2622       if (scannerName != null) {
2623         try {
2624           this.leases.cancelLease(scannerName);
2625         } catch (LeaseException le) {
2626           LOG.info("Server shutting down and client tried to access missing scanner "
2627               + scannerName);
2628         }
2629       }
2630       throw e;
2631     }
2632     Leases.Lease lease = null;
2633     try {
2634       // Remove lease while its being processed in server; protects against case
2635       // where processing of request takes > lease expiration time.
2636       try {
2637         if (scannerName != null) {
2638           lease = this.leases.removeLease(scannerName);
2639         }
2640       } catch (LeaseException le) {
2641         // What it really means is that there's no such scanner.
2642         LOG.info("Client tried to access missing scanner " + scannerName + " (no lease)");
2643         throw new UnknownScannerException("No lease for " + scannerName + ": " + le.getMessage());
2644       }
2645       List<Result> results = new ArrayList<Result>(nbRows);
2646       long currentScanResultSize = 0;
2647       List<KeyValue> values = new ArrayList<KeyValue>();
2648 
2649       // Call coprocessor. Get region info from scanner.
2650       HRegion region = getRegion(s.getRegionInfo().getRegionName());
2651       if (region != null && region.getCoprocessorHost() != null) {
2652         Boolean bypass = region.getCoprocessorHost().preScannerNext(s,
2653             results, nbRows);
2654         if (!results.isEmpty()) {
2655           for (Result r : results) {
2656             if (maxScannerResultSize < Long.MAX_VALUE){
2657               for (KeyValue kv : r.raw()) {
2658                 currentScanResultSize += kv.heapSize();
2659               }
2660             }
2661           }
2662         }
2663         if (bypass != null) {
2664           return s.isFilterDone() && results.isEmpty() ? null
2665               : results.toArray(new Result[0]);
2666         }
2667       }
2668 
2669       MultiVersionConsistencyControl.setThreadReadPoint(s.getMvccReadPoint());
2670       region.startRegionOperation();
2671       try {
2672         int i = 0;
2673         synchronized(s) {
2674           for (; i < nbRows
2675               && currentScanResultSize < maxScannerResultSize; ) {
2676             // Collect values to be returned here
2677             boolean moreRows = s.nextRaw(values, SchemaMetrics.METRIC_NEXTSIZE);
2678             if (!values.isEmpty()) {
2679               if (maxScannerResultSize < Long.MAX_VALUE){
2680                 for (KeyValue kv : values) {
2681                   currentScanResultSize += kv.heapSize();
2682                 }
2683               }
2684               results.add(new Result(values));
2685               i++;
2686             }
2687             if (!moreRows) {
2688               break;
2689             }
2690             values.clear();
2691           }
2692         }
2693         requestCount.addAndGet(i);
2694         region.readRequestsCount.add(i);
2695         region.setOpMetricsReadRequestCount(region.readRequestsCount.get());
2696       } finally {
2697         region.closeRegionOperation();
2698       }
2699       // coprocessor postNext hook
2700       if (region != null && region.getCoprocessorHost() != null) {
2701         region.getCoprocessorHost().postScannerNext(s, results, nbRows, true);
2702       }
2703 
2704       // If the scanner's filter - if any - is done with the scan
2705       // and wants to tell the client to stop the scan. This is done by passing
2706       // a null result.
2707       return s.isFilterDone() && results.isEmpty() ? null
2708           : results.toArray(new Result[0]);
2709     } catch (Throwable t) {
2710       if (t instanceof NotServingRegionException && scannerName != null) {
2711         this.scanners.remove(scannerName);
2712       }
2713       throw convertThrowableToIOE(cleanup(t));
2714     } finally {
2715       // We're done. On way out readd the above removed lease.  Adding resets
2716       // expiration time on lease.
2717       if (scannerName != null && this.scanners.containsKey(scannerName)) {
2718         if (lease != null) this.leases.addLease(lease);
2719       }
2720     }
2721   }
2722 
2723   public void close(final long scannerId) throws IOException {
2724     String scannerName = String.valueOf(scannerId);
2725     RegionScanner s = scanners.get(scannerName);
2726     internalCloseScanner(s, scannerName);
2727   }
2728 
2729   private void internalCloseScanner(final RegionScanner s, String scannerName)
2730       throws IOException {
2731     try {
2732       checkOpen();
2733       requestCount.incrementAndGet();
2734 
2735       HRegion region = null;
2736       if (s != null) {
2737         // call coprocessor.
2738         region = getRegion(s.getRegionInfo().getRegionName());
2739         if (region != null && region.getCoprocessorHost() != null) {
2740           if (region.getCoprocessorHost().preScannerClose(s)) {
2741             return; // bypass
2742           }
2743         }
2744       }
2745       RegionScanner toCloseScanner = s;
2746       if (scannerName != null) {
2747         toCloseScanner = scanners.remove(scannerName);
2748       }
2749       if (toCloseScanner != null) {
2750         toCloseScanner.close();
2751         if (scannerName != null) {
2752           this.leases.cancelLease(scannerName);
2753         }
2754 
2755         if (region != null && region.getCoprocessorHost() != null) {
2756           region.getCoprocessorHost().postScannerClose(toCloseScanner);
2757         }
2758       }
2759     } catch (Throwable t) {
2760       throw convertThrowableToIOE(cleanup(t));
2761     }
2762   }
2763 
2764   @Override
2765   public Result[] scan(byte[] regionName, Scan scan, int numberOfRows)
2766       throws IOException {
2767     RegionScanner s = internalOpenScanner(regionName, scan);
2768     try {
2769       Result[] results = internalNext(s, numberOfRows, null);
2770       return results;
2771     } finally {
2772       internalCloseScanner(s, null);
2773     }
2774   }
2775 
2776   /**
2777    * Instantiated as a scanner lease. If the lease times out, the scanner is
2778    * closed
2779    */
2780   private class ScannerListener implements LeaseListener {
2781     private final String scannerName;
2782 
2783     ScannerListener(final String n) {
2784       this.scannerName = n;
2785     }
2786 
2787     public void leaseExpired() {
2788       RegionScanner s = scanners.remove(this.scannerName);
2789       if (s != null) {
2790         LOG.info("Scanner " + this.scannerName + " lease expired on region "
2791             + s.getRegionInfo().getRegionNameAsString());
2792         try {
2793           HRegion region = getRegion(s.getRegionInfo().getRegionName());
2794           if (region != null && region.getCoprocessorHost() != null) {
2795             region.getCoprocessorHost().preScannerClose(s);
2796           }
2797 
2798           s.close();
2799           if (region != null && region.getCoprocessorHost() != null) {
2800             region.getCoprocessorHost().postScannerClose(s);
2801           }
2802         } catch (IOException e) {
2803           LOG.error("Closing scanner for "
2804               + s.getRegionInfo().getRegionNameAsString(), e);
2805         }
2806       } else {
2807         LOG.info("Scanner " + this.scannerName + " lease expired");
2808       }
2809     }
2810   }
2811 
2812   //
2813   // Methods that do the actual work for the remote API
2814   //
2815   public void delete(final byte[] regionName, final Delete delete)
2816       throws IOException {
2817     checkOpen();
2818     try {
2819       boolean writeToWAL = delete.getWriteToWAL();
2820       this.requestCount.incrementAndGet();
2821       HRegion region = getRegion(regionName);
2822       if (!region.getRegionInfo().isMetaTable()) {
2823         this.cacheFlusher.reclaimMemStoreMemory();
2824       }
2825       Integer lid = getLockFromId(delete.getLockId());
2826       region.delete(delete, lid, writeToWAL);
2827     } catch (Throwable t) {
2828       throw convertThrowableToIOE(cleanup(t));
2829     }
2830   }
2831 
2832   public int delete(final byte[] regionName, final List<Delete> deletes)
2833       throws IOException {
2834     checkOpen();
2835     // Count of Deletes processed.
2836     int i = 0;
2837     HRegion region = null;
2838     try {
2839       region = getRegion(regionName);
2840       if (!region.getRegionInfo().isMetaTable()) {
2841         this.cacheFlusher.reclaimMemStoreMemory();
2842       }
2843       int size = deletes.size();
2844       Integer[] locks = new Integer[size];
2845       for (Delete delete : deletes) {
2846         this.requestCount.incrementAndGet();
2847         locks[i] = getLockFromId(delete.getLockId());
2848         region.delete(delete, locks[i], delete.getWriteToWAL());
2849         i++;
2850       }
2851     } catch (WrongRegionException ex) {
2852       LOG.debug("Batch deletes: " + i, ex);
2853       return i;
2854     } catch (NotServingRegionException ex) {
2855       return i;
2856     } catch (Throwable t) {
2857       throw convertThrowableToIOE(cleanup(t));
2858     }
2859     return -1;
2860   }
2861 
2862   /**
2863    * @deprecated {@link RowLock} and associated operations are deprecated.
2864    */
2865   public long lockRow(byte[] regionName, byte[] row) throws IOException {
2866     checkOpen();
2867     NullPointerException npe = null;
2868     if (regionName == null) {
2869       npe = new NullPointerException("regionName is null");
2870     } else if (row == null) {
2871       npe = new NullPointerException("row to lock is null");
2872     }
2873     if (npe != null) {
2874       IOException io = new IOException("Invalid arguments to lockRow");
2875       io.initCause(npe);
2876       throw io;
2877     }
2878     requestCount.incrementAndGet();
2879     try {
2880       HRegion region = getRegion(regionName);
2881       if (region.getCoprocessorHost() != null) {
2882         region.getCoprocessorHost().preLockRow(regionName, row);
2883       }
2884       Integer r = region.obtainRowLock(row);
2885       long lockId = addRowLock(r, region);
2886       LOG.debug("Row lock " + lockId + " explicitly acquired by client");
2887       return lockId;
2888     } catch (Throwable t) {
2889       throw convertThrowableToIOE(cleanup(t, "Error obtaining row lock (fsOk: "
2890           + this.fsOk + ")"));
2891     }
2892   }
2893 
2894   protected long addRowLock(Integer r, HRegion region)
2895       throws LeaseStillHeldException {
2896     long lockId = -1L;
2897     lockId = rand.nextLong();
2898     String lockName = String.valueOf(lockId);
2899     rowlocks.put(lockName, r);
2900     this.leases.createLease(lockName, new RowLockListener(lockName, region));
2901     return lockId;
2902   }
2903 
2904   /**
2905    * Method to get the Integer lock identifier used internally from the long
2906    * lock identifier used by the client.
2907    *
2908    * @param lockId
2909    *          long row lock identifier from client
2910    * @return intId Integer row lock used internally in HRegion
2911    * @throws IOException
2912    *           Thrown if this is not a valid client lock id.
2913    */
2914   Integer getLockFromId(long lockId) throws IOException {
2915     if (lockId == -1L) {
2916       return null;
2917     }
2918     String lockName = String.valueOf(lockId);
2919     Integer rl = rowlocks.get(lockName);
2920     if (rl == null) {
2921       throw new UnknownRowLockException("Invalid row lock");
2922     }
2923     this.leases.renewLease(lockName);
2924     return rl;
2925   }
2926 
2927   /**
2928    * @deprecated {@link RowLock} and associated operations are deprecated.
2929    */
2930   @Override
2931   @QosPriority(priority=HConstants.HIGH_QOS)
2932   public void unlockRow(byte[] regionName, long lockId) throws IOException {
2933     checkOpen();
2934     NullPointerException npe = null;
2935     if (regionName == null) {
2936       npe = new NullPointerException("regionName is null");
2937     } else if (lockId == -1L) {
2938       npe = new NullPointerException("lockId is null");
2939     }
2940     if (npe != null) {
2941       IOException io = new IOException("Invalid arguments to unlockRow");
2942       io.initCause(npe);
2943       throw io;
2944     }
2945     requestCount.incrementAndGet();
2946     try {
2947       HRegion region = getRegion(regionName);
2948       if (region.getCoprocessorHost() != null) {
2949         region.getCoprocessorHost().preUnLockRow(regionName, lockId);
2950       }
2951       String lockName = String.valueOf(lockId);
2952       Integer r = rowlocks.remove(lockName);
2953       if (r == null) {
2954         throw new UnknownRowLockException(lockName);
2955       }
2956       region.releaseRowLock(r);
2957       this.leases.cancelLease(lockName);
2958       LOG.debug("Row lock " + lockId
2959           + " has been explicitly released by client");
2960     } catch (Throwable t) {
2961       throw convertThrowableToIOE(cleanup(t));
2962     }
2963   }
2964 
2965   /**
2966    * Atomically bulk load several HFiles into an open region
2967    * @return true if successful, false is failed but recoverably (no action)
2968    * @throws IOException if failed unrecoverably
2969    */
2970   @Override
2971   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
2972       byte[] regionName) throws IOException {
2973     return bulkLoadHFiles(familyPaths, regionName, false);
2974   }
2975 
2976   /**
2977    * Atomically bulk load several HFiles into an open region
2978    * @return true if successful, false is failed but recoverably (no action)
2979    * @throws IOException if failed unrecoverably
2980    */
2981   @Override
2982   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
2983       byte[] regionName, boolean assignSeqNum) throws IOException {
2984     checkOpen();
2985     HRegion region = getRegion(regionName);
2986     boolean bypass = false;
2987     if (region.getCoprocessorHost() != null) {
2988       bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
2989     }
2990     boolean loaded = false;
2991     if (!bypass) {
2992       loaded = region.bulkLoadHFiles(familyPaths, assignSeqNum);
2993     }
2994     if (region.getCoprocessorHost() != null) {
2995       loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
2996     }
2997     return loaded;
2998   }
2999 
3000   Map<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
3001 
3002   /**
3003    * Instantiated as a row lock lease. If the lease times out, the row lock is
3004    * released
3005    */
3006   private class RowLockListener implements LeaseListener {
3007     private final String lockName;
3008     private final HRegion region;
3009 
3010     RowLockListener(final String lockName, final HRegion region) {
3011       this.lockName = lockName;
3012       this.region = region;
3013     }
3014 
3015     public void leaseExpired() {
3016       LOG.info("Row Lock " + this.lockName + " lease expired");
3017       Integer r = rowlocks.remove(this.lockName);
3018       if (r != null) {
3019         region.releaseRowLock(r);
3020       }
3021     }
3022   }
3023 
3024   // Region open/close direct RPCs
3025 
3026   @Override
3027   @QosPriority(priority=HConstants.HIGH_QOS)
3028   public RegionOpeningState openRegion(HRegionInfo region)
3029   throws IOException {
3030     return openRegion(region, -1);
3031   }
3032 
3033   @Override
3034   @QosPriority(priority = HConstants.HIGH_QOS)
3035   public RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode)
3036       throws IOException {
3037     return openRegion(region, versionOfOfflineNode, null);
3038   }
3039 
3040   private RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode,
3041       Map<String, HTableDescriptor> htds) throws IOException {
3042     checkOpen();
3043     HRegion onlineRegion = this.getFromOnlineRegions(region.getEncodedName());
3044     if (null != onlineRegion) {
3045       // See HBASE-5094. Cross check with META if still this RS is owning the
3046       // region.
3047       Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(
3048           this.catalogTracker, region.getRegionName());
3049       if (this.getServerName().equals(p.getSecond())) {
3050         LOG.warn("Attempted open of " + region.getEncodedName()
3051             + " but already online on this server");
3052         return RegionOpeningState.ALREADY_OPENED;
3053       } else {
3054         LOG.warn("The region " + region.getEncodedName()
3055             + " is online on this server but META does not have this server.");
3056         this.removeFromOnlineRegions(region.getEncodedName());
3057       }
3058     }
3059     // Added to in-memory RS RIT that we are trying to open this region.
3060     // Clear it if we fail queuing an open executor.
3061     boolean isNewRit = addRegionsInTransition(region, OPEN);
3062     if (!isNewRit) {
3063       // An open is in progress. This is supported, but let's log this.
3064       LOG.info("Receiving OPEN for the region:" +
3065           region.getRegionNameAsString() + " , which we are already trying to OPEN" +
3066           " - ignoring this new request for this region.");
3067       return RegionOpeningState.OPENED;
3068     }
3069     try {
3070       LOG.info("Received request to open region: " +
3071         region.getRegionNameAsString());
3072       HTableDescriptor htd = null;
3073       if (htds == null) {
3074         htd = this.tableDescriptors.get(region.getTableName());
3075       } else {
3076         htd = htds.get(region.getTableNameAsString());
3077         if (htd == null) {
3078           htd = this.tableDescriptors.get(region.getTableName());
3079           htds.put(region.getTableNameAsString(), htd);
3080         }
3081       }
3082 
3083       // Mark the region as OPENING up in zk.  This is how we tell the master control of the
3084       // region has passed to this regionserver.
3085       int version = transitionZookeeperOfflineToOpening(region, versionOfOfflineNode);
3086       // Need to pass the expected version in the constructor.
3087       if (region.isRootRegion()) {
3088         this.service.submit(new OpenRootHandler(this, this, region, htd, version));
3089       } else if (region.isMetaRegion()) {
3090         this.service.submit(new OpenMetaHandler(this, this, region, htd, version));
3091       } else {
3092         this.service.submit(new OpenRegionHandler(this, this, region, htd, version));
3093       }
3094     } catch (IOException ie) {
3095       // Clear from this server's RIT list else will stick around for ever.
3096       removeFromRegionsInTransition(region);
3097       throw ie;
3098     }
3099     return RegionOpeningState.OPENED;
3100   }
3101 
3102   /**
3103    * Transition ZK node from OFFLINE to OPENING. The master will get a callback
3104    * and will know that the region is now ours.
3105    *
3106    * @param hri
3107    *          HRegionInfo whose znode we are updating
3108    * @param versionOfOfflineNode
3109    *          Version Of OfflineNode that needs to be compared before changing
3110    *          the node's state from OFFLINE
3111    * @throws IOException
3112    */
3113   int transitionZookeeperOfflineToOpening(final HRegionInfo hri, int versionOfOfflineNode)
3114       throws IOException {
3115     // TODO: should also handle transition from CLOSED?
3116     int version = -1;
3117     try {
3118       // Initialize the znode version.
3119       version = ZKAssign.transitionNode(this.zooKeeper, hri, this.getServerName(),
3120           EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_OPENING, versionOfOfflineNode);
3121     } catch (KeeperException e) {
3122       LOG.error("Error transition from OFFLINE to OPENING for region=" + hri.getEncodedName(), e);
3123     }
3124     if (version == -1) {
3125       // TODO: Fix this sloppyness. The exception should be coming off zk
3126       // directly, not an
3127       // intepretation at this high-level (-1 when we call transitionNode can
3128       // mean many things).
3129       throw new IOException("Failed transition from OFFLINE to OPENING for region="
3130           + hri.getEncodedName());
3131     }
3132     return version;
3133   }
3134 
3135    /**
3136     * String currentAction) throws RegionAlreadyInTransitionException { Add
3137     * region to this regionservers list of in transitions regions ONLY if its not
3138     * already byte[] encodedName = region.getEncodedNameAsBytes(); in transition.
3139     * If a region already in RIT, we throw
3140     * {@link RegionAlreadyInTransitionException}. if
3141     * (this.regionsInTransitionInRS.containsKey(encodedName)) { Callers need to
3142     * call {@link #removeFromRegionsInTransition(HRegionInfo)} when done or if
3143     * boolean openAction = this.regionsInTransitionInRS.get(encodedName); error
3144     * processing.
3145     *
3146     * @param region
3147     *          Region to add
3148     * @param currentAction
3149     *          Whether OPEN or CLOSE.
3150     * @throws RegionAlreadyInTransitionException
3151     */
3152    protected boolean addRegionsInTransition(final HRegionInfo region, final String currentAction)
3153        throws RegionAlreadyInTransitionException {
3154      boolean isOpen = currentAction.equals(OPEN);
3155      Boolean action = this.regionsInTransitionInRS.putIfAbsent(
3156          region.getEncodedNameAsBytes(), isOpen);
3157      if (action == null) return true;
3158      if (isOpen && action.booleanValue()) {
3159        return false;
3160      }
3161      // The below exception message will be used in master.
3162      throw new RegionAlreadyInTransitionException("Received:" + currentAction
3163          + " for the region:" + region.getRegionNameAsString()
3164          + ", which we are already trying to " + (action ? OPEN : CLOSE) + ".");
3165    }
3166 
3167   @Override
3168   @QosPriority(priority=HConstants.HIGH_QOS)
3169   public void openRegions(List<HRegionInfo> regions)
3170   throws IOException {
3171     checkOpen();
3172     LOG.info("Received request to open " + regions.size() + " region(s)");
3173     Map<String, HTableDescriptor> htds = new HashMap<String, HTableDescriptor>(regions.size());
3174     for (HRegionInfo region : regions) openRegion(region, -1, htds);
3175   }
3176 
3177   @Override
3178   @QosPriority(priority=HConstants.HIGH_QOS)
3179   public boolean closeRegion(HRegionInfo region)
3180   throws IOException {
3181     return closeRegion(region, true, -1);
3182   }
3183 
3184   @Override
3185   @QosPriority(priority=HConstants.HIGH_QOS)
3186   public boolean closeRegion(final HRegionInfo region,
3187     final int versionOfClosingNode)
3188   throws IOException {
3189     return closeRegion(region, true, versionOfClosingNode);
3190   }
3191 
3192   @Override
3193   @QosPriority(priority=HConstants.HIGH_QOS)
3194   public boolean closeRegion(HRegionInfo region, final boolean zk)
3195   throws IOException {
3196     return closeRegion(region, zk, -1);
3197   }
3198 
3199   @QosPriority(priority=HConstants.HIGH_QOS)
3200   protected boolean closeRegion(HRegionInfo region, final boolean zk,
3201     final int versionOfClosingNode)
3202   throws IOException {
3203     checkOpen();
3204     //Check for permissions to close.
3205     HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
3206     if (actualRegion != null && actualRegion.getCoprocessorHost() != null) {
3207       actualRegion.getCoprocessorHost().preClose(false);
3208     }
3209     LOG.info("Received close region: " + region.getRegionNameAsString() +
3210       ". Version of ZK closing node:" + versionOfClosingNode);
3211     boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
3212     if (!hasit) {
3213       LOG.warn("Received close for region we are not serving; " +
3214         region.getEncodedName());
3215       throw new NotServingRegionException("Received close for "
3216         + region.getRegionNameAsString() + " but we are not serving it");
3217     }
3218     return closeRegion(region, false, zk, versionOfClosingNode);
3219   }
3220 
3221   @Override
3222   @QosPriority(priority=HConstants.HIGH_QOS)
3223   public boolean closeRegion(byte[] encodedRegionName, boolean zk)
3224     throws IOException {
3225     return closeRegion(encodedRegionName, false, zk);
3226   }
3227 
3228   /**
3229    * @param region Region to close
3230    * @param abort True if we are aborting
3231    * @param zk True if we are to update zk about the region close; if the close
3232    * was orchestrated by master, then update zk.  If the close is being run by
3233    * the regionserver because its going down, don't update zk.
3234    * @return True if closed a region.
3235    */
3236   protected boolean closeRegion(HRegionInfo region, final boolean abort,
3237       final boolean zk) {
3238     return closeRegion(region, abort, zk, -1);
3239   }
3240 
3241 
3242   /**
3243    * @param region Region to close
3244    * @param abort True if we are aborting
3245    * @param zk True if we are to update zk about the region close; if the close
3246    * was orchestrated by master, then update zk.  If the close is being run by
3247    * the regionserver because its going down, don't update zk.
3248    * @param versionOfClosingNode
3249    *   the version of znode to compare when RS transitions the znode from
3250    *   CLOSING state.
3251    * @return True if closed a region.
3252    */
3253   protected boolean closeRegion(HRegionInfo region, final boolean abort,
3254       final boolean zk, final int versionOfClosingNode) {
3255     
3256     HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
3257     if (actualRegion != null && actualRegion.getCoprocessorHost() != null) {
3258       try {
3259         actualRegion.getCoprocessorHost().preClose(abort);
3260       } catch (IOException e) {
3261         LOG.warn(e);
3262         return false;
3263       }
3264     }
3265     try {
3266       addRegionsInTransition(region, CLOSE);
3267     } catch (RegionAlreadyInTransitionException rate) {
3268       LOG.warn("Received close for region we are already opening or closing; "
3269           + region.getEncodedName());
3270       return false;
3271     }
3272     boolean success = false;
3273     try {
3274       CloseRegionHandler crh = null;
3275       if (region.isRootRegion()) {
3276         crh = new CloseRootHandler(this, this, region, abort, zk, versionOfClosingNode);
3277       } else if (region.isMetaRegion()) {
3278         crh = new CloseMetaHandler(this, this, region, abort, zk, versionOfClosingNode);
3279       } else {
3280         crh = new CloseRegionHandler(this, this, region, abort, zk, versionOfClosingNode);
3281       }
3282       this.service.submit(crh);
3283       success = true;
3284     } finally {
3285       // Remove from this server's RIT.
3286       if (!success) removeFromRegionsInTransition(region);
3287     }
3288     return true;
3289   }
3290 
3291   /**
3292    * @param encodedRegionName
3293    *          encodedregionName to close
3294    * @param abort
3295    *          True if we are aborting
3296    * @param zk
3297    *          True if we are to update zk about the region close; if the close
3298    *          was orchestrated by master, then update zk. If the close is being
3299    *          run by the regionserver because its going down, don't update zk.
3300    * @return True if closed a region.
3301    */
3302   protected boolean closeRegion(byte[] encodedRegionName, final boolean abort,
3303       final boolean zk) throws IOException {
3304     String encodedRegionNameStr = Bytes.toString(encodedRegionName);
3305     HRegion region = this.getFromOnlineRegions(encodedRegionNameStr);
3306     if (null != region) {
3307       return closeRegion(region.getRegionInfo(), abort, zk);
3308     }
3309     LOG.error("The specified region name" + encodedRegionNameStr
3310         + " does not exist to close the region.");
3311     return false;
3312   }
3313 
3314   // Manual remote region administration RPCs
3315 
3316   @Override
3317   @QosPriority(priority=HConstants.HIGH_QOS)
3318   public void flushRegion(HRegionInfo regionInfo)
3319       throws NotServingRegionException, IOException {
3320     checkOpen();
3321     LOG.info("Flushing " + regionInfo.getRegionNameAsString());
3322     HRegion region = getRegion(regionInfo.getRegionName());
3323     boolean needsCompaction = region.flushcache();
3324     if (needsCompaction) {
3325       this.compactSplitThread.requestCompaction(region, "Compaction through user triggered flush");
3326     }
3327   }
3328 
3329   @Override
3330   @QosPriority(priority=HConstants.HIGH_QOS)
3331   public void splitRegion(HRegionInfo regionInfo)
3332       throws NotServingRegionException, IOException {
3333     splitRegion(regionInfo, null);
3334   }
3335 
3336   @Override
3337   public void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
3338       throws NotServingRegionException, IOException {
3339     checkOpen();
3340     HRegion region = getRegion(regionInfo.getRegionName());
3341     region.flushcache();
3342     region.forceSplit(splitPoint);
3343     compactSplitThread.requestSplit(region, region.checkSplit());
3344   }
3345 
3346   @Override
3347   @QosPriority(priority=HConstants.HIGH_QOS)
3348   public void compactRegion(HRegionInfo regionInfo, boolean major)
3349       throws NotServingRegionException, IOException {
3350     compactRegion(regionInfo, major, null);
3351   }
3352 
3353   @Override
3354   @QosPriority(priority=HConstants.HIGH_QOS)
3355   public void compactRegion(HRegionInfo regionInfo, boolean major,  byte[] family)
3356       throws NotServingRegionException, IOException {
3357     checkOpen();
3358     HRegion region = getRegion(regionInfo.getRegionName());
3359     Store store = null;
3360     if (family != null) {
3361       store = region.getStore(family);
3362       if (store == null) {
3363         throw new IOException("column family " + Bytes.toString(family) +
3364           " does not exist in region " + new String(region.getRegionNameAsString()));
3365       }
3366     }
3367 
3368     if (major) {
3369       if (family != null) {
3370         store.triggerMajorCompaction();
3371       } else {
3372         region.triggerMajorCompaction();
3373       }
3374     }
3375     String familyLogMsg = (family != null)?" for column family: " + Bytes.toString(family):"";
3376     LOG.trace("User-triggered compaction requested for region " +
3377       region.getRegionNameAsString() + familyLogMsg);
3378     String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
3379     if (family != null) {
3380       compactSplitThread.requestCompaction(region, store, log,
3381         Store.PRIORITY_USER, null);
3382     } else {
3383       compactSplitThread.requestCompaction(region, log,
3384         Store.PRIORITY_USER, null);
3385     }
3386   }
3387 
3388   /** @return the info server */
3389   public InfoServer getInfoServer() {
3390     return infoServer;
3391   }
3392 
3393   /**
3394    * @return true if a stop has been requested.
3395    */
3396   public boolean isStopped() {
3397     return this.stopped;
3398   }
3399 
3400   @Override
3401   public boolean isStopping() {
3402     return this.stopping;
3403   }
3404 
3405   /**
3406    *
3407    * @return the configuration
3408    */
3409   public Configuration getConfiguration() {
3410     return conf;
3411   }
3412 
3413   /** @return the write lock for the server */
3414   ReentrantReadWriteLock.WriteLock getWriteLock() {
3415     return lock.writeLock();
3416   }
3417 
3418   @Override
3419   @QosPriority(priority=HConstants.HIGH_QOS)
3420   public List<HRegionInfo> getOnlineRegions() throws IOException {
3421     checkOpen();
3422     List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
3423     for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
3424       list.add(e.getValue().getRegionInfo());
3425     }
3426     Collections.sort(list);
3427     return list;
3428   }
3429 
3430   public int getNumberOfOnlineRegions() {
3431     return this.onlineRegions.size();
3432   }
3433 
3434   boolean isOnlineRegionsEmpty() {
3435     return this.onlineRegions.isEmpty();
3436   }
3437 
3438   /**
3439    * @param encodedRegionName
3440    * @return JSON Map of labels to values for passed in <code>encodedRegionName</code>
3441    * @throws IOException
3442    */
3443   public byte [] getRegionStats(final String encodedRegionName)
3444   throws IOException {
3445     HRegion r = null;
3446     synchronized (this.onlineRegions) {
3447       r = this.onlineRegions.get(encodedRegionName);
3448     }
3449     if (r == null) return null;
3450     ObjectMapper mapper = new ObjectMapper();
3451     int stores = 0;
3452     int storefiles = 0;
3453     int storefileSizeMB = 0;
3454     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
3455     int storefileIndexSizeMB = 0;
3456     long totalCompactingKVs = 0;
3457     long currentCompactedKVs = 0;
3458     synchronized (r.stores) {
3459       stores += r.stores.size();
3460       for (Store store : r.stores.values()) {
3461         storefiles += store.getStorefilesCount();
3462         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
3463         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
3464       }
3465     }
3466     Map<String, Integer> map = new TreeMap<String, Integer>();
3467     map.put("stores", stores);
3468     map.put("storefiles", storefiles);
3469     map.put("storefileSizeMB", storefileIndexSizeMB);
3470     map.put("memstoreSizeMB", memstoreSizeMB);
3471     StringWriter w = new StringWriter();
3472     mapper.writeValue(w, map);
3473     w.close();
3474     return Bytes.toBytes(w.toString());
3475   }
3476 
3477   /**
3478    * For tests and web ui.
3479    * This method will only work if HRegionServer is in the same JVM as client;
3480    * HRegion cannot be serialized to cross an rpc.
3481    * @see #getOnlineRegions()
3482    */
3483   public Collection<HRegion> getOnlineRegionsLocalContext() {
3484     Collection<HRegion> regions = this.onlineRegions.values();
3485     return Collections.unmodifiableCollection(regions);
3486   }
3487 
3488   @Override
3489   public void addToOnlineRegions(HRegion region) {
3490     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
3491   }
3492 
3493   @Override
3494   public boolean removeFromOnlineRegions(final String encodedName) {
3495     HRegion toReturn = null;
3496     toReturn = this.onlineRegions.remove(encodedName);
3497     
3498     //Clear all of the dynamic metrics as they are now probably useless.
3499     //This is a clear because dynamic metrics could include metrics per cf and
3500     //per hfile.  Figuring out which cfs, hfiles, and regions are still relevant to
3501     //this region server would be an onerous task.  Instead just clear everything
3502     //and on the next tick of the metrics everything that is still relevant will be
3503     //re-added.
3504     this.dynamicMetrics.clear();
3505     return toReturn != null;
3506   }
3507 
3508   /**
3509    * @return A new Map of online regions sorted by region size with the first
3510    *         entry being the biggest.
3511    */
3512   public SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
3513     // we'll sort the regions in reverse
3514     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
3515         new Comparator<Long>() {
3516           public int compare(Long a, Long b) {
3517             return -1 * a.compareTo(b);
3518           }
3519         });
3520     // Copy over all regions. Regions are sorted by size with biggest first.
3521     for (HRegion region : this.onlineRegions.values()) {
3522       sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
3523     }
3524     return sortedRegions;
3525   }
3526 
3527   @Override
3528   public HRegion getFromOnlineRegions(final String encodedRegionName) {
3529     HRegion r = null;
3530     r = this.onlineRegions.get(encodedRegionName);
3531     return r;
3532   }
3533 
3534   /**
3535    * @param regionName
3536    * @return HRegion for the passed binary <code>regionName</code> or null if
3537    *         named region is not member of the online regions.
3538    */
3539   public HRegion getOnlineRegion(final byte[] regionName) {
3540     return getFromOnlineRegions(HRegionInfo.encodeRegionName(regionName));
3541   }
3542 
3543   /** @return the request count */
3544   public AtomicInteger getRequestCount() {
3545     return this.requestCount;
3546   }
3547 
3548   /**
3549    * @return time stamp in millis of when this region server was started
3550    */
3551   public long getStartcode() {
3552     return this.startcode;
3553   }
3554 
3555   /** @return reference to FlushRequester */
3556   public FlushRequester getFlushRequester() {
3557     return this.cacheFlusher;
3558   }
3559 
3560   /**
3561    * Protected utility method for safely obtaining an HRegion handle.
3562    *
3563    * @param regionName
3564    *          Name of online {@link HRegion} to return
3565    * @return {@link HRegion} for <code>regionName</code>
3566    * @throws NotServingRegionException
3567    */
3568   protected HRegion getRegion(final byte[] regionName)
3569       throws NotServingRegionException {
3570     HRegion region = null;
3571     region = getOnlineRegion(regionName);
3572     if (region == null) {
3573       throw new NotServingRegionException("Region is not online: " +
3574         Bytes.toStringBinary(regionName));
3575     }
3576     return region;
3577   }
3578 
3579   /**
3580    * Get the top N most loaded regions this server is serving so we can tell the
3581    * master which regions it can reallocate if we're overloaded. TODO: actually
3582    * calculate which regions are most loaded. (Right now, we're just grabbing
3583    * the first N regions being served regardless of load.)
3584    */
3585   protected HRegionInfo[] getMostLoadedRegions() {
3586     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
3587     for (HRegion r : onlineRegions.values()) {
3588       if (!r.isAvailable()) {
3589         continue;
3590       }
3591       if (regions.size() < numRegionsToReport) {
3592         regions.add(r.getRegionInfo());
3593       } else {
3594         break;
3595       }
3596     }
3597     return regions.toArray(new HRegionInfo[regions.size()]);
3598   }
3599 
3600   /**
3601    * Called to verify that this server is up and running.
3602    *
3603    * @throws IOException
3604    */
3605   protected void checkOpen() throws IOException {
3606     if (this.stopped || this.abortRequested) {
3607       throw new RegionServerStoppedException("Server " + getServerName() +
3608         " not running" + (this.abortRequested ? ", aborting" : ""));
3609     }
3610     if (!fsOk) {
3611       throw new RegionServerStoppedException("File system not available");
3612     }
3613   }
3614 
3615   @Override
3616   @QosPriority(priority=HConstants.HIGH_QOS)
3617   public ProtocolSignature getProtocolSignature(
3618       String protocol, long version, int clientMethodsHashCode)
3619   throws IOException {
3620     if (protocol.equals(HRegionInterface.class.getName())) {
3621       return new ProtocolSignature(HRegionInterface.VERSION, null);
3622     }
3623     throw new IOException("Unknown protocol: " + protocol);
3624   }
3625 
3626   @Override
3627   @QosPriority(priority=HConstants.HIGH_QOS)
3628   public long getProtocolVersion(final String protocol, final long clientVersion)
3629   throws IOException {
3630     if (protocol.equals(HRegionInterface.class.getName())) {
3631       return HRegionInterface.VERSION;
3632     }
3633     throw new IOException("Unknown protocol: " + protocol);
3634   }
3635 
3636   @Override
3637   public Leases getLeases() {
3638     return leases;
3639   }
3640 
3641   /**
3642    * @return Return the rootDir.
3643    */
3644   protected Path getRootDir() {
3645     return rootDir;
3646   }
3647 
3648   /**
3649    * @return Return the fs.
3650    */
3651   public FileSystem getFileSystem() {
3652     return fs;
3653   }
3654 
3655   /**
3656    * @return This servers {@link HServerInfo}
3657    */
3658   // TODO: Deprecate and do getServerName instead.
3659   public HServerInfo getServerInfo() {
3660     try {
3661       return getHServerInfo();
3662     } catch (IOException e) {
3663       e.printStackTrace();
3664     }
3665     return null;
3666   }
3667 
3668   @Override
3669   public void mutateRow(byte[] regionName, RowMutations rm)
3670       throws IOException {
3671     checkOpen();
3672     if (regionName == null) {
3673       throw new IOException("Invalid arguments to mutateRow " +
3674       "regionName is null");
3675     }
3676     requestCount.incrementAndGet();
3677     try {
3678       HRegion region = getRegion(regionName);
3679       if (!region.getRegionInfo().isMetaTable()) {
3680         this.cacheFlusher.reclaimMemStoreMemory();
3681       }
3682       region.mutateRow(rm);
3683     } catch (IOException e) {
3684       checkFileSystem();
3685       throw e;
3686     }
3687   }
3688 
3689   @Override
3690   public Result append(byte[] regionName, Append append)
3691   throws IOException {
3692     checkOpen();
3693     if (regionName == null) {
3694       throw new IOException("Invalid arguments to increment " +
3695       "regionName is null");
3696     }
3697     requestCount.incrementAndGet();
3698     try {
3699       HRegion region = getRegion(regionName);
3700       Integer lock = getLockFromId(append.getLockId());
3701       Append appVal = append;
3702       Result resVal;
3703       if (region.getCoprocessorHost() != null) {
3704         resVal = region.getCoprocessorHost().preAppend(appVal);
3705         if (resVal != null) {
3706           return resVal;
3707         }
3708       }
3709       resVal = region.append(appVal, lock, append.getWriteToWAL());
3710       if (region.getCoprocessorHost() != null) {
3711         region.getCoprocessorHost().postAppend(appVal, resVal);
3712       }
3713       return resVal;
3714     } catch (IOException e) {
3715       checkFileSystem();
3716       throw e;
3717     }
3718   }
3719 
3720   @Override
3721   public Result increment(byte[] regionName, Increment increment)
3722   throws IOException {
3723     checkOpen();
3724     if (regionName == null) {
3725       throw new IOException("Invalid arguments to increment " +
3726       "regionName is null");
3727     }
3728     requestCount.incrementAndGet();
3729     try {
3730       HRegion region = getRegion(regionName);
3731       Integer lock = getLockFromId(increment.getLockId());
3732       Increment incVal = increment;
3733       Result resVal;
3734       if (region.getCoprocessorHost() != null) {
3735         resVal = region.getCoprocessorHost().preIncrement(incVal);
3736         if (resVal != null) {
3737           return resVal;
3738         }
3739       }
3740       resVal = region.increment(incVal, lock,
3741           increment.getWriteToWAL());
3742       if (region.getCoprocessorHost() != null) {
3743         resVal = region.getCoprocessorHost().postIncrement(incVal, resVal);
3744       }
3745       return resVal;
3746     } catch (IOException e) {
3747       checkFileSystem();
3748       throw e;
3749     }
3750   }
3751 
3752   /** {@inheritDoc} */
3753   public long incrementColumnValue(byte[] regionName, byte[] row,
3754       byte[] family, byte[] qualifier, long amount, boolean writeToWAL)
3755       throws IOException {
3756     checkOpen();
3757 
3758     if (regionName == null) {
3759       throw new IOException("Invalid arguments to incrementColumnValue "
3760           + "regionName is null");
3761     }
3762     requestCount.incrementAndGet();
3763     try {
3764       HRegion region = getRegion(regionName);
3765       if (region.getCoprocessorHost() != null) {
3766         Long amountVal = region.getCoprocessorHost().preIncrementColumnValue(row,
3767           family, qualifier, amount, writeToWAL);
3768         if (amountVal != null) {
3769           return amountVal.longValue();
3770         }
3771       }
3772       long retval = region.incrementColumnValue(row, family, qualifier, amount,
3773         writeToWAL);
3774       if (region.getCoprocessorHost() != null) {
3775         retval = region.getCoprocessorHost().postIncrementColumnValue(row,
3776           family, qualifier, amount, writeToWAL, retval);
3777       }
3778       return retval;
3779     } catch (IOException e) {
3780       checkFileSystem();
3781       throw e;
3782     }
3783   }
3784 
3785   /** {@inheritDoc}
3786    * @deprecated Use {@link #getServerName()} instead.
3787    */
3788   @Override
3789   @QosPriority(priority=HConstants.HIGH_QOS)
3790   public HServerInfo getHServerInfo() throws IOException {
3791     checkOpen();
3792     return new HServerInfo(new HServerAddress(this.isa),
3793       this.startcode, this.webuiport);
3794   }
3795 
3796   @SuppressWarnings("unchecked")
3797   @Override
3798   public <R> MultiResponse multi(MultiAction<R> multi) throws IOException {
3799     checkOpen();
3800     MultiResponse response = new MultiResponse();
3801     for (Map.Entry<byte[], List<Action<R>>> e : multi.actions.entrySet()) {
3802       byte[] regionName = e.getKey();
3803       List<Action<R>> actionsForRegion = e.getValue();
3804       // sort based on the row id - this helps in the case where we reach the
3805       // end of a region, so that we don't have to try the rest of the
3806       // actions in the list.
3807       Collections.sort(actionsForRegion);
3808       Row action;
3809       List<Action<R>> mutations = new ArrayList<Action<R>>();
3810       for (Action<R> a : actionsForRegion) {
3811         action = a.getAction();
3812         int originalIndex = a.getOriginalIndex();
3813 
3814         try {
3815           if (action instanceof Delete || action instanceof Put) {
3816             mutations.add(a); 
3817           } else if (action instanceof Get) {
3818             response.add(regionName, originalIndex,
3819                 get(regionName, (Get)action));
3820           } else if (action instanceof Exec) {
3821             ExecResult result = execCoprocessor(regionName, (Exec)action);
3822             response.add(regionName, new Pair<Integer, Object>(
3823                 a.getOriginalIndex(), result.getValue()
3824             ));
3825           } else if (action instanceof Increment) {
3826             response.add(regionName, originalIndex,
3827                 increment(regionName, (Increment)action));
3828           } else if (action instanceof Append) {
3829             response.add(regionName, originalIndex,
3830                 append(regionName, (Append)action));
3831           } else if (action instanceof RowMutations) {
3832             mutateRow(regionName, (RowMutations)action);
3833             response.add(regionName, originalIndex, new Result());
3834           } else {
3835             LOG.debug("Error: invalid Action, row must be a Get, Delete, " +
3836                 "Put, Exec, Increment, or Append.");
3837             throw new DoNotRetryIOException("Invalid Action, row must be a " +
3838                 "Get, Delete, Put, Exec, Increment, or Append.");
3839           }
3840         } catch (IOException ex) {
3841           response.add(regionName, originalIndex, ex);
3842         }
3843       }
3844 
3845       // We do the puts with result.put so we can get the batching efficiency
3846       // we so need. All this data munging doesn't seem great, but at least
3847       // we arent copying bytes or anything.
3848       if (!mutations.isEmpty()) {
3849         try {
3850           HRegion region = getRegion(regionName);
3851 
3852           if (!region.getRegionInfo().isMetaTable()) {
3853             this.cacheFlusher.reclaimMemStoreMemory();
3854           }
3855 
3856           List<Pair<Mutation,Integer>> mutationsWithLocks =
3857               Lists.newArrayListWithCapacity(mutations.size());
3858           for (Action<R> a : mutations) {
3859             Mutation m = (Mutation) a.getAction();
3860 
3861             Integer lock;
3862             try {
3863               lock = getLockFromId(m.getLockId());
3864             } catch (UnknownRowLockException ex) {
3865               response.add(regionName, a.getOriginalIndex(), ex);
3866               continue;
3867             }
3868             mutationsWithLocks.add(new Pair<Mutation, Integer>(m, lock));
3869           }
3870 
3871           this.requestCount.addAndGet(mutations.size());
3872 
3873           OperationStatus[] codes =
3874               region.batchMutate(mutationsWithLocks.toArray(new Pair[]{}));
3875 
3876           for( int i = 0 ; i < codes.length ; i++) {
3877             OperationStatus code = codes[i];
3878 
3879             Action<R> theAction = mutations.get(i);
3880             Object result = null;
3881 
3882             if (code.getOperationStatusCode() == OperationStatusCode.SUCCESS) {
3883               result = new Result();
3884             } else if (code.getOperationStatusCode()
3885                 == OperationStatusCode.SANITY_CHECK_FAILURE) {
3886               // Don't send a FailedSanityCheckException as older clients will not know about
3887               // that class being a subclass of DoNotRetryIOException
3888               // and will retry mutations that will never succeed.
3889               result = new DoNotRetryIOException(code.getExceptionMsg());
3890             } else if (code.getOperationStatusCode() == OperationStatusCode.BAD_FAMILY) {
3891               result = new NoSuchColumnFamilyException(code.getExceptionMsg());
3892             }
3893             // FAILURE && NOT_RUN becomes null, aka: need to run again.
3894 
3895             response.add(regionName, theAction.getOriginalIndex(), result);
3896           }
3897         } catch (IOException ioe) {
3898           // fail all the puts with the ioe in question.
3899           for (Action<R> a: mutations) {
3900             response.add(regionName, a.getOriginalIndex(), ioe);
3901           }
3902         }
3903       }
3904     }
3905     return response;
3906   }
3907 
3908   /**
3909    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
3910    * method using the registered protocol handlers.
3911    * {@link CoprocessorProtocol} implementations must be registered per-region
3912    * via the
3913    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
3914    * method before they are available.
3915    *
3916    * @param regionName name of the region against which the invocation is executed
3917    * @param call an {@code Exec} instance identifying the protocol, method name,
3918    *     and parameters for the method invocation
3919    * @return an {@code ExecResult} instance containing the region name of the
3920    *     invocation and the return value
3921    * @throws IOException if no registered protocol handler is found or an error
3922    *     occurs during the invocation
3923    * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
3924    */
3925   @Override
3926   public ExecResult execCoprocessor(byte[] regionName, Exec call)
3927       throws IOException {
3928     checkOpen();
3929     requestCount.incrementAndGet();
3930     try {
3931       HRegion region = getRegion(regionName);
3932       return region.exec(call);
3933     } catch (Throwable t) {
3934       throw convertThrowableToIOE(cleanup(t));
3935     }
3936   }
3937 
3938   public String toString() {
3939     return getServerName().toString();
3940   }
3941 
3942   /**
3943    * Interval at which threads should run
3944    *
3945    * @return the interval
3946    */
3947   public int getThreadWakeFrequency() {
3948     return threadWakeFrequency;
3949   }
3950 
3951   @Override
3952   public ZooKeeperWatcher getZooKeeper() {
3953     return zooKeeper;
3954   }
3955 
3956   @Override
3957   public ServerName getServerName() {
3958     // Our servername could change after we talk to the master.
3959     return this.serverNameFromMasterPOV == null?
3960       new ServerName(this.isa.getHostName(), this.isa.getPort(), this.startcode):
3961         this.serverNameFromMasterPOV;
3962   }
3963 
3964   @Override
3965   public CompactionRequestor getCompactionRequester() {
3966     return this.compactSplitThread;
3967   }
3968 
3969   public ZooKeeperWatcher getZooKeeperWatcher() {
3970     return this.zooKeeper;
3971   }
3972 
3973   public RegionServerCoprocessorHost getCoprocessorHost(){
3974     return this.rsHost;
3975   }
3976 
3977   @Override
3978   public boolean removeFromRegionsInTransition(final HRegionInfo hri) {
3979     Boolean res = this.regionsInTransitionInRS.remove(hri.getEncodedNameAsBytes());
3980     return res != null && res.booleanValue();
3981   }
3982 
3983   @Override
3984   public boolean containsKeyInRegionsInTransition(final HRegionInfo hri) {
3985     return this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes());
3986   }
3987 
3988   public ExecutorService getExecutorService() {
3989     return service;
3990   }
3991 
3992   //
3993   // Main program and support routines
3994   //
3995 
3996   /**
3997    * Load the replication service objects, if any
3998    */
3999   static private void createNewReplicationInstance(Configuration conf,
4000     HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
4001 
4002     // If replication is not enabled, then return immediately.
4003     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
4004       return;
4005     }
4006 
4007     // read in the name of the source replication class from the config file.
4008     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
4009                                HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
4010 
4011     // read in the name of the sink replication class from the config file.
4012     String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
4013                              HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
4014 
4015     // If both the sink and the source class names are the same, then instantiate
4016     // only one object.
4017     if (sourceClassname.equals(sinkClassname)) {
4018       server.replicationSourceHandler = (ReplicationSourceService)
4019                                          newReplicationInstance(sourceClassname,
4020                                          conf, server, fs, logDir, oldLogDir);
4021       server.replicationSinkHandler = (ReplicationSinkService)
4022                                          server.replicationSourceHandler;
4023     }
4024     else {
4025       server.replicationSourceHandler = (ReplicationSourceService)
4026                                          newReplicationInstance(sourceClassname,
4027                                          conf, server, fs, logDir, oldLogDir);
4028       server.replicationSinkHandler = (ReplicationSinkService)
4029                                          newReplicationInstance(sinkClassname,
4030                                          conf, server, fs, logDir, oldLogDir);
4031     }
4032   }
4033 
4034   static private ReplicationService newReplicationInstance(String classname,
4035     Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
4036     Path oldLogDir) throws IOException{
4037 
4038     Class<?> clazz = null;
4039     try {
4040       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
4041       clazz = Class.forName(classname, true, classLoader);
4042     } catch (java.lang.ClassNotFoundException nfe) {
4043       throw new IOException("Cound not find class for " + classname);
4044     }
4045 
4046     // create an instance of the replication object.
4047     ReplicationService service = (ReplicationService)
4048                               ReflectionUtils.newInstance(clazz, conf);
4049     service.initialize(server, fs, logDir, oldLogDir);
4050     return service;
4051   }
4052 
4053   /**
4054    * @param hrs
4055    * @return Thread the RegionServer is running in correctly named.
4056    * @throws IOException
4057    */
4058   public static Thread startRegionServer(final HRegionServer hrs)
4059       throws IOException {
4060     return startRegionServer(hrs, "regionserver" + hrs.isa.getPort());
4061   }
4062 
4063   /**
4064    * @param hrs
4065    * @param name
4066    * @return Thread the RegionServer is running in correctly named.
4067    * @throws IOException
4068    */
4069   public static Thread startRegionServer(final HRegionServer hrs,
4070       final String name) throws IOException {
4071     Thread t = new Thread(hrs);
4072     t.setName(name);
4073     t.start();
4074     // Install shutdown hook that will catch signals and run an orderly shutdown
4075     // of the hrs.
4076     ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
4077         .getConfiguration()), hrs, t);
4078     return t;
4079   }
4080 
4081   /**
4082    * Utility for constructing an instance of the passed HRegionServer class.
4083    *
4084    * @param regionServerClass
4085    * @param conf2
4086    * @return HRegionServer instance.
4087    */
4088   public static HRegionServer constructRegionServer(
4089       Class<? extends HRegionServer> regionServerClass,
4090       final Configuration conf2) {
4091     try {
4092       Constructor<? extends HRegionServer> c = regionServerClass
4093           .getConstructor(Configuration.class);
4094       return c.newInstance(conf2);
4095     } catch (Exception e) {
4096       throw new RuntimeException("Failed construction of " + "Regionserver: "
4097           + regionServerClass.toString(), e);
4098     }
4099   }
4100 
4101   @Override
4102   @QosPriority(priority=HConstants.REPLICATION_QOS)
4103   public void replicateLogEntries(final HLog.Entry[] entries)
4104   throws IOException {
4105     checkOpen();
4106     if (this.replicationSinkHandler == null) return;
4107     this.replicationSinkHandler.replicateLogEntries(entries);
4108   }
4109 
4110   /**
4111    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
4112    */
4113   public static void main(String[] args) throws Exception {
4114 	VersionInfo.logVersion();
4115     Configuration conf = HBaseConfiguration.create();
4116     @SuppressWarnings("unchecked")
4117     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
4118         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
4119 
4120     new HRegionServerCommandLine(regionServerClass).doMain(args);
4121   }
4122 
4123   @Override
4124   public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries() throws IOException {
4125     BlockCache c = new CacheConfig(this.conf).getBlockCache();
4126     return c.getBlockCacheColumnFamilySummaries(this.conf);
4127   }
4128 
4129   @Override
4130   public byte[][] rollHLogWriter() throws IOException, FailedLogCloseException {
4131     HLog wal = this.getWAL();
4132     return wal.rollWriter(true);
4133   }
4134 
4135   /**
4136    * Gets the online regions of the specified table.
4137    * This method looks at the in-memory onlineRegions.  It does not go to <code>.META.</code>.
4138    * Only returns <em>online</em> regions.  If a region on this table has been
4139    * closed during a disable, etc., it will not be included in the returned list.
4140    * So, the returned list may not necessarily be ALL regions in this table, its
4141    * all the ONLINE regions in the table.
4142    * @param tableName
4143    * @return Online regions from <code>tableName</code>
4144    */
4145    public List<HRegion> getOnlineRegions(byte[] tableName) {
4146      List<HRegion> tableRegions = new ArrayList<HRegion>();
4147      synchronized (this.onlineRegions) {
4148        for (HRegion region: this.onlineRegions.values()) {
4149          HRegionInfo regionInfo = region.getRegionInfo();
4150          if(Bytes.equals(regionInfo.getTableName(), tableName)) {
4151            tableRegions.add(region);
4152          }
4153        }
4154      }
4155      return tableRegions;
4156    }
4157 
4158   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
4159   public String[] getCoprocessors() {
4160     TreeSet<String> coprocessors = new TreeSet<String>(
4161         this.hlog.getCoprocessorHost().getCoprocessors());
4162     Collection<HRegion> regions = getOnlineRegionsLocalContext();
4163     for (HRegion region: regions) {
4164       coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
4165     }
4166     return coprocessors.toArray(new String[0]);
4167   }
4168 
4169   /**
4170    * Register bean with platform management server
4171    */
4172   @SuppressWarnings("deprecation")
4173   void registerMBean() {
4174     MXBeanImpl mxBeanInfo = MXBeanImpl.init(this);
4175     mxBean = MBeanUtil.registerMBean("RegionServer", "RegionServer",
4176         mxBeanInfo);
4177     LOG.info("Registered RegionServer MXBean");
4178   }
4179 
4180   /**
4181    * Get the current compaction state of the region.
4182    *
4183    * @param regionName the name of the region to check compaction statte.
4184    * @return the compaction state name.
4185    * @throws IOException exception
4186    */
4187   public String getCompactionState(final byte[] regionName) throws IOException {
4188       checkOpen();
4189       requestCount.incrementAndGet();
4190       HRegion region = getRegion(regionName);
4191       HRegionInfo info = region.getRegionInfo();
4192       return CompactionRequest.getCompactionState(info.getRegionId()).name();
4193   }
4194 
4195   public long getResponseQueueSize(){
4196     if (server != null) {
4197       return server.getResponseQueueSize();
4198     }
4199     return 0;
4200   }
4201 
4202   private boolean isHealthCheckerConfigured() {
4203     String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
4204     return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
4205   }
4206 
4207   /**
4208    * @return the underlying {@link CompactSplitThread} for the servers
4209    */
4210   public CompactSplitThread getCompactSplitThread() {
4211     return this.compactSplitThread;
4212   }
4213 }