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