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