View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.lang.Thread.UncaughtExceptionHandler;
24  import java.lang.management.ManagementFactory;
25  import java.lang.management.MemoryUsage;
26  import java.lang.reflect.Constructor;
27  import java.net.BindException;
28  import java.net.InetSocketAddress;
29  import java.util.ArrayList;
30  import java.util.Collection;
31  import java.util.Collections;
32  import java.util.Comparator;
33  import java.util.HashMap;
34  import java.util.HashSet;
35  import java.util.Iterator;
36  import java.util.List;
37  import java.util.Map;
38  import java.util.Map.Entry;
39  import java.util.Set;
40  import java.util.SortedMap;
41  import java.util.TreeMap;
42  import java.util.TreeSet;
43  import java.util.concurrent.ConcurrentHashMap;
44  import java.util.concurrent.ConcurrentMap;
45  import java.util.concurrent.ConcurrentSkipListMap;
46  import java.util.concurrent.atomic.AtomicBoolean;
47  import java.util.concurrent.locks.ReentrantReadWriteLock;
48  import java.net.InetAddress;
49  
50  import javax.management.ObjectName;
51  import javax.servlet.http.HttpServlet;
52  
53  import org.apache.commons.lang.math.RandomUtils;
54  import org.apache.commons.logging.Log;
55  import org.apache.commons.logging.LogFactory;
56  import org.apache.hadoop.conf.Configuration;
57  import org.apache.hadoop.fs.FileSystem;
58  import org.apache.hadoop.fs.Path;
59  import org.apache.hadoop.hbase.Chore;
60  import org.apache.hadoop.hbase.ClockOutOfSyncException;
61  import org.apache.hadoop.hbase.CoordinatedStateManager;
62  import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
63  import org.apache.hadoop.hbase.HBaseConfiguration;
64  import org.apache.hadoop.hbase.HConstants;
65  import org.apache.hadoop.hbase.HRegionInfo;
66  import org.apache.hadoop.hbase.HealthCheckChore;
67  import org.apache.hadoop.hbase.MetaTableAccessor;
68  import org.apache.hadoop.hbase.NotServingRegionException;
69  import org.apache.hadoop.hbase.ServerName;
70  import org.apache.hadoop.hbase.Stoppable;
71  import org.apache.hadoop.hbase.TableDescriptors;
72  import org.apache.hadoop.hbase.TableName;
73  import org.apache.hadoop.hbase.YouAreDeadException;
74  import org.apache.hadoop.hbase.ZNodeClearer;
75  import org.apache.hadoop.hbase.classification.InterfaceAudience;
76  import org.apache.hadoop.hbase.client.ConnectionUtils;
77  import org.apache.hadoop.hbase.client.HConnection;
78  import org.apache.hadoop.hbase.client.HConnectionManager;
79  import org.apache.hadoop.hbase.conf.ConfigurationManager;
80  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
81  import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
82  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
83  import org.apache.hadoop.hbase.exceptions.RegionMovedException;
84  import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
85  import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
86  import org.apache.hadoop.hbase.executor.ExecutorService;
87  import org.apache.hadoop.hbase.executor.ExecutorType;
88  import org.apache.hadoop.hbase.fs.HFileSystem;
89  import org.apache.hadoop.hbase.http.InfoServer;
90  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
91  import org.apache.hadoop.hbase.ipc.RpcClient;
92  import org.apache.hadoop.hbase.ipc.RpcServerInterface;
93  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
94  import org.apache.hadoop.hbase.ipc.ServerRpcController;
95  import org.apache.hadoop.hbase.master.HMaster;
96  import org.apache.hadoop.hbase.master.RegionState.State;
97  import org.apache.hadoop.hbase.master.TableLockManager;
98  import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
99  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
100 import org.apache.hadoop.hbase.protobuf.RequestConverter;
101 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
102 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
103 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
104 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
105 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
106 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
107 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
108 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
109 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
110 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
111 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
112 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
113 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
114 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
115 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
116 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
117 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
118 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
119 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
121 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
122 import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
123 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
124 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
125 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
126 import org.apache.hadoop.hbase.regionserver.wal.HLog;
127 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
128 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
129 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
130 import org.apache.hadoop.hbase.security.UserProvider;
131 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
132 import org.apache.hadoop.hbase.util.Addressing;
133 import org.apache.hadoop.hbase.util.ByteStringer;
134 import org.apache.hadoop.hbase.util.Bytes;
135 import org.apache.hadoop.hbase.util.CompressionTest;
136 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
137 import org.apache.hadoop.hbase.util.FSTableDescriptors;
138 import org.apache.hadoop.hbase.util.FSUtils;
139 import org.apache.hadoop.hbase.util.HasThread;
140 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
141 import org.apache.hadoop.hbase.util.Sleeper;
142 import org.apache.hadoop.hbase.util.Threads;
143 import org.apache.hadoop.hbase.util.VersionInfo;
144 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
145 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
146 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
147 import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
148 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
149 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
150 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
151 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
152 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
153 import org.apache.hadoop.ipc.RemoteException;
154 import org.apache.hadoop.metrics.util.MBeanUtil;
155 import org.apache.hadoop.util.ReflectionUtils;
156 import org.apache.hadoop.util.StringUtils;
157 import org.apache.zookeeper.KeeperException;
158 import org.apache.zookeeper.KeeperException.NoNodeException;
159 import org.apache.zookeeper.data.Stat;
160 
161 import com.google.common.annotations.VisibleForTesting;
162 import com.google.common.base.Preconditions;
163 import com.google.common.collect.Maps;
164 import com.google.protobuf.BlockingRpcChannel;
165 import com.google.protobuf.Descriptors;
166 import com.google.protobuf.Message;
167 import com.google.protobuf.RpcCallback;
168 import com.google.protobuf.RpcController;
169 import com.google.protobuf.Service;
170 import com.google.protobuf.ServiceException;
171 
172 /**
173  * HRegionServer makes a set of HRegions available to clients. It checks in with
174  * the HMaster. There are many HRegionServers in a single HBase deployment.
175  */
176 @InterfaceAudience.Private
177 @SuppressWarnings("deprecation")
178 public class HRegionServer extends HasThread implements
179     RegionServerServices, LastSequenceId {
180 
181   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
182 
183   /**
184    * For testing only!  Set to true to skip notifying region assignment to master .
185    */
186   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
187   public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
188 
189   /*
190    * Strings to be used in forming the exception message for
191    * RegionsAlreadyInTransitionException.
192    */
193   protected static final String OPEN = "OPEN";
194   protected static final String CLOSE = "CLOSE";
195 
196   //RegionName vs current action in progress
197   //true - if open region action in progress
198   //false - if close region action in progress
199   protected final ConcurrentMap<byte[], Boolean> regionsInTransitionInRS =
200     new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
201 
202   // Cache flushing
203   protected MemStoreFlusher cacheFlusher;
204 
205   protected HeapMemoryManager hMemManager;
206 
207   /*
208    * Short-circuit (ie. bypassing RPC layer) HConnection to this Server
209    * to be used internally for miscellaneous needs. Initialized at the server startup
210    * and closed when server shuts down. Clients must never close it explicitly.
211    */
212   protected HConnection shortCircuitConnection;
213 
214   /*
215    * Long-living meta table locator, which is created when the server is started and stopped
216    * when server shuts down. References to this locator shall be used to perform according
217    * operations in EventHandlers. Primary reason for this decision is to make it mockable
218    * for tests.
219    */
220   protected MetaTableLocator metaTableLocator;
221 
222   // Watch if a region is out of recovering state from ZooKeeper
223   @SuppressWarnings("unused")
224   private RecoveringRegionWatcher recoveringRegionWatcher;
225 
226   /**
227    * Go here to get table descriptors.
228    */
229   protected TableDescriptors tableDescriptors;
230 
231   // Replication services. If no replication, this handler will be null.
232   protected ReplicationSourceService replicationSourceHandler;
233   protected ReplicationSinkService replicationSinkHandler;
234 
235   // Compactions
236   public CompactSplitThread compactSplitThread;
237 
238   /**
239    * Map of regions currently being served by this region server. Key is the
240    * encoded region name.  All access should be synchronized.
241    */
242   protected final Map<String, HRegion> onlineRegions =
243     new ConcurrentHashMap<String, HRegion>();
244 
245   /**
246    * Map of encoded region names to the DataNode locations they should be hosted on
247    * We store the value as InetSocketAddress since this is used only in HDFS
248    * API (create() that takes favored nodes as hints for placing file blocks).
249    * We could have used ServerName here as the value class, but we'd need to
250    * convert it to InetSocketAddress at some point before the HDFS API call, and
251    * it seems a bit weird to store ServerName since ServerName refers to RegionServers
252    * and here we really mean DataNode locations.
253    */
254   protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap =
255       new ConcurrentHashMap<String, InetSocketAddress[]>();
256 
257   /**
258    * Set of regions currently being in recovering state which means it can accept writes(edits from
259    * previous failed region server) but not reads. A recovering region is also an online region.
260    */
261   protected final Map<String, HRegion> recoveringRegions = Collections
262       .synchronizedMap(new HashMap<String, HRegion>());
263 
264   // Leases
265   protected Leases leases;
266 
267   // Instance of the hbase executor service.
268   protected ExecutorService service;
269 
270   // If false, the file system has become unavailable
271   protected volatile boolean fsOk;
272   protected HFileSystem fs;
273 
274   // Set when a report to the master comes back with a message asking us to
275   // shutdown. Also set by call to stop when debugging or running unit tests
276   // of HRegionServer in isolation.
277   private volatile boolean stopped = false;
278 
279   // Go down hard. Used if file system becomes unavailable and also in
280   // debugging and unit tests.
281   private volatile boolean abortRequested;
282 
283   ConcurrentMap<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
284 
285   // A state before we go into stopped state.  At this stage we're closing user
286   // space regions.
287   private boolean stopping = false;
288 
289   private volatile boolean killed = false;
290 
291   protected final Configuration conf;
292 
293   private Path rootDir;
294 
295   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
296 
297   final int numRetries;
298   protected final int threadWakeFrequency;
299   protected final int msgInterval;
300 
301   protected final int numRegionsToReport;
302 
303   // Stub to do region server status calls against the master.
304   private volatile RegionServerStatusService.BlockingInterface rssStub;
305   // RPC client. Used to make the stub above that does region server status checking.
306   RpcClient rpcClient;
307 
308   private UncaughtExceptionHandler uncaughtExceptionHandler;
309 
310   // Info server. Default access so can be used by unit tests. REGIONSERVER
311   // is name of the webapp and the attribute name used stuffing this instance
312   // into web context.
313   protected InfoServer infoServer;
314   private JvmPauseMonitor pauseMonitor;
315 
316   /** region server process name */
317   public static final String REGIONSERVER = "regionserver";
318 
319   MetricsRegionServer metricsRegionServer;
320   private SpanReceiverHost spanReceiverHost;
321 
322   /*
323    * Check for compactions requests.
324    */
325   Chore compactionChecker;
326 
327   /*
328    * Check for flushes
329    */
330   Chore periodicFlusher;
331 
332   // HLog and HLog roller. log is protected rather than private to avoid
333   // eclipse warning when accessed by inner classes
334   protected volatile HLog hlog;
335   // The meta updates are written to a different hlog. If this
336   // regionserver holds meta regions, then this field will be non-null.
337   protected volatile HLog hlogForMeta;
338 
339   LogRoller hlogRoller;
340   LogRoller metaHLogRoller;
341 
342   // flag set after we're done setting up server threads
343   final AtomicBoolean online = new AtomicBoolean(false);
344 
345   // zookeeper connection and watcher
346   protected ZooKeeperWatcher zooKeeper;
347 
348   // master address tracker
349   private MasterAddressTracker masterAddressTracker;
350 
351   // Cluster Status Tracker
352   protected ClusterStatusTracker clusterStatusTracker;
353 
354   // Log Splitting Worker
355   private SplitLogWorker splitLogWorker;
356 
357   // A sleeper that sleeps for msgInterval.
358   protected final Sleeper sleeper;
359 
360   private final int operationTimeout;
361 
362   private final RegionServerAccounting regionServerAccounting;
363 
364   // Cache configuration and block cache reference
365   protected CacheConfig cacheConfig;
366 
367   /** The health check chore. */
368   private HealthCheckChore healthCheckChore;
369 
370   /** The nonce manager chore. */
371   private Chore nonceManagerChore;
372 
373   private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
374 
375   /**
376    * The server name the Master sees us as.  Its made from the hostname the
377    * master passes us, port, and server startcode. Gets set after registration
378    * against  Master.
379    */
380   protected ServerName serverName;
381 
382   /**
383    * This servers startcode.
384    */
385   protected final long startcode;
386 
387   /**
388    * Unique identifier for the cluster we are a part of.
389    */
390   private String clusterId;
391 
392   /**
393    * MX Bean for RegionServerInfo
394    */
395   private ObjectName mxBean = null;
396 
397   /**
398    * Chore to clean periodically the moved region list
399    */
400   private MovedRegionsCleaner movedRegionsCleaner;
401 
402   // chore for refreshing store files for secondary regions
403   private StorefileRefresherChore storefileRefresher;
404 
405   private RegionServerCoprocessorHost rsHost;
406 
407   private RegionServerProcedureManagerHost rspmHost;
408 
409   private RegionServerQuotaManager rsQuotaManager;
410 
411   // Table level lock manager for locking for region operations
412   protected TableLockManager tableLockManager;
413 
414   /**
415    * Nonce manager. Nonces are used to make operations like increment and append idempotent
416    * in the case where client doesn't receive the response from a successful operation and
417    * retries. We track the successful ops for some time via a nonce sent by client and handle
418    * duplicate operations (currently, by failing them; in future we might use MVCC to return
419    * result). Nonces are also recovered from WAL during, recovery; however, the caveats (from
420    * HBASE-3787) are:
421    * - WAL recovery is optimized, and under high load we won't read nearly nonce-timeout worth
422    *   of past records. If we don't read the records, we don't read and recover the nonces.
423    *   Some WALs within nonce-timeout at recovery may not even be present due to rolling/cleanup.
424    * - There's no WAL recovery during normal region move, so nonces will not be transfered.
425    * We can have separate additional "Nonce WAL". It will just contain bunch of numbers and
426    * won't be flushed on main path - because WAL itself also contains nonces, if we only flush
427    * it before memstore flush, for a given nonce we will either see it in the WAL (if it was
428    * never flushed to disk, it will be part of recovery), or we'll see it as part of the nonce
429    * log (or both occasionally, which doesn't matter). Nonce log file can be deleted after the
430    * latest nonce in it expired. It can also be recovered during move.
431    */
432   final ServerNonceManager nonceManager;
433 
434   private UserProvider userProvider;
435 
436   protected final RSRpcServices rpcServices;
437 
438   protected BaseCoordinatedStateManager csm;
439 
440   /**
441    * Configuration manager is used to register/deregister and notify the configuration observers
442    * when the regionserver is notified that there was a change in the on disk configs.
443    */
444   private final ConfigurationManager configurationManager;
445 
446   /**
447    * Starts a HRegionServer at the default location.
448    * @param conf
449    * @throws IOException
450    * @throws InterruptedException
451    */
452   public HRegionServer(Configuration conf) throws IOException, InterruptedException {
453     this(conf, CoordinatedStateManagerFactory.getCoordinatedStateManager(conf));
454   }
455 
456   /**
457    * Starts a HRegionServer at the default location
458    * @param conf
459    * @param csm implementation of CoordinatedStateManager to be used
460    * @throws IOException
461    */
462   public HRegionServer(Configuration conf, CoordinatedStateManager csm)
463       throws IOException {
464     this.fsOk = true;
465     this.conf = conf;
466     checkCodecs(this.conf);
467     this.userProvider = UserProvider.instantiate(conf);
468     FSUtils.setupShortCircuitRead(this.conf);
469 
470     // Config'ed params
471     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
472         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
473     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
474     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
475 
476     this.sleeper = new Sleeper(this.msgInterval, this);
477 
478     boolean isNoncesEnabled = conf.getBoolean(HConstants.HBASE_RS_NONCES_ENABLED, true);
479     this.nonceManager = isNoncesEnabled ? new ServerNonceManager(this.conf) : null;
480 
481     this.numRegionsToReport = conf.getInt(
482       "hbase.regionserver.numregionstoreport", 10);
483 
484     this.operationTimeout = conf.getInt(
485       HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
486       HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
487 
488     this.abortRequested = false;
489     this.stopped = false;
490 
491     rpcServices = createRpcServices();
492     this.startcode = System.currentTimeMillis();
493     String hostName = rpcServices.isa.getHostName();
494     serverName = ServerName.valueOf(hostName, rpcServices.isa.getPort(), startcode);
495 
496     // login the zookeeper client principal (if using security)
497     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
498       "hbase.zookeeper.client.kerberos.principal", hostName);
499     // login the server principal (if using secure Hadoop)
500     login(userProvider, hostName);
501 
502     regionServerAccounting = new RegionServerAccounting();
503     uncaughtExceptionHandler = new UncaughtExceptionHandler() {
504       @Override
505       public void uncaughtException(Thread t, Throwable e) {
506         abort("Uncaught exception in service thread " + t.getName(), e);
507       }
508     };
509 
510     // Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
511     // underlying hadoop hdfs accessors will be going against wrong filesystem
512     // (unless all is set to defaults).
513     FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
514     // Get fs instance used by this RS.  Do we use checksum verification in the hbase? If hbase
515     // checksum verification enabled, then automatically switch off hdfs checksum verification.
516     boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
517     this.fs = new HFileSystem(this.conf, useHBaseChecksum);
518     this.rootDir = FSUtils.getRootDir(this.conf);
519     this.tableDescriptors = new FSTableDescriptors(this.conf,
520       this.fs, this.rootDir, !canUpdateTableDescriptor());
521 
522     service = new ExecutorService(getServerName().toShortString());
523     spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
524 
525     // Some unit tests don't need a cluster, so no zookeeper at all
526     if (!conf.getBoolean("hbase.testing.nocluster", false)) {
527       // Open connection to zookeeper and set primary watcher
528       zooKeeper = new ZooKeeperWatcher(conf, getProcessName() + ":" +
529         rpcServices.isa.getPort(), this, canCreateBaseZNode());
530 
531       this.csm = (BaseCoordinatedStateManager) csm;
532       this.csm.initialize(this);
533       this.csm.start();
534 
535       tableLockManager = TableLockManager.createTableLockManager(
536         conf, zooKeeper, serverName);
537 
538       masterAddressTracker = new MasterAddressTracker(getZooKeeper(), this);
539       masterAddressTracker.start();
540 
541       clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this);
542       clusterStatusTracker.start();
543     }
544     this.configurationManager = new ConfigurationManager();
545 
546     rpcServices.start();
547     putUpWebUI();
548   }
549 
550   protected void login(UserProvider user, String host) throws IOException {
551     user.login("hbase.regionserver.keytab.file",
552       "hbase.regionserver.kerberos.principal", host);
553   }
554 
555   protected void waitForMasterActive(){
556   }
557 
558   protected String getProcessName() {
559     return REGIONSERVER;
560   }
561 
562   protected boolean canCreateBaseZNode() {
563     return false;
564   }
565 
566   protected boolean canUpdateTableDescriptor() {
567     return false;
568   }
569 
570   protected RSRpcServices createRpcServices() throws IOException {
571     return new RSRpcServices(this);
572   }
573 
574   protected void configureInfoServer() {
575     infoServer.addServlet("rs-status", "/rs-status", RSStatusServlet.class);
576     infoServer.setAttribute(REGIONSERVER, this);
577   }
578 
579   protected Class<? extends HttpServlet> getDumpServlet() {
580     return RSDumpServlet.class;
581   }
582 
583   protected void doMetrics() {
584   }
585 
586   @Override
587   public boolean registerService(Service instance) {
588     /*
589      * No stacking of instances is allowed for a single service name
590      */
591     Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
592     if (coprocessorServiceHandlers.containsKey(serviceDesc.getFullName())) {
593       LOG.error("Coprocessor service " + serviceDesc.getFullName()
594           + " already registered, rejecting request from " + instance);
595       return false;
596     }
597 
598     coprocessorServiceHandlers.put(serviceDesc.getFullName(), instance);
599     if (LOG.isDebugEnabled()) {
600       LOG.debug("Registered regionserver coprocessor service: service=" + serviceDesc.getFullName());
601     }
602     return true;
603   }
604 
605   /**
606    * Create wrapped short-circuit connection to this server.
607    * In its own method so can intercept and mock it over in tests.
608    * @throws IOException
609    */
610   protected HConnection createShortCircuitConnection() throws IOException {
611     return ConnectionUtils.createShortCircuitHConnection(
612       HConnectionManager.getConnection(conf), serverName, rpcServices, rpcServices);
613   }
614   
615   /**
616    * Run test on configured codecs to make sure supporting libs are in place.
617    * @param c
618    * @throws IOException
619    */
620   private static void checkCodecs(final Configuration c) throws IOException {
621     // check to see if the codec list is available:
622     String [] codecs = c.getStrings("hbase.regionserver.codecs", (String[])null);
623     if (codecs == null) return;
624     for (String codec : codecs) {
625       if (!CompressionTest.testCompression(codec)) {
626         throw new IOException("Compression codec " + codec +
627           " not supported, aborting RS construction");
628       }
629     }
630   }
631 
632   public String getClusterId() {
633     return this.clusterId;
634   }
635 
636   /**
637    * All initialization needed before we go register with Master.
638    *
639    * @throws IOException
640    * @throws InterruptedException
641    */
642   private void preRegistrationInitialization(){
643     try {
644       synchronized (this) {
645         if (shortCircuitConnection == null) {
646           shortCircuitConnection = createShortCircuitConnection();
647           metaTableLocator = new MetaTableLocator();
648         }
649       }
650 
651       // Health checker thread.
652       if (isHealthCheckerConfigured()) {
653         int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
654           HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
655         healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
656       }
657       this.pauseMonitor = new JvmPauseMonitor(conf);
658       pauseMonitor.start();
659 
660       initializeZooKeeper();
661       if (!isStopped() && !isAborted()) {
662         initializeThreads();
663       }
664     } catch (Throwable t) {
665       // Call stop if error or process will stick around for ever since server
666       // puts up non-daemon threads.
667       this.rpcServices.stop();
668       abort("Initialization of RS failed.  Hence aborting RS.", t);
669     }
670   }
671 
672   /**
673    * Bring up connection to zk ensemble and then wait until a master for this
674    * cluster and then after that, wait until cluster 'up' flag has been set.
675    * This is the order in which master does things.
676    * Finally open long-living server short-circuit connection.
677    * @throws IOException
678    * @throws InterruptedException
679    */
680   private void initializeZooKeeper() throws IOException, InterruptedException {
681     // Create the master address tracker, register with zk, and start it.  Then
682     // block until a master is available.  No point in starting up if no master
683     // running.
684     blockAndCheckIfStopped(this.masterAddressTracker);
685 
686     // Wait on cluster being up.  Master will set this flag up in zookeeper
687     // when ready.
688     blockAndCheckIfStopped(this.clusterStatusTracker);
689 
690     // Retrieve clusterId
691     // Since cluster status is now up
692     // ID should have already been set by HMaster
693     try {
694       clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
695       if (clusterId == null) {
696         this.abort("Cluster ID has not been set");
697       }
698       LOG.info("ClusterId : "+clusterId);
699     } catch (KeeperException e) {
700       this.abort("Failed to retrieve Cluster ID",e);
701     }
702 
703     // In case colocated master, wait here till it's active.
704     // So backup masters won't start as regionservers.
705     // This is to avoid showing backup masters as regionservers
706     // in master web UI, or assigning any region to them.
707     waitForMasterActive();
708     if (isStopped() || isAborted()) {
709       return; // No need for further initialization
710     }
711 
712     // watch for snapshots and other procedures
713     try {
714       rspmHost = new RegionServerProcedureManagerHost();
715       rspmHost.loadProcedures(conf);
716       rspmHost.initialize(this);
717     } catch (KeeperException e) {
718       this.abort("Failed to reach zk cluster when creating procedure handler.", e);
719     }
720     // register watcher for recovering regions
721     this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);
722   }
723 
724   /**
725    * Utilty method to wait indefinitely on a znode availability while checking
726    * if the region server is shut down
727    * @param tracker znode tracker to use
728    * @throws IOException any IO exception, plus if the RS is stopped
729    * @throws InterruptedException
730    */
731   private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
732       throws IOException, InterruptedException {
733     while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
734       if (this.stopped) {
735         throw new IOException("Received the shutdown message while waiting.");
736       }
737     }
738   }
739 
740   /**
741    * @return False if cluster shutdown in progress
742    */
743   private boolean isClusterUp() {
744     return clusterStatusTracker != null && clusterStatusTracker.isClusterUp();
745   }
746 
747   private void initializeThreads() throws IOException {
748     // Cache flushing thread.
749     this.cacheFlusher = new MemStoreFlusher(conf, this);
750 
751     // Compaction thread
752     this.compactSplitThread = new CompactSplitThread(this);
753 
754     // Background thread to check for compactions; needed if region has not gotten updates
755     // in a while. It will take care of not checking too frequently on store-by-store basis.
756     this.compactionChecker = new CompactionChecker(this, this.threadWakeFrequency, this);
757     this.periodicFlusher = new PeriodicMemstoreFlusher(this.threadWakeFrequency, this);
758     this.leases = new Leases(this.threadWakeFrequency);
759 
760     // Create the thread to clean the moved regions list
761     movedRegionsCleaner = MovedRegionsCleaner.createAndStart(this);
762 
763     if (this.nonceManager != null) {
764       // Create the chore that cleans up nonces.
765       nonceManagerChore = this.nonceManager.createCleanupChore(this);
766     }
767 
768     // Setup the Quota Manager
769     rsQuotaManager = new RegionServerQuotaManager(this);
770 
771     // Setup RPC client for master communication
772     rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress(
773       rpcServices.isa.getAddress(), 0));
774 
775     int storefileRefreshPeriod = conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD
776       , StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
777     if (storefileRefreshPeriod > 0) {
778       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod, this, this);
779     }
780     registerConfigurationObservers();
781   }
782 
783   private void registerConfigurationObservers() {
784     // Registering the compactSplitThread object with the ConfigurationManager.
785     configurationManager.registerObserver(this.compactSplitThread);
786   }
787 
788   /**
789    * The HRegionServer sticks in this loop until closed.
790    */
791   @Override
792   public void run() {
793     try {
794       // Do pre-registration initializations; zookeeper, lease threads, etc.
795       preRegistrationInitialization();
796     } catch (Throwable e) {
797       abort("Fatal exception during initialization", e);
798     }
799 
800     try {
801       if (!isStopped() && !isAborted()) {
802         ShutdownHook.install(conf, fs, this, Thread.currentThread());
803         // Set our ephemeral znode up in zookeeper now we have a name.
804         createMyEphemeralNode();
805         // Initialize the RegionServerCoprocessorHost now that our ephemeral
806         // node was created, in case any coprocessors want to use ZooKeeper
807         this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
808       }
809 
810       // Try and register with the Master; tell it we are here.  Break if
811       // server is stopped or the clusterup flag is down or hdfs went wacky.
812       while (keepLooping()) {
813         RegionServerStartupResponse w = reportForDuty();
814         if (w == null) {
815           LOG.warn("reportForDuty failed; sleeping and then retrying.");
816           this.sleeper.sleep();
817         } else {
818           handleReportForDutyResponse(w);
819           break;
820         }
821       }
822 
823       if (!isStopped() && isHealthy()){
824         // start the snapshot handler and other procedure handlers,
825         // since the server is ready to run
826         rspmHost.start();
827 
828         // Start the Quota Manager
829         rsQuotaManager.start(getRpcServer().getScheduler());
830       }
831 
832       // We registered with the Master.  Go into run mode.
833       long lastMsg = System.currentTimeMillis();
834       long oldRequestCount = -1;
835       // The main run loop.
836       while (!isStopped() && isHealthy()) {
837         if (!isClusterUp()) {
838           if (isOnlineRegionsEmpty()) {
839             stop("Exiting; cluster shutdown set and not carrying any regions");
840           } else if (!this.stopping) {
841             this.stopping = true;
842             LOG.info("Closing user regions");
843             closeUserRegions(this.abortRequested);
844           } else if (this.stopping) {
845             boolean allUserRegionsOffline = areAllUserRegionsOffline();
846             if (allUserRegionsOffline) {
847               // Set stopped if no more write requests tp meta tables
848               // since last time we went around the loop.  Any open
849               // meta regions will be closed on our way out.
850               if (oldRequestCount == getWriteRequestCount()) {
851                 stop("Stopped; only catalog regions remaining online");
852                 break;
853               }
854               oldRequestCount = getWriteRequestCount();
855             } else {
856               // Make sure all regions have been closed -- some regions may
857               // have not got it because we were splitting at the time of
858               // the call to closeUserRegions.
859               closeUserRegions(this.abortRequested);
860             }
861             LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
862           }
863         }
864         long now = System.currentTimeMillis();
865         if ((now - lastMsg) >= msgInterval) {
866           tryRegionServerReport(lastMsg, now);
867           lastMsg = System.currentTimeMillis();
868           doMetrics();
869         }
870         if (!isStopped() && !isAborted()) {
871           this.sleeper.sleep();
872         }
873       } // for
874     } catch (Throwable t) {
875       if (!rpcServices.checkOOME(t)) {
876         String prefix = t instanceof YouAreDeadException? "": "Unhandled: ";
877         abort(prefix + t.getMessage(), t);
878       }
879     }
880     // Run shutdown.
881     if (mxBean != null) {
882       MBeanUtil.unregisterMBean(mxBean);
883       mxBean = null;
884     }
885     if (this.leases != null) this.leases.closeAfterLeasesExpire();
886     if (this.splitLogWorker != null) {
887       splitLogWorker.stop();
888     }
889     if (this.infoServer != null) {
890       LOG.info("Stopping infoServer");
891       try {
892         this.infoServer.stop();
893       } catch (Exception e) {
894         LOG.error("Failed to stop infoServer", e);
895       }
896     }
897     // Send cache a shutdown.
898     if (cacheConfig != null && cacheConfig.isBlockCacheEnabled()) {
899       cacheConfig.getBlockCache().shutdown();
900     }
901 
902     if (movedRegionsCleaner != null) {
903       movedRegionsCleaner.stop("Region Server stopping");
904     }
905 
906     // Send interrupts to wake up threads if sleeping so they notice shutdown.
907     // TODO: Should we check they are alive? If OOME could have exited already
908     if(this.hMemManager != null) this.hMemManager.stop();
909     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
910     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
911     if (this.compactionChecker != null)
912       this.compactionChecker.interrupt();
913     if (this.healthCheckChore != null) {
914       this.healthCheckChore.interrupt();
915     }
916     if (this.nonceManagerChore != null) {
917       this.nonceManagerChore.interrupt();
918     }
919     if (this.storefileRefresher != null) {
920       this.storefileRefresher.interrupt();
921     }
922 
923     // Stop the quota manager
924     if (rsQuotaManager != null) {
925       rsQuotaManager.stop();
926     }
927 
928     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
929     if (rspmHost != null) {
930       rspmHost.stop(this.abortRequested || this.killed);
931     }
932 
933     if (this.killed) {
934       // Just skip out w/o closing regions.  Used when testing.
935     } else if (abortRequested) {
936       if (this.fsOk) {
937         closeUserRegions(abortRequested); // Don't leave any open file handles
938       }
939       LOG.info("aborting server " + this.serverName);
940     } else {
941       closeUserRegions(abortRequested);
942       LOG.info("stopping server " + this.serverName);
943     }
944 
945     // so callers waiting for meta without timeout can stop
946     if (this.metaTableLocator != null) this.metaTableLocator.stop();
947     if (this.shortCircuitConnection != null && !shortCircuitConnection.isClosed()) {
948       try {
949         this.shortCircuitConnection.close();
950       } catch (IOException e) {
951         // Although the {@link Closeable} interface throws an {@link
952         // IOException}, in reality, the implementation would never do that.
953         LOG.error("Attempt to close server's short circuit HConnection failed.", e);
954       }
955     }
956 
957     // Closing the compactSplit thread before closing meta regions
958     if (!this.killed && containsMetaTableRegions()) {
959       if (!abortRequested || this.fsOk) {
960         if (this.compactSplitThread != null) {
961           this.compactSplitThread.join();
962           this.compactSplitThread = null;
963         }
964         closeMetaTableRegions(abortRequested);
965       }
966     }
967 
968     if (!this.killed && this.fsOk) {
969       waitOnAllRegionsToClose(abortRequested);
970       LOG.info("stopping server " + this.serverName +
971         "; all regions closed.");
972     }
973 
974     //fsOk flag may be changed when closing regions throws exception.
975     if (this.fsOk) {
976       closeWAL(!abortRequested);
977     }
978 
979     // Make sure the proxy is down.
980     if (this.rssStub != null) {
981       this.rssStub = null;
982     }
983     if (this.rpcClient != null) {
984       this.rpcClient.stop();
985     }
986     if (this.leases != null) {
987       this.leases.close();
988     }
989     if (this.pauseMonitor != null) {
990       this.pauseMonitor.stop();
991     }
992 
993     if (!killed) {
994       stopServiceThreads();
995     }
996 
997     if (this.rpcServices != null) {
998       this.rpcServices.stop();
999     }
1000 
1001     try {
1002       deleteMyEphemeralNode();
1003     } catch (KeeperException.NoNodeException nn) {
1004     } catch (KeeperException e) {
1005       LOG.warn("Failed deleting my ephemeral node", e);
1006     }
1007     // We may have failed to delete the znode at the previous step, but
1008     //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
1009     ZNodeClearer.deleteMyEphemeralNodeOnDisk();
1010 
1011     if (this.zooKeeper != null) {
1012       this.zooKeeper.close();
1013     }
1014     LOG.info("stopping server " + this.serverName +
1015       "; zookeeper connection closed.");
1016 
1017     LOG.info(Thread.currentThread().getName() + " exiting");
1018   }
1019 
1020   private boolean containsMetaTableRegions() {
1021     return onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
1022   }
1023 
1024   private boolean areAllUserRegionsOffline() {
1025     if (getNumberOfOnlineRegions() > 2) return false;
1026     boolean allUserRegionsOffline = true;
1027     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1028       if (!e.getValue().getRegionInfo().isMetaTable()) {
1029         allUserRegionsOffline = false;
1030         break;
1031       }
1032     }
1033     return allUserRegionsOffline;
1034   }
1035 
1036   /**
1037    * @return Current write count for all online regions.
1038    */
1039   private long getWriteRequestCount() {
1040     int writeCount = 0;
1041     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1042       writeCount += e.getValue().getWriteRequestsCount();
1043     }
1044     return writeCount;
1045   }
1046 
1047   @VisibleForTesting
1048   protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
1049   throws IOException {
1050     RegionServerStatusService.BlockingInterface rss = rssStub;
1051     if (rss == null) {
1052       // the current server could be stopping.
1053       return;
1054     }
1055     ClusterStatusProtos.ServerLoad sl = buildServerLoad(reportStartTime, reportEndTime);
1056     try {
1057       RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();
1058       ServerName sn = ServerName.parseVersionedServerName(
1059         this.serverName.getVersionedBytes());
1060       request.setServer(ProtobufUtil.toServerName(sn));
1061       request.setLoad(sl);
1062       rss.regionServerReport(null, request.build());
1063     } catch (ServiceException se) {
1064       IOException ioe = ProtobufUtil.getRemoteException(se);
1065       if (ioe instanceof YouAreDeadException) {
1066         // This will be caught and handled as a fatal error in run()
1067         throw ioe;
1068       }
1069       if (rssStub == rss) {
1070         rssStub = null;
1071       }
1072       // Couldn't connect to the master, get location from zk and reconnect
1073       // Method blocks until new master is found or we are stopped
1074       createRegionServerStatusStub();
1075     }
1076   }
1077 
1078   ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime) {
1079     // We're getting the MetricsRegionServerWrapper here because the wrapper computes requests
1080     // per second, and other metrics  As long as metrics are part of ServerLoad it's best to use
1081     // the wrapper to compute those numbers in one place.
1082     // In the long term most of these should be moved off of ServerLoad and the heart beat.
1083     // Instead they should be stored in an HBase table so that external visibility into HBase is
1084     // improved; Additionally the load balancer will be able to take advantage of a more complete
1085     // history.
1086     MetricsRegionServerWrapper regionServerWrapper = this.metricsRegionServer.getRegionServerWrapper();
1087     Collection<HRegion> regions = getOnlineRegionsLocalContext();
1088     MemoryUsage memory =
1089       ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
1090 
1091     ClusterStatusProtos.ServerLoad.Builder serverLoad =
1092       ClusterStatusProtos.ServerLoad.newBuilder();
1093     serverLoad.setNumberOfRequests((int) regionServerWrapper.getRequestsPerSecond());
1094     serverLoad.setTotalNumberOfRequests((int) regionServerWrapper.getTotalRequestCount());
1095     serverLoad.setUsedHeapMB((int)(memory.getUsed() / 1024 / 1024));
1096     serverLoad.setMaxHeapMB((int) (memory.getMax() / 1024 / 1024));
1097     Set<String> coprocessors = this.hlog.getCoprocessorHost().getCoprocessors();
1098     for (String coprocessor : coprocessors) {
1099       serverLoad.addCoprocessors(
1100         Coprocessor.newBuilder().setName(coprocessor).build());
1101     }
1102     RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder();
1103     RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
1104     for (HRegion region : regions) {
1105       serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier));
1106     }
1107     serverLoad.setReportStartTime(reportStartTime);
1108     serverLoad.setReportEndTime(reportEndTime);
1109     if (this.infoServer != null) {
1110       serverLoad.setInfoServerPort(this.infoServer.getPort());
1111     } else {
1112       serverLoad.setInfoServerPort(-1);
1113     }
1114     return serverLoad.build();
1115   }
1116 
1117   String getOnlineRegionsAsPrintableString() {
1118     StringBuilder sb = new StringBuilder();
1119     for (HRegion r: this.onlineRegions.values()) {
1120       if (sb.length() > 0) sb.append(", ");
1121       sb.append(r.getRegionInfo().getEncodedName());
1122     }
1123     return sb.toString();
1124   }
1125 
1126   /**
1127    * Wait on regions close.
1128    */
1129   private void waitOnAllRegionsToClose(final boolean abort) {
1130     // Wait till all regions are closed before going out.
1131     int lastCount = -1;
1132     long previousLogTime = 0;
1133     Set<String> closedRegions = new HashSet<String>();
1134     boolean interrupted = false;
1135     try {
1136       while (!isOnlineRegionsEmpty()) {
1137         int count = getNumberOfOnlineRegions();
1138         // Only print a message if the count of regions has changed.
1139         if (count != lastCount) {
1140           // Log every second at most
1141           if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1142             previousLogTime = System.currentTimeMillis();
1143             lastCount = count;
1144             LOG.info("Waiting on " + count + " regions to close");
1145             // Only print out regions still closing if a small number else will
1146             // swamp the log.
1147             if (count < 10 && LOG.isDebugEnabled()) {
1148               LOG.debug(this.onlineRegions);
1149             }
1150           }
1151         }
1152         // Ensure all user regions have been sent a close. Use this to
1153         // protect against the case where an open comes in after we start the
1154         // iterator of onlineRegions to close all user regions.
1155         for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1156           HRegionInfo hri = e.getValue().getRegionInfo();
1157           if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
1158               && !closedRegions.contains(hri.getEncodedName())) {
1159             closedRegions.add(hri.getEncodedName());
1160             // Don't update zk with this close transition; pass false.
1161             closeRegionIgnoreErrors(hri, abort);
1162               }
1163         }
1164         // No regions in RIT, we could stop waiting now.
1165         if (this.regionsInTransitionInRS.isEmpty()) {
1166           if (!isOnlineRegionsEmpty()) {
1167             LOG.info("We were exiting though online regions are not empty," +
1168                 " because some regions failed closing");
1169           }
1170           break;
1171         }
1172         if (sleep(200)) {
1173           interrupted = true;
1174         }
1175       }
1176     } finally {
1177       if (interrupted) {
1178         Thread.currentThread().interrupt();
1179       }
1180     }
1181   }
1182 
1183   private boolean sleep(long millis) {
1184     boolean interrupted = false;
1185     try {
1186       Thread.sleep(millis);
1187     } catch (InterruptedException e) {
1188       LOG.warn("Interrupted while sleeping");
1189       interrupted = true;
1190     }
1191     return interrupted;
1192   }
1193 
1194   private void closeWAL(final boolean delete) {
1195     if (this.hlogForMeta != null) {
1196       // All hlogs (meta and non-meta) are in the same directory. Don't call
1197       // closeAndDelete here since that would delete all hlogs not just the
1198       // meta ones. We will just 'close' the hlog for meta here, and leave
1199       // the directory cleanup to the follow-on closeAndDelete call.
1200       try {
1201         this.hlogForMeta.close();
1202       } catch (Throwable e) {
1203         e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
1204         LOG.error("Metalog close and delete failed", e);
1205       }
1206     }
1207     if (this.hlog != null) {
1208       try {
1209         if (delete) {
1210           hlog.closeAndDelete();
1211         } else {
1212           hlog.close();
1213         }
1214       } catch (Throwable e) {
1215         e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
1216         LOG.error("Close and delete failed", e);
1217       }
1218     }
1219   }
1220 
1221   /*
1222    * Run init. Sets up hlog and starts up all server threads.
1223    *
1224    * @param c Extra configuration.
1225    */
1226   protected void handleReportForDutyResponse(final RegionServerStartupResponse c)
1227   throws IOException {
1228     try {
1229       for (NameStringPair e : c.getMapEntriesList()) {
1230         String key = e.getName();
1231         // The hostname the master sees us as.
1232         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1233           String hostnameFromMasterPOV = e.getValue();
1234           this.serverName = ServerName.valueOf(hostnameFromMasterPOV,
1235             rpcServices.isa.getPort(), this.startcode);
1236           if (!hostnameFromMasterPOV.equals(rpcServices.isa.getHostName())) {
1237             LOG.info("Master passed us a different hostname to use; was=" +
1238               rpcServices.isa.getHostName() + ", but now=" + hostnameFromMasterPOV);
1239           }
1240           continue;
1241         }
1242         String value = e.getValue();
1243         if (LOG.isDebugEnabled()) {
1244           LOG.info("Config from master: " + key + "=" + value);
1245         }
1246         this.conf.set(key, value);
1247       }
1248 
1249       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
1250       // config param for task trackers, but we can piggyback off of it.
1251       if (this.conf.get("mapreduce.task.attempt.id") == null) {
1252         this.conf.set("mapreduce.task.attempt.id", "hb_rs_" +
1253           this.serverName.toString());
1254       }
1255 
1256       // Save it in a file, this will allow to see if we crash
1257       ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
1258 
1259       this.cacheConfig = new CacheConfig(conf);
1260       this.hlog = setupWALAndReplication();
1261       // Init in here rather than in constructor after thread name has been set
1262       this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this));
1263 
1264       startServiceThreads();
1265       startHeapMemoryManager();
1266       LOG.info("Serving as " + this.serverName +
1267         ", RpcServer on " + rpcServices.isa +
1268         ", sessionid=0x" +
1269         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1270 
1271       // Wake up anyone waiting for this server to online
1272       synchronized (online) {
1273         online.set(true);
1274         online.notifyAll();
1275       }
1276     } catch (Throwable e) {
1277       stop("Failed initialization");
1278       throw convertThrowableToIOE(cleanup(e, "Failed init"),
1279           "Region server startup failed");
1280     } finally {
1281       sleeper.skipSleepCycle();
1282     }
1283   }
1284 
1285   private void startHeapMemoryManager() {
1286     this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher, this);
1287     if (this.hMemManager != null) {
1288       this.hMemManager.start();
1289     }
1290   }
1291 
1292   private void createMyEphemeralNode() throws KeeperException, IOException {
1293     RegionServerInfo.Builder rsInfo = RegionServerInfo.newBuilder();
1294     rsInfo.setInfoPort(infoServer != null ? infoServer.getPort() : -1);
1295     byte[] data = ProtobufUtil.prependPBMagic(rsInfo.build().toByteArray());
1296     ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper,
1297       getMyEphemeralNodePath(), data);
1298   }
1299 
1300   private void deleteMyEphemeralNode() throws KeeperException {
1301     ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1302   }
1303 
1304   @Override
1305   public RegionServerAccounting getRegionServerAccounting() {
1306     return regionServerAccounting;
1307   }
1308 
1309   @Override
1310   public TableLockManager getTableLockManager() {
1311     return tableLockManager;
1312   }
1313 
1314   /*
1315    * @param r Region to get RegionLoad for.
1316    * @param regionLoadBldr the RegionLoad.Builder, can be null
1317    * @param regionSpecifier the RegionSpecifier.Builder, can be null
1318    * @return RegionLoad instance.
1319    *
1320    * @throws IOException
1321    */
1322   private RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr,
1323       RegionSpecifier.Builder regionSpecifier) {
1324     byte[] name = r.getRegionName();
1325     int stores = 0;
1326     int storefiles = 0;
1327     int storeUncompressedSizeMB = 0;
1328     int storefileSizeMB = 0;
1329     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
1330     int storefileIndexSizeMB = 0;
1331     int rootIndexSizeKB = 0;
1332     int totalStaticIndexSizeKB = 0;
1333     int totalStaticBloomSizeKB = 0;
1334     long totalCompactingKVs = 0;
1335     long currentCompactedKVs = 0;
1336     synchronized (r.stores) {
1337       stores += r.stores.size();
1338       for (Store store : r.stores.values()) {
1339         storefiles += store.getStorefilesCount();
1340         storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
1341             / 1024 / 1024);
1342         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1343         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
1344         CompactionProgress progress = store.getCompactionProgress();
1345         if (progress != null) {
1346           totalCompactingKVs += progress.totalCompactingKVs;
1347           currentCompactedKVs += progress.currentCompactedKVs;
1348         }
1349 
1350         rootIndexSizeKB +=
1351             (int) (store.getStorefilesIndexSize() / 1024);
1352 
1353         totalStaticIndexSizeKB +=
1354           (int) (store.getTotalStaticIndexSize() / 1024);
1355 
1356         totalStaticBloomSizeKB +=
1357           (int) (store.getTotalStaticBloomSize() / 1024);
1358       }
1359     }
1360     if (regionLoadBldr == null) {
1361       regionLoadBldr = RegionLoad.newBuilder();
1362     }
1363     if (regionSpecifier == null) {
1364       regionSpecifier = RegionSpecifier.newBuilder();
1365     }
1366     regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
1367     regionSpecifier.setValue(ByteStringer.wrap(name));
1368     regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
1369       .setStores(stores)
1370       .setStorefiles(storefiles)
1371       .setStoreUncompressedSizeMB(storeUncompressedSizeMB)
1372       .setStorefileSizeMB(storefileSizeMB)
1373       .setMemstoreSizeMB(memstoreSizeMB)
1374       .setStorefileIndexSizeMB(storefileIndexSizeMB)
1375       .setRootIndexSizeKB(rootIndexSizeKB)
1376       .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
1377       .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
1378       .setReadRequestsCount(r.readRequestsCount.get())
1379       .setWriteRequestsCount(r.writeRequestsCount.get())
1380       .setTotalCompactingKVs(totalCompactingKVs)
1381       .setCurrentCompactedKVs(currentCompactedKVs)
1382       .setCompleteSequenceId(r.lastFlushSeqId);
1383 
1384     return regionLoadBldr.build();
1385   }
1386 
1387   /**
1388    * @param encodedRegionName
1389    * @return An instance of RegionLoad.
1390    */
1391   public RegionLoad createRegionLoad(final String encodedRegionName) {
1392     HRegion r = null;
1393     r = this.onlineRegions.get(encodedRegionName);
1394     return r != null ? createRegionLoad(r, null, null) : null;
1395   }
1396 
1397   /*
1398    * Inner class that runs on a long period checking if regions need compaction.
1399    */
1400   private static class CompactionChecker extends Chore {
1401     private final HRegionServer instance;
1402     private final int majorCompactPriority;
1403     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1404     private long iteration = 0;
1405 
1406     CompactionChecker(final HRegionServer h, final int sleepTime,
1407         final Stoppable stopper) {
1408       super("CompactionChecker", sleepTime, h);
1409       this.instance = h;
1410       LOG.info(this.getName() + " runs every " + StringUtils.formatTime(sleepTime));
1411 
1412       /* MajorCompactPriority is configurable.
1413        * If not set, the compaction will use default priority.
1414        */
1415       this.majorCompactPriority = this.instance.conf.
1416         getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1417         DEFAULT_PRIORITY);
1418     }
1419 
1420     @Override
1421     protected void chore() {
1422       for (HRegion r : this.instance.onlineRegions.values()) {
1423         if (r == null)
1424           continue;
1425         for (Store s : r.getStores().values()) {
1426           try {
1427             long multiplier = s.getCompactionCheckMultiplier();
1428             assert multiplier > 0;
1429             if (iteration % multiplier != 0) continue;
1430             if (s.needsCompaction()) {
1431               // Queue a compaction. Will recognize if major is needed.
1432               this.instance.compactSplitThread.requestSystemCompaction(r, s, getName()
1433                   + " requests compaction");
1434             } else if (s.isMajorCompaction()) {
1435               if (majorCompactPriority == DEFAULT_PRIORITY
1436                   || majorCompactPriority > r.getCompactPriority()) {
1437                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1438                     + " requests major compaction; use default priority", null);
1439               } else {
1440                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1441                     + " requests major compaction; use configured priority",
1442                   this.majorCompactPriority, null);
1443               }
1444             }
1445           } catch (IOException e) {
1446             LOG.warn("Failed major compaction check on " + r, e);
1447           }
1448         }
1449       }
1450       iteration = (iteration == Long.MAX_VALUE) ? 0 : (iteration + 1);
1451     }
1452   }
1453 
1454   class PeriodicMemstoreFlusher extends Chore {
1455     final HRegionServer server;
1456     final static int RANGE_OF_DELAY = 20000; //millisec
1457     final static int MIN_DELAY_TIME = 3000; //millisec
1458     public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1459       super(server.getServerName() + "-MemstoreFlusherChore", cacheFlushInterval, server);
1460       this.server = server;
1461     }
1462 
1463     @Override
1464     protected void chore() {
1465       for (HRegion r : this.server.onlineRegions.values()) {
1466         if (r == null)
1467           continue;
1468         if (r.shouldFlush()) {
1469           FlushRequester requester = server.getFlushRequester();
1470           if (requester != null) {
1471             long randomDelay = RandomUtils.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME;
1472             LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() +
1473                 " after a delay of " + randomDelay);
1474             //Throttle the flushes by putting a delay. If we don't throttle, and there
1475             //is a balanced write-load on the regions in a table, we might end up
1476             //overwhelming the filesystem with too many flushes at once.
1477             requester.requestDelayedFlush(r, randomDelay);
1478           }
1479         }
1480       }
1481     }
1482   }
1483 
1484   /**
1485    * Report the status of the server. A server is online once all the startup is
1486    * completed (setting up filesystem, starting service threads, etc.). This
1487    * method is designed mostly to be useful in tests.
1488    *
1489    * @return true if online, false if not.
1490    */
1491   public boolean isOnline() {
1492     return online.get();
1493   }
1494 
1495   /**
1496    * Setup WAL log and replication if enabled.
1497    * Replication setup is done in here because it wants to be hooked up to WAL.
1498    * @return A WAL instance.
1499    * @throws IOException
1500    */
1501   private HLog setupWALAndReplication() throws IOException {
1502     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1503     final String logName
1504       = HLogUtil.getHLogDirectoryName(this.serverName.toString());
1505 
1506     Path logdir = new Path(rootDir, logName);
1507     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1508     if (this.fs.exists(logdir)) {
1509       throw new RegionServerRunningException("Region server has already " +
1510         "created directory at " + this.serverName.toString());
1511     }
1512 
1513     // Instantiate replication manager if replication enabled.  Pass it the
1514     // log directories.
1515     createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
1516 
1517     return instantiateHLog(rootDir, logName);
1518   }
1519 
1520   private HLog getMetaWAL() throws IOException {
1521     if (this.hlogForMeta != null) return this.hlogForMeta;
1522     final String logName = HLogUtil.getHLogDirectoryName(this.serverName.toString());
1523     Path logdir = new Path(rootDir, logName);
1524     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1525     this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(), rootDir, logName,
1526       this.conf, getMetaWALActionListeners(), this.serverName.toString());
1527     return this.hlogForMeta;
1528   }
1529 
1530   /**
1531    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1532    * @param rootdir
1533    * @param logName
1534    * @return WAL instance.
1535    * @throws IOException
1536    */
1537   protected HLog instantiateHLog(Path rootdir, String logName) throws IOException {
1538     return HLogFactory.createHLog(this.fs.getBackingFs(), rootdir, logName, this.conf,
1539       getWALActionListeners(), this.serverName.toString());
1540   }
1541 
1542   /**
1543    * Called by {@link #instantiateHLog(Path, String)} setting up WAL instance.
1544    * Add any {@link WALActionsListener}s you want inserted before WAL startup.
1545    * @return List of WALActionsListener that will be passed in to
1546    * {@link org.apache.hadoop.hbase.regionserver.wal.FSHLog} on construction.
1547    */
1548   protected List<WALActionsListener> getWALActionListeners() {
1549     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1550     // Log roller.
1551     this.hlogRoller = new LogRoller(this, this);
1552     listeners.add(this.hlogRoller);
1553     if (this.replicationSourceHandler != null &&
1554         this.replicationSourceHandler.getWALActionsListener() != null) {
1555       // Replication handler is an implementation of WALActionsListener.
1556       listeners.add(this.replicationSourceHandler.getWALActionsListener());
1557     }
1558     return listeners;
1559   }
1560 
1561   protected List<WALActionsListener> getMetaWALActionListeners() {
1562     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1563     // Using a tmp log roller to ensure metaLogRoller is alive once it is not
1564     // null
1565     MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this);
1566     String n = Thread.currentThread().getName();
1567     Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
1568         n + "-MetaLogRoller", uncaughtExceptionHandler);
1569     this.metaHLogRoller = tmpLogRoller;
1570     tmpLogRoller = null;
1571     listeners.add(this.metaHLogRoller);
1572     return listeners;
1573   }
1574 
1575   protected LogRoller getLogRoller() {
1576     return hlogRoller;
1577   }
1578 
1579   public MetricsRegionServer getRegionServerMetrics() {
1580     return this.metricsRegionServer;
1581   }
1582 
1583   /**
1584    * @return Master address tracker instance.
1585    */
1586   public MasterAddressTracker getMasterAddressTracker() {
1587     return this.masterAddressTracker;
1588   }
1589 
1590   /*
1591    * Start maintenance Threads, Server, Worker and lease checker threads.
1592    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1593    * get an unhandled exception. We cannot set the handler on all threads.
1594    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1595    * waits a while then retries. Meantime, a flush or a compaction that tries to
1596    * run should trigger same critical condition and the shutdown will run. On
1597    * its way out, this server will shut down Server. Leases are sort of
1598    * inbetween. It has an internal thread that while it inherits from Chore, it
1599    * keeps its own internal stop mechanism so needs to be stopped by this
1600    * hosting server. Worker logs the exception and exits.
1601    */
1602   private void startServiceThreads() throws IOException {
1603     // Start executor services
1604     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1605       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1606     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1607       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1608     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1609       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1610     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1611       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1612     if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
1613       this.service.startExecutorService(ExecutorType.RS_PARALLEL_SEEK,
1614         conf.getInt("hbase.storescanner.parallel.seek.threads", 10));
1615     }
1616     this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, conf.getInt(
1617        "hbase.regionserver.wal.max.splitters", SplitLogWorkerCoordination.DEFAULT_MAX_SPLITTERS));
1618 
1619     Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), getName() + ".logRoller",
1620         uncaughtExceptionHandler);
1621     this.cacheFlusher.start(uncaughtExceptionHandler);
1622     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), getName() +
1623       ".compactionChecker", uncaughtExceptionHandler);
1624     Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), getName() +
1625         ".periodicFlusher", uncaughtExceptionHandler);
1626     if (this.healthCheckChore != null) {
1627       Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), getName() + ".healthChecker",
1628             uncaughtExceptionHandler);
1629     }
1630     if (this.nonceManagerChore != null) {
1631       Threads.setDaemonThreadRunning(this.nonceManagerChore.getThread(), getName() + ".nonceCleaner",
1632             uncaughtExceptionHandler);
1633     }
1634     if (this.storefileRefresher != null) {
1635       Threads.setDaemonThreadRunning(this.storefileRefresher.getThread(), getName() + ".storefileRefresher",
1636             uncaughtExceptionHandler);
1637     }
1638 
1639     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1640     // an unhandled exception, it will just exit.
1641     this.leases.setName(getName() + ".leaseChecker");
1642     this.leases.start();
1643 
1644     if (this.replicationSourceHandler == this.replicationSinkHandler &&
1645         this.replicationSourceHandler != null) {
1646       this.replicationSourceHandler.startReplicationService();
1647     } else {
1648       if (this.replicationSourceHandler != null) {
1649         this.replicationSourceHandler.startReplicationService();
1650       }
1651       if (this.replicationSinkHandler != null) {
1652         this.replicationSinkHandler.startReplicationService();
1653       }
1654     }
1655 
1656     // Create the log splitting worker and start it
1657     // set a smaller retries to fast fail otherwise splitlogworker could be blocked for
1658     // quite a while inside HConnection layer. The worker won't be available for other
1659     // tasks even after current task is preempted after a split task times out.
1660     Configuration sinkConf = HBaseConfiguration.create(conf);
1661     sinkConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
1662       conf.getInt("hbase.log.replay.retries.number", 8)); // 8 retries take about 23 seconds
1663     sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
1664       conf.getInt("hbase.log.replay.rpc.timeout", 30000)); // default 30 seconds
1665     sinkConf.setInt("hbase.client.serverside.retries.multiplier", 1);
1666     this.splitLogWorker = new SplitLogWorker(this, sinkConf, this, this);
1667     splitLogWorker.start();
1668   }
1669 
1670   /**
1671    * Puts up the webui.
1672    * @return Returns final port -- maybe different from what we started with.
1673    * @throws IOException
1674    */
1675   private int putUpWebUI() throws IOException {
1676     int port = this.conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
1677       HConstants.DEFAULT_REGIONSERVER_INFOPORT);
1678     // -1 is for disabling info server
1679     if (port < 0) return port;
1680     String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1681     if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
1682       String msg =
1683           "Failed to start http info server. Address " + addr
1684               + " does not belong to this host. Correct configuration parameter: "
1685               + "hbase.regionserver.info.bindAddress";
1686       LOG.error(msg);
1687       throw new IOException(msg);
1688     }
1689     // check if auto port bind enabled
1690     boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
1691         false);
1692     while (true) {
1693       try {
1694         this.infoServer = new InfoServer(getProcessName(), addr, port, false, this.conf);
1695         infoServer.addServlet("dump", "/dump", getDumpServlet());
1696         configureInfoServer();
1697         this.infoServer.start();
1698         break;
1699       } catch (BindException e) {
1700         if (!auto) {
1701           // auto bind disabled throw BindException
1702           LOG.error("Failed binding http info server to port: " + port);
1703           throw e;
1704         }
1705         // auto bind enabled, try to use another port
1706         LOG.info("Failed binding http info server to port: " + port);
1707         port++;
1708       }
1709     }
1710     port = this.infoServer.getPort();
1711     conf.setInt(HConstants.REGIONSERVER_INFO_PORT, port);
1712     int masterInfoPort = conf.getInt(HConstants.MASTER_INFO_PORT,
1713       HConstants.DEFAULT_MASTER_INFOPORT);
1714     conf.setInt("hbase.master.info.port.orig", masterInfoPort);
1715     conf.setInt(HConstants.MASTER_INFO_PORT, port);
1716     return port;
1717   }
1718 
1719   /*
1720    * Verify that server is healthy
1721    */
1722   private boolean isHealthy() {
1723     if (!fsOk) {
1724       // File system problem
1725       return false;
1726     }
1727     // Verify that all threads are alive
1728     if (!(leases.isAlive()
1729         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1730         && this.compactionChecker.isAlive()
1731         && this.periodicFlusher.isAlive())) {
1732       stop("One or more threads are no longer alive -- stop");
1733       return false;
1734     }
1735     if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
1736       stop("Meta HLog roller thread is no longer alive -- stop");
1737       return false;
1738     }
1739     return true;
1740   }
1741 
1742   public HLog getWAL() {
1743     try {
1744       return getWAL(null);
1745     } catch (IOException e) {
1746       LOG.warn("getWAL threw exception " + e);
1747       return null;
1748     }
1749   }
1750 
1751   @Override
1752   public HLog getWAL(HRegionInfo regionInfo) throws IOException {
1753     //TODO: at some point this should delegate to the HLogFactory
1754     //currently, we don't care about the region as much as we care about the
1755     //table.. (hence checking the tablename below)
1756     //_ROOT_ and hbase:meta regions have separate WAL.
1757     if (regionInfo != null && regionInfo.isMetaTable()) {
1758       return getMetaWAL();
1759     }
1760     return this.hlog;
1761   }
1762 
1763   @Override
1764   public HConnection getShortCircuitConnection() {
1765     return this.shortCircuitConnection;
1766   }
1767 
1768   @Override
1769   public MetaTableLocator getMetaTableLocator() {
1770     return this.metaTableLocator;
1771   }
1772 
1773   @Override
1774   public void stop(final String msg) {
1775     if (!this.stopped) {
1776       try {
1777         if (this.rsHost != null) {
1778           this.rsHost.preStop(msg);
1779         }
1780         this.stopped = true;
1781         LOG.info("STOPPED: " + msg);
1782         // Wakes run() if it is sleeping
1783         sleeper.skipSleepCycle();
1784       } catch (IOException exp) {
1785         LOG.warn("The region server did not stop", exp);
1786       }
1787     }
1788   }
1789 
1790   public void waitForServerOnline(){
1791     while (!isStopped() && !isOnline()) {
1792       synchronized (online) {
1793         try {
1794           online.wait(msgInterval);
1795         } catch (InterruptedException ie) {
1796           Thread.currentThread().interrupt();
1797           break;
1798         }
1799       }
1800     }
1801   }
1802 
1803   @Override
1804   public void postOpenDeployTasks(final HRegion r)
1805   throws KeeperException, IOException {
1806     rpcServices.checkOpen();
1807     LOG.info("Post open deploy tasks for " + r.getRegionNameAsString());
1808     // Do checks to see if we need to compact (references or too many files)
1809     for (Store s : r.getStores().values()) {
1810       if (s.hasReferences() || s.needsCompaction()) {
1811        this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
1812       }
1813     }
1814     long openSeqNum = r.getOpenSeqNum();
1815     if (openSeqNum == HConstants.NO_SEQNUM) {
1816       // If we opened a region, we should have read some sequence number from it.
1817       LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
1818       openSeqNum = 0;
1819     }
1820 
1821     // Update flushed sequence id of a recovering region in ZK
1822     updateRecoveringRegionLastFlushedSequenceId(r);
1823 
1824     // Notify master
1825     if (!reportRegionStateTransition(
1826         TransitionCode.OPENED, openSeqNum, r.getRegionInfo())) {
1827       throw new IOException("Failed to report opened region to master: "
1828         + r.getRegionNameAsString());
1829     }
1830 
1831     LOG.debug("Finished post open deploy task for " + r.getRegionNameAsString());
1832   }
1833 
1834   @Override
1835   public boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris) {
1836     return reportRegionStateTransition(code, HConstants.NO_SEQNUM, hris);
1837   }
1838 
1839   @Override
1840   public boolean reportRegionStateTransition(
1841       TransitionCode code, long openSeqNum, HRegionInfo... hris) {
1842     if (TEST_SKIP_REPORTING_TRANSITION) {
1843       // This is for testing only in case there is no master
1844       // to handle the region transition report at all.
1845       if (code == TransitionCode.OPENED) {
1846         Preconditions.checkArgument(hris != null && hris.length == 1);
1847         if (hris[0].isMetaRegion()) {
1848           try {
1849             MetaTableLocator.setMetaLocation(getZooKeeper(), serverName, State.OPEN);
1850           } catch (KeeperException e) {
1851             LOG.info("Failed to update meta location", e);
1852             return false;
1853           }
1854         } else {
1855           try {
1856             MetaTableAccessor.updateRegionLocation(shortCircuitConnection,
1857               hris[0], serverName, openSeqNum);
1858           } catch (IOException e) {
1859             LOG.info("Failed to update meta", e);
1860             return false;
1861           }
1862         }
1863       }
1864       return true;
1865     }
1866 
1867     ReportRegionStateTransitionRequest.Builder builder =
1868       ReportRegionStateTransitionRequest.newBuilder();
1869     builder.setServer(ProtobufUtil.toServerName(serverName));
1870     RegionStateTransition.Builder transition = builder.addTransitionBuilder();
1871     transition.setTransitionCode(code);
1872     if (code == TransitionCode.OPENED && openSeqNum >= 0) {
1873       transition.setOpenSeqNum(openSeqNum);
1874     }
1875     for (HRegionInfo hri: hris) {
1876       transition.addRegionInfo(HRegionInfo.convert(hri));
1877     }
1878     ReportRegionStateTransitionRequest request = builder.build();
1879     while (keepLooping()) {
1880       RegionServerStatusService.BlockingInterface rss = rssStub;
1881       try {
1882         if (rss == null) {
1883           createRegionServerStatusStub();
1884           continue;
1885         }
1886         ReportRegionStateTransitionResponse response =
1887           rss.reportRegionStateTransition(null, request);
1888         if (response.hasErrorMessage()) {
1889           LOG.info("Failed to transition " + hris[0]
1890             + " to " + code + ": " + response.getErrorMessage());
1891           return false;
1892         }
1893         return true;
1894       } catch (ServiceException se) {
1895         IOException ioe = ProtobufUtil.getRemoteException(se);
1896         LOG.info("Failed to report region transition, will retry", ioe);
1897         if (rssStub == rss) {
1898           rssStub = null;
1899         }
1900       }
1901     }
1902     return false;
1903   }
1904 
1905   @Override
1906   public RpcServerInterface getRpcServer() {
1907     return rpcServices.rpcServer;
1908   }
1909 
1910   @VisibleForTesting
1911   public RSRpcServices getRSRpcServices() {
1912     return rpcServices;
1913   }
1914 
1915   /**
1916    * Cause the server to exit without closing the regions it is serving, the log
1917    * it is using and without notifying the master. Used unit testing and on
1918    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1919    *
1920    * @param reason
1921    *          the reason we are aborting
1922    * @param cause
1923    *          the exception that caused the abort, or null
1924    */
1925   @Override
1926   public void abort(String reason, Throwable cause) {
1927     String msg = "ABORTING region server " + this + ": " + reason;
1928     if (cause != null) {
1929       LOG.fatal(msg, cause);
1930     } else {
1931       LOG.fatal(msg);
1932     }
1933     this.abortRequested = true;
1934     // HBASE-4014: show list of coprocessors that were loaded to help debug
1935     // regionserver crashes.Note that we're implicitly using
1936     // java.util.HashSet's toString() method to print the coprocessor names.
1937     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1938         CoprocessorHost.getLoadedCoprocessors());
1939     // Do our best to report our abort to the master, but this may not work
1940     try {
1941       if (cause != null) {
1942         msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1943       }
1944       // Report to the master but only if we have already registered with the master.
1945       if (rssStub != null && this.serverName != null) {
1946         ReportRSFatalErrorRequest.Builder builder =
1947           ReportRSFatalErrorRequest.newBuilder();
1948         ServerName sn =
1949           ServerName.parseVersionedServerName(this.serverName.getVersionedBytes());
1950         builder.setServer(ProtobufUtil.toServerName(sn));
1951         builder.setErrorMessage(msg);
1952         rssStub.reportRSFatalError(null, builder.build());
1953       }
1954     } catch (Throwable t) {
1955       LOG.warn("Unable to report fatal error to master", t);
1956     }
1957     stop(reason);
1958   }
1959 
1960   /**
1961    * @see HRegionServer#abort(String, Throwable)
1962    */
1963   public void abort(String reason) {
1964     abort(reason, null);
1965   }
1966 
1967   @Override
1968   public boolean isAborted() {
1969     return this.abortRequested;
1970   }
1971 
1972   /*
1973    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1974    * logs but it does close socket in case want to bring up server on old
1975    * hostname+port immediately.
1976    */
1977   protected void kill() {
1978     this.killed = true;
1979     abort("Simulated kill");
1980   }
1981 
1982   /**
1983    * Wait on all threads to finish. Presumption is that all closes and stops
1984    * have already been called.
1985    */
1986   protected void stopServiceThreads() {
1987     if (this.nonceManagerChore != null) {
1988       Threads.shutdown(this.nonceManagerChore.getThread());
1989     }
1990     if (this.compactionChecker != null) {
1991       Threads.shutdown(this.compactionChecker.getThread());
1992     }
1993     if (this.periodicFlusher != null) {
1994       Threads.shutdown(this.periodicFlusher.getThread());
1995     }
1996     if (this.cacheFlusher != null) {
1997       this.cacheFlusher.join();
1998     }
1999     if (this.healthCheckChore != null) {
2000       Threads.shutdown(this.healthCheckChore.getThread());
2001     }
2002     if (this.spanReceiverHost != null) {
2003       this.spanReceiverHost.closeReceivers();
2004     }
2005     if (this.hlogRoller != null) {
2006       Threads.shutdown(this.hlogRoller.getThread());
2007     }
2008     if (this.metaHLogRoller != null) {
2009       Threads.shutdown(this.metaHLogRoller.getThread());
2010     }
2011     if (this.compactSplitThread != null) {
2012       this.compactSplitThread.join();
2013     }
2014     if (this.service != null) this.service.shutdown();
2015     if (this.replicationSourceHandler != null &&
2016         this.replicationSourceHandler == this.replicationSinkHandler) {
2017       this.replicationSourceHandler.stopReplicationService();
2018     } else {
2019       if (this.replicationSourceHandler != null) {
2020         this.replicationSourceHandler.stopReplicationService();
2021       }
2022       if (this.replicationSinkHandler != null) {
2023         this.replicationSinkHandler.stopReplicationService();
2024       }
2025     }
2026     if (this.storefileRefresher != null) {
2027       Threads.shutdown(this.storefileRefresher.getThread());
2028     }
2029   }
2030 
2031   /**
2032    * @return Return the object that implements the replication
2033    * source service.
2034    */
2035   ReplicationSourceService getReplicationSourceService() {
2036     return replicationSourceHandler;
2037   }
2038 
2039   /**
2040    * @return Return the object that implements the replication
2041    * sink service.
2042    */
2043   ReplicationSinkService getReplicationSinkService() {
2044     return replicationSinkHandler;
2045   }
2046 
2047   /**
2048    * Get the current master from ZooKeeper and open the RPC connection to it.
2049    *
2050    * Method will block until a master is available. You can break from this
2051    * block by requesting the server stop.
2052    *
2053    * @return master + port, or null if server has been stopped
2054    */
2055   private synchronized ServerName createRegionServerStatusStub() {
2056     if (rssStub != null) {
2057       return masterAddressTracker.getMasterAddress();
2058     }
2059     ServerName sn = null;
2060     long previousLogTime = 0;
2061     boolean refresh = false; // for the first time, use cached data
2062     RegionServerStatusService.BlockingInterface intf = null;
2063     boolean interrupted = false;
2064     try {
2065       while (keepLooping()) {
2066         sn = this.masterAddressTracker.getMasterAddress(refresh);
2067         if (sn == null) {
2068           if (!keepLooping()) {
2069             // give up with no connection.
2070             LOG.debug("No master found and cluster is stopped; bailing out");
2071             return null;
2072           }
2073           if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2074             LOG.debug("No master found; retry");
2075             previousLogTime = System.currentTimeMillis();
2076           }
2077           refresh = true; // let's try pull it from ZK directly
2078           if (sleep(200)) {
2079             interrupted = true;
2080           }
2081           continue;
2082         }
2083 
2084         // If we are on the active master, use the shortcut
2085         if (this instanceof HMaster && sn.equals(getServerName())) {
2086           intf = ((HMaster)this).getMasterRpcServices();
2087           break;
2088         }
2089         try {
2090           BlockingRpcChannel channel =
2091             this.rpcClient.createBlockingRpcChannel(sn, userProvider.getCurrent(), operationTimeout);
2092           intf = RegionServerStatusService.newBlockingStub(channel);
2093           break;
2094         } catch (IOException e) {
2095           if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2096             e = e instanceof RemoteException ?
2097               ((RemoteException)e).unwrapRemoteException() : e;
2098             if (e instanceof ServerNotRunningYetException) {
2099               LOG.info("Master isn't available yet, retrying");
2100             } else {
2101               LOG.warn("Unable to connect to master. Retrying. Error was:", e);
2102             }
2103             previousLogTime = System.currentTimeMillis();
2104           }
2105           if (sleep(200)) {
2106             interrupted = true;
2107           }
2108         }
2109       }
2110     } finally {
2111       if (interrupted) {
2112         Thread.currentThread().interrupt();
2113       }
2114     }
2115     rssStub = intf;
2116     return sn;
2117   }
2118 
2119   /**
2120    * @return True if we should break loop because cluster is going down or
2121    * this server has been stopped or hdfs has gone bad.
2122    */
2123   private boolean keepLooping() {
2124     return !this.stopped && isClusterUp();
2125   }
2126 
2127   /*
2128    * Let the master know we're here Run initialization using parameters passed
2129    * us by the master.
2130    * @return A Map of key/value configurations we got from the Master else
2131    * null if we failed to register.
2132    * @throws IOException
2133    */
2134   private RegionServerStartupResponse reportForDuty() throws IOException {
2135     ServerName masterServerName = createRegionServerStatusStub();
2136     if (masterServerName == null) return null;
2137     RegionServerStartupResponse result = null;
2138     try {
2139       rpcServices.requestCount.set(0);
2140       LOG.info("reportForDuty to master=" + masterServerName + " with port="
2141         + rpcServices.isa.getPort() + ", startcode=" + this.startcode);
2142       long now = EnvironmentEdgeManager.currentTime();
2143       int port = rpcServices.isa.getPort();
2144       RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
2145       request.setPort(port);
2146       request.setServerStartCode(this.startcode);
2147       request.setServerCurrentTime(now);
2148       result = this.rssStub.regionServerStartup(null, request.build());
2149     } catch (ServiceException se) {
2150       IOException ioe = ProtobufUtil.getRemoteException(se);
2151       if (ioe instanceof ClockOutOfSyncException) {
2152         LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2153         // Re-throw IOE will cause RS to abort
2154         throw ioe;
2155       } else if (ioe instanceof ServerNotRunningYetException) {
2156         LOG.debug("Master is not running yet");
2157       } else {
2158         LOG.warn("error telling master we are up", se);
2159       }
2160     }
2161     return result;
2162   }
2163 
2164   @Override
2165   public long getLastSequenceId(byte[] region) {
2166     Long lastFlushedSequenceId = -1l;
2167     try {
2168       GetLastFlushedSequenceIdRequest req = RequestConverter
2169           .buildGetLastFlushedSequenceIdRequest(region);
2170       RegionServerStatusService.BlockingInterface rss = rssStub;
2171       if (rss == null) { // Try to connect one more time
2172         createRegionServerStatusStub();
2173         rss = rssStub;
2174         if (rss == null) {
2175           // Still no luck, we tried
2176           LOG.warn("Unable to connect to the master to check "
2177             + "the last flushed sequence id");
2178           return -1l;
2179         }
2180       }
2181       lastFlushedSequenceId = rss.getLastFlushedSequenceId(null, req)
2182           .getLastFlushedSequenceId();
2183     } catch (ServiceException e) {
2184       lastFlushedSequenceId = -1l;
2185       LOG.warn("Unable to connect to the master to check "
2186         + "the last flushed sequence id", e);
2187     }
2188     return lastFlushedSequenceId;
2189   }
2190 
2191   /**
2192    * Closes all regions.  Called on our way out.
2193    * Assumes that its not possible for new regions to be added to onlineRegions
2194    * while this method runs.
2195    */
2196   protected void closeAllRegions(final boolean abort) {
2197     closeUserRegions(abort);
2198     closeMetaTableRegions(abort);
2199   }
2200 
2201   /**
2202    * Close meta region if we carry it
2203    * @param abort Whether we're running an abort.
2204    */
2205   void closeMetaTableRegions(final boolean abort) {
2206     HRegion meta = null;
2207     this.lock.writeLock().lock();
2208     try {
2209       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2210         HRegionInfo hri = e.getValue().getRegionInfo();
2211         if (hri.isMetaRegion()) {
2212           meta = e.getValue();
2213         }
2214         if (meta != null) break;
2215       }
2216     } finally {
2217       this.lock.writeLock().unlock();
2218     }
2219     if (meta != null) closeRegionIgnoreErrors(meta.getRegionInfo(), abort);
2220   }
2221 
2222   /**
2223    * Schedule closes on all user regions.
2224    * Should be safe calling multiple times because it wont' close regions
2225    * that are already closed or that are closing.
2226    * @param abort Whether we're running an abort.
2227    */
2228   void closeUserRegions(final boolean abort) {
2229     this.lock.writeLock().lock();
2230     try {
2231       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2232         HRegion r = e.getValue();
2233         if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2234           // Don't update zk with this close transition; pass false.
2235           closeRegionIgnoreErrors(r.getRegionInfo(), abort);
2236         }
2237       }
2238     } finally {
2239       this.lock.writeLock().unlock();
2240     }
2241   }
2242 
2243   /** @return the info server */
2244   public InfoServer getInfoServer() {
2245     return infoServer;
2246   }
2247 
2248   /**
2249    * @return true if a stop has been requested.
2250    */
2251   @Override
2252   public boolean isStopped() {
2253     return this.stopped;
2254   }
2255 
2256   @Override
2257   public boolean isStopping() {
2258     return this.stopping;
2259   }
2260 
2261   @Override
2262   public Map<String, HRegion> getRecoveringRegions() {
2263     return this.recoveringRegions;
2264   }
2265 
2266   /**
2267    *
2268    * @return the configuration
2269    */
2270   @Override
2271   public Configuration getConfiguration() {
2272     return conf;
2273   }
2274 
2275   /** @return the write lock for the server */
2276   ReentrantReadWriteLock.WriteLock getWriteLock() {
2277     return lock.writeLock();
2278   }
2279 
2280   public int getNumberOfOnlineRegions() {
2281     return this.onlineRegions.size();
2282   }
2283 
2284   boolean isOnlineRegionsEmpty() {
2285     return this.onlineRegions.isEmpty();
2286   }
2287 
2288   /**
2289    * For tests, web ui and metrics.
2290    * This method will only work if HRegionServer is in the same JVM as client;
2291    * HRegion cannot be serialized to cross an rpc.
2292    */
2293   public Collection<HRegion> getOnlineRegionsLocalContext() {
2294     Collection<HRegion> regions = this.onlineRegions.values();
2295     return Collections.unmodifiableCollection(regions);
2296   }
2297 
2298   @Override
2299   public void addToOnlineRegions(HRegion region) {
2300     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2301     configurationManager.registerObserver(region);
2302   }
2303 
2304   /**
2305    * @return A new Map of online regions sorted by region size with the first entry being the
2306    * biggest.  If two regions are the same size, then the last one found wins; i.e. this method
2307    * may NOT return all regions.
2308    */
2309   SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
2310     // we'll sort the regions in reverse
2311     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
2312         new Comparator<Long>() {
2313           @Override
2314           public int compare(Long a, Long b) {
2315             return -1 * a.compareTo(b);
2316           }
2317         });
2318     // Copy over all regions. Regions are sorted by size with biggest first.
2319     for (HRegion region : this.onlineRegions.values()) {
2320       sortedRegions.put(region.memstoreSize.get(), region);
2321     }
2322     return sortedRegions;
2323   }
2324 
2325   /**
2326    * @return time stamp in millis of when this region server was started
2327    */
2328   public long getStartcode() {
2329     return this.startcode;
2330   }
2331 
2332   /** @return reference to FlushRequester */
2333   @Override
2334   public FlushRequester getFlushRequester() {
2335     return this.cacheFlusher;
2336   }
2337 
2338   /**
2339    * Get the top N most loaded regions this server is serving so we can tell the
2340    * master which regions it can reallocate if we're overloaded. TODO: actually
2341    * calculate which regions are most loaded. (Right now, we're just grabbing
2342    * the first N regions being served regardless of load.)
2343    */
2344   protected HRegionInfo[] getMostLoadedRegions() {
2345     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2346     for (HRegion r : onlineRegions.values()) {
2347       if (!r.isAvailable()) {
2348         continue;
2349       }
2350       if (regions.size() < numRegionsToReport) {
2351         regions.add(r.getRegionInfo());
2352       } else {
2353         break;
2354       }
2355     }
2356     return regions.toArray(new HRegionInfo[regions.size()]);
2357   }
2358 
2359   @Override
2360   public Leases getLeases() {
2361     return leases;
2362   }
2363 
2364   /**
2365    * @return Return the rootDir.
2366    */
2367   protected Path getRootDir() {
2368     return rootDir;
2369   }
2370 
2371   /**
2372    * @return Return the fs.
2373    */
2374   @Override
2375   public FileSystem getFileSystem() {
2376     return fs;
2377   }
2378 
2379   @Override
2380   public String toString() {
2381     return getServerName().toString();
2382   }
2383 
2384   /**
2385    * Interval at which threads should run
2386    *
2387    * @return the interval
2388    */
2389   public int getThreadWakeFrequency() {
2390     return threadWakeFrequency;
2391   }
2392 
2393   @Override
2394   public ZooKeeperWatcher getZooKeeper() {
2395     return zooKeeper;
2396   }
2397 
2398   @Override
2399   public BaseCoordinatedStateManager getCoordinatedStateManager() {
2400     return csm;
2401   }
2402 
2403   @Override
2404   public ServerName getServerName() {
2405     return serverName;
2406   }
2407 
2408   @Override
2409   public CompactionRequestor getCompactionRequester() {
2410     return this.compactSplitThread;
2411   }
2412 
2413   public RegionServerCoprocessorHost getRegionServerCoprocessorHost(){
2414     return this.rsHost;
2415   }
2416 
2417   @Override
2418   public ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS() {
2419     return this.regionsInTransitionInRS;
2420   }
2421 
2422   @Override
2423   public ExecutorService getExecutorService() {
2424     return service;
2425   }
2426 
2427   @Override
2428   public RegionServerQuotaManager getRegionServerQuotaManager() {
2429     return rsQuotaManager;
2430   }
2431 
2432   //
2433   // Main program and support routines
2434   //
2435 
2436   /**
2437    * Load the replication service objects, if any
2438    */
2439   static private void createNewReplicationInstance(Configuration conf,
2440     HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
2441 
2442     // If replication is not enabled, then return immediately.
2443     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
2444         HConstants.REPLICATION_ENABLE_DEFAULT)) {
2445       return;
2446     }
2447 
2448     // read in the name of the source replication class from the config file.
2449     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
2450                                HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2451 
2452     // read in the name of the sink replication class from the config file.
2453     String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
2454                              HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2455 
2456     // If both the sink and the source class names are the same, then instantiate
2457     // only one object.
2458     if (sourceClassname.equals(sinkClassname)) {
2459       server.replicationSourceHandler = (ReplicationSourceService)
2460                                          newReplicationInstance(sourceClassname,
2461                                          conf, server, fs, logDir, oldLogDir);
2462       server.replicationSinkHandler = (ReplicationSinkService)
2463                                          server.replicationSourceHandler;
2464     } else {
2465       server.replicationSourceHandler = (ReplicationSourceService)
2466                                          newReplicationInstance(sourceClassname,
2467                                          conf, server, fs, logDir, oldLogDir);
2468       server.replicationSinkHandler = (ReplicationSinkService)
2469                                          newReplicationInstance(sinkClassname,
2470                                          conf, server, fs, logDir, oldLogDir);
2471     }
2472   }
2473 
2474   static private ReplicationService newReplicationInstance(String classname,
2475     Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
2476     Path oldLogDir) throws IOException{
2477 
2478     Class<?> clazz = null;
2479     try {
2480       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
2481       clazz = Class.forName(classname, true, classLoader);
2482     } catch (java.lang.ClassNotFoundException nfe) {
2483       throw new IOException("Could not find class for " + classname);
2484     }
2485 
2486     // create an instance of the replication object.
2487     ReplicationService service = (ReplicationService)
2488                               ReflectionUtils.newInstance(clazz, conf);
2489     service.initialize(server, fs, logDir, oldLogDir);
2490     return service;
2491   }
2492 
2493   /**
2494    * Utility for constructing an instance of the passed HRegionServer class.
2495    *
2496    * @param regionServerClass
2497    * @param conf2
2498    * @return HRegionServer instance.
2499    */
2500   public static HRegionServer constructRegionServer(
2501       Class<? extends HRegionServer> regionServerClass,
2502       final Configuration conf2, CoordinatedStateManager cp) {
2503     try {
2504       Constructor<? extends HRegionServer> c = regionServerClass
2505           .getConstructor(Configuration.class, CoordinatedStateManager.class);
2506       return c.newInstance(conf2, cp);
2507     } catch (Exception e) {
2508       throw new RuntimeException("Failed construction of " + "Regionserver: "
2509           + regionServerClass.toString(), e);
2510     }
2511   }
2512 
2513   /**
2514    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
2515    */
2516   public static void main(String[] args) throws Exception {
2517 	VersionInfo.logVersion();
2518     Configuration conf = HBaseConfiguration.create();
2519     @SuppressWarnings("unchecked")
2520     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
2521         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
2522 
2523     new HRegionServerCommandLine(regionServerClass).doMain(args);
2524   }
2525 
2526   /**
2527    * Gets the online regions of the specified table.
2528    * This method looks at the in-memory onlineRegions.  It does not go to <code>hbase:meta</code>.
2529    * Only returns <em>online</em> regions.  If a region on this table has been
2530    * closed during a disable, etc., it will not be included in the returned list.
2531    * So, the returned list may not necessarily be ALL regions in this table, its
2532    * all the ONLINE regions in the table.
2533    * @param tableName
2534    * @return Online regions from <code>tableName</code>
2535    */
2536   @Override
2537   public List<HRegion> getOnlineRegions(TableName tableName) {
2538      List<HRegion> tableRegions = new ArrayList<HRegion>();
2539      synchronized (this.onlineRegions) {
2540        for (HRegion region: this.onlineRegions.values()) {
2541          HRegionInfo regionInfo = region.getRegionInfo();
2542          if(regionInfo.getTable().equals(tableName)) {
2543            tableRegions.add(region);
2544          }
2545        }
2546      }
2547      return tableRegions;
2548    }
2549 
2550   /**
2551    * Gets the online tables in this RS.
2552    * This method looks at the in-memory onlineRegions.
2553    * @return all the online tables in this RS
2554    */
2555   @Override
2556   public Set<TableName> getOnlineTables() {
2557     Set<TableName> tables = new HashSet<TableName>();
2558     synchronized (this.onlineRegions) {
2559       for (HRegion region: this.onlineRegions.values()) {
2560         tables.add(region.getTableDesc().getTableName());
2561       }
2562     }
2563     return tables;
2564   }
2565 
2566   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
2567   public String[] getRegionServerCoprocessors() {
2568     TreeSet<String> coprocessors = new TreeSet<String>(
2569         this.hlog.getCoprocessorHost().getCoprocessors());
2570     Collection<HRegion> regions = getOnlineRegionsLocalContext();
2571     for (HRegion region: regions) {
2572       coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
2573     }
2574     return coprocessors.toArray(new String[coprocessors.size()]);
2575   }
2576 
2577   /**
2578    * Try to close the region, logs a warning on failure but continues.
2579    * @param region Region to close
2580    */
2581   private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
2582     try {
2583       if (!closeRegion(region.getEncodedName(), abort, null)) {
2584         LOG.warn("Failed to close " + region.getRegionNameAsString() +
2585             " - ignoring and continuing");
2586       }
2587     } catch (IOException e) {
2588       LOG.warn("Failed to close " + region.getRegionNameAsString() +
2589           " - ignoring and continuing", e);
2590     }
2591   }
2592 
2593   /**
2594    * Close asynchronously a region, can be called from the master or internally by the regionserver
2595    * when stopping. If called from the master, the region will update the znode status.
2596    *
2597    * <p>
2598    * If an opening was in progress, this method will cancel it, but will not start a new close. The
2599    * coprocessors are not called in this case. A NotServingRegionException exception is thrown.
2600    * </p>
2601 
2602    * <p>
2603    *   If a close was in progress, this new request will be ignored, and an exception thrown.
2604    * </p>
2605    *
2606    * @param encodedName Region to close
2607    * @param abort True if we are aborting
2608    * @return True if closed a region.
2609    * @throws NotServingRegionException if the region is not online
2610    */
2611   protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
2612       throws NotServingRegionException {
2613     //Check for permissions to close.
2614     HRegion actualRegion = this.getFromOnlineRegions(encodedName);
2615     if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
2616       try {
2617         actualRegion.getCoprocessorHost().preClose(false);
2618       } catch (IOException exp) {
2619         LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
2620         return false;
2621       }
2622     }
2623 
2624     final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(encodedName.getBytes(),
2625         Boolean.FALSE);
2626 
2627     if (Boolean.TRUE.equals(previous)) {
2628       LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
2629           "trying to OPEN. Cancelling OPENING.");
2630       if (!regionsInTransitionInRS.replace(encodedName.getBytes(), previous, Boolean.FALSE)){
2631         // The replace failed. That should be an exceptional case, but theoretically it can happen.
2632         // We're going to try to do a standard close then.
2633         LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
2634             " Doing a standard close now");
2635         return closeRegion(encodedName, abort, sn);
2636       }
2637       // Let's get the region from the online region list again
2638       actualRegion = this.getFromOnlineRegions(encodedName);
2639       if (actualRegion == null) { // If already online, we still need to close it.
2640         LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
2641         // The master deletes the znode when it receives this exception.
2642         throw new NotServingRegionException("The region " + encodedName +
2643           " was opening but not yet served. Opening is cancelled.");
2644       }
2645     } else if (Boolean.FALSE.equals(previous)) {
2646       LOG.info("Received CLOSE for the region: " + encodedName +
2647         ", which we are already trying to CLOSE, but not completed yet");
2648       return true;
2649     }
2650 
2651     if (actualRegion == null) {
2652       LOG.debug("Received CLOSE for a region which is not online, and we're not opening.");
2653       this.regionsInTransitionInRS.remove(encodedName.getBytes());
2654       // The master deletes the znode when it receives this exception.
2655       throw new NotServingRegionException("The region " + encodedName +
2656           " is not online, and is not opening.");
2657     }
2658 
2659     CloseRegionHandler crh;
2660     final HRegionInfo hri = actualRegion.getRegionInfo();
2661     if (hri.isMetaRegion()) {
2662       crh = new CloseMetaHandler(this, this, hri, abort);
2663     } else {
2664       crh = new CloseRegionHandler(this, this, hri, abort, sn);
2665     }
2666     this.service.submit(crh);
2667     return true;
2668   }
2669 
2670    /**
2671    * @param regionName
2672    * @return HRegion for the passed binary <code>regionName</code> or null if
2673    *         named region is not member of the online regions.
2674    */
2675   public HRegion getOnlineRegion(final byte[] regionName) {
2676     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2677     return this.onlineRegions.get(encodedRegionName);
2678   }
2679 
2680   public InetSocketAddress[] getRegionBlockLocations(final String encodedRegionName) {
2681     return this.regionFavoredNodesMap.get(encodedRegionName);
2682   }
2683 
2684   @Override
2685   public HRegion getFromOnlineRegions(final String encodedRegionName) {
2686     return this.onlineRegions.get(encodedRegionName);
2687   }
2688 
2689 
2690   @Override
2691   public boolean removeFromOnlineRegions(final HRegion r, ServerName destination) {
2692     HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
2693 
2694     if (destination != null) {
2695       HLog wal = getWAL();
2696       long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes());
2697       if (closeSeqNum == HConstants.NO_SEQNUM) {
2698         // No edits in WAL for this region; get the sequence number when the region was opened.
2699         closeSeqNum = r.getOpenSeqNum();
2700         if (closeSeqNum == HConstants.NO_SEQNUM) {
2701           closeSeqNum = 0;
2702         }
2703       }
2704       addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
2705     }
2706     this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
2707     return toReturn != null;
2708   }
2709 
2710   /**
2711    * Protected utility method for safely obtaining an HRegion handle.
2712    *
2713    * @param regionName
2714    *          Name of online {@link HRegion} to return
2715    * @return {@link HRegion} for <code>regionName</code>
2716    * @throws NotServingRegionException
2717    */
2718   protected HRegion getRegion(final byte[] regionName)
2719       throws NotServingRegionException {
2720     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2721     return getRegionByEncodedName(regionName, encodedRegionName);
2722   }
2723 
2724   public HRegion getRegionByEncodedName(String encodedRegionName)
2725       throws NotServingRegionException {
2726     return getRegionByEncodedName(null, encodedRegionName);
2727   }
2728 
2729   protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName)
2730     throws NotServingRegionException {
2731     HRegion region = this.onlineRegions.get(encodedRegionName);
2732     if (region == null) {
2733       MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
2734       if (moveInfo != null) {
2735         throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
2736       }
2737       Boolean isOpening = this.regionsInTransitionInRS.get(Bytes.toBytes(encodedRegionName));
2738       String regionNameStr = regionName == null?
2739         encodedRegionName: Bytes.toStringBinary(regionName);
2740       if (isOpening != null && isOpening.booleanValue()) {
2741         throw new RegionOpeningException("Region " + regionNameStr +
2742           " is opening on " + this.serverName);
2743       }
2744       throw new NotServingRegionException("Region " + regionNameStr +
2745         " is not online on " + this.serverName);
2746     }
2747     return region;
2748   }
2749 
2750   /*
2751    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
2752    * IOE if it isn't already.
2753    *
2754    * @param t Throwable
2755    *
2756    * @param msg Message to log in error. Can be null.
2757    *
2758    * @return Throwable converted to an IOE; methods can only let out IOEs.
2759    */
2760   private Throwable cleanup(final Throwable t, final String msg) {
2761     // Don't log as error if NSRE; NSRE is 'normal' operation.
2762     if (t instanceof NotServingRegionException) {
2763       LOG.debug("NotServingRegionException; " + t.getMessage());
2764       return t;
2765     }
2766     Throwable e = t instanceof RemoteException ? ((RemoteException) t).unwrapRemoteException() : t;
2767     if (msg == null) {
2768       LOG.error("", e);
2769     } else {
2770       LOG.error(msg, e);
2771     }
2772     if (!rpcServices.checkOOME(t)) {
2773       checkFileSystem();
2774     }
2775     return t;
2776   }
2777 
2778   /*
2779    * @param t
2780    *
2781    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
2782    *
2783    * @return Make <code>t</code> an IOE if it isn't already.
2784    */
2785   protected IOException convertThrowableToIOE(final Throwable t, final String msg) {
2786     return (t instanceof IOException ? (IOException) t : msg == null
2787         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
2788   }
2789 
2790   /**
2791    * Checks to see if the file system is still accessible. If not, sets
2792    * abortRequested and stopRequested
2793    *
2794    * @return false if file system is not available
2795    */
2796   public boolean checkFileSystem() {
2797     if (this.fsOk && this.fs != null) {
2798       try {
2799         FSUtils.checkFileSystemAvailable(this.fs);
2800       } catch (IOException e) {
2801         abort("File System not available", e);
2802         this.fsOk = false;
2803       }
2804     }
2805     return this.fsOk;
2806   }
2807 
2808   @Override
2809   public void updateRegionFavoredNodesMapping(String encodedRegionName,
2810       List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName> favoredNodes) {
2811     InetSocketAddress[] addr = new InetSocketAddress[favoredNodes.size()];
2812     // Refer to the comment on the declaration of regionFavoredNodesMap on why
2813     // it is a map of region name to InetSocketAddress[]
2814     for (int i = 0; i < favoredNodes.size(); i++) {
2815       addr[i] = InetSocketAddress.createUnresolved(favoredNodes.get(i).getHostName(),
2816           favoredNodes.get(i).getPort());
2817     }
2818     regionFavoredNodesMap.put(encodedRegionName, addr);
2819   }
2820 
2821   /**
2822    * Return the favored nodes for a region given its encoded name. Look at the
2823    * comment around {@link #regionFavoredNodesMap} on why it is InetSocketAddress[]
2824    * @param encodedRegionName
2825    * @return array of favored locations
2826    */
2827   @Override
2828   public InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName) {
2829     return regionFavoredNodesMap.get(encodedRegionName);
2830   }
2831 
2832   @Override
2833   public ServerNonceManager getNonceManager() {
2834     return this.nonceManager;
2835   }
2836 
2837   private static class MovedRegionInfo {
2838     private final ServerName serverName;
2839     private final long seqNum;
2840     private final long ts;
2841 
2842     public MovedRegionInfo(ServerName serverName, long closeSeqNum) {
2843       this.serverName = serverName;
2844       this.seqNum = closeSeqNum;
2845       ts = EnvironmentEdgeManager.currentTime();
2846      }
2847 
2848     public ServerName getServerName() {
2849       return serverName;
2850     }
2851 
2852     public long getSeqNum() {
2853       return seqNum;
2854     }
2855 
2856     public long getMoveTime() {
2857       return ts;
2858     }
2859   }
2860 
2861   // This map will contains all the regions that we closed for a move.
2862   //  We add the time it was moved as we don't want to keep too old information
2863   protected Map<String, MovedRegionInfo> movedRegions =
2864       new ConcurrentHashMap<String, MovedRegionInfo>(3000);
2865 
2866   // We need a timeout. If not there is a risk of giving a wrong information: this would double
2867   //  the number of network calls instead of reducing them.
2868   private static final int TIMEOUT_REGION_MOVED = (2 * 60 * 1000);
2869 
2870   protected void addToMovedRegions(String encodedName, ServerName destination, long closeSeqNum) {
2871     if (ServerName.isSameHostnameAndPort(destination, this.getServerName())) {
2872       LOG.warn("Not adding moved region record: " + encodedName + " to self.");
2873       return;
2874     }
2875     LOG.info("Adding moved region record: "
2876       + encodedName + " to " + destination + " as of " + closeSeqNum);
2877     movedRegions.put(encodedName, new MovedRegionInfo(destination, closeSeqNum));
2878   }
2879 
2880   void removeFromMovedRegions(String encodedName) {
2881     movedRegions.remove(encodedName);
2882   }
2883 
2884   private MovedRegionInfo getMovedRegion(final String encodedRegionName) {
2885     MovedRegionInfo dest = movedRegions.get(encodedRegionName);
2886 
2887     long now = EnvironmentEdgeManager.currentTime();
2888     if (dest != null) {
2889       if (dest.getMoveTime() > (now - TIMEOUT_REGION_MOVED)) {
2890         return dest;
2891       } else {
2892         movedRegions.remove(encodedRegionName);
2893       }
2894     }
2895 
2896     return null;
2897   }
2898 
2899   /**
2900    * Remove the expired entries from the moved regions list.
2901    */
2902   protected void cleanMovedRegions() {
2903     final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED;
2904     Iterator<Entry<String, MovedRegionInfo>> it = movedRegions.entrySet().iterator();
2905 
2906     while (it.hasNext()){
2907       Map.Entry<String, MovedRegionInfo> e = it.next();
2908       if (e.getValue().getMoveTime() < cutOff) {
2909         it.remove();
2910       }
2911     }
2912   }
2913 
2914   /**
2915    * Creates a Chore thread to clean the moved region cache.
2916    */
2917   protected static class MovedRegionsCleaner extends Chore implements Stoppable {
2918     private HRegionServer regionServer;
2919     Stoppable stoppable;
2920 
2921     private MovedRegionsCleaner(
2922       HRegionServer regionServer, Stoppable stoppable){
2923       super("MovedRegionsCleaner for region "+regionServer, TIMEOUT_REGION_MOVED, stoppable);
2924       this.regionServer = regionServer;
2925       this.stoppable = stoppable;
2926     }
2927 
2928     static MovedRegionsCleaner createAndStart(HRegionServer rs){
2929       Stoppable stoppable = new Stoppable() {
2930         private volatile boolean isStopped = false;
2931         @Override public void stop(String why) { isStopped = true;}
2932         @Override public boolean isStopped() {return isStopped;}
2933       };
2934 
2935       return new MovedRegionsCleaner(rs, stoppable);
2936     }
2937 
2938     @Override
2939     protected void chore() {
2940       regionServer.cleanMovedRegions();
2941     }
2942 
2943     @Override
2944     public void stop(String why) {
2945       stoppable.stop(why);
2946     }
2947 
2948     @Override
2949     public boolean isStopped() {
2950       return stoppable.isStopped();
2951     }
2952   }
2953 
2954   private String getMyEphemeralNodePath() {
2955     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
2956   }
2957 
2958   private boolean isHealthCheckerConfigured() {
2959     String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
2960     return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
2961   }
2962 
2963   /**
2964    * @return the underlying {@link CompactSplitThread} for the servers
2965    */
2966   public CompactSplitThread getCompactSplitThread() {
2967     return this.compactSplitThread;
2968   }
2969 
2970   /**
2971    * A helper function to store the last flushed sequence Id with the previous failed RS for a
2972    * recovering region. The Id is used to skip wal edits which are flushed. Since the flushed
2973    * sequence id is only valid for each RS, we associate the Id with corresponding failed RS.
2974    * @throws KeeperException
2975    * @throws IOException
2976    */
2977   private void updateRecoveringRegionLastFlushedSequenceId(HRegion r) throws KeeperException,
2978       IOException {
2979     if (!r.isRecovering()) {
2980       // return immdiately for non-recovering regions
2981       return;
2982     }
2983 
2984     HRegionInfo region = r.getRegionInfo();
2985     ZooKeeperWatcher zkw = getZooKeeper();
2986     String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName());
2987     Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqIdForLogReplay();
2988     long minSeqIdForLogReplay = -1;
2989     for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) {
2990       if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) {
2991         minSeqIdForLogReplay = storeSeqIdForReplay;
2992       }
2993     }
2994 
2995     try {
2996       long lastRecordedFlushedSequenceId = -1;
2997       String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
2998         region.getEncodedName());
2999       // recovering-region level
3000       byte[] data;
3001       try {
3002         data = ZKUtil.getData(zkw, nodePath);
3003       } catch (InterruptedException e) {
3004         throw new InterruptedIOException();
3005       }
3006       if (data != null) {
3007       lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data);
3008       }
3009       if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
3010         ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
3011       }
3012       if (previousRSName != null) {
3013         // one level deeper for the failed RS
3014         nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
3015         ZKUtil.setData(zkw, nodePath,
3016           ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
3017         LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for "
3018             + previousRSName);
3019       } else {
3020         LOG.warn("Can't find failed region server for recovering region " +
3021           region.getEncodedName());
3022       }
3023     } catch (NoNodeException ignore) {
3024       LOG.debug("Region " + region.getEncodedName() +
3025         " must have completed recovery because its recovery znode has been removed", ignore);
3026     }
3027   }
3028 
3029   /**
3030    * Return the last failed RS name under /hbase/recovering-regions/encodedRegionName
3031    * @param encodedRegionName
3032    * @throws KeeperException
3033    */
3034   private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
3035     String result = null;
3036     long maxZxid = 0;
3037     ZooKeeperWatcher zkw = this.getZooKeeper();
3038     String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
3039     List<String> failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath);
3040     if (failedServers == null || failedServers.isEmpty()) {
3041       return result;
3042     }
3043     for (String failedServer : failedServers) {
3044       String rsPath = ZKUtil.joinZNode(nodePath, failedServer);
3045       Stat stat = new Stat();
3046       ZKUtil.getDataNoWatch(zkw, rsPath, stat);
3047       if (maxZxid < stat.getCzxid()) {
3048         maxZxid = stat.getCzxid();
3049         result = failedServer;
3050       }
3051     }
3052     return result;
3053   }
3054   
3055   public CoprocessorServiceResponse execRegionServerService(final RpcController controller,
3056       final CoprocessorServiceRequest serviceRequest) throws ServiceException {
3057     try {
3058       ServerRpcController execController = new ServerRpcController();
3059       CoprocessorServiceCall call = serviceRequest.getCall();
3060       String serviceName = call.getServiceName();
3061       String methodName = call.getMethodName();
3062       if (!coprocessorServiceHandlers.containsKey(serviceName)) {
3063         throw new UnknownProtocolException(null,
3064             "No registered coprocessor service found for name " + serviceName);
3065       }
3066       Service service = coprocessorServiceHandlers.get(serviceName);
3067       Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
3068       Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
3069       if (methodDesc == null) {
3070         throw new UnknownProtocolException(service.getClass(), "Unknown method " + methodName
3071             + " called on service " + serviceName);
3072       }
3073       Message request =
3074           service.getRequestPrototype(methodDesc).newBuilderForType().mergeFrom(call.getRequest())
3075               .build();
3076       final Message.Builder responseBuilder =
3077           service.getResponsePrototype(methodDesc).newBuilderForType();
3078       service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
3079         @Override
3080         public void run(Message message) {
3081           if (message != null) {
3082             responseBuilder.mergeFrom(message);
3083           }
3084         }
3085       });
3086       Message execResult = responseBuilder.build();
3087       if (execController.getFailedOn() != null) {
3088         throw execController.getFailedOn();
3089       }
3090       ClientProtos.CoprocessorServiceResponse.Builder builder =
3091           ClientProtos.CoprocessorServiceResponse.newBuilder();
3092       builder.setRegion(RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME,
3093         HConstants.EMPTY_BYTE_ARRAY));
3094       builder.setValue(builder.getValueBuilder().setName(execResult.getClass().getName())
3095           .setValue(execResult.toByteString()));
3096       return builder.build();
3097     } catch (IOException ie) {
3098       throw new ServiceException(ie);
3099     }
3100   }
3101   
3102   /**
3103    * @return The cache config instance used by the regionserver.
3104    */
3105   public CacheConfig getCacheConfig() {
3106     return this.cacheConfig;
3107   }
3108 
3109   /**
3110    * @return : Returns the ConfigurationManager object for testing purposes.
3111    */
3112   protected ConfigurationManager getConfigurationManager() {
3113     return configurationManager;
3114   }
3115     
3116   /**
3117    * Reload the configuration from disk.
3118    */
3119   public void updateConfiguration() {
3120     LOG.info("Reloading the configuration from disk.");
3121     // Reload the configuration from disk.
3122     conf.reloadConfiguration();
3123     configurationManager.notifyAllObservers(conf);  
3124   }
3125 }