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