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