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.annotation.Retention;
25  import java.lang.annotation.RetentionPolicy;
26  import java.lang.management.ManagementFactory;
27  import java.lang.management.MemoryUsage;
28  import java.lang.reflect.Constructor;
29  import java.net.BindException;
30  import java.net.InetSocketAddress;
31  import java.util.ArrayList;
32  import java.util.Collection;
33  import java.util.Collections;
34  import java.util.Comparator;
35  import java.util.HashMap;
36  import java.util.HashSet;
37  import java.util.Iterator;
38  import java.util.List;
39  import java.util.Map;
40  import java.util.Map.Entry;
41  import java.util.Random;
42  import java.util.Set;
43  import java.util.SortedMap;
44  import java.util.TreeMap;
45  import java.util.TreeSet;
46  import java.util.concurrent.atomic.AtomicLong;
47  import java.util.concurrent.ConcurrentHashMap;
48  import java.util.concurrent.ConcurrentMap;
49  import java.util.concurrent.ConcurrentSkipListMap;
50  import java.util.concurrent.locks.ReentrantReadWriteLock;
51  
52  import javax.management.ObjectName;
53  
54  import com.google.common.annotations.VisibleForTesting;
55  import com.google.protobuf.HBaseZeroCopyByteString;
56  import org.apache.commons.logging.Log;
57  import org.apache.commons.logging.LogFactory;
58  import org.apache.hadoop.classification.InterfaceAudience;
59  import org.apache.hadoop.conf.Configuration;
60  import org.apache.hadoop.fs.FileSystem;
61  import org.apache.hadoop.fs.Path;
62  import org.apache.hadoop.hbase.Cell;
63  import org.apache.hadoop.hbase.CellScannable;
64  import org.apache.hadoop.hbase.CellScanner;
65  import org.apache.hadoop.hbase.CellUtil;
66  import org.apache.hadoop.hbase.Chore;
67  import org.apache.hadoop.hbase.ClockOutOfSyncException;
68  import org.apache.hadoop.hbase.DoNotRetryIOException;
69  import org.apache.hadoop.hbase.HBaseConfiguration;
70  import org.apache.hadoop.hbase.HBaseIOException;
71  import org.apache.hadoop.hbase.HConstants;
72  import org.apache.hadoop.hbase.HRegionInfo;
73  import org.apache.hadoop.hbase.HTableDescriptor;
74  import org.apache.hadoop.hbase.HealthCheckChore;
75  import org.apache.hadoop.hbase.KeyValueUtil;
76  import org.apache.hadoop.hbase.NotServingRegionException;
77  import org.apache.hadoop.hbase.RemoteExceptionHandler;
78  import org.apache.hadoop.hbase.ServerName;
79  import org.apache.hadoop.hbase.Stoppable;
80  import org.apache.hadoop.hbase.TableDescriptors;
81  import org.apache.hadoop.hbase.TableName;
82  import org.apache.hadoop.hbase.UnknownScannerException;
83  import org.apache.hadoop.hbase.YouAreDeadException;
84  import org.apache.hadoop.hbase.ZNodeClearer;
85  import org.apache.hadoop.hbase.catalog.CatalogTracker;
86  import org.apache.hadoop.hbase.catalog.MetaEditor;
87  import org.apache.hadoop.hbase.catalog.MetaReader;
88  import org.apache.hadoop.hbase.client.Append;
89  import org.apache.hadoop.hbase.client.ConnectionUtils;
90  import org.apache.hadoop.hbase.client.Delete;
91  import org.apache.hadoop.hbase.client.Get;
92  import org.apache.hadoop.hbase.client.Increment;
93  import org.apache.hadoop.hbase.client.Mutation;
94  import org.apache.hadoop.hbase.client.Put;
95  import org.apache.hadoop.hbase.client.Result;
96  import org.apache.hadoop.hbase.client.RowMutations;
97  import org.apache.hadoop.hbase.client.Scan;
98  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
99  import org.apache.hadoop.hbase.DroppedSnapshotException;
100 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
101 import org.apache.hadoop.hbase.exceptions.OperationConflictException;
102 import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
103 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
104 import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
105 import org.apache.hadoop.hbase.executor.ExecutorService;
106 import org.apache.hadoop.hbase.executor.ExecutorType;
107 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
108 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
109 import org.apache.hadoop.hbase.fs.HFileSystem;
110 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
111 import org.apache.hadoop.hbase.io.hfile.HFile;
112 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
113 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
114 import org.apache.hadoop.hbase.ipc.PriorityFunction;
115 import org.apache.hadoop.hbase.ipc.RpcCallContext;
116 import org.apache.hadoop.hbase.ipc.RpcClient;
117 import org.apache.hadoop.hbase.ipc.RpcServer;
118 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
119 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
120 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
121 import org.apache.hadoop.hbase.ipc.ServerRpcController;
122 import org.apache.hadoop.hbase.master.SplitLogManager;
123 import org.apache.hadoop.hbase.master.TableLockManager;
124 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
125 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
126 import org.apache.hadoop.hbase.protobuf.RequestConverter;
127 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
128 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
129 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
130 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
131 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
132 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse;
133 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
134 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
135 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
136 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
137 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
138 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
139 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest;
140 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
141 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRequest;
142 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileResponse;
143 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest;
144 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse;
145 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
146 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
147 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
148 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
149 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
150 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
151 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
152 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
153 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
154 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse;
155 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
156 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse;
157 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
158 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
159 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
160 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
161 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
162 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
163 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
164 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition;
165 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
166 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
167 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
168 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
169 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
170 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
171 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
172 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
173 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
174 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
175 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
176 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
177 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
178 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
179 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
180 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
181 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
182 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
183 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
184 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
185 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
186 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
187 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
188 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
189 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
190 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
191 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
192 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
193 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
194 import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
195 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
196 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
197 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
198 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
199 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
200 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
201 import org.apache.hadoop.hbase.regionserver.wal.HLog;
202 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
203 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
204 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
205 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
206 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
207 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
208 import org.apache.hadoop.hbase.security.UserProvider;
209 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
210 import org.apache.hadoop.hbase.util.Bytes;
211 import org.apache.hadoop.hbase.util.CompressionTest;
212 import org.apache.hadoop.hbase.util.Counter;
213 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
214 import org.apache.hadoop.hbase.util.FSTableDescriptors;
215 import org.apache.hadoop.hbase.util.FSUtils;
216 import org.apache.hadoop.hbase.util.InfoServer;
217 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
218 import org.apache.hadoop.hbase.util.Pair;
219 import org.apache.hadoop.hbase.util.Sleeper;
220 import org.apache.hadoop.hbase.util.Strings;
221 import org.apache.hadoop.hbase.util.Threads;
222 import org.apache.hadoop.hbase.util.VersionInfo;
223 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
224 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
225 import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
226 import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
227 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
228 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
229 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
230 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
231 import org.apache.hadoop.ipc.RemoteException;
232 import org.apache.hadoop.metrics.util.MBeanUtil;
233 import org.apache.hadoop.net.DNS;
234 import org.apache.hadoop.util.ReflectionUtils;
235 import org.apache.hadoop.util.StringUtils;
236 import org.apache.zookeeper.KeeperException;
237 import org.apache.zookeeper.data.Stat;
238 
239 import com.google.protobuf.BlockingRpcChannel;
240 import com.google.protobuf.ByteString;
241 import com.google.protobuf.Message;
242 import com.google.protobuf.RpcController;
243 import com.google.protobuf.ServiceException;
244 import com.google.protobuf.TextFormat;
245 
246 /**
247  * HRegionServer makes a set of HRegions available to clients. It checks in with
248  * the HMaster. There are many HRegionServers in a single HBase deployment.
249  */
250 @InterfaceAudience.Private
251 @SuppressWarnings("deprecation")
252 public class HRegionServer implements ClientProtos.ClientService.BlockingInterface,
253   AdminProtos.AdminService.BlockingInterface, Runnable, RegionServerServices,
254   HBaseRPCErrorHandler, LastSequenceId {
255 
256   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
257 
258   private final Random rand;
259 
260   private final AtomicLong scannerIdGen = new AtomicLong(0L);
261 
262   /*
263    * Strings to be used in forming the exception message for
264    * RegionsAlreadyInTransitionException.
265    */
266   protected static final String OPEN = "OPEN";
267   protected static final String CLOSE = "CLOSE";
268 
269   //RegionName vs current action in progress
270   //true - if open region action in progress
271   //false - if close region action in progress
272   protected final ConcurrentMap<byte[], Boolean> regionsInTransitionInRS =
273     new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
274 
275   /** RPC scheduler to use for the region server. */
276   public static final String REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS =
277       "hbase.region.server.rpc.scheduler.factory.class";
278 
279   protected long maxScannerResultSize;
280 
281   // Cache flushing
282   protected MemStoreFlusher cacheFlusher;
283 
284   protected HeapMemoryManager hMemManager;
285 
286   // catalog tracker
287   protected CatalogTracker catalogTracker;
288 
289   // Watch if a region is out of recovering state from ZooKeeper
290   @SuppressWarnings("unused")
291   private RecoveringRegionWatcher recoveringRegionWatcher;
292 
293   /**
294    * Go here to get table descriptors.
295    */
296   protected TableDescriptors tableDescriptors;
297 
298   // Replication services. If no replication, this handler will be null.
299   protected ReplicationSourceService replicationSourceHandler;
300   protected ReplicationSinkService replicationSinkHandler;
301 
302   // Compactions
303   public CompactSplitThread compactSplitThread;
304 
305   final ConcurrentHashMap<String, RegionScannerHolder> scanners =
306       new ConcurrentHashMap<String, RegionScannerHolder>();
307 
308   /**
309    * Map of regions currently being served by this region server. Key is the
310    * encoded region name.  All access should be synchronized.
311    */
312   protected final Map<String, HRegion> onlineRegions =
313     new ConcurrentHashMap<String, HRegion>();
314 
315   /**
316    * Map of encoded region names to the DataNode locations they should be hosted on
317    * We store the value as InetSocketAddress since this is used only in HDFS
318    * API (create() that takes favored nodes as hints for placing file blocks).
319    * We could have used ServerName here as the value class, but we'd need to
320    * convert it to InetSocketAddress at some point before the HDFS API call, and
321    * it seems a bit weird to store ServerName since ServerName refers to RegionServers
322    * and here we really mean DataNode locations.
323    */
324   protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap =
325       new ConcurrentHashMap<String, InetSocketAddress[]>();
326 
327   /**
328    * Set of regions currently being in recovering state which means it can accept writes(edits from
329    * previous failed region server) but not reads. A recovering region is also an online region.
330    */
331   protected final Map<String, HRegion> recoveringRegions = Collections
332       .synchronizedMap(new HashMap<String, HRegion>());
333 
334   // Leases
335   protected Leases leases;
336 
337   // Instance of the hbase executor service.
338   protected ExecutorService service;
339 
340   // Request counter. (Includes requests that are not serviced by regions.)
341   final Counter requestCount = new Counter();
342 
343   // If false, the file system has become unavailable
344   protected volatile boolean fsOk;
345   protected HFileSystem fs;
346 
347   // Set when a report to the master comes back with a message asking us to
348   // shutdown. Also set by call to stop when debugging or running unit tests
349   // of HRegionServer in isolation.
350   protected volatile boolean stopped = false;
351 
352   // Go down hard. Used if file system becomes unavailable and also in
353   // debugging and unit tests.
354   protected volatile boolean abortRequested;
355 
356   // region server static info like info port
357   private RegionServerInfo.Builder rsInfo;
358 
359   ConcurrentMap<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
360 
361   // A state before we go into stopped state.  At this stage we're closing user
362   // space regions.
363   private boolean stopping = false;
364 
365   private volatile boolean killed = false;
366 
367   protected final Configuration conf;
368 
369   private Path rootDir;
370 
371   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
372 
373   final int numRetries;
374   protected final int threadWakeFrequency;
375   private final int msgInterval;
376 
377   protected final int numRegionsToReport;
378 
379   // Stub to do region server status calls against the master.
380   private RegionServerStatusService.BlockingInterface rssStub;
381   // RPC client. Used to make the stub above that does region server status checking.
382   RpcClient rpcClient;
383 
384   // Server to handle client requests. Default access so can be accessed by
385   // unit tests.
386   RpcServerInterface rpcServer;
387 
388   private final InetSocketAddress isa;
389   private UncaughtExceptionHandler uncaughtExceptionHandler;
390 
391   // Info server. Default access so can be used by unit tests. REGIONSERVER
392   // is name of the webapp and the attribute name used stuffing this instance
393   // into web context.
394   InfoServer infoServer;
395   private JvmPauseMonitor pauseMonitor;
396 
397   /** region server process name */
398   public static final String REGIONSERVER = "regionserver";
399 
400   /** region server configuration name */
401   public static final String REGIONSERVER_CONF = "regionserver_conf";
402 
403   private MetricsRegionServer metricsRegionServer;
404   private SpanReceiverHost spanReceiverHost;
405 
406   /*
407    * Check for compactions requests.
408    */
409   Chore compactionChecker;
410 
411   /*
412    * Check for flushes
413    */
414   Chore periodicFlusher;
415 
416   // HLog and HLog roller. log is protected rather than private to avoid
417   // eclipse warning when accessed by inner classes
418   protected volatile HLog hlog;
419   // The meta updates are written to a different hlog. If this
420   // regionserver holds meta regions, then this field will be non-null.
421   protected volatile HLog hlogForMeta;
422 
423   LogRoller hlogRoller;
424   LogRoller metaHLogRoller;
425 
426   // flag set after we're done setting up server threads (used for testing)
427   protected volatile boolean isOnline;
428 
429   // zookeeper connection and watcher
430   private ZooKeeperWatcher zooKeeper;
431 
432   // master address tracker
433   private MasterAddressTracker masterAddressTracker;
434 
435   // Cluster Status Tracker
436   private ClusterStatusTracker clusterStatusTracker;
437 
438   // Log Splitting Worker
439   private SplitLogWorker splitLogWorker;
440 
441   // A sleeper that sleeps for msgInterval.
442   private final Sleeper sleeper;
443 
444   private final int operationTimeout;
445 
446   private final RegionServerAccounting regionServerAccounting;
447 
448   // Cache configuration and block cache reference
449   final CacheConfig cacheConfig;
450 
451   /** The health check chore. */
452   private HealthCheckChore healthCheckChore;
453 
454   /** The nonce manager chore. */
455   private Chore nonceManagerChore;
456 
457   /**
458    * The server name the Master sees us as.  Its made from the hostname the
459    * master passes us, port, and server startcode. Gets set after registration
460    * against  Master.  The hostname can differ from the hostname in {@link #isa}
461    * but usually doesn't if both servers resolve .
462    */
463   private ServerName serverNameFromMasterPOV;
464 
465   /**
466    * This servers startcode.
467    */
468   private final long startcode;
469 
470   /**
471    * Unique identifier for the cluster we are a part of.
472    */
473   private String clusterId;
474 
475   /**
476    * MX Bean for RegionServerInfo
477    */
478   private ObjectName mxBean = null;
479 
480   /**
481    * Chore to clean periodically the moved region list
482    */
483   private MovedRegionsCleaner movedRegionsCleaner;
484 
485   /**
486    * The lease timeout period for client scanners (milliseconds).
487    */
488   private final int scannerLeaseTimeoutPeriod;
489 
490   /**
491    * The reference to the priority extraction function
492    */
493   private final PriorityFunction priority;
494 
495   private RegionServerCoprocessorHost rsHost;
496 
497   private RegionServerProcedureManagerHost rspmHost;
498 
499   // configuration setting on if replay WAL edits directly to another RS
500   private final boolean distributedLogReplay;
501 
502   // Table level lock manager for locking for region operations
503   private TableLockManager tableLockManager;
504 
505   /**
506    * Nonce manager. Nonces are used to make operations like increment and append idempotent
507    * in the case where client doesn't receive the response from a successful operation and
508    * retries. We track the successful ops for some time via a nonce sent by client and handle
509    * duplicate operations (currently, by failing them; in future we might use MVCC to return
510    * result). Nonces are also recovered from WAL during, recovery; however, the caveats (from
511    * HBASE-3787) are:
512    * - WAL recovery is optimized, and under high load we won't read nearly nonce-timeout worth
513    *   of past records. If we don't read the records, we don't read and recover the nonces.
514    *   Some WALs within nonce-timeout at recovery may not even be present due to rolling/cleanup.
515    * - There's no WAL recovery during normal region move, so nonces will not be transfered.
516    * We can have separate additional "Nonce WAL". It will just contain bunch of numbers and
517    * won't be flushed on main path - because WAL itself also contains nonces, if we only flush
518    * it before memstore flush, for a given nonce we will either see it in the WAL (if it was
519    * never flushed to disk, it will be part of recovery), or we'll see it as part of the nonce
520    * log (or both occasionally, which doesn't matter). Nonce log file can be deleted after the
521    * latest nonce in it expired. It can also be recovered during move.
522    */
523   private final ServerNonceManager nonceManager;
524 
525   private UserProvider userProvider;
526 
527   /**
528    * Starts a HRegionServer at the default location
529    *
530    * @param conf
531    * @throws IOException
532    * @throws InterruptedException
533    */
534   public HRegionServer(Configuration conf)
535   throws IOException, InterruptedException {
536     this.fsOk = true;
537     this.conf = conf;
538     this.isOnline = false;
539     checkCodecs(this.conf);
540     this.userProvider = UserProvider.instantiate(conf);
541 
542     FSUtils.setupShortCircuitRead(this.conf);
543 
544     // Config'ed params
545     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
546         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
547     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
548     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
549 
550     this.sleeper = new Sleeper(this.msgInterval, this);
551 
552     boolean isNoncesEnabled = conf.getBoolean(HConstants.HBASE_RS_NONCES_ENABLED, true);
553     this.nonceManager = isNoncesEnabled ? new ServerNonceManager(this.conf) : null;
554 
555     this.maxScannerResultSize = conf.getLong(
556       HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
557       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
558 
559     this.numRegionsToReport = conf.getInt(
560       "hbase.regionserver.numregionstoreport", 10);
561 
562     this.operationTimeout = conf.getInt(
563       HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
564       HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
565 
566     this.abortRequested = false;
567     this.stopped = false;
568 
569     this.scannerLeaseTimeoutPeriod = HBaseConfiguration.getInt(conf,
570       HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
571       HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
572       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
573 
574     // Server to handle client requests.
575     String hostname = conf.get("hbase.regionserver.ipc.address",
576       Strings.domainNamePointerToHostName(DNS.getDefaultHost(
577         conf.get("hbase.regionserver.dns.interface", "default"),
578         conf.get("hbase.regionserver.dns.nameserver", "default"))));
579     int port = conf.getInt(HConstants.REGIONSERVER_PORT,
580       HConstants.DEFAULT_REGIONSERVER_PORT);
581     // Creation of a HSA will force a resolve.
582     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
583     if (initialIsa.getAddress() == null) {
584       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
585     }
586     this.rand = new Random(initialIsa.hashCode());
587     String name = "regionserver/" + initialIsa.toString();
588     // Set how many times to retry talking to another server over HConnection.
589     ConnectionUtils.setServerSideHConnectionRetriesConfig(this.conf, name, LOG);
590     this.priority = new AnnotationReadingPriorityFunction(this);
591     RpcSchedulerFactory rpcSchedulerFactory;
592     try {
593       Class<?> rpcSchedulerFactoryClass = conf.getClass(
594           REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,
595           SimpleRpcSchedulerFactory.class);
596       rpcSchedulerFactory = ((RpcSchedulerFactory) rpcSchedulerFactoryClass.newInstance());
597     } catch (InstantiationException e) {
598       throw new IllegalArgumentException(e);
599     } catch (IllegalAccessException e) {
600       throw new IllegalArgumentException(e);
601     }
602     this.rpcServer = new RpcServer(this, name, getServices(),
603       /*HBaseRPCErrorHandler.class, OnlineRegions.class},*/
604       initialIsa, // BindAddress is IP we got for this server.
605       conf,
606       rpcSchedulerFactory.create(conf, this));
607 
608     // Set our address.
609     this.isa = this.rpcServer.getListenerAddress();
610 
611     this.rpcServer.setErrorHandler(this);
612     this.startcode = System.currentTimeMillis();
613 
614     // login the zookeeper client principal (if using security)
615     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
616       "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
617 
618     // login the server principal (if using secure Hadoop)
619     userProvider.login("hbase.regionserver.keytab.file",
620       "hbase.regionserver.kerberos.principal", this.isa.getHostName());
621     regionServerAccounting = new RegionServerAccounting();
622     cacheConfig = new CacheConfig(conf);
623     uncaughtExceptionHandler = new UncaughtExceptionHandler() {
624       @Override
625       public void uncaughtException(Thread t, Throwable e) {
626         abort("Uncaught exception in service thread " + t.getName(), e);
627       }
628     };
629 
630     this.rsInfo = RegionServerInfo.newBuilder();
631     // Put up the webui. Webui may come up on port other than configured if
632     // that port is occupied. Adjust serverInfo if this is the case.
633     this.rsInfo.setInfoPort(putUpWebUI());
634     this.distributedLogReplay = HLogSplitter.isDistributedLogReplay(this.conf);
635   }
636 
637   /**
638    * @return list of blocking services and their security info classes that this server supports
639    */
640   private List<BlockingServiceAndInterface> getServices() {
641     List<BlockingServiceAndInterface> bssi = new ArrayList<BlockingServiceAndInterface>(2);
642     bssi.add(new BlockingServiceAndInterface(
643         ClientProtos.ClientService.newReflectiveBlockingService(this),
644         ClientProtos.ClientService.BlockingInterface.class));
645     bssi.add(new BlockingServiceAndInterface(
646         AdminProtos.AdminService.newReflectiveBlockingService(this),
647         AdminProtos.AdminService.BlockingInterface.class));
648     return bssi;
649   }
650 
651   /**
652    * Run test on configured codecs to make sure supporting libs are in place.
653    * @param c
654    * @throws IOException
655    */
656   private static void checkCodecs(final Configuration c) throws IOException {
657     // check to see if the codec list is available:
658     String [] codecs = c.getStrings("hbase.regionserver.codecs", (String[])null);
659     if (codecs == null) return;
660     for (String codec : codecs) {
661       if (!CompressionTest.testCompression(codec)) {
662         throw new IOException("Compression codec " + codec +
663           " not supported, aborting RS construction");
664       }
665     }
666   }
667 
668   String getClusterId() {
669     return this.clusterId;
670   }
671 
672   @Override
673   public int getPriority(RequestHeader header, Message param) {
674     return priority.getPriority(header, param);
675   }
676 
677   @Retention(RetentionPolicy.RUNTIME)
678   protected @interface QosPriority {
679     int priority() default 0;
680   }
681 
682   PriorityFunction getPriority() {
683     return priority;
684   }
685 
686   RegionScanner getScanner(long scannerId) {
687     String scannerIdString = Long.toString(scannerId);
688     RegionScannerHolder scannerHolder = scanners.get(scannerIdString);
689     if (scannerHolder != null) {
690       return scannerHolder.s;
691     }
692     return null;
693   }
694 
695   /**
696    * All initialization needed before we go register with Master.
697    *
698    * @throws IOException
699    * @throws InterruptedException
700    */
701   private void preRegistrationInitialization(){
702     try {
703       initializeZooKeeper();
704       initializeThreads();
705     } catch (Throwable t) {
706       // Call stop if error or process will stick around for ever since server
707       // puts up non-daemon threads.
708       this.rpcServer.stop();
709       abort("Initialization of RS failed.  Hence aborting RS.", t);
710     }
711   }
712 
713   /**
714    * Bring up connection to zk ensemble and then wait until a master for this
715    * cluster and then after that, wait until cluster 'up' flag has been set.
716    * This is the order in which master does things.
717    * Finally put up a catalog tracker.
718    * @throws IOException
719    * @throws InterruptedException
720    */
721   private void initializeZooKeeper() throws IOException, InterruptedException {
722     // Open connection to zookeeper and set primary watcher
723     this.zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER + ":" +
724       this.isa.getPort(), this);
725 
726     // Create the master address tracker, register with zk, and start it.  Then
727     // block until a master is available.  No point in starting up if no master
728     // running.
729     this.masterAddressTracker = new MasterAddressTracker(this.zooKeeper, this);
730     this.masterAddressTracker.start();
731     blockAndCheckIfStopped(this.masterAddressTracker);
732 
733     // Wait on cluster being up.  Master will set this flag up in zookeeper
734     // when ready.
735     this.clusterStatusTracker = new ClusterStatusTracker(this.zooKeeper, this);
736     this.clusterStatusTracker.start();
737     blockAndCheckIfStopped(this.clusterStatusTracker);
738 
739     // Create the catalog tracker and start it;
740     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf, this);
741     catalogTracker.start();
742 
743     // Retrieve clusterId
744     // Since cluster status is now up
745     // ID should have already been set by HMaster
746     try {
747       clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
748       if (clusterId == null) {
749         this.abort("Cluster ID has not been set");
750       }
751       LOG.info("ClusterId : "+clusterId);
752     } catch (KeeperException e) {
753       this.abort("Failed to retrieve Cluster ID",e);
754     }
755 
756     // watch for snapshots and other procedures
757     try {
758       rspmHost = new RegionServerProcedureManagerHost();
759       rspmHost.loadProcedures(conf);
760       rspmHost.initialize(this);
761     } catch (KeeperException e) {
762       this.abort("Failed to reach zk cluster when creating procedure handler.", e);
763     }
764     this.tableLockManager = TableLockManager.createTableLockManager(conf, zooKeeper,
765         ServerName.valueOf(isa.getHostName(), isa.getPort(), startcode));
766 
767     // register watcher for recovering regions
768     if(this.distributedLogReplay) {
769       this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);
770     }
771   }
772 
773   /**
774    * Utilty method to wait indefinitely on a znode availability while checking
775    * if the region server is shut down
776    * @param tracker znode tracker to use
777    * @throws IOException any IO exception, plus if the RS is stopped
778    * @throws InterruptedException
779    */
780   private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
781       throws IOException, InterruptedException {
782     while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
783       if (this.stopped) {
784         throw new IOException("Received the shutdown message while waiting.");
785       }
786     }
787   }
788 
789   /**
790    * @return False if cluster shutdown in progress
791    */
792   private boolean isClusterUp() {
793     return this.clusterStatusTracker.isClusterUp();
794   }
795 
796   private void initializeThreads() throws IOException {
797     // Cache flushing thread.
798     this.cacheFlusher = new MemStoreFlusher(conf, this);
799 
800     // Compaction thread
801     this.compactSplitThread = new CompactSplitThread(this);
802 
803     // Background thread to check for compactions; needed if region has not gotten updates
804     // in a while. It will take care of not checking too frequently on store-by-store basis.
805     this.compactionChecker = new CompactionChecker(this, this.threadWakeFrequency, this);
806     this.periodicFlusher = new PeriodicMemstoreFlusher(this.threadWakeFrequency, this);
807     // Health checker thread.
808     int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
809       HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
810     if (isHealthCheckerConfigured()) {
811       healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
812     }
813 
814     this.leases = new Leases(this.threadWakeFrequency);
815 
816     // Create the thread to clean the moved regions list
817     movedRegionsCleaner = MovedRegionsCleaner.createAndStart(this);
818 
819     if (this.nonceManager != null) {
820       // Create the chore that cleans up nonces.
821       nonceManagerChore = this.nonceManager.createCleanupChore(this);
822     }
823 
824     // Setup RPC client for master communication
825     rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress(
826         this.isa.getAddress(), 0));
827     this.pauseMonitor = new JvmPauseMonitor(conf);
828     pauseMonitor.start();
829   }
830 
831   /**
832    * The HRegionServer sticks in this loop until closed.
833    */
834   @Override
835   public void run() {
836     try {
837       // Do pre-registration initializations; zookeeper, lease threads, etc.
838       preRegistrationInitialization();
839     } catch (Throwable e) {
840       abort("Fatal exception during initialization", e);
841     }
842 
843     try {
844       // Set our ephemeral znode up in zookeeper now we have a name.
845       createMyEphemeralNode();
846 
847       // Initialize the RegionServerCoprocessorHost now that our ephemeral
848       // node was created, in case any coprocessors want to use ZooKeeper
849       this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
850 
851       // Try and register with the Master; tell it we are here.  Break if
852       // server is stopped or the clusterup flag is down or hdfs went wacky.
853       while (keepLooping()) {
854         RegionServerStartupResponse w = reportForDuty();
855         if (w == null) {
856           LOG.warn("reportForDuty failed; sleeping and then retrying.");
857           this.sleeper.sleep();
858         } else {
859           handleReportForDutyResponse(w);
860           break;
861         }
862       }
863 
864       if (!this.stopped && isHealthy()){
865         // start the snapshot handler and other procedure handlers,
866         // since the server is ready to run
867         rspmHost.start();
868       }
869 
870       // We registered with the Master.  Go into run mode.
871       long lastMsg = 0;
872       long oldRequestCount = -1;
873       // The main run loop.
874       while (!this.stopped && isHealthy()) {
875         if (!isClusterUp()) {
876           if (isOnlineRegionsEmpty()) {
877             stop("Exiting; cluster shutdown set and not carrying any regions");
878           } else if (!this.stopping) {
879             this.stopping = true;
880             LOG.info("Closing user regions");
881             closeUserRegions(this.abortRequested);
882           } else if (this.stopping) {
883             boolean allUserRegionsOffline = areAllUserRegionsOffline();
884             if (allUserRegionsOffline) {
885               // Set stopped if no more write requests tp meta tables
886               // since last time we went around the loop.  Any open
887               // meta regions will be closed on our way out.
888               if (oldRequestCount == getWriteRequestCount()) {
889                 stop("Stopped; only catalog regions remaining online");
890                 break;
891               }
892               oldRequestCount = getWriteRequestCount();
893             } else {
894               // Make sure all regions have been closed -- some regions may
895               // have not got it because we were splitting at the time of
896               // the call to closeUserRegions.
897               closeUserRegions(this.abortRequested);
898             }
899             LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
900           }
901         }
902         long now = System.currentTimeMillis();
903         if ((now - lastMsg) >= msgInterval) {
904           tryRegionServerReport(lastMsg, now);
905           lastMsg = System.currentTimeMillis();
906         }
907         if (!this.stopped) this.sleeper.sleep();
908       } // for
909     } catch (Throwable t) {
910       if (!checkOOME(t)) {
911         String prefix = t instanceof YouAreDeadException? "": "Unhandled: ";
912         abort(prefix + t.getMessage(), t);
913       }
914     }
915     // Run shutdown.
916     if (mxBean != null) {
917       MBeanUtil.unregisterMBean(mxBean);
918       mxBean = null;
919     }
920     if (this.leases != null) this.leases.closeAfterLeasesExpire();
921     this.rpcServer.stop();
922     if (this.splitLogWorker != null) {
923       splitLogWorker.stop();
924     }
925     if (this.infoServer != null) {
926       LOG.info("Stopping infoServer");
927       try {
928         this.infoServer.stop();
929       } catch (Exception e) {
930         e.printStackTrace();
931       }
932     }
933     // Send cache a shutdown.
934     if (cacheConfig.isBlockCacheEnabled()) {
935       cacheConfig.getBlockCache().shutdown();
936     }
937 
938     movedRegionsCleaner.stop("Region Server stopping");
939 
940     // Send interrupts to wake up threads if sleeping so they notice shutdown.
941     // TODO: Should we check they are alive? If OOME could have exited already
942     if(this.hMemManager != null) this.hMemManager.stop();
943     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
944     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
945     if (this.hlogRoller != null) this.hlogRoller.interruptIfNecessary();
946     if (this.metaHLogRoller != null) this.metaHLogRoller.interruptIfNecessary();
947     if (this.compactionChecker != null)
948       this.compactionChecker.interrupt();
949     if (this.healthCheckChore != null) {
950       this.healthCheckChore.interrupt();
951     }
952     if (this.nonceManagerChore != null) {
953       this.nonceManagerChore.interrupt();
954     }
955 
956     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
957     rspmHost.stop(this.abortRequested || this.killed);
958 
959     if (this.killed) {
960       // Just skip out w/o closing regions.  Used when testing.
961     } else if (abortRequested) {
962       if (this.fsOk) {
963         closeUserRegions(abortRequested); // Don't leave any open file handles
964       }
965       LOG.info("aborting server " + this.serverNameFromMasterPOV);
966     } else {
967       closeUserRegions(abortRequested);
968       closeAllScanners();
969       LOG.info("stopping server " + this.serverNameFromMasterPOV);
970     }
971     // Interrupt catalog tracker here in case any regions being opened out in
972     // handlers are stuck waiting on meta.
973     if (this.catalogTracker != null) this.catalogTracker.stop();
974 
975     // Closing the compactSplit thread before closing meta regions
976     if (!this.killed && containsMetaTableRegions()) {
977       if (!abortRequested || this.fsOk) {
978         if (this.compactSplitThread != null) {
979           this.compactSplitThread.join();
980           this.compactSplitThread = null;
981         }
982         closeMetaTableRegions(abortRequested);
983       }
984     }
985 
986     if (!this.killed && this.fsOk) {
987       waitOnAllRegionsToClose(abortRequested);
988       LOG.info("stopping server " + this.serverNameFromMasterPOV +
989         "; all regions closed.");
990     }
991 
992     //fsOk flag may be changed when closing regions throws exception.
993     if (this.fsOk) {
994       closeWAL(!abortRequested);
995     }
996 
997     // Make sure the proxy is down.
998     if (this.rssStub != null) {
999       this.rssStub = null;
1000     }
1001     this.rpcClient.stop();
1002     this.leases.close();
1003     if (this.pauseMonitor != null) {
1004       this.pauseMonitor.stop();
1005     }
1006 
1007     if (!killed) {
1008       join();
1009     }
1010 
1011     try {
1012       deleteMyEphemeralNode();
1013     } catch (KeeperException e) {
1014       LOG.warn("Failed deleting my ephemeral node", e);
1015     }
1016     // We may have failed to delete the znode at the previous step, but
1017     //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
1018     ZNodeClearer.deleteMyEphemeralNodeOnDisk();
1019     this.zooKeeper.close();
1020     LOG.info("stopping server " + this.serverNameFromMasterPOV +
1021       "; zookeeper connection closed.");
1022 
1023     LOG.info(Thread.currentThread().getName() + " exiting");
1024   }
1025 
1026   private boolean containsMetaTableRegions() {
1027     return onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
1028   }
1029 
1030   private boolean areAllUserRegionsOffline() {
1031     if (getNumberOfOnlineRegions() > 2) return false;
1032     boolean allUserRegionsOffline = true;
1033     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1034       if (!e.getValue().getRegionInfo().isMetaTable()) {
1035         allUserRegionsOffline = false;
1036         break;
1037       }
1038     }
1039     return allUserRegionsOffline;
1040   }
1041 
1042   /**
1043    * @return Current write count for all online regions.
1044    */
1045   private long getWriteRequestCount() {
1046     int writeCount = 0;
1047     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1048       writeCount += e.getValue().getWriteRequestsCount();
1049     }
1050     return writeCount;
1051   }
1052 
1053   @VisibleForTesting
1054   protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
1055   throws IOException {
1056     if (this.rssStub == null) {
1057       // the current server is stopping.
1058       return;
1059     }
1060     ClusterStatusProtos.ServerLoad sl = buildServerLoad(reportStartTime, reportEndTime);
1061     try {
1062       RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();
1063       ServerName sn = ServerName.parseVersionedServerName(
1064         this.serverNameFromMasterPOV.getVersionedBytes());
1065       request.setServer(ProtobufUtil.toServerName(sn));
1066       request.setLoad(sl);
1067       this.rssStub.regionServerReport(null, request.build());
1068     } catch (ServiceException se) {
1069       IOException ioe = ProtobufUtil.getRemoteException(se);
1070       if (ioe instanceof YouAreDeadException) {
1071         // This will be caught and handled as a fatal error in run()
1072         throw ioe;
1073       }
1074       // Couldn't connect to the master, get location from zk and reconnect
1075       // Method blocks until new master is found or we are stopped
1076       Pair<ServerName, RegionServerStatusService.BlockingInterface> p =
1077         createRegionServerStatusStub();
1078       this.rssStub = p.getSecond();
1079     }
1080   }
1081 
1082   ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime) {
1083     // We're getting the MetricsRegionServerWrapper here because the wrapper computes requests
1084     // per second, and other metrics  As long as metrics are part of ServerLoad it's best to use
1085     // the wrapper to compute those numbers in one place.
1086     // In the long term most of these should be moved off of ServerLoad and the heart beat.
1087     // Instead they should be stored in an HBase table so that external visibility into HBase is
1088     // improved; Additionally the load balancer will be able to take advantage of a more complete
1089     // history.
1090     MetricsRegionServerWrapper regionServerWrapper = this.metricsRegionServer.getRegionServerWrapper();
1091     Collection<HRegion> regions = getOnlineRegionsLocalContext();
1092     MemoryUsage memory =
1093       ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
1094 
1095     ClusterStatusProtos.ServerLoad.Builder serverLoad =
1096       ClusterStatusProtos.ServerLoad.newBuilder();
1097     serverLoad.setNumberOfRequests((int) regionServerWrapper.getRequestsPerSecond());
1098     serverLoad.setTotalNumberOfRequests((int) regionServerWrapper.getTotalRequestCount());
1099     serverLoad.setUsedHeapMB((int)(memory.getUsed() / 1024 / 1024));
1100     serverLoad.setMaxHeapMB((int) (memory.getMax() / 1024 / 1024));
1101     Set<String> coprocessors = this.hlog.getCoprocessorHost().getCoprocessors();
1102     for (String coprocessor : coprocessors) {
1103       serverLoad.addCoprocessors(
1104         Coprocessor.newBuilder().setName(coprocessor).build());
1105     }
1106     for (HRegion region : regions) {
1107       serverLoad.addRegionLoads(createRegionLoad(region));
1108     }
1109     serverLoad.setReportStartTime(reportStartTime);
1110     serverLoad.setReportEndTime(reportEndTime);
1111     if (this.infoServer != null) {
1112       serverLoad.setInfoServerPort(this.infoServer.getPort());
1113     } else {
1114       serverLoad.setInfoServerPort(-1);
1115     }
1116     return serverLoad.build();
1117   }
1118 
1119   String getOnlineRegionsAsPrintableString() {
1120     StringBuilder sb = new StringBuilder();
1121     for (HRegion r: this.onlineRegions.values()) {
1122       if (sb.length() > 0) sb.append(", ");
1123       sb.append(r.getRegionInfo().getEncodedName());
1124     }
1125     return sb.toString();
1126   }
1127 
1128   /**
1129    * Wait on regions close.
1130    */
1131   private void waitOnAllRegionsToClose(final boolean abort) {
1132     // Wait till all regions are closed before going out.
1133     int lastCount = -1;
1134     long previousLogTime = 0;
1135     Set<String> closedRegions = new HashSet<String>();
1136     boolean interrupted = false;
1137     try {
1138       while (!isOnlineRegionsEmpty()) {
1139         int count = getNumberOfOnlineRegions();
1140         // Only print a message if the count of regions has changed.
1141         if (count != lastCount) {
1142           // Log every second at most
1143           if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1144             previousLogTime = System.currentTimeMillis();
1145             lastCount = count;
1146             LOG.info("Waiting on " + count + " regions to close");
1147             // Only print out regions still closing if a small number else will
1148             // swamp the log.
1149             if (count < 10 && LOG.isDebugEnabled()) {
1150               LOG.debug(this.onlineRegions);
1151             }
1152           }
1153         }
1154         // Ensure all user regions have been sent a close. Use this to
1155         // protect against the case where an open comes in after we start the
1156         // iterator of onlineRegions to close all user regions.
1157         for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1158           HRegionInfo hri = e.getValue().getRegionInfo();
1159           if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
1160               && !closedRegions.contains(hri.getEncodedName())) {
1161             closedRegions.add(hri.getEncodedName());
1162             // Don't update zk with this close transition; pass false.
1163             closeRegionIgnoreErrors(hri, abort);
1164               }
1165         }
1166         // No regions in RIT, we could stop waiting now.
1167         if (this.regionsInTransitionInRS.isEmpty()) {
1168           if (!isOnlineRegionsEmpty()) {
1169             LOG.info("We were exiting though online regions are not empty," +
1170                 " because some regions failed closing");
1171           }
1172           break;
1173         }
1174         try {
1175           Thread.sleep(200);
1176         } catch (InterruptedException e) {
1177           interrupted = true;
1178           LOG.warn("Interrupted while sleeping");
1179         }
1180       }
1181     } finally {
1182       if (interrupted) {
1183         Thread.currentThread().interrupt();
1184       }
1185     }
1186   }
1187 
1188   private void closeWAL(final boolean delete) {
1189     if (this.hlogForMeta != null) {
1190       // All hlogs (meta and non-meta) are in the same directory. Don't call
1191       // closeAndDelete here since that would delete all hlogs not just the
1192       // meta ones. We will just 'close' the hlog for meta here, and leave
1193       // the directory cleanup to the follow-on closeAndDelete call.
1194       try {
1195         this.hlogForMeta.close();
1196       } catch (Throwable e) {
1197         LOG.error("Metalog close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1198       }
1199     }
1200     if (this.hlog != null) {
1201       try {
1202         if (delete) {
1203           hlog.closeAndDelete();
1204         } else {
1205           hlog.close();
1206         }
1207       } catch (Throwable e) {
1208         LOG.error("Close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1209       }
1210     }
1211   }
1212 
1213   private void closeAllScanners() {
1214     // Close any outstanding scanners. Means they'll get an UnknownScanner
1215     // exception next time they come in.
1216     for (Map.Entry<String, RegionScannerHolder> e : this.scanners.entrySet()) {
1217       try {
1218         e.getValue().s.close();
1219       } catch (IOException ioe) {
1220         LOG.warn("Closing scanner " + e.getKey(), ioe);
1221       }
1222     }
1223   }
1224 
1225   /*
1226    * Run init. Sets up hlog and starts up all server threads.
1227    *
1228    * @param c Extra configuration.
1229    */
1230   protected void handleReportForDutyResponse(final RegionServerStartupResponse c)
1231   throws IOException {
1232     try {
1233       for (NameStringPair e : c.getMapEntriesList()) {
1234         String key = e.getName();
1235         // The hostname the master sees us as.
1236         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1237           String hostnameFromMasterPOV = e.getValue();
1238           this.serverNameFromMasterPOV = ServerName.valueOf(hostnameFromMasterPOV,
1239               this.isa.getPort(), this.startcode);
1240           if (!hostnameFromMasterPOV.equals(this.isa.getHostName())) {
1241             LOG.info("Master passed us a different hostname to use; was=" +
1242               this.isa.getHostName() + ", but now=" + hostnameFromMasterPOV);
1243           }
1244           continue;
1245         }
1246         String value = e.getValue();
1247         if (LOG.isDebugEnabled()) {
1248           LOG.debug("Config from master: " + key + "=" + value);
1249         }
1250         this.conf.set(key, value);
1251       }
1252 
1253       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
1254       // config param for task trackers, but we can piggyback off of it.
1255       if (this.conf.get("mapred.task.id") == null) {
1256         this.conf.set("mapred.task.id", "hb_rs_" +
1257           this.serverNameFromMasterPOV.toString());
1258       }
1259 
1260       // Save it in a file, this will allow to see if we crash
1261       ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
1262 
1263       // Master sent us hbase.rootdir to use. Should be fully qualified
1264       // path with file system specification included. Set 'fs.defaultFS'
1265       // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
1266       // accessors will be going against wrong filesystem (unless all is set
1267       // to defaults).
1268       FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
1269       // Get fs instance used by this RS.  Do we use checksum verification in the hbase? If hbase
1270       // checksum verification enabled, then automatically switch off hdfs checksum verification.
1271       boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
1272       this.fs = new HFileSystem(this.conf, useHBaseChecksum);
1273       this.rootDir = FSUtils.getRootDir(this.conf);
1274       this.tableDescriptors = new FSTableDescriptors(this.fs, this.rootDir, true);
1275       this.hlog = setupWALAndReplication();
1276       // Init in here rather than in constructor after thread name has been set
1277       this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this));
1278 
1279       spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
1280 
1281       startServiceThreads();
1282       startHeapMemoryManager();
1283       LOG.info("Serving as " + this.serverNameFromMasterPOV +
1284         ", RpcServer on " + this.isa +
1285         ", sessionid=0x" +
1286         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1287       isOnline = true;
1288     } catch (Throwable e) {
1289       this.isOnline = false;
1290       stop("Failed initialization");
1291       throw convertThrowableToIOE(cleanup(e, "Failed init"),
1292           "Region server startup failed");
1293     } finally {
1294       sleeper.skipSleepCycle();
1295     }
1296   }
1297 
1298   private void startHeapMemoryManager() {
1299     this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher, this);
1300     if (this.hMemManager != null) {
1301       this.hMemManager.start();
1302     }
1303   }
1304 
1305   private void createMyEphemeralNode() throws KeeperException, IOException {
1306     byte[] data = ProtobufUtil.prependPBMagic(rsInfo.build().toByteArray());
1307     ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper,
1308       getMyEphemeralNodePath(), data);
1309   }
1310 
1311   private void deleteMyEphemeralNode() throws KeeperException {
1312     ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1313   }
1314 
1315   @Override
1316   public RegionServerAccounting getRegionServerAccounting() {
1317     return regionServerAccounting;
1318   }
1319 
1320   @Override
1321   public TableLockManager getTableLockManager() {
1322     return tableLockManager;
1323   }
1324 
1325   /*
1326    * @param r Region to get RegionLoad for.
1327    *
1328    * @return RegionLoad instance.
1329    *
1330    * @throws IOException
1331    */
1332   private RegionLoad createRegionLoad(final HRegion r) {
1333     byte[] name = r.getRegionName();
1334     int stores = 0;
1335     int storefiles = 0;
1336     int storeUncompressedSizeMB = 0;
1337     int storefileSizeMB = 0;
1338     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
1339     int storefileIndexSizeMB = 0;
1340     int rootIndexSizeKB = 0;
1341     int totalStaticIndexSizeKB = 0;
1342     int totalStaticBloomSizeKB = 0;
1343     long totalCompactingKVs = 0;
1344     long currentCompactedKVs = 0;
1345     synchronized (r.stores) {
1346       stores += r.stores.size();
1347       for (Store store : r.stores.values()) {
1348         storefiles += store.getStorefilesCount();
1349         storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
1350             / 1024 / 1024);
1351         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1352         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
1353         CompactionProgress progress = store.getCompactionProgress();
1354         if (progress != null) {
1355           totalCompactingKVs += progress.totalCompactingKVs;
1356           currentCompactedKVs += progress.currentCompactedKVs;
1357         }
1358 
1359         rootIndexSizeKB +=
1360             (int) (store.getStorefilesIndexSize() / 1024);
1361 
1362         totalStaticIndexSizeKB +=
1363           (int) (store.getTotalStaticIndexSize() / 1024);
1364 
1365         totalStaticBloomSizeKB +=
1366           (int) (store.getTotalStaticBloomSize() / 1024);
1367       }
1368     }
1369     RegionLoad.Builder regionLoad = RegionLoad.newBuilder();
1370     RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
1371     regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
1372     regionSpecifier.setValue(HBaseZeroCopyByteString.wrap(name));
1373     regionLoad.setRegionSpecifier(regionSpecifier.build())
1374       .setStores(stores)
1375       .setStorefiles(storefiles)
1376       .setStoreUncompressedSizeMB(storeUncompressedSizeMB)
1377       .setStorefileSizeMB(storefileSizeMB)
1378       .setMemstoreSizeMB(memstoreSizeMB)
1379       .setStorefileIndexSizeMB(storefileIndexSizeMB)
1380       .setRootIndexSizeKB(rootIndexSizeKB)
1381       .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
1382       .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
1383       .setReadRequestsCount((int) r.readRequestsCount.get())
1384       .setWriteRequestsCount((int) r.writeRequestsCount.get())
1385       .setTotalCompactingKVs(totalCompactingKVs)
1386       .setCurrentCompactedKVs(currentCompactedKVs)
1387       .setCompleteSequenceId(r.completeSequenceId);
1388 
1389     return regionLoad.build();
1390   }
1391 
1392   /**
1393    * @param encodedRegionName
1394    * @return An instance of RegionLoad.
1395    */
1396   public RegionLoad createRegionLoad(final String encodedRegionName) {
1397     HRegion r = null;
1398     r = this.onlineRegions.get(encodedRegionName);
1399     return r != null ? createRegionLoad(r) : null;
1400   }
1401 
1402   /*
1403    * Inner class that runs on a long period checking if regions need compaction.
1404    */
1405   private static class CompactionChecker extends Chore {
1406     private final HRegionServer instance;
1407     private final int majorCompactPriority;
1408     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1409     private long iteration = 0;
1410 
1411     CompactionChecker(final HRegionServer h, final int sleepTime,
1412         final Stoppable stopper) {
1413       super("CompactionChecker", sleepTime, h);
1414       this.instance = h;
1415       LOG.info(this.getName() + " runs every " + StringUtils.formatTime(sleepTime));
1416 
1417       /* MajorCompactPriority is configurable.
1418        * If not set, the compaction will use default priority.
1419        */
1420       this.majorCompactPriority = this.instance.conf.
1421         getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1422         DEFAULT_PRIORITY);
1423     }
1424 
1425     @Override
1426     protected void chore() {
1427       for (HRegion r : this.instance.onlineRegions.values()) {
1428         if (r == null)
1429           continue;
1430         for (Store s : r.getStores().values()) {
1431           try {
1432             long multiplier = s.getCompactionCheckMultiplier();
1433             assert multiplier > 0;
1434             if (iteration % multiplier != 0) continue;
1435             if (s.needsCompaction()) {
1436               // Queue a compaction. Will recognize if major is needed.
1437               this.instance.compactSplitThread.requestSystemCompaction(r, s, getName()
1438                   + " requests compaction");
1439             } else if (s.isMajorCompaction()) {
1440               if (majorCompactPriority == DEFAULT_PRIORITY
1441                   || majorCompactPriority > r.getCompactPriority()) {
1442                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1443                     + " requests major compaction; use default priority", null);
1444               } else {
1445                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1446                     + " requests major compaction; use configured priority",
1447                   this.majorCompactPriority, null);
1448               }
1449             }
1450           } catch (IOException e) {
1451             LOG.warn("Failed major compaction check on " + r, e);
1452           }
1453         }
1454       }
1455       iteration = (iteration == Long.MAX_VALUE) ? 0 : (iteration + 1);
1456     }
1457   }
1458 
1459   class PeriodicMemstoreFlusher extends Chore {
1460     final HRegionServer server;
1461     final static int RANGE_OF_DELAY = 20000; //millisec
1462     final static int MIN_DELAY_TIME = 3000; //millisec
1463     public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1464       super(server.getServerName() + "-MemstoreFlusherChore", cacheFlushInterval, server);
1465       this.server = server;
1466     }
1467 
1468     @Override
1469     protected void chore() {
1470       for (HRegion r : this.server.onlineRegions.values()) {
1471         if (r == null)
1472           continue;
1473         if (r.shouldFlush()) {
1474           FlushRequester requester = server.getFlushRequester();
1475           if (requester != null) {
1476             long randomDelay = rand.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME;
1477             LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() +
1478                 " after a delay of " + randomDelay);
1479             //Throttle the flushes by putting a delay. If we don't throttle, and there
1480             //is a balanced write-load on the regions in a table, we might end up
1481             //overwhelming the filesystem with too many flushes at once.
1482             requester.requestDelayedFlush(r, randomDelay);
1483           }
1484         }
1485       }
1486     }
1487   }
1488 
1489   /**
1490    * Report the status of the server. A server is online once all the startup is
1491    * completed (setting up filesystem, starting service threads, etc.). This
1492    * method is designed mostly to be useful in tests.
1493    *
1494    * @return true if online, false if not.
1495    */
1496   public boolean isOnline() {
1497     return isOnline;
1498   }
1499 
1500   /**
1501    * Setup WAL log and replication if enabled.
1502    * Replication setup is done in here because it wants to be hooked up to WAL.
1503    * @return A WAL instance.
1504    * @throws IOException
1505    */
1506   private HLog setupWALAndReplication() throws IOException {
1507     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1508     final String logName
1509       = HLogUtil.getHLogDirectoryName(this.serverNameFromMasterPOV.toString());
1510 
1511     Path logdir = new Path(rootDir, logName);
1512     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1513     if (this.fs.exists(logdir)) {
1514       throw new RegionServerRunningException("Region server has already " +
1515         "created directory at " + this.serverNameFromMasterPOV.toString());
1516     }
1517 
1518     // Instantiate replication manager if replication enabled.  Pass it the
1519     // log directories.
1520     createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
1521 
1522     return instantiateHLog(rootDir, logName);
1523   }
1524 
1525   private HLog getMetaWAL() throws IOException {
1526     if (this.hlogForMeta != null) return this.hlogForMeta;
1527     final String logName = HLogUtil.getHLogDirectoryName(this.serverNameFromMasterPOV.toString());
1528     Path logdir = new Path(rootDir, logName);
1529     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1530     this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(), rootDir, logName,
1531       this.conf, getMetaWALActionListeners(), this.serverNameFromMasterPOV.toString());
1532     return this.hlogForMeta;
1533   }
1534 
1535   /**
1536    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1537    * @param rootdir
1538    * @param logName
1539    * @return WAL instance.
1540    * @throws IOException
1541    */
1542   protected HLog instantiateHLog(Path rootdir, String logName) throws IOException {
1543     return HLogFactory.createHLog(this.fs.getBackingFs(), rootdir, logName, this.conf,
1544       getWALActionListeners(), this.serverNameFromMasterPOV.toString());
1545   }
1546 
1547   /**
1548    * Called by {@link #instantiateHLog(Path, String)} setting up WAL instance.
1549    * Add any {@link WALActionsListener}s you want inserted before WAL startup.
1550    * @return List of WALActionsListener that will be passed in to
1551    * {@link org.apache.hadoop.hbase.regionserver.wal.FSHLog} on construction.
1552    */
1553   protected List<WALActionsListener> getWALActionListeners() {
1554     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1555     // Log roller.
1556     this.hlogRoller = new LogRoller(this, this);
1557     listeners.add(this.hlogRoller);
1558     if (this.replicationSourceHandler != null &&
1559         this.replicationSourceHandler.getWALActionsListener() != null) {
1560       // Replication handler is an implementation of WALActionsListener.
1561       listeners.add(this.replicationSourceHandler.getWALActionsListener());
1562     }
1563     return listeners;
1564   }
1565 
1566   protected List<WALActionsListener> getMetaWALActionListeners() {
1567     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1568     // Using a tmp log roller to ensure metaLogRoller is alive once it is not
1569     // null
1570     MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this);
1571     String n = Thread.currentThread().getName();
1572     Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
1573         n + "-MetaLogRoller", uncaughtExceptionHandler);
1574     this.metaHLogRoller = tmpLogRoller;
1575     tmpLogRoller = null;
1576     listeners.add(this.metaHLogRoller);
1577     return listeners;
1578   }
1579 
1580   protected LogRoller getLogRoller() {
1581     return hlogRoller;
1582   }
1583 
1584   public MetricsRegionServer getMetrics() {
1585     return this.metricsRegionServer;
1586   }
1587 
1588   /**
1589    * @return Master address tracker instance.
1590    */
1591   public MasterAddressTracker getMasterAddressTracker() {
1592     return this.masterAddressTracker;
1593   }
1594 
1595   /*
1596    * Start maintenance Threads, Server, Worker and lease checker threads.
1597    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1598    * get an unhandled exception. We cannot set the handler on all threads.
1599    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1600    * waits a while then retries. Meantime, a flush or a compaction that tries to
1601    * run should trigger same critical condition and the shutdown will run. On
1602    * its way out, this server will shut down Server. Leases are sort of
1603    * inbetween. It has an internal thread that while it inherits from Chore, it
1604    * keeps its own internal stop mechanism so needs to be stopped by this
1605    * hosting server. Worker logs the exception and exits.
1606    */
1607   private void startServiceThreads() throws IOException {
1608     String n = Thread.currentThread().getName();
1609     // Start executor services
1610     this.service = new ExecutorService(getServerName().toShortString());
1611     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1612       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1613     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1614       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1615     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1616       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1617     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1618       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1619     if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
1620       this.service.startExecutorService(ExecutorType.RS_PARALLEL_SEEK,
1621         conf.getInt("hbase.storescanner.parallel.seek.threads", 10));
1622     }
1623     this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS,
1624       conf.getInt("hbase.regionserver.wal.max.splitters", SplitLogWorker.DEFAULT_MAX_SPLITTERS));
1625 
1626     Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), n + ".logRoller",
1627         uncaughtExceptionHandler);
1628     this.cacheFlusher.start(uncaughtExceptionHandler);
1629     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), n +
1630       ".compactionChecker", uncaughtExceptionHandler);
1631     Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), n +
1632         ".periodicFlusher", uncaughtExceptionHandler);
1633     if (this.healthCheckChore != null) {
1634       Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
1635             uncaughtExceptionHandler);
1636     }
1637     if (this.nonceManagerChore != null) {
1638       Threads.setDaemonThreadRunning(this.nonceManagerChore.getThread(), n + ".nonceCleaner",
1639             uncaughtExceptionHandler);
1640     }
1641 
1642     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1643     // an unhandled exception, it will just exit.
1644     this.leases.setName(n + ".leaseChecker");
1645     this.leases.start();
1646 
1647     if (this.replicationSourceHandler == this.replicationSinkHandler &&
1648         this.replicationSourceHandler != null) {
1649       this.replicationSourceHandler.startReplicationService();
1650     } else {
1651       if (this.replicationSourceHandler != null) {
1652         this.replicationSourceHandler.startReplicationService();
1653       }
1654       if (this.replicationSinkHandler != null) {
1655         this.replicationSinkHandler.startReplicationService();
1656       }
1657     }
1658 
1659     // Start Server.  This service is like leases in that it internally runs
1660     // a thread.
1661     this.rpcServer.start();
1662 
1663     // Create the log splitting worker and start it
1664     // set a smaller retries to fast fail otherwise splitlogworker could be blocked for
1665     // quite a while inside HConnection layer. The worker won't be available for other
1666     // tasks even after current task is preempted after a split task times out.
1667     Configuration sinkConf = HBaseConfiguration.create(conf);
1668     sinkConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
1669       conf.getInt("hbase.log.replay.retries.number", 8)); // 8 retries take about 23 seconds
1670     sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
1671       conf.getInt("hbase.log.replay.rpc.timeout", 30000)); // default 30 seconds
1672     sinkConf.setInt("hbase.client.serverside.retries.multiplier", 1);
1673     this.splitLogWorker = new SplitLogWorker(this.zooKeeper, sinkConf, this, this);
1674     splitLogWorker.start();
1675   }
1676 
1677   /**
1678    * Puts up the webui.
1679    * @return Returns final port -- maybe different from what we started with.
1680    * @throws IOException
1681    */
1682   private int putUpWebUI() throws IOException {
1683     int port = this.conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
1684 				HConstants.DEFAULT_REGIONSERVER_INFOPORT);
1685     // -1 is for disabling info server
1686     if (port < 0) return port;
1687     String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1688     // check if auto port bind enabled
1689     boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
1690         false);
1691     while (true) {
1692       try {
1693         this.infoServer = new InfoServer("regionserver", addr, port, false, this.conf);
1694         this.infoServer.addServlet("status", "/rs-status", RSStatusServlet.class);
1695         this.infoServer.addServlet("dump", "/dump", RSDumpServlet.class);
1696         this.infoServer.setAttribute(REGIONSERVER, this);
1697         this.infoServer.setAttribute(REGIONSERVER_CONF, conf);
1698         this.infoServer.start();
1699         break;
1700       } catch (BindException e) {
1701         if (!auto) {
1702           // auto bind disabled throw BindException
1703           LOG.error("Failed binding http info server to port: " + port);
1704           throw e;
1705         }
1706         // auto bind enabled, try to use another port
1707         LOG.info("Failed binding http info server to port: " + port);
1708         port++;
1709       }
1710     }
1711     return this.infoServer.getPort();
1712   }
1713 
1714   /*
1715    * Verify that server is healthy
1716    */
1717   private boolean isHealthy() {
1718     if (!fsOk) {
1719       // File system problem
1720       return false;
1721     }
1722     // Verify that all threads are alive
1723     if (!(leases.isAlive()
1724         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1725         && this.compactionChecker.isAlive()
1726         && this.periodicFlusher.isAlive())) {
1727       stop("One or more threads are no longer alive -- stop");
1728       return false;
1729     }
1730     if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
1731       stop("Meta HLog roller thread is no longer alive -- stop");
1732       return false;
1733     }
1734     return true;
1735   }
1736 
1737   public HLog getWAL() {
1738     try {
1739       return getWAL(null);
1740     } catch (IOException e) {
1741       LOG.warn("getWAL threw exception " + e);
1742       return null;
1743     }
1744   }
1745 
1746   @Override
1747   public HLog getWAL(HRegionInfo regionInfo) throws IOException {
1748     //TODO: at some point this should delegate to the HLogFactory
1749     //currently, we don't care about the region as much as we care about the
1750     //table.. (hence checking the tablename below)
1751     //_ROOT_ and hbase:meta regions have separate WAL.
1752     if (regionInfo != null && regionInfo.isMetaTable()) {
1753       return getMetaWAL();
1754     }
1755     return this.hlog;
1756   }
1757 
1758   @Override
1759   public CatalogTracker getCatalogTracker() {
1760     return this.catalogTracker;
1761   }
1762 
1763   @Override
1764   public void stop(final String msg) {
1765     try {
1766     	if (this.rsHost != null) {
1767     		this.rsHost.preStop(msg);
1768     	}
1769       this.stopped = true;
1770       LOG.info("STOPPED: " + msg);
1771       // Wakes run() if it is sleeping
1772       sleeper.skipSleepCycle();
1773     } catch (IOException exp) {
1774       LOG.warn("The region server did not stop", exp);
1775     }
1776   }
1777 
1778   public void waitForServerOnline(){
1779     while (!isOnline() && !isStopped()){
1780        sleeper.sleep();
1781     }
1782   }
1783 
1784   @Override
1785   public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct)
1786   throws KeeperException, IOException {
1787     checkOpen();
1788     LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString());
1789     // Do checks to see if we need to compact (references or too many files)
1790     for (Store s : r.getStores().values()) {
1791       if (s.hasReferences() || s.needsCompaction()) {
1792        this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
1793       }
1794     }
1795     long openSeqNum = r.getOpenSeqNum();
1796     if (openSeqNum == HConstants.NO_SEQNUM) {
1797       // If we opened a region, we should have read some sequence number from it.
1798       LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
1799       openSeqNum = 0;
1800     }
1801 
1802     // Update flushed sequence id of a recovering region in ZK
1803     updateRecoveringRegionLastFlushedSequenceId(r);
1804 
1805     // Update ZK, or META
1806     if (r.getRegionInfo().isMetaRegion()) {
1807       MetaRegionTracker.setMetaLocation(getZooKeeper(),
1808           this.serverNameFromMasterPOV);
1809     } else {
1810       MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
1811         this.serverNameFromMasterPOV, openSeqNum);
1812     }
1813     LOG.info("Finished post open deploy task for " + r.getRegionNameAsString());
1814 
1815   }
1816 
1817   @Override
1818   public RpcServerInterface getRpcServer() {
1819     return rpcServer;
1820   }
1821 
1822   /**
1823    * Cause the server to exit without closing the regions it is serving, the log
1824    * it is using and without notifying the master. Used unit testing and on
1825    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1826    *
1827    * @param reason
1828    *          the reason we are aborting
1829    * @param cause
1830    *          the exception that caused the abort, or null
1831    */
1832   @Override
1833   public void abort(String reason, Throwable cause) {
1834     String msg = "ABORTING region server " + this + ": " + reason;
1835     if (cause != null) {
1836       LOG.fatal(msg, cause);
1837     } else {
1838       LOG.fatal(msg);
1839     }
1840     this.abortRequested = true;
1841     // HBASE-4014: show list of coprocessors that were loaded to help debug
1842     // regionserver crashes.Note that we're implicitly using
1843     // java.util.HashSet's toString() method to print the coprocessor names.
1844     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1845         CoprocessorHost.getLoadedCoprocessors());
1846     // Do our best to report our abort to the master, but this may not work
1847     try {
1848       if (cause != null) {
1849         msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1850       }
1851       // Report to the master but only if we have already registered with the master.
1852       if (rssStub != null && this.serverNameFromMasterPOV != null) {
1853         ReportRSFatalErrorRequest.Builder builder =
1854           ReportRSFatalErrorRequest.newBuilder();
1855         ServerName sn =
1856           ServerName.parseVersionedServerName(this.serverNameFromMasterPOV.getVersionedBytes());
1857         builder.setServer(ProtobufUtil.toServerName(sn));
1858         builder.setErrorMessage(msg);
1859         rssStub.reportRSFatalError(null, builder.build());
1860       }
1861     } catch (Throwable t) {
1862       LOG.warn("Unable to report fatal error to master", t);
1863     }
1864     stop(reason);
1865   }
1866 
1867   /**
1868    * @see HRegionServer#abort(String, Throwable)
1869    */
1870   public void abort(String reason) {
1871     abort(reason, null);
1872   }
1873 
1874   @Override
1875   public boolean isAborted() {
1876     return this.abortRequested;
1877   }
1878 
1879   /*
1880    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1881    * logs but it does close socket in case want to bring up server on old
1882    * hostname+port immediately.
1883    */
1884   protected void kill() {
1885     this.killed = true;
1886     abort("Simulated kill");
1887   }
1888 
1889   /**
1890    * Wait on all threads to finish. Presumption is that all closes and stops
1891    * have already been called.
1892    */
1893   protected void join() {
1894     if (this.nonceManagerChore != null) {
1895       Threads.shutdown(this.nonceManagerChore.getThread());
1896     }
1897     Threads.shutdown(this.compactionChecker.getThread());
1898     Threads.shutdown(this.periodicFlusher.getThread());
1899     this.cacheFlusher.join();
1900     if (this.healthCheckChore != null) {
1901       Threads.shutdown(this.healthCheckChore.getThread());
1902     }
1903     if (this.spanReceiverHost != null) {
1904       this.spanReceiverHost.closeReceivers();
1905     }
1906     if (this.hlogRoller != null) {
1907       Threads.shutdown(this.hlogRoller.getThread());
1908     }
1909     if (this.metaHLogRoller != null) {
1910       Threads.shutdown(this.metaHLogRoller.getThread());
1911     }
1912     if (this.compactSplitThread != null) {
1913       this.compactSplitThread.join();
1914     }
1915     if (this.service != null) this.service.shutdown();
1916     if (this.replicationSourceHandler != null &&
1917         this.replicationSourceHandler == this.replicationSinkHandler) {
1918       this.replicationSourceHandler.stopReplicationService();
1919     } else {
1920       if (this.replicationSourceHandler != null) {
1921         this.replicationSourceHandler.stopReplicationService();
1922       }
1923       if (this.replicationSinkHandler != null) {
1924         this.replicationSinkHandler.stopReplicationService();
1925       }
1926     }
1927   }
1928 
1929   /**
1930    * @return Return the object that implements the replication
1931    * source service.
1932    */
1933   ReplicationSourceService getReplicationSourceService() {
1934     return replicationSourceHandler;
1935   }
1936 
1937   /**
1938    * @return Return the object that implements the replication
1939    * sink service.
1940    */
1941   ReplicationSinkService getReplicationSinkService() {
1942     return replicationSinkHandler;
1943   }
1944 
1945   /**
1946    * Get the current master from ZooKeeper and open the RPC connection to it.
1947    *
1948    * Method will block until a master is available. You can break from this
1949    * block by requesting the server stop.
1950    *
1951    * @return master + port, or null if server has been stopped
1952    */
1953   private Pair<ServerName, RegionServerStatusService.BlockingInterface>
1954   createRegionServerStatusStub() {
1955     ServerName sn = null;
1956     long previousLogTime = 0;
1957     RegionServerStatusService.BlockingInterface master = null;
1958     boolean refresh = false; // for the first time, use cached data
1959     RegionServerStatusService.BlockingInterface intf = null;
1960     boolean interrupted = false;
1961     try {
1962       while (keepLooping() && master == null) {
1963         sn = this.masterAddressTracker.getMasterAddress(refresh);
1964         if (sn == null) {
1965           if (!keepLooping()) {
1966             // give up with no connection.
1967             LOG.debug("No master found and cluster is stopped; bailing out");
1968             return null;
1969           }
1970           LOG.debug("No master found; retry");
1971           previousLogTime = System.currentTimeMillis();
1972           refresh = true; // let's try pull it from ZK directly
1973           sleeper.sleep();
1974           continue;
1975         }
1976 
1977         new InetSocketAddress(sn.getHostname(), sn.getPort());
1978         try {
1979           BlockingRpcChannel channel =
1980             this.rpcClient.createBlockingRpcChannel(sn, userProvider.getCurrent(), operationTimeout);
1981           intf = RegionServerStatusService.newBlockingStub(channel);
1982           break;
1983         } catch (IOException e) {
1984           e = e instanceof RemoteException ?
1985             ((RemoteException)e).unwrapRemoteException() : e;
1986           if (e instanceof ServerNotRunningYetException) {
1987             if (System.currentTimeMillis() > (previousLogTime+1000)){
1988               LOG.info("Master isn't available yet, retrying");
1989               previousLogTime = System.currentTimeMillis();
1990             }
1991           } else {
1992             if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1993               LOG.warn("Unable to connect to master. Retrying. Error was:", e);
1994               previousLogTime = System.currentTimeMillis();
1995             }
1996           }
1997           try {
1998             Thread.sleep(200);
1999           } catch (InterruptedException ex) {
2000             interrupted = true;
2001             LOG.warn("Interrupted while sleeping");
2002           }
2003         }
2004       }
2005     } finally {
2006       if (interrupted) {
2007         Thread.currentThread().interrupt();
2008       }
2009     }
2010     return new Pair<ServerName, RegionServerStatusService.BlockingInterface>(sn, intf);
2011   }
2012 
2013   /**
2014    * @return True if we should break loop because cluster is going down or
2015    * this server has been stopped or hdfs has gone bad.
2016    */
2017   private boolean keepLooping() {
2018     return !this.stopped && isClusterUp();
2019   }
2020 
2021   /*
2022    * Let the master know we're here Run initialization using parameters passed
2023    * us by the master.
2024    * @return A Map of key/value configurations we got from the Master else
2025    * null if we failed to register.
2026    * @throws IOException
2027    */
2028   private RegionServerStartupResponse reportForDuty() throws IOException {
2029     RegionServerStartupResponse result = null;
2030     Pair<ServerName, RegionServerStatusService.BlockingInterface> p =
2031       createRegionServerStatusStub();
2032     this.rssStub = p.getSecond();
2033     ServerName masterServerName = p.getFirst();
2034     if (masterServerName == null) return result;
2035     try {
2036       this.requestCount.set(0);
2037       LOG.info("reportForDuty to master=" + masterServerName + " with port=" + this.isa.getPort() +
2038         ", startcode=" + this.startcode);
2039       long now = EnvironmentEdgeManager.currentTimeMillis();
2040       int port = this.isa.getPort();
2041       RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
2042       request.setPort(port);
2043       request.setServerStartCode(this.startcode);
2044       request.setServerCurrentTime(now);
2045       result = this.rssStub.regionServerStartup(null, request.build());
2046     } catch (ServiceException se) {
2047       IOException ioe = ProtobufUtil.getRemoteException(se);
2048       if (ioe instanceof ClockOutOfSyncException) {
2049         LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2050         // Re-throw IOE will cause RS to abort
2051         throw ioe;
2052       } else if (ioe instanceof ServerNotRunningYetException) {
2053         LOG.debug("Master is not running yet");
2054       } else {
2055         LOG.warn("error telling master we are up", se);
2056       }
2057     }
2058     return result;
2059   }
2060 
2061   @Override
2062   public long getLastSequenceId(byte[] region) {
2063     Long lastFlushedSequenceId = -1l;
2064     try {
2065       GetLastFlushedSequenceIdRequest req = RequestConverter
2066           .buildGetLastFlushedSequenceIdRequest(region);
2067       lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req)
2068           .getLastFlushedSequenceId();
2069     } catch (ServiceException e) {
2070       lastFlushedSequenceId = -1l;
2071       LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id", e);
2072     }
2073     return lastFlushedSequenceId;
2074   }
2075 
2076   /**
2077    * Closes all regions.  Called on our way out.
2078    * Assumes that its not possible for new regions to be added to onlineRegions
2079    * while this method runs.
2080    */
2081   protected void closeAllRegions(final boolean abort) {
2082     closeUserRegions(abort);
2083     closeMetaTableRegions(abort);
2084   }
2085 
2086   /**
2087    * Close meta region if we carry it
2088    * @param abort Whether we're running an abort.
2089    */
2090   void closeMetaTableRegions(final boolean abort) {
2091     HRegion meta = null;
2092     this.lock.writeLock().lock();
2093     try {
2094       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2095         HRegionInfo hri = e.getValue().getRegionInfo();
2096         if (hri.isMetaRegion()) {
2097           meta = e.getValue();
2098         }
2099         if (meta != null) break;
2100       }
2101     } finally {
2102       this.lock.writeLock().unlock();
2103     }
2104     if (meta != null) closeRegionIgnoreErrors(meta.getRegionInfo(), abort);
2105   }
2106 
2107   /**
2108    * Schedule closes on all user regions.
2109    * Should be safe calling multiple times because it wont' close regions
2110    * that are already closed or that are closing.
2111    * @param abort Whether we're running an abort.
2112    */
2113   void closeUserRegions(final boolean abort) {
2114     this.lock.writeLock().lock();
2115     try {
2116       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2117         HRegion r = e.getValue();
2118         if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2119           // Don't update zk with this close transition; pass false.
2120           closeRegionIgnoreErrors(r.getRegionInfo(), abort);
2121         }
2122       }
2123     } finally {
2124       this.lock.writeLock().unlock();
2125     }
2126   }
2127 
2128   /** @return the info server */
2129   public InfoServer getInfoServer() {
2130     return infoServer;
2131   }
2132 
2133   /**
2134    * @return true if a stop has been requested.
2135    */
2136   @Override
2137   public boolean isStopped() {
2138     return this.stopped;
2139   }
2140 
2141   @Override
2142   public boolean isStopping() {
2143     return this.stopping;
2144   }
2145 
2146   @Override
2147   public Map<String, HRegion> getRecoveringRegions() {
2148     return this.recoveringRegions;
2149   }
2150 
2151   /**
2152    *
2153    * @return the configuration
2154    */
2155   @Override
2156   public Configuration getConfiguration() {
2157     return conf;
2158   }
2159 
2160   /** @return the write lock for the server */
2161   ReentrantReadWriteLock.WriteLock getWriteLock() {
2162     return lock.writeLock();
2163   }
2164 
2165   public int getNumberOfOnlineRegions() {
2166     return this.onlineRegions.size();
2167   }
2168 
2169   boolean isOnlineRegionsEmpty() {
2170     return this.onlineRegions.isEmpty();
2171   }
2172 
2173   /**
2174    * For tests, web ui and metrics.
2175    * This method will only work if HRegionServer is in the same JVM as client;
2176    * HRegion cannot be serialized to cross an rpc.
2177    */
2178   public Collection<HRegion> getOnlineRegionsLocalContext() {
2179     Collection<HRegion> regions = this.onlineRegions.values();
2180     return Collections.unmodifiableCollection(regions);
2181   }
2182 
2183   @Override
2184   public void addToOnlineRegions(HRegion region) {
2185     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2186   }
2187 
2188   /**
2189    * @return A new Map of online regions sorted by region size with the first entry being the
2190    * biggest.  If two regions are the same size, then the last one found wins; i.e. this method
2191    * may NOT return all regions.
2192    */
2193   SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
2194     // we'll sort the regions in reverse
2195     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
2196         new Comparator<Long>() {
2197           @Override
2198           public int compare(Long a, Long b) {
2199             return -1 * a.compareTo(b);
2200           }
2201         });
2202     // Copy over all regions. Regions are sorted by size with biggest first.
2203     for (HRegion region : this.onlineRegions.values()) {
2204       sortedRegions.put(region.memstoreSize.get(), region);
2205     }
2206     return sortedRegions;
2207   }
2208 
2209   /**
2210    * @return time stamp in millis of when this region server was started
2211    */
2212   public long getStartcode() {
2213     return this.startcode;
2214   }
2215 
2216   /** @return reference to FlushRequester */
2217   @Override
2218   public FlushRequester getFlushRequester() {
2219     return this.cacheFlusher;
2220   }
2221 
2222   /**
2223    * Get the top N most loaded regions this server is serving so we can tell the
2224    * master which regions it can reallocate if we're overloaded. TODO: actually
2225    * calculate which regions are most loaded. (Right now, we're just grabbing
2226    * the first N regions being served regardless of load.)
2227    */
2228   protected HRegionInfo[] getMostLoadedRegions() {
2229     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2230     for (HRegion r : onlineRegions.values()) {
2231       if (!r.isAvailable()) {
2232         continue;
2233       }
2234       if (regions.size() < numRegionsToReport) {
2235         regions.add(r.getRegionInfo());
2236       } else {
2237         break;
2238       }
2239     }
2240     return regions.toArray(new HRegionInfo[regions.size()]);
2241   }
2242 
2243   @Override
2244   public Leases getLeases() {
2245     return leases;
2246   }
2247 
2248   /**
2249    * @return Return the rootDir.
2250    */
2251   protected Path getRootDir() {
2252     return rootDir;
2253   }
2254 
2255   /**
2256    * @return Return the fs.
2257    */
2258   @Override
2259   public FileSystem getFileSystem() {
2260     return fs;
2261   }
2262 
2263   @Override
2264   public String toString() {
2265     return getServerName().toString();
2266   }
2267 
2268   /**
2269    * Interval at which threads should run
2270    *
2271    * @return the interval
2272    */
2273   public int getThreadWakeFrequency() {
2274     return threadWakeFrequency;
2275   }
2276 
2277   @Override
2278   public ZooKeeperWatcher getZooKeeper() {
2279     return zooKeeper;
2280   }
2281 
2282   @Override
2283   public ServerName getServerName() {
2284     // Our servername could change after we talk to the master.
2285     return this.serverNameFromMasterPOV == null?
2286         ServerName.valueOf(this.isa.getHostName(), this.isa.getPort(), this.startcode) :
2287         this.serverNameFromMasterPOV;
2288   }
2289 
2290   @Override
2291   public CompactionRequestor getCompactionRequester() {
2292     return this.compactSplitThread;
2293   }
2294 
2295   public ZooKeeperWatcher getZooKeeperWatcher() {
2296     return this.zooKeeper;
2297   }
2298 
2299   public RegionServerCoprocessorHost getCoprocessorHost(){
2300     return this.rsHost;
2301   }
2302 
2303   @Override
2304   public ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS() {
2305     return this.regionsInTransitionInRS;
2306   }
2307 
2308   @Override
2309   public ExecutorService getExecutorService() {
2310     return service;
2311   }
2312 
2313   //
2314   // Main program and support routines
2315   //
2316 
2317   /**
2318    * Load the replication service objects, if any
2319    */
2320   static private void createNewReplicationInstance(Configuration conf,
2321     HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
2322 
2323     // If replication is not enabled, then return immediately.
2324     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
2325         HConstants.REPLICATION_ENABLE_DEFAULT)) {
2326       return;
2327     }
2328 
2329     // read in the name of the source replication class from the config file.
2330     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
2331                                HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2332 
2333     // read in the name of the sink replication class from the config file.
2334     String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
2335                              HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2336 
2337     // If both the sink and the source class names are the same, then instantiate
2338     // only one object.
2339     if (sourceClassname.equals(sinkClassname)) {
2340       server.replicationSourceHandler = (ReplicationSourceService)
2341                                          newReplicationInstance(sourceClassname,
2342                                          conf, server, fs, logDir, oldLogDir);
2343       server.replicationSinkHandler = (ReplicationSinkService)
2344                                          server.replicationSourceHandler;
2345     } else {
2346       server.replicationSourceHandler = (ReplicationSourceService)
2347                                          newReplicationInstance(sourceClassname,
2348                                          conf, server, fs, logDir, oldLogDir);
2349       server.replicationSinkHandler = (ReplicationSinkService)
2350                                          newReplicationInstance(sinkClassname,
2351                                          conf, server, fs, logDir, oldLogDir);
2352     }
2353   }
2354 
2355   static private ReplicationService newReplicationInstance(String classname,
2356     Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
2357     Path oldLogDir) throws IOException{
2358 
2359     Class<?> clazz = null;
2360     try {
2361       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
2362       clazz = Class.forName(classname, true, classLoader);
2363     } catch (java.lang.ClassNotFoundException nfe) {
2364       throw new IOException("Could not find class for " + classname);
2365     }
2366 
2367     // create an instance of the replication object.
2368     ReplicationService service = (ReplicationService)
2369                               ReflectionUtils.newInstance(clazz, conf);
2370     service.initialize(server, fs, logDir, oldLogDir);
2371     return service;
2372   }
2373 
2374   /**
2375    * @param hrs
2376    * @return Thread the RegionServer is running in correctly named.
2377    * @throws IOException
2378    */
2379   public static Thread startRegionServer(final HRegionServer hrs)
2380       throws IOException {
2381     return startRegionServer(hrs, "regionserver" + hrs.isa.getPort());
2382   }
2383 
2384   /**
2385    * @param hrs
2386    * @param name
2387    * @return Thread the RegionServer is running in correctly named.
2388    * @throws IOException
2389    */
2390   public static Thread startRegionServer(final HRegionServer hrs,
2391       final String name) throws IOException {
2392     Thread t = new Thread(hrs);
2393     t.setName(name);
2394     t.start();
2395     // Install shutdown hook that will catch signals and run an orderly shutdown
2396     // of the hrs.
2397     ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
2398         .getConfiguration()), hrs, t);
2399     return t;
2400   }
2401 
2402   /**
2403    * Utility for constructing an instance of the passed HRegionServer class.
2404    *
2405    * @param regionServerClass
2406    * @param conf2
2407    * @return HRegionServer instance.
2408    */
2409   public static HRegionServer constructRegionServer(
2410       Class<? extends HRegionServer> regionServerClass,
2411       final Configuration conf2) {
2412     try {
2413       Constructor<? extends HRegionServer> c = regionServerClass
2414           .getConstructor(Configuration.class);
2415       return c.newInstance(conf2);
2416     } catch (Exception e) {
2417       throw new RuntimeException("Failed construction of " + "Regionserver: "
2418           + regionServerClass.toString(), e);
2419     }
2420   }
2421 
2422   /**
2423    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
2424    */
2425   public static void main(String[] args) throws Exception {
2426 	VersionInfo.logVersion();
2427     Configuration conf = HBaseConfiguration.create();
2428     @SuppressWarnings("unchecked")
2429     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
2430         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
2431 
2432     new HRegionServerCommandLine(regionServerClass).doMain(args);
2433   }
2434 
2435   /**
2436    * Gets the online regions of the specified table.
2437    * This method looks at the in-memory onlineRegions.  It does not go to <code>hbase:meta</code>.
2438    * Only returns <em>online</em> regions.  If a region on this table has been
2439    * closed during a disable, etc., it will not be included in the returned list.
2440    * So, the returned list may not necessarily be ALL regions in this table, its
2441    * all the ONLINE regions in the table.
2442    * @param tableName
2443    * @return Online regions from <code>tableName</code>
2444    */
2445   @Override
2446   public List<HRegion> getOnlineRegions(TableName tableName) {
2447      List<HRegion> tableRegions = new ArrayList<HRegion>();
2448      synchronized (this.onlineRegions) {
2449        for (HRegion region: this.onlineRegions.values()) {
2450          HRegionInfo regionInfo = region.getRegionInfo();
2451          if(regionInfo.getTable().equals(tableName)) {
2452            tableRegions.add(region);
2453          }
2454        }
2455      }
2456      return tableRegions;
2457    }
2458 
2459   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
2460   public String[] getCoprocessors() {
2461     TreeSet<String> coprocessors = new TreeSet<String>(
2462         this.hlog.getCoprocessorHost().getCoprocessors());
2463     Collection<HRegion> regions = getOnlineRegionsLocalContext();
2464     for (HRegion region: regions) {
2465       coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
2466     }
2467     return coprocessors.toArray(new String[coprocessors.size()]);
2468   }
2469 
2470   /**
2471    * Instantiated as a scanner lease. If the lease times out, the scanner is
2472    * closed
2473    */
2474   private class ScannerListener implements LeaseListener {
2475     private final String scannerName;
2476 
2477     ScannerListener(final String n) {
2478       this.scannerName = n;
2479     }
2480 
2481     @Override
2482     public void leaseExpired() {
2483       RegionScannerHolder rsh = scanners.remove(this.scannerName);
2484       if (rsh != null) {
2485         RegionScanner s = rsh.s;
2486         LOG.info("Scanner " + this.scannerName + " lease expired on region "
2487             + s.getRegionInfo().getRegionNameAsString());
2488         try {
2489           HRegion region = getRegion(s.getRegionInfo().getRegionName());
2490           if (region != null && region.getCoprocessorHost() != null) {
2491             region.getCoprocessorHost().preScannerClose(s);
2492           }
2493 
2494           s.close();
2495           if (region != null && region.getCoprocessorHost() != null) {
2496             region.getCoprocessorHost().postScannerClose(s);
2497           }
2498         } catch (IOException e) {
2499           LOG.error("Closing scanner for "
2500               + s.getRegionInfo().getRegionNameAsString(), e);
2501         }
2502       } else {
2503         LOG.warn("Scanner " + this.scannerName + " lease expired, but no related" +
2504             " scanner found, hence no chance to close that related scanner!");
2505       }
2506     }
2507   }
2508 
2509   /**
2510    * Called to verify that this server is up and running.
2511    *
2512    * @throws IOException
2513    */
2514   protected void checkOpen() throws IOException {
2515     if (this.stopped || this.abortRequested) {
2516       throw new RegionServerStoppedException("Server " + getServerName() +
2517         " not running" + (this.abortRequested ? ", aborting" : ""));
2518     }
2519     if (!fsOk) {
2520       throw new RegionServerStoppedException("File system not available");
2521     }
2522   }
2523 
2524 
2525   /**
2526    * Try to close the region, logs a warning on failure but continues.
2527    * @param region Region to close
2528    */
2529   private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
2530     try {
2531       if (!closeRegion(region.getEncodedName(), abort, false, -1, null)) {
2532         LOG.warn("Failed to close " + region.getRegionNameAsString() +
2533             " - ignoring and continuing");
2534       }
2535     } catch (IOException e) {
2536       LOG.warn("Failed to close " + region.getRegionNameAsString() +
2537           " - ignoring and continuing", e);
2538     }
2539   }
2540 
2541   /**
2542    * Close asynchronously a region, can be called from the master or internally by the regionserver
2543    * when stopping. If called from the master, the region will update the znode status.
2544    *
2545    * <p>
2546    * If an opening was in progress, this method will cancel it, but will not start a new close. The
2547    * coprocessors are not called in this case. A NotServingRegionException exception is thrown.
2548    * </p>
2549 
2550    * <p>
2551    *   If a close was in progress, this new request will be ignored, and an exception thrown.
2552    * </p>
2553    *
2554    * @param encodedName Region to close
2555    * @param abort True if we are aborting
2556    * @param zk True if we are to update zk about the region close; if the close
2557    * was orchestrated by master, then update zk.  If the close is being run by
2558    * the regionserver because its going down, don't update zk.
2559    * @param versionOfClosingNode the version of znode to compare when RS transitions the znode from
2560    *   CLOSING state.
2561    * @return True if closed a region.
2562    * @throws NotServingRegionException if the region is not online
2563    * @throws RegionAlreadyInTransitionException if the region is already closing
2564    */
2565   protected boolean closeRegion(String encodedName, final boolean abort,
2566       final boolean zk, final int versionOfClosingNode, final ServerName sn)
2567       throws NotServingRegionException, RegionAlreadyInTransitionException {
2568     //Check for permissions to close.
2569     HRegion actualRegion = this.getFromOnlineRegions(encodedName);
2570     if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
2571       try {
2572         actualRegion.getCoprocessorHost().preClose(false);
2573       } catch (IOException exp) {
2574         LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
2575         return false;
2576       }
2577     }
2578 
2579     final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(encodedName.getBytes(),
2580         Boolean.FALSE);
2581 
2582     if (Boolean.TRUE.equals(previous)) {
2583       LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
2584           "trying to OPEN. Cancelling OPENING.");
2585       if (!regionsInTransitionInRS.replace(encodedName.getBytes(), previous, Boolean.FALSE)){
2586         // The replace failed. That should be an exceptional case, but theoretically it can happen.
2587         // We're going to try to do a standard close then.
2588         LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
2589             " Doing a standard close now");
2590         return closeRegion(encodedName, abort, zk, versionOfClosingNode, sn);
2591       }
2592       // Let's get the region from the online region list again
2593       actualRegion = this.getFromOnlineRegions(encodedName);
2594       if (actualRegion == null) { // If already online, we still need to close it.
2595         LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
2596         // The master deletes the znode when it receives this exception.
2597         throw new NotServingRegionException("The region " + encodedName +
2598           " was opening but not yet served. Opening is cancelled.");
2599       }
2600     } else if (Boolean.FALSE.equals(previous)) {
2601       LOG.info("Received CLOSE for the region: " + encodedName +
2602         " ,which we are already trying to CLOSE, but not completed yet");
2603       // The master will retry till the region is closed. We need to do this since
2604       // the region could fail to close somehow. If we mark the region closed in master
2605       // while it is not, there could be data loss.
2606       // If the region stuck in closing for a while, and master runs out of retries,
2607       // master will move the region to failed_to_close. Later on, if the region
2608       // is indeed closed, master can properly re-assign it.
2609       throw new RegionAlreadyInTransitionException("The region " + encodedName +
2610         " was already closing. New CLOSE request is ignored.");
2611     }
2612 
2613     if (actualRegion == null) {
2614       LOG.error("Received CLOSE for a region which is not online, and we're not opening.");
2615       this.regionsInTransitionInRS.remove(encodedName.getBytes());
2616       // The master deletes the znode when it receives this exception.
2617       throw new NotServingRegionException("The region " + encodedName +
2618           " is not online, and is not opening.");
2619     }
2620 
2621     CloseRegionHandler crh;
2622     final HRegionInfo hri = actualRegion.getRegionInfo();
2623     if (hri.isMetaRegion()) {
2624       crh = new CloseMetaHandler(this, this, hri, abort, zk, versionOfClosingNode);
2625     } else {
2626       crh = new CloseRegionHandler(this, this, hri, abort, zk, versionOfClosingNode, sn);
2627     }
2628     this.service.submit(crh);
2629     return true;
2630   }
2631 
2632    /**
2633    * @param regionName
2634    * @return HRegion for the passed binary <code>regionName</code> or null if
2635    *         named region is not member of the online regions.
2636    */
2637   public HRegion getOnlineRegion(final byte[] regionName) {
2638     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2639     return this.onlineRegions.get(encodedRegionName);
2640   }
2641 
2642   public InetSocketAddress[] getRegionBlockLocations(final String encodedRegionName) {
2643     return this.regionFavoredNodesMap.get(encodedRegionName);
2644   }
2645 
2646   @Override
2647   public HRegion getFromOnlineRegions(final String encodedRegionName) {
2648     return this.onlineRegions.get(encodedRegionName);
2649   }
2650 
2651 
2652   @Override
2653   public boolean removeFromOnlineRegions(final HRegion r, ServerName destination) {
2654     HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
2655 
2656     if (destination != null) {
2657       HLog wal = getWAL();
2658       long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes());
2659       if (closeSeqNum == HConstants.NO_SEQNUM) {
2660         // No edits in WAL for this region; get the sequence number when the region was opened.
2661         closeSeqNum = r.getOpenSeqNum();
2662         if (closeSeqNum == HConstants.NO_SEQNUM) {
2663           closeSeqNum = 0;
2664         }
2665       }
2666       addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
2667     }
2668     this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
2669     return toReturn != null;
2670   }
2671 
2672   /**
2673    * Protected utility method for safely obtaining an HRegion handle.
2674    *
2675    * @param regionName
2676    *          Name of online {@link HRegion} to return
2677    * @return {@link HRegion} for <code>regionName</code>
2678    * @throws NotServingRegionException
2679    */
2680   protected HRegion getRegion(final byte[] regionName)
2681       throws NotServingRegionException {
2682     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2683     return getRegionByEncodedName(regionName, encodedRegionName);
2684   }
2685 
2686   protected HRegion getRegionByEncodedName(String encodedRegionName)
2687       throws NotServingRegionException {
2688     return getRegionByEncodedName(null, encodedRegionName);
2689   }
2690 
2691   protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName)
2692     throws NotServingRegionException {
2693     HRegion region = this.onlineRegions.get(encodedRegionName);
2694     if (region == null) {
2695       MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
2696       if (moveInfo != null) {
2697         throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
2698       }
2699       Boolean isOpening = this.regionsInTransitionInRS.get(Bytes.toBytes(encodedRegionName));
2700       String regionNameStr = regionName == null?
2701         encodedRegionName: Bytes.toStringBinary(regionName);
2702       if (isOpening != null && isOpening.booleanValue()) {
2703         throw new RegionOpeningException("Region " + regionNameStr + " is opening");
2704       }
2705       throw new NotServingRegionException("Region " + regionNameStr + " is not online");
2706     }
2707     return region;
2708   }
2709 
2710   /*
2711    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
2712    * IOE if it isn't already.
2713    *
2714    * @param t Throwable
2715    *
2716    * @return Throwable converted to an IOE; methods can only let out IOEs.
2717    */
2718   protected Throwable cleanup(final Throwable t) {
2719     return cleanup(t, null);
2720   }
2721 
2722   /*
2723    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
2724    * IOE if it isn't already.
2725    *
2726    * @param t Throwable
2727    *
2728    * @param msg Message to log in error. Can be null.
2729    *
2730    * @return Throwable converted to an IOE; methods can only let out IOEs.
2731    */
2732   protected Throwable cleanup(final Throwable t, final String msg) {
2733     // Don't log as error if NSRE; NSRE is 'normal' operation.
2734     if (t instanceof NotServingRegionException) {
2735       LOG.debug("NotServingRegionException; " + t.getMessage());
2736       return t;
2737     }
2738     if (msg == null) {
2739       LOG.error("", RemoteExceptionHandler.checkThrowable(t));
2740     } else {
2741       LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
2742     }
2743     if (!checkOOME(t)) {
2744       checkFileSystem();
2745     }
2746     return t;
2747   }
2748 
2749   /*
2750    * @param t
2751    *
2752    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
2753    *
2754    * @return Make <code>t</code> an IOE if it isn't already.
2755    */
2756   protected IOException convertThrowableToIOE(final Throwable t, final String msg) {
2757     return (t instanceof IOException ? (IOException) t : msg == null
2758         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
2759   }
2760 
2761   /*
2762    * Check if an OOME and, if so, abort immediately to avoid creating more objects.
2763    *
2764    * @param e
2765    *
2766    * @return True if we OOME'd and are aborting.
2767    */
2768   @Override
2769   public boolean checkOOME(final Throwable e) {
2770     boolean stop = false;
2771     try {
2772       if (e instanceof OutOfMemoryError
2773           || (e.getCause() != null && e.getCause() instanceof OutOfMemoryError)
2774           || (e.getMessage() != null && e.getMessage().contains(
2775               "java.lang.OutOfMemoryError"))) {
2776         stop = true;
2777         LOG.fatal(
2778           "Run out of memory; HRegionServer will abort itself immediately", e);
2779       }
2780     } finally {
2781       if (stop) {
2782         Runtime.getRuntime().halt(1);
2783       }
2784     }
2785     return stop;
2786   }
2787 
2788   /**
2789    * Checks to see if the file system is still accessible. If not, sets
2790    * abortRequested and stopRequested
2791    *
2792    * @return false if file system is not available
2793    */
2794   public boolean checkFileSystem() {
2795     if (this.fsOk && this.fs != null) {
2796       try {
2797         FSUtils.checkFileSystemAvailable(this.fs);
2798       } catch (IOException e) {
2799         abort("File System not available", e);
2800         this.fsOk = false;
2801       }
2802     }
2803     return this.fsOk;
2804   }
2805 
2806   protected long addScanner(RegionScanner s, HRegion r) throws LeaseStillHeldException {
2807     long scannerId = this.scannerIdGen.incrementAndGet();
2808     String scannerName = String.valueOf(scannerId);
2809 
2810     RegionScannerHolder existing =
2811       scanners.putIfAbsent(scannerName, new RegionScannerHolder(s, r));
2812     assert existing == null : "scannerId must be unique within regionserver's whole lifecycle!";
2813 
2814     this.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod,
2815         new ScannerListener(scannerName));
2816 
2817     return scannerId;
2818   }
2819 
2820   // Start Client methods
2821 
2822   /**
2823    * Get data from a table.
2824    *
2825    * @param controller the RPC controller
2826    * @param request the get request
2827    * @throws ServiceException
2828    */
2829   @Override
2830   public GetResponse get(final RpcController controller,
2831       final GetRequest request) throws ServiceException {
2832     long before = EnvironmentEdgeManager.currentTimeMillis();
2833     try {
2834       checkOpen();
2835       requestCount.increment();
2836       HRegion region = getRegion(request.getRegion());
2837 
2838       GetResponse.Builder builder = GetResponse.newBuilder();
2839       ClientProtos.Get get = request.getGet();
2840       Boolean existence = null;
2841       Result r = null;
2842 
2843       if (get.hasClosestRowBefore() && get.getClosestRowBefore()) {
2844         if (get.getColumnCount() != 1) {
2845           throw new DoNotRetryIOException(
2846             "get ClosestRowBefore supports one and only one family now, not "
2847               + get.getColumnCount() + " families");
2848         }
2849         byte[] row = get.getRow().toByteArray();
2850         byte[] family = get.getColumn(0).getFamily().toByteArray();
2851         r = region.getClosestRowBefore(row, family);
2852       } else {
2853         Get clientGet = ProtobufUtil.toGet(get);
2854         if (get.getExistenceOnly() && region.getCoprocessorHost() != null) {
2855           existence = region.getCoprocessorHost().preExists(clientGet);
2856         }
2857         if (existence == null) {
2858           r = region.get(clientGet);
2859           if (get.getExistenceOnly()) {
2860             boolean exists = r.getExists();
2861             if (region.getCoprocessorHost() != null) {
2862               exists = region.getCoprocessorHost().postExists(clientGet, exists);
2863             }
2864             existence = exists;
2865           }
2866         }
2867       }
2868       if (existence != null){
2869         ClientProtos.Result pbr = ProtobufUtil.toResult(existence);
2870         builder.setResult(pbr);
2871       } else  if (r != null) {
2872         ClientProtos.Result pbr = ProtobufUtil.toResult(r);
2873         builder.setResult(pbr);
2874       }
2875       return builder.build();
2876     } catch (IOException ie) {
2877       throw new ServiceException(ie);
2878     } finally {
2879       metricsRegionServer.updateGet(EnvironmentEdgeManager.currentTimeMillis() - before);
2880     }
2881   }
2882 
2883 
2884   /**
2885    * Mutate data in a table.
2886    *
2887    * @param rpcc the RPC controller
2888    * @param request the mutate request
2889    * @throws ServiceException
2890    */
2891   @Override
2892   public MutateResponse mutate(final RpcController rpcc,
2893       final MutateRequest request) throws ServiceException {
2894     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
2895     // It is also the conduit via which we pass back data.
2896     PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
2897     CellScanner cellScanner = controller != null? controller.cellScanner(): null;
2898     // Clear scanner so we are not holding on to reference across call.
2899     if (controller != null) controller.setCellScanner(null);
2900     try {
2901       checkOpen();
2902       requestCount.increment();
2903       HRegion region = getRegion(request.getRegion());
2904       MutateResponse.Builder builder = MutateResponse.newBuilder();
2905       MutationProto mutation = request.getMutation();
2906       if (!region.getRegionInfo().isMetaTable()) {
2907         cacheFlusher.reclaimMemStoreMemory();
2908       }
2909       long nonceGroup = request.hasNonceGroup()
2910           ? request.getNonceGroup() : HConstants.NO_NONCE;
2911       Result r = null;
2912       Boolean processed = null;
2913       MutationType type = mutation.getMutateType();
2914       switch (type) {
2915       case APPEND:
2916         // TODO: this doesn't actually check anything.
2917         r = append(region, mutation, cellScanner, nonceGroup);
2918         break;
2919       case INCREMENT:
2920         // TODO: this doesn't actually check anything.
2921         r = increment(region, mutation, cellScanner, nonceGroup);
2922         break;
2923       case PUT:
2924         Put put = ProtobufUtil.toPut(mutation, cellScanner);
2925         if (request.hasCondition()) {
2926           Condition condition = request.getCondition();
2927           byte[] row = condition.getRow().toByteArray();
2928           byte[] family = condition.getFamily().toByteArray();
2929           byte[] qualifier = condition.getQualifier().toByteArray();
2930           CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
2931           ByteArrayComparable comparator =
2932             ProtobufUtil.toComparator(condition.getComparator());
2933           if (region.getCoprocessorHost() != null) {
2934             processed = region.getCoprocessorHost().preCheckAndPut(
2935               row, family, qualifier, compareOp, comparator, put);
2936           }
2937           if (processed == null) {
2938             boolean result = region.checkAndMutate(row, family,
2939               qualifier, compareOp, comparator, put, true);
2940             if (region.getCoprocessorHost() != null) {
2941               result = region.getCoprocessorHost().postCheckAndPut(row, family,
2942                 qualifier, compareOp, comparator, put, result);
2943             }
2944             processed = result;
2945           }
2946         } else {
2947           region.put(put);
2948           processed = Boolean.TRUE;
2949         }
2950         break;
2951       case DELETE:
2952         Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
2953         if (request.hasCondition()) {
2954           Condition condition = request.getCondition();
2955           byte[] row = condition.getRow().toByteArray();
2956           byte[] family = condition.getFamily().toByteArray();
2957           byte[] qualifier = condition.getQualifier().toByteArray();
2958           CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
2959           ByteArrayComparable comparator =
2960             ProtobufUtil.toComparator(condition.getComparator());
2961           if (region.getCoprocessorHost() != null) {
2962             processed = region.getCoprocessorHost().preCheckAndDelete(
2963               row, family, qualifier, compareOp, comparator, delete);
2964           }
2965           if (processed == null) {
2966             boolean result = region.checkAndMutate(row, family,
2967               qualifier, compareOp, comparator, delete, true);
2968             if (region.getCoprocessorHost() != null) {
2969               result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2970                 qualifier, compareOp, comparator, delete, result);
2971             }
2972             processed = result;
2973           }
2974         } else {
2975           region.delete(delete);
2976           processed = Boolean.TRUE;
2977         }
2978         break;
2979         default:
2980           throw new DoNotRetryIOException(
2981             "Unsupported mutate type: " + type.name());
2982       }
2983       if (processed != null) builder.setProcessed(processed.booleanValue());
2984       addResult(builder, r, controller);
2985       return builder.build();
2986     } catch (IOException ie) {
2987       checkFileSystem();
2988       throw new ServiceException(ie);
2989     }
2990   }
2991 
2992 
2993   /**
2994    * @return True if current call supports cellblocks
2995    */
2996   private boolean isClientCellBlockSupport() {
2997     RpcCallContext context = RpcServer.getCurrentCall();
2998     return context != null && context.isClientCellBlockSupport();
2999   }
3000 
3001   private void addResult(final MutateResponse.Builder builder,
3002       final Result result, final PayloadCarryingRpcController rpcc) {
3003     if (result == null) return;
3004     if (isClientCellBlockSupport()) {
3005       builder.setResult(ProtobufUtil.toResultNoData(result));
3006       rpcc.setCellScanner(result.cellScanner());
3007     } else {
3008       ClientProtos.Result pbr = ProtobufUtil.toResult(result);
3009       builder.setResult(pbr);
3010     }
3011   }
3012 
3013   //
3014   // remote scanner interface
3015   //
3016 
3017   /**
3018    * Scan data in a table.
3019    *
3020    * @param controller the RPC controller
3021    * @param request the scan request
3022    * @throws ServiceException
3023    */
3024   @Override
3025   public ScanResponse scan(final RpcController controller, final ScanRequest request)
3026   throws ServiceException {
3027     Leases.Lease lease = null;
3028     String scannerName = null;
3029     try {
3030       if (!request.hasScannerId() && !request.hasScan()) {
3031         throw new DoNotRetryIOException(
3032           "Missing required input: scannerId or scan");
3033       }
3034       long scannerId = -1;
3035       if (request.hasScannerId()) {
3036         scannerId = request.getScannerId();
3037         scannerName = String.valueOf(scannerId);
3038       }
3039       try {
3040         checkOpen();
3041       } catch (IOException e) {
3042         // If checkOpen failed, server not running or filesystem gone,
3043         // cancel this lease; filesystem is gone or we're closing or something.
3044         if (scannerName != null) {
3045           try {
3046             leases.cancelLease(scannerName);
3047           } catch (LeaseException le) {
3048             LOG.info("Server shutting down and client tried to access missing scanner " +
3049               scannerName);
3050           }
3051         }
3052         throw e;
3053       }
3054       requestCount.increment();
3055 
3056       int ttl = 0;
3057       HRegion region = null;
3058       RegionScanner scanner = null;
3059       RegionScannerHolder rsh = null;
3060       boolean moreResults = true;
3061       boolean closeScanner = false;
3062       ScanResponse.Builder builder = ScanResponse.newBuilder();
3063       if (request.hasCloseScanner()) {
3064         closeScanner = request.getCloseScanner();
3065       }
3066       int rows = 1;
3067       if (request.hasNumberOfRows()) {
3068         rows = request.getNumberOfRows();
3069       }
3070       if (request.hasScannerId()) {
3071         rsh = scanners.get(scannerName);
3072         if (rsh == null) {
3073           LOG.info("Client tried to access missing scanner " + scannerName);
3074           throw new UnknownScannerException(
3075             "Name: " + scannerName + ", already closed?");
3076         }
3077         scanner = rsh.s;
3078         HRegionInfo hri = scanner.getRegionInfo();
3079         region = getRegion(hri.getRegionName());
3080         if (region != rsh.r) { // Yes, should be the same instance
3081           throw new NotServingRegionException("Region was re-opened after the scanner"
3082             + scannerName + " was created: " + hri.getRegionNameAsString());
3083         }
3084       } else {
3085         region = getRegion(request.getRegion());
3086         ClientProtos.Scan protoScan = request.getScan();
3087         boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();
3088         Scan scan = ProtobufUtil.toScan(protoScan);
3089         // if the request doesn't set this, get the default region setting.
3090         if (!isLoadingCfsOnDemandSet) {
3091           scan.setLoadColumnFamiliesOnDemand(region.isLoadingCfsOnDemandDefault());
3092         }
3093         scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE);
3094         region.prepareScanner(scan);
3095         if (region.getCoprocessorHost() != null) {
3096           scanner = region.getCoprocessorHost().preScannerOpen(scan);
3097         }
3098         if (scanner == null) {
3099           scanner = region.getScanner(scan);
3100         }
3101         if (region.getCoprocessorHost() != null) {
3102           scanner = region.getCoprocessorHost().postScannerOpen(scan, scanner);
3103         }
3104         scannerId = addScanner(scanner, region);
3105         scannerName = String.valueOf(scannerId);
3106         ttl = this.scannerLeaseTimeoutPeriod;
3107       }
3108 
3109       if (rows > 0) {
3110         // if nextCallSeq does not match throw Exception straight away. This needs to be
3111         // performed even before checking of Lease.
3112         // See HBASE-5974
3113         if (request.hasNextCallSeq()) {
3114           if (rsh == null) {
3115             rsh = scanners.get(scannerName);
3116           }
3117           if (rsh != null) {
3118             if (request.getNextCallSeq() != rsh.nextCallSeq) {
3119               throw new OutOfOrderScannerNextException("Expected nextCallSeq: " + rsh.nextCallSeq
3120                 + " But the nextCallSeq got from client: " + request.getNextCallSeq() +
3121                 "; request=" + TextFormat.shortDebugString(request));
3122             }
3123             // Increment the nextCallSeq value which is the next expected from client.
3124             rsh.nextCallSeq++;
3125           }
3126         }
3127         try {
3128           // Remove lease while its being processed in server; protects against case
3129           // where processing of request takes > lease expiration time.
3130           lease = leases.removeLease(scannerName);
3131           List<Result> results = new ArrayList<Result>(rows);
3132           long currentScanResultSize = 0;
3133 
3134           boolean done = false;
3135           // Call coprocessor. Get region info from scanner.
3136           if (region != null && region.getCoprocessorHost() != null) {
3137             Boolean bypass = region.getCoprocessorHost().preScannerNext(
3138               scanner, results, rows);
3139             if (!results.isEmpty()) {
3140               for (Result r : results) {
3141                 if (maxScannerResultSize < Long.MAX_VALUE){
3142                   for (Cell kv : r.rawCells()) {
3143                     // TODO
3144                     currentScanResultSize += KeyValueUtil.ensureKeyValue(kv).heapSize();
3145                   }
3146                 }
3147               }
3148             }
3149             if (bypass != null && bypass.booleanValue()) {
3150               done = true;
3151             }
3152           }
3153 
3154           if (!done) {
3155             long maxResultSize = scanner.getMaxResultSize();
3156             if (maxResultSize <= 0) {
3157               maxResultSize = maxScannerResultSize;
3158             }
3159             List<Cell> values = new ArrayList<Cell>();
3160             region.startRegionOperation(Operation.SCAN);
3161             try {
3162               int i = 0;
3163               synchronized(scanner) {
3164                 for (; i < rows
3165                     && currentScanResultSize < maxResultSize; ) {
3166                   // Collect values to be returned here
3167                   boolean moreRows = scanner.nextRaw(values);
3168                   if (!values.isEmpty()) {
3169                     if (maxScannerResultSize < Long.MAX_VALUE){
3170                       for (Cell kv : values) {
3171                         currentScanResultSize += KeyValueUtil.ensureKeyValue(kv).heapSize();
3172                       }
3173                     }
3174                     results.add(Result.create(values));
3175                     i++;
3176                   }
3177                   if (!moreRows) {
3178                     break;
3179                   }
3180                   values.clear();
3181                 }
3182               }
3183               region.readRequestsCount.add(i);
3184             } finally {
3185               region.closeRegionOperation();
3186             }
3187 
3188             // coprocessor postNext hook
3189             if (region != null && region.getCoprocessorHost() != null) {
3190               region.getCoprocessorHost().postScannerNext(scanner, results, rows, true);
3191             }
3192           }
3193 
3194           // If the scanner's filter - if any - is done with the scan
3195           // and wants to tell the client to stop the scan. This is done by passing
3196           // a null result, and setting moreResults to false.
3197           if (scanner.isFilterDone() && results.isEmpty()) {
3198             moreResults = false;
3199             results = null;
3200           } else {
3201             addResults(builder, results, controller);
3202           }
3203         } finally {
3204           // We're done. On way out re-add the above removed lease.
3205           // Adding resets expiration time on lease.
3206           if (scanners.containsKey(scannerName)) {
3207             if (lease != null) leases.addLease(lease);
3208             ttl = this.scannerLeaseTimeoutPeriod;
3209           }
3210         }
3211       }
3212 
3213       if (!moreResults || closeScanner) {
3214         ttl = 0;
3215         moreResults = false;
3216         if (region != null && region.getCoprocessorHost() != null) {
3217           if (region.getCoprocessorHost().preScannerClose(scanner)) {
3218             return builder.build(); // bypass
3219           }
3220         }
3221         rsh = scanners.remove(scannerName);
3222         if (rsh != null) {
3223           scanner = rsh.s;
3224           scanner.close();
3225           leases.cancelLease(scannerName);
3226           if (region != null && region.getCoprocessorHost() != null) {
3227             region.getCoprocessorHost().postScannerClose(scanner);
3228           }
3229         }
3230       }
3231 
3232       if (ttl > 0) {
3233         builder.setTtl(ttl);
3234       }
3235       builder.setScannerId(scannerId);
3236       builder.setMoreResults(moreResults);
3237       return builder.build();
3238     } catch (IOException ie) {
3239       if (scannerName != null && ie instanceof NotServingRegionException) {
3240         RegionScannerHolder rsh = scanners.remove(scannerName);
3241         if (rsh != null) {
3242           try {
3243             RegionScanner scanner = rsh.s;
3244             scanner.close();
3245             leases.cancelLease(scannerName);
3246           } catch (IOException e) {}
3247         }
3248       }
3249       throw new ServiceException(ie);
3250     }
3251   }
3252 
3253   private void addResults(final ScanResponse.Builder builder, final List<Result> results,
3254       final RpcController controller) {
3255     if (results == null || results.isEmpty()) return;
3256     if (isClientCellBlockSupport()) {
3257       for (Result res : results) {
3258         builder.addCellsPerResult(res.size());
3259       }
3260       ((PayloadCarryingRpcController)controller).
3261         setCellScanner(CellUtil.createCellScanner(results));
3262     } else {
3263       for (Result res: results) {
3264         ClientProtos.Result pbr = ProtobufUtil.toResult(res);
3265         builder.addResults(pbr);
3266       }
3267     }
3268   }
3269 
3270   /**
3271    * Atomically bulk load several HFiles into an open region
3272    * @return true if successful, false is failed but recoverably (no action)
3273    * @throws IOException if failed unrecoverably
3274    */
3275   @Override
3276   public BulkLoadHFileResponse bulkLoadHFile(final RpcController controller,
3277       final BulkLoadHFileRequest request) throws ServiceException {
3278     try {
3279       checkOpen();
3280       requestCount.increment();
3281       HRegion region = getRegion(request.getRegion());
3282       List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
3283       for (FamilyPath familyPath: request.getFamilyPathList()) {
3284         familyPaths.add(new Pair<byte[], String>(familyPath.getFamily().toByteArray(),
3285           familyPath.getPath()));
3286       }
3287       boolean bypass = false;
3288       if (region.getCoprocessorHost() != null) {
3289         bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
3290       }
3291       boolean loaded = false;
3292       if (!bypass) {
3293         loaded = region.bulkLoadHFiles(familyPaths, request.getAssignSeqNum());
3294       }
3295       if (region.getCoprocessorHost() != null) {
3296         loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
3297       }
3298       BulkLoadHFileResponse.Builder builder = BulkLoadHFileResponse.newBuilder();
3299       builder.setLoaded(loaded);
3300       return builder.build();
3301     } catch (IOException ie) {
3302       throw new ServiceException(ie);
3303     }
3304   }
3305 
3306   @Override
3307   public CoprocessorServiceResponse execService(final RpcController controller,
3308       final CoprocessorServiceRequest request) throws ServiceException {
3309     try {
3310       checkOpen();
3311       requestCount.increment();
3312       HRegion region = getRegion(request.getRegion());
3313       Message result = execServiceOnRegion(region, request.getCall());
3314       CoprocessorServiceResponse.Builder builder =
3315           CoprocessorServiceResponse.newBuilder();
3316       builder.setRegion(RequestConverter.buildRegionSpecifier(
3317           RegionSpecifierType.REGION_NAME, region.getRegionName()));
3318       builder.setValue(
3319           builder.getValueBuilder().setName(result.getClass().getName())
3320               .setValue(result.toByteString()));
3321       return builder.build();
3322     } catch (IOException ie) {
3323       throw new ServiceException(ie);
3324     }
3325   }
3326 
3327   private Message execServiceOnRegion(HRegion region,
3328       final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException {
3329     // ignore the passed in controller (from the serialized call)
3330     ServerRpcController execController = new ServerRpcController();
3331     Message result = region.execService(execController, serviceCall);
3332     if (execController.getFailedOn() != null) {
3333       throw execController.getFailedOn();
3334     }
3335     return result;
3336   }
3337 
3338   /**
3339    * Execute multiple actions on a table: get, mutate, and/or execCoprocessor
3340    *
3341    * @param rpcc the RPC controller
3342    * @param request the multi request
3343    * @throws ServiceException
3344    */
3345   @Override
3346   public MultiResponse multi(final RpcController rpcc, final MultiRequest request)
3347   throws ServiceException {
3348     try {
3349       checkOpen();
3350     } catch (IOException ie) {
3351       throw new ServiceException(ie);
3352     }
3353 
3354     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
3355     // It is also the conduit via which we pass back data.
3356     PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
3357     CellScanner cellScanner = controller != null ? controller.cellScanner(): null;
3358     if (controller != null) controller.setCellScanner(null);
3359 
3360     long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
3361 
3362     // this will contain all the cells that we need to return. It's created later, if needed.
3363     List<CellScannable> cellsToReturn = null;
3364     MultiResponse.Builder responseBuilder = MultiResponse.newBuilder();
3365     RegionActionResult.Builder regionActionResultBuilder = RegionActionResult.newBuilder();
3366 
3367     for (RegionAction regionAction : request.getRegionActionList()) {
3368       this.requestCount.add(regionAction.getActionCount());
3369       HRegion region;
3370       regionActionResultBuilder.clear();
3371       try {
3372         region = getRegion(regionAction.getRegion());
3373       } catch (IOException e) {
3374         regionActionResultBuilder.setException(ResponseConverter.buildException(e));
3375         responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
3376         continue;  // For this region it's a failure.
3377       }
3378 
3379       if (regionAction.hasAtomic() && regionAction.getAtomic()) {
3380         // How does this call happen?  It may need some work to play well w/ the surroundings.
3381         // Need to return an item per Action along w/ Action index.  TODO.
3382         try {
3383           mutateRows(region, regionAction.getActionList(), cellScanner);
3384         } catch (IOException e) {
3385           // As it's atomic, we may expect it's a global failure.
3386           regionActionResultBuilder.setException(ResponseConverter.buildException(e));
3387         }
3388       } else {
3389         // doNonAtomicRegionMutation manages the exception internally
3390         cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner,
3391             regionActionResultBuilder, cellsToReturn, nonceGroup);
3392       }
3393       responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
3394     }
3395     // Load the controller with the Cells to return.
3396     if (cellsToReturn != null && !cellsToReturn.isEmpty() && controller != null) {
3397       controller.setCellScanner(CellUtil.createCellScanner(cellsToReturn));
3398     }
3399     return responseBuilder.build();
3400   }
3401 
3402   /**
3403    * Run through the regionMutation <code>rm</code> and per Mutation, do the work, and then when
3404    * done, add an instance of a {@link ResultOrException} that corresponds to each Mutation.
3405    * @param region
3406    * @param actions
3407    * @param cellScanner
3408    * @param builder
3409    * @param cellsToReturn  Could be null. May be allocated in this method.  This is what this
3410    * method returns as a 'result'.
3411    * @return Return the <code>cellScanner</code> passed
3412    */
3413   private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
3414       final RegionAction actions, final CellScanner cellScanner,
3415       final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup) {
3416     // Gather up CONTIGUOUS Puts and Deletes in this mutations List.  Idea is that rather than do
3417     // one at a time, we instead pass them in batch.  Be aware that the corresponding
3418     // ResultOrException instance that matches each Put or Delete is then added down in the
3419     // doBatchOp call.  We should be staying aligned though the Put and Delete are deferred/batched
3420     List<ClientProtos.Action> mutations = null;
3421     for (ClientProtos.Action action: actions.getActionList()) {
3422       ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = null;
3423       try {
3424         Result r = null;
3425         if (action.hasGet()) {
3426           Get get = ProtobufUtil.toGet(action.getGet());
3427           r = region.get(get);
3428         } else if (action.hasServiceCall()) {
3429           resultOrExceptionBuilder = ResultOrException.newBuilder();
3430           try {
3431             Message result = execServiceOnRegion(region, action.getServiceCall());
3432             ClientProtos.CoprocessorServiceResult.Builder serviceResultBuilder =
3433                 ClientProtos.CoprocessorServiceResult.newBuilder();
3434             resultOrExceptionBuilder.setServiceResult(
3435                 serviceResultBuilder.setValue(
3436                   serviceResultBuilder.getValueBuilder()
3437                     .setName(result.getClass().getName())
3438                     .setValue(result.toByteString())));
3439           } catch (IOException ioe) {
3440             resultOrExceptionBuilder.setException(ResponseConverter.buildException(ioe));
3441           }
3442         } else if (action.hasMutation()) {
3443           MutationType type = action.getMutation().getMutateType();
3444           if (type != MutationType.PUT && type != MutationType.DELETE && mutations != null &&
3445               !mutations.isEmpty()) {
3446             // Flush out any Puts or Deletes already collected.
3447             doBatchOp(builder, region, mutations, cellScanner);
3448             mutations.clear();
3449           }
3450           switch (type) {
3451           case APPEND:
3452             r = append(region, action.getMutation(), cellScanner, nonceGroup);
3453             break;
3454           case INCREMENT:
3455             r = increment(region, action.getMutation(), cellScanner,  nonceGroup);
3456             break;
3457           case PUT:
3458           case DELETE:
3459             // Collect the individual mutations and apply in a batch
3460             if (mutations == null) {
3461               mutations = new ArrayList<ClientProtos.Action>(actions.getActionCount());
3462             }
3463             mutations.add(action);
3464             break;
3465           default:
3466             throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
3467           }
3468         } else {
3469           throw new HBaseIOException("Unexpected Action type");
3470         }
3471         if (r != null) {
3472           ClientProtos.Result pbResult = null;
3473           if (isClientCellBlockSupport()) {
3474             pbResult = ProtobufUtil.toResultNoData(r);
3475             //  Hard to guess the size here.  Just make a rough guess.
3476             if (cellsToReturn == null) cellsToReturn = new ArrayList<CellScannable>();
3477             cellsToReturn.add(r);
3478           } else {
3479             pbResult = ProtobufUtil.toResult(r);
3480           }
3481           resultOrExceptionBuilder =
3482             ClientProtos.ResultOrException.newBuilder().setResult(pbResult);
3483         }
3484         // Could get to here and there was no result and no exception.  Presumes we added
3485         // a Put or Delete to the collecting Mutations List for adding later.  In this
3486         // case the corresponding ResultOrException instance for the Put or Delete will be added
3487         // down in the doBatchOp method call rather than up here.
3488       } catch (IOException ie) {
3489         resultOrExceptionBuilder = ResultOrException.newBuilder().
3490           setException(ResponseConverter.buildException(ie));
3491       }
3492       if (resultOrExceptionBuilder != null) {
3493         // Propagate index.
3494         resultOrExceptionBuilder.setIndex(action.getIndex());
3495         builder.addResultOrException(resultOrExceptionBuilder.build());
3496       }
3497     }
3498     // Finish up any outstanding mutations
3499     if (mutations != null && !mutations.isEmpty()) {
3500       doBatchOp(builder, region, mutations, cellScanner);
3501     }
3502     return cellsToReturn;
3503   }
3504 
3505 // End Client methods
3506 // Start Admin methods
3507 
3508   @Override
3509   @QosPriority(priority=HConstants.HIGH_QOS)
3510   public GetRegionInfoResponse getRegionInfo(final RpcController controller,
3511       final GetRegionInfoRequest request) throws ServiceException {
3512     try {
3513       checkOpen();
3514       requestCount.increment();
3515       HRegion region = getRegion(request.getRegion());
3516       HRegionInfo info = region.getRegionInfo();
3517       GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
3518       builder.setRegionInfo(HRegionInfo.convert(info));
3519       if (request.hasCompactionState() && request.getCompactionState()) {
3520         builder.setCompactionState(region.getCompactionState());
3521       }
3522       builder.setIsRecovering(region.isRecovering());
3523       return builder.build();
3524     } catch (IOException ie) {
3525       throw new ServiceException(ie);
3526     }
3527   }
3528 
3529   @Override
3530   public GetStoreFileResponse getStoreFile(final RpcController controller,
3531       final GetStoreFileRequest request) throws ServiceException {
3532     try {
3533       checkOpen();
3534       HRegion region = getRegion(request.getRegion());
3535       requestCount.increment();
3536       Set<byte[]> columnFamilies;
3537       if (request.getFamilyCount() == 0) {
3538         columnFamilies = region.getStores().keySet();
3539       } else {
3540         columnFamilies = new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR);
3541         for (ByteString cf: request.getFamilyList()) {
3542           columnFamilies.add(cf.toByteArray());
3543         }
3544       }
3545       int nCF = columnFamilies.size();
3546       List<String>  fileList = region.getStoreFileList(
3547         columnFamilies.toArray(new byte[nCF][]));
3548       GetStoreFileResponse.Builder builder = GetStoreFileResponse.newBuilder();
3549       builder.addAllStoreFile(fileList);
3550       return builder.build();
3551     } catch (IOException ie) {
3552       throw new ServiceException(ie);
3553     }
3554   }
3555 
3556   @Override
3557   @QosPriority(priority=HConstants.HIGH_QOS)
3558   public GetOnlineRegionResponse getOnlineRegion(final RpcController controller,
3559       final GetOnlineRegionRequest request) throws ServiceException {
3560     try {
3561       checkOpen();
3562       requestCount.increment();
3563       List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
3564       for (HRegion region: this.onlineRegions.values()) {
3565         list.add(region.getRegionInfo());
3566       }
3567       Collections.sort(list);
3568       return ResponseConverter.buildGetOnlineRegionResponse(list);
3569     } catch (IOException ie) {
3570       throw new ServiceException(ie);
3571     }
3572   }
3573 
3574   // Region open/close direct RPCs
3575 
3576   /**
3577    * Open asynchronously a region or a set of regions on the region server.
3578    *
3579    * The opening is coordinated by ZooKeeper, and this method requires the znode to be created
3580    *  before being called. As a consequence, this method should be called only from the master.
3581    * <p>
3582    * Different manages states for the region are:<ul>
3583    *  <li>region not opened: the region opening will start asynchronously.</li>
3584    *  <li>a close is already in progress: this is considered as an error.</li>
3585    *  <li>an open is already in progress: this new open request will be ignored. This is important
3586    *  because the Master can do multiple requests if it crashes.</li>
3587    *  <li>the region is already opened:  this new open request will be ignored./li>
3588    *  </ul>
3589    * </p>
3590    * <p>
3591    * Bulk assign: If there are more than 1 region to open, it will be considered as a bulk assign.
3592    * For a single region opening, errors are sent through a ServiceException. For bulk assign,
3593    * errors are put in the response as FAILED_OPENING.
3594    * </p>
3595    * @param controller the RPC controller
3596    * @param request the request
3597    * @throws ServiceException
3598    */
3599   @Override
3600   @QosPriority(priority=HConstants.HIGH_QOS)
3601   public OpenRegionResponse openRegion(final RpcController controller,
3602       final OpenRegionRequest request) throws ServiceException {
3603     try {
3604       checkOpen();
3605     } catch (IOException ie) {
3606       throw new ServiceException(ie);
3607     }
3608     requestCount.increment();
3609     if (request.hasServerStartCode() && this.serverNameFromMasterPOV != null) {
3610       // check that we are the same server that this RPC is intended for.
3611       long serverStartCode = request.getServerStartCode();
3612       if (this.serverNameFromMasterPOV.getStartcode() !=  serverStartCode) {
3613         throw new ServiceException(new DoNotRetryIOException("This RPC was intended for a " +
3614             "different server with startCode: " + serverStartCode + ", this server is: "
3615             + this.serverNameFromMasterPOV));
3616       }
3617     }
3618     OpenRegionResponse.Builder builder = OpenRegionResponse.newBuilder();
3619     final int regionCount = request.getOpenInfoCount();
3620     final Map<TableName, HTableDescriptor> htds =
3621         new HashMap<TableName, HTableDescriptor>(regionCount);
3622     final boolean isBulkAssign = regionCount > 1;
3623     for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
3624       final HRegionInfo region = HRegionInfo.convert(regionOpenInfo.getRegion());
3625 
3626       int versionOfOfflineNode = -1;
3627       if (regionOpenInfo.hasVersionOfOfflineNode()) {
3628         versionOfOfflineNode = regionOpenInfo.getVersionOfOfflineNode();
3629       }
3630       HTableDescriptor htd;
3631       try {
3632         final HRegion onlineRegion = getFromOnlineRegions(region.getEncodedName());
3633         if (onlineRegion != null) {
3634           //Check if the region can actually be opened.
3635           if (onlineRegion.getCoprocessorHost() != null) {
3636             onlineRegion.getCoprocessorHost().preOpen();
3637           }
3638           // See HBASE-5094. Cross check with hbase:meta if still this RS is owning
3639           // the region.
3640           Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(
3641               this.catalogTracker, region.getRegionName());
3642           if (this.getServerName().equals(p.getSecond())) {
3643             Boolean closing = regionsInTransitionInRS.get(region.getEncodedNameAsBytes());
3644             // Map regionsInTransitionInRSOnly has an entry for a region only if the region
3645             // is in transition on this RS, so here closing can be null. If not null, it can
3646             // be true or false. True means the region is opening on this RS; while false
3647             // means the region is closing. Only return ALREADY_OPENED if not closing (i.e.
3648             // not in transition any more, or still transition to open.
3649             if (!Boolean.FALSE.equals(closing)
3650                 && getFromOnlineRegions(region.getEncodedName()) != null) {
3651               LOG.warn("Attempted open of " + region.getEncodedName()
3652                 + " but already online on this server");
3653               builder.addOpeningState(RegionOpeningState.ALREADY_OPENED);
3654               continue;
3655             }
3656           } else {
3657             LOG.warn("The region " + region.getEncodedName() + " is online on this server" +
3658                 " but hbase:meta does not have this server - continue opening.");
3659             removeFromOnlineRegions(onlineRegion, null);
3660           }
3661         }
3662         LOG.info("Open " + region.getRegionNameAsString());
3663         htd = htds.get(region.getTable());
3664         if (htd == null) {
3665           htd = this.tableDescriptors.get(region.getTable());
3666           htds.put(region.getTable(), htd);
3667         }
3668 
3669         final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(
3670             region.getEncodedNameAsBytes(), Boolean.TRUE);
3671 
3672         if (Boolean.FALSE.equals(previous)) {
3673           // There is a close in progress. We need to mark this open as failed in ZK.
3674           OpenRegionHandler.
3675               tryTransitionFromOfflineToFailedOpen(this, region, versionOfOfflineNode);
3676 
3677           throw new RegionAlreadyInTransitionException("Received OPEN for the region:" +
3678               region.getRegionNameAsString() + " , which we are already trying to CLOSE ");
3679         }
3680 
3681         if (Boolean.TRUE.equals(previous)) {
3682           // An open is in progress. This is supported, but let's log this.
3683           LOG.info("Receiving OPEN for the region:" +
3684               region.getRegionNameAsString() + " , which we are already trying to OPEN" +
3685               " - ignoring this new request for this region.");
3686         }
3687 
3688         // We are opening this region. If it moves back and forth for whatever reason, we don't
3689         // want to keep returning the stale moved record while we are opening/if we close again.
3690         removeFromMovedRegions(region.getEncodedName());
3691 
3692         if (previous == null) {
3693           // check if the region to be opened is marked in recovering state in ZK
3694           if (this.distributedLogReplay
3695               && SplitLogManager.isRegionMarkedRecoveringInZK(this.getZooKeeper(),
3696             region.getEncodedName())) {
3697             this.recoveringRegions.put(region.getEncodedName(), null);
3698           }
3699           // If there is no action in progress, we can submit a specific handler.
3700           // Need to pass the expected version in the constructor.
3701           if (region.isMetaRegion()) {
3702             this.service.submit(new OpenMetaHandler(this, this, region, htd,
3703                 versionOfOfflineNode));
3704           } else {
3705             updateRegionFavoredNodesMapping(region.getEncodedName(),
3706                 regionOpenInfo.getFavoredNodesList());
3707             this.service.submit(new OpenRegionHandler(this, this, region, htd,
3708                 versionOfOfflineNode));
3709           }
3710         }
3711 
3712         builder.addOpeningState(RegionOpeningState.OPENED);
3713 
3714       } catch (KeeperException zooKeeperEx) {
3715         LOG.error("Can't retrieve recovering state from zookeeper", zooKeeperEx);
3716         throw new ServiceException(zooKeeperEx);
3717       } catch (IOException ie) {
3718         LOG.warn("Failed opening region " + region.getRegionNameAsString(), ie);
3719         if (isBulkAssign) {
3720           builder.addOpeningState(RegionOpeningState.FAILED_OPENING);
3721         } else {
3722           throw new ServiceException(ie);
3723         }
3724       }
3725     }
3726 
3727     return builder.build();
3728   }
3729 
3730   @Override
3731   public void updateRegionFavoredNodesMapping(String encodedRegionName,
3732       List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName> favoredNodes) {
3733     InetSocketAddress[] addr = new InetSocketAddress[favoredNodes.size()];
3734     // Refer to the comment on the declaration of regionFavoredNodesMap on why
3735     // it is a map of region name to InetSocketAddress[]
3736     for (int i = 0; i < favoredNodes.size(); i++) {
3737       addr[i] = InetSocketAddress.createUnresolved(favoredNodes.get(i).getHostName(),
3738           favoredNodes.get(i).getPort());
3739     }
3740     regionFavoredNodesMap.put(encodedRegionName, addr);
3741   }
3742 
3743   /**
3744    * Return the favored nodes for a region given its encoded name. Look at the
3745    * comment around {@link #regionFavoredNodesMap} on why it is InetSocketAddress[]
3746    * @param encodedRegionName
3747    * @return array of favored locations
3748    */
3749   @Override
3750   public InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName) {
3751     return regionFavoredNodesMap.get(encodedRegionName);
3752   }
3753 
3754   /**
3755    * Close a region on the region server.
3756    *
3757    * @param controller the RPC controller
3758    * @param request the request
3759    * @throws ServiceException
3760    */
3761   @Override
3762   @QosPriority(priority=HConstants.HIGH_QOS)
3763   public CloseRegionResponse closeRegion(final RpcController controller,
3764       final CloseRegionRequest request) throws ServiceException {
3765     int versionOfClosingNode = -1;
3766     if (request.hasVersionOfClosingNode()) {
3767       versionOfClosingNode = request.getVersionOfClosingNode();
3768     }
3769     boolean zk = request.getTransitionInZK();
3770     final ServerName sn = (request.hasDestinationServer() ?
3771       ProtobufUtil.toServerName(request.getDestinationServer()) : null);
3772 
3773     try {
3774       checkOpen();
3775       if (request.hasServerStartCode() && this.serverNameFromMasterPOV != null) {
3776         // check that we are the same server that this RPC is intended for.
3777         long serverStartCode = request.getServerStartCode();
3778         if (this.serverNameFromMasterPOV.getStartcode() !=  serverStartCode) {
3779           throw new ServiceException(new DoNotRetryIOException("This RPC was intended for a " +
3780               "different server with startCode: " + serverStartCode + ", this server is: "
3781               + this.serverNameFromMasterPOV));
3782         }
3783       }
3784       final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion());
3785 
3786       // Can be null if we're calling close on a region that's not online
3787       final HRegion region = this.getFromOnlineRegions(encodedRegionName);
3788       if ((region  != null) && (region .getCoprocessorHost() != null)) {
3789         region.getCoprocessorHost().preClose(false);
3790       }
3791 
3792       requestCount.increment();
3793       LOG.info("Close " + encodedRegionName + ", via zk=" + (zk ? "yes" : "no") +
3794         ", znode version=" + versionOfClosingNode + ", on " + sn);
3795 
3796       boolean closed = closeRegion(encodedRegionName, false, zk, versionOfClosingNode, sn);
3797       CloseRegionResponse.Builder builder = CloseRegionResponse.newBuilder().setClosed(closed);
3798       return builder.build();
3799     } catch (IOException ie) {
3800       throw new ServiceException(ie);
3801     }
3802   }
3803 
3804   /**
3805    * Flush a region on the region server.
3806    *
3807    * @param controller the RPC controller
3808    * @param request the request
3809    * @throws ServiceException
3810    */
3811   @Override
3812   @QosPriority(priority=HConstants.HIGH_QOS)
3813   public FlushRegionResponse flushRegion(final RpcController controller,
3814       final FlushRegionRequest request) throws ServiceException {
3815     try {
3816       checkOpen();
3817       requestCount.increment();
3818       HRegion region = getRegion(request.getRegion());
3819       LOG.info("Flushing " + region.getRegionNameAsString());
3820       boolean shouldFlush = true;
3821       if (request.hasIfOlderThanTs()) {
3822         shouldFlush = region.getLastFlushTime() < request.getIfOlderThanTs();
3823       }
3824       FlushRegionResponse.Builder builder = FlushRegionResponse.newBuilder();
3825       if (shouldFlush) {
3826         boolean result = region.flushcache();
3827         if (result) {
3828           this.compactSplitThread.requestSystemCompaction(region,
3829               "Compaction through user triggered flush");
3830         }
3831         builder.setFlushed(result);
3832       }
3833       builder.setLastFlushTime(region.getLastFlushTime());
3834       return builder.build();
3835     } catch (DroppedSnapshotException ex) {
3836       // Cache flush can fail in a few places. If it fails in a critical
3837       // section, we get a DroppedSnapshotException and a replay of hlog
3838       // is required. Currently the only way to do this is a restart of
3839       // the server.
3840       abort("Replay of HLog required. Forcing server shutdown", ex);
3841       throw new ServiceException(ex);
3842     } catch (IOException ie) {
3843       throw new ServiceException(ie);
3844     }
3845   }
3846 
3847   /**
3848    * Split a region on the region server.
3849    *
3850    * @param controller the RPC controller
3851    * @param request the request
3852    * @throws ServiceException
3853    */
3854   @Override
3855   @QosPriority(priority=HConstants.HIGH_QOS)
3856   public SplitRegionResponse splitRegion(final RpcController controller,
3857       final SplitRegionRequest request) throws ServiceException {
3858     try {
3859       checkOpen();
3860       requestCount.increment();
3861       HRegion region = getRegion(request.getRegion());
3862       region.startRegionOperation(Operation.SPLIT_REGION);
3863       LOG.info("Splitting " + region.getRegionNameAsString());
3864       region.flushcache();
3865       byte[] splitPoint = null;
3866       if (request.hasSplitPoint()) {
3867         splitPoint = request.getSplitPoint().toByteArray();
3868       }
3869       region.forceSplit(splitPoint);
3870       compactSplitThread.requestSplit(region, region.checkSplit());
3871       return SplitRegionResponse.newBuilder().build();
3872     } catch (IOException ie) {
3873       throw new ServiceException(ie);
3874     }
3875   }
3876 
3877   /**
3878    * Merge regions on the region server.
3879    *
3880    * @param controller the RPC controller
3881    * @param request the request
3882    * @return merge regions response
3883    * @throws ServiceException
3884    */
3885   @Override
3886   @QosPriority(priority = HConstants.HIGH_QOS)
3887   public MergeRegionsResponse mergeRegions(final RpcController controller,
3888       final MergeRegionsRequest request) throws ServiceException {
3889     try {
3890       checkOpen();
3891       requestCount.increment();
3892       HRegion regionA = getRegion(request.getRegionA());
3893       HRegion regionB = getRegion(request.getRegionB());
3894       boolean forcible = request.getForcible();
3895       regionA.startRegionOperation(Operation.MERGE_REGION);
3896       regionB.startRegionOperation(Operation.MERGE_REGION);
3897       LOG.info("Receiving merging request for  " + regionA + ", " + regionB
3898           + ",forcible=" + forcible);
3899       regionA.flushcache();
3900       regionB.flushcache();
3901       compactSplitThread.requestRegionsMerge(regionA, regionB, forcible);
3902       return MergeRegionsResponse.newBuilder().build();
3903     } catch (IOException ie) {
3904       throw new ServiceException(ie);
3905     }
3906   }
3907 
3908   /**
3909    * Compact a region on the region server.
3910    *
3911    * @param controller the RPC controller
3912    * @param request the request
3913    * @throws ServiceException
3914    */
3915   @Override
3916   @QosPriority(priority=HConstants.HIGH_QOS)
3917   public CompactRegionResponse compactRegion(final RpcController controller,
3918       final CompactRegionRequest request) throws ServiceException {
3919     try {
3920       checkOpen();
3921       requestCount.increment();
3922       HRegion region = getRegion(request.getRegion());
3923       region.startRegionOperation(Operation.COMPACT_REGION);
3924       LOG.info("Compacting " + region.getRegionNameAsString());
3925       boolean major = false;
3926       byte [] family = null;
3927       Store store = null;
3928       if (request.hasFamily()) {
3929         family = request.getFamily().toByteArray();
3930         store = region.getStore(family);
3931         if (store == null) {
3932           throw new ServiceException(new IOException("column family " + Bytes.toString(family) +
3933             " does not exist in region " + region.getRegionNameAsString()));
3934         }
3935       }
3936       if (request.hasMajor()) {
3937         major = request.getMajor();
3938       }
3939       if (major) {
3940         if (family != null) {
3941           store.triggerMajorCompaction();
3942         } else {
3943           region.triggerMajorCompaction();
3944         }
3945       }
3946 
3947       String familyLogMsg = (family != null)?" for column family: " + Bytes.toString(family):"";
3948       LOG.trace("User-triggered compaction requested for region " +
3949         region.getRegionNameAsString() + familyLogMsg);
3950       String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
3951       if(family != null) {
3952         compactSplitThread.requestCompaction(region, store, log,
3953           Store.PRIORITY_USER, null);
3954       } else {
3955         compactSplitThread.requestCompaction(region, log,
3956           Store.PRIORITY_USER, null);
3957       }
3958       return CompactRegionResponse.newBuilder().build();
3959     } catch (IOException ie) {
3960       throw new ServiceException(ie);
3961     }
3962   }
3963 
3964   /**
3965    * Replicate WAL entries on the region server.
3966    *
3967    * @param controller the RPC controller
3968    * @param request the request
3969    * @throws ServiceException
3970    */
3971   @Override
3972   @QosPriority(priority=HConstants.REPLICATION_QOS)
3973   public ReplicateWALEntryResponse replicateWALEntry(final RpcController controller,
3974       final ReplicateWALEntryRequest request)
3975   throws ServiceException {
3976     try {
3977       if (replicationSinkHandler != null) {
3978         checkOpen();
3979         requestCount.increment();
3980         this.replicationSinkHandler.replicateLogEntries(request.getEntryList(),
3981           ((PayloadCarryingRpcController)controller).cellScanner());
3982       }
3983       return ReplicateWALEntryResponse.newBuilder().build();
3984     } catch (IOException ie) {
3985       throw new ServiceException(ie);
3986     }
3987   }
3988 
3989   /**
3990    * Replay the given changes when distributedLogReplay WAL edits from a failed RS. The guarantee is
3991    * that the given mutations will be durable on the receiving RS if this method returns without any
3992    * exception.
3993    * @param controller the RPC controller
3994    * @param request the request
3995    * @throws ServiceException
3996    */
3997   @Override
3998   @QosPriority(priority = HConstants.REPLAY_QOS)
3999   public ReplicateWALEntryResponse replay(final RpcController controller,
4000       final ReplicateWALEntryRequest request) throws ServiceException {
4001     long before = EnvironmentEdgeManager.currentTimeMillis();
4002     CellScanner cells = ((PayloadCarryingRpcController) controller).cellScanner();
4003     try {
4004       checkOpen();
4005       List<WALEntry> entries = request.getEntryList();
4006       if (entries == null || entries.isEmpty()) {
4007         // empty input
4008         return ReplicateWALEntryResponse.newBuilder().build();
4009       }
4010       HRegion region = this.getRegionByEncodedName(
4011         entries.get(0).getKey().getEncodedRegionName().toStringUtf8());
4012       RegionCoprocessorHost coprocessorHost = region.getCoprocessorHost();
4013       List<Pair<HLogKey, WALEdit>> walEntries = new ArrayList<Pair<HLogKey, WALEdit>>();
4014       List<HLogSplitter.MutationReplay> mutations = new ArrayList<HLogSplitter.MutationReplay>();
4015       // when tag is enabled, we need tag replay edits with log sequence number
4016       boolean needAddReplayTag = (HFile.getFormatVersion(this.conf) >= 3);
4017       for (WALEntry entry : entries) {
4018         if (nonceManager != null) {
4019           long nonceGroup = entry.getKey().hasNonceGroup()
4020               ? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
4021           long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
4022           nonceManager.reportOperationFromWal(nonceGroup, nonce, entry.getKey().getWriteTime());
4023         }
4024         Pair<HLogKey, WALEdit> walEntry = (coprocessorHost == null) ? null :
4025           new Pair<HLogKey, WALEdit>();
4026         List<HLogSplitter.MutationReplay> edits = HLogSplitter.getMutationsFromWALEntry(entry,
4027           cells, walEntry, needAddReplayTag);
4028         if (coprocessorHost != null) {
4029           // Start coprocessor replay here. The coprocessor is for each WALEdit instead of a
4030           // KeyValue.
4031           if (coprocessorHost.preWALRestore(region.getRegionInfo(), walEntry.getFirst(),
4032             walEntry.getSecond())) {
4033             // if bypass this log entry, ignore it ...
4034             continue;
4035           }
4036           walEntries.add(walEntry);
4037         }
4038         mutations.addAll(edits);
4039       }
4040 
4041       if (!mutations.isEmpty()) {
4042         OperationStatus[] result = doReplayBatchOp(region, mutations);
4043         // check if it's a partial success
4044         for (int i = 0; result != null && i < result.length; i++) {
4045           if (result[i] != OperationStatus.SUCCESS) {
4046             throw new IOException(result[i].getExceptionMsg());
4047           }
4048         }
4049       }
4050       if (coprocessorHost != null) {
4051         for (Pair<HLogKey, WALEdit> wal : walEntries) {
4052           coprocessorHost.postWALRestore(region.getRegionInfo(), wal.getFirst(),
4053             wal.getSecond());
4054         }
4055       }
4056       return ReplicateWALEntryResponse.newBuilder().build();
4057     } catch (IOException ie) {
4058       throw new ServiceException(ie);
4059     } finally {
4060       metricsRegionServer.updateReplay(EnvironmentEdgeManager.currentTimeMillis() - before);
4061     }
4062   }
4063 
4064   /**
4065    * Roll the WAL writer of the region server.
4066    * @param controller the RPC controller
4067    * @param request the request
4068    * @throws ServiceException
4069    */
4070   @Override
4071   public RollWALWriterResponse rollWALWriter(final RpcController controller,
4072       final RollWALWriterRequest request) throws ServiceException {
4073     try {
4074       checkOpen();
4075       requestCount.increment();
4076       HLog wal = this.getWAL();
4077       byte[][] regionsToFlush = wal.rollWriter(true);
4078       RollWALWriterResponse.Builder builder = RollWALWriterResponse.newBuilder();
4079       if (regionsToFlush != null) {
4080         for (byte[] region: regionsToFlush) {
4081           builder.addRegionToFlush(HBaseZeroCopyByteString.wrap(region));
4082         }
4083       }
4084       return builder.build();
4085     } catch (IOException ie) {
4086       throw new ServiceException(ie);
4087     }
4088   }
4089 
4090   /**
4091    * Stop the region server.
4092    *
4093    * @param controller the RPC controller
4094    * @param request the request
4095    * @throws ServiceException
4096    */
4097   @Override
4098   public StopServerResponse stopServer(final RpcController controller,
4099       final StopServerRequest request) throws ServiceException {
4100     requestCount.increment();
4101     String reason = request.getReason();
4102     stop(reason);
4103     return StopServerResponse.newBuilder().build();
4104   }
4105 
4106   /**
4107    * Get some information of the region server.
4108    *
4109    * @param controller the RPC controller
4110    * @param request the request
4111    * @throws ServiceException
4112    */
4113   @Override
4114   public GetServerInfoResponse getServerInfo(final RpcController controller,
4115       final GetServerInfoRequest request) throws ServiceException {
4116     try {
4117       checkOpen();
4118     } catch (IOException ie) {
4119       throw new ServiceException(ie);
4120     }
4121     ServerName serverName = getServerName();
4122     requestCount.increment();
4123     return ResponseConverter.buildGetServerInfoResponse(serverName, rsInfo.getInfoPort());
4124   }
4125 
4126 // End Admin methods
4127 
4128   /**
4129    * Find the HRegion based on a region specifier
4130    *
4131    * @param regionSpecifier the region specifier
4132    * @return the corresponding region
4133    * @throws IOException if the specifier is not null,
4134    *    but failed to find the region
4135    */
4136   protected HRegion getRegion(
4137       final RegionSpecifier regionSpecifier) throws IOException {
4138     return getRegionByEncodedName(regionSpecifier.getValue().toByteArray(),
4139         ProtobufUtil.getRegionEncodedName(regionSpecifier));
4140   }
4141 
4142   /**
4143    * Execute an append mutation.
4144    *
4145    * @param region
4146    * @param m
4147    * @param cellScanner
4148    * @return result to return to client if default operation should be
4149    * bypassed as indicated by RegionObserver, null otherwise
4150    * @throws IOException
4151    */
4152   protected Result append(final HRegion region,
4153       final MutationProto m, final CellScanner cellScanner, long nonceGroup) throws IOException {
4154     long before = EnvironmentEdgeManager.currentTimeMillis();
4155     Append append = ProtobufUtil.toAppend(m, cellScanner);
4156     Result r = null;
4157     if (region.getCoprocessorHost() != null) {
4158       r = region.getCoprocessorHost().preAppend(append);
4159     }
4160     if (r == null) {
4161       long nonce = startNonceOperation(m, nonceGroup);
4162       boolean success = false;
4163       try {
4164         r = region.append(append, nonceGroup, nonce);
4165         success = true;
4166       } finally {
4167         endNonceOperation(m, nonceGroup, success);
4168       }
4169       if (region.getCoprocessorHost() != null) {
4170         region.getCoprocessorHost().postAppend(append, r);
4171       }
4172     }
4173     metricsRegionServer.updateAppend(EnvironmentEdgeManager.currentTimeMillis() - before);
4174     return r;
4175   }
4176 
4177   /**
4178    * Execute an increment mutation.
4179    *
4180    * @param region
4181    * @param mutation
4182    * @return the Result
4183    * @throws IOException
4184    */
4185   protected Result increment(final HRegion region, final MutationProto mutation,
4186       final CellScanner cells, long nonceGroup) throws IOException {
4187     long before = EnvironmentEdgeManager.currentTimeMillis();
4188     Increment increment = ProtobufUtil.toIncrement(mutation, cells);
4189     Result r = null;
4190     if (region.getCoprocessorHost() != null) {
4191       r = region.getCoprocessorHost().preIncrement(increment);
4192     }
4193     if (r == null) {
4194       long nonce = startNonceOperation(mutation, nonceGroup);
4195       boolean success = false;
4196       try {
4197         r = region.increment(increment, nonceGroup, nonce);
4198         success = true;
4199       } finally {
4200         endNonceOperation(mutation, nonceGroup, success);
4201       }
4202       if (region.getCoprocessorHost() != null) {
4203         r = region.getCoprocessorHost().postIncrement(increment, r);
4204       }
4205     }
4206     metricsRegionServer.updateIncrement(EnvironmentEdgeManager.currentTimeMillis() - before);
4207     return r;
4208   }
4209 
4210   /**
4211    * Starts the nonce operation for a mutation, if needed.
4212    * @param mutation Mutation.
4213    * @param nonceGroup Nonce group from the request.
4214    * @returns Nonce used (can be NO_NONCE).
4215    */
4216   private long startNonceOperation(final MutationProto mutation, long nonceGroup)
4217       throws IOException, OperationConflictException {
4218     if (nonceManager == null || !mutation.hasNonce()) return HConstants.NO_NONCE;
4219     boolean canProceed = false;
4220     try {
4221       canProceed = nonceManager.startOperation(nonceGroup, mutation.getNonce(), this);
4222     } catch (InterruptedException ex) {
4223       throw new InterruptedIOException("Nonce start operation interrupted");
4224     }
4225     if (!canProceed) {
4226       // TODO: instead, we could convert append/increment to get w/mvcc
4227       String message = "The operation with nonce {" + nonceGroup + ", " + mutation.getNonce()
4228           + "} on row [" + Bytes.toString(mutation.getRow().toByteArray())
4229           + "] may have already completed";
4230       throw new OperationConflictException(message);
4231     }
4232     return mutation.getNonce();
4233   }
4234 
4235   /**
4236    * Ends nonce operation for a mutation, if needed.
4237    * @param mutation Mutation.
4238    * @param nonceGroup Nonce group from the request. Always 0 in initial implementation.
4239    * @param success Whether the operation for this nonce has succeeded.
4240    */
4241   private void endNonceOperation(final MutationProto mutation, long nonceGroup,
4242       boolean success) {
4243     if (nonceManager == null || !mutation.hasNonce()) return;
4244     nonceManager.endOperation(nonceGroup, mutation.getNonce(), success);
4245   }
4246 
4247   @Override
4248   public ServerNonceManager getNonceManager() {
4249     return this.nonceManager;
4250   }
4251 
4252   /**
4253    * Execute a list of Put/Delete mutations.
4254    *
4255    * @param builder
4256    * @param region
4257    * @param mutations
4258    */
4259   protected void doBatchOp(final RegionActionResult.Builder builder, final HRegion region,
4260       final List<ClientProtos.Action> mutations, final CellScanner cells) {
4261     Mutation[] mArray = new Mutation[mutations.size()];
4262     long before = EnvironmentEdgeManager.currentTimeMillis();
4263     boolean batchContainsPuts = false, batchContainsDelete = false;
4264     try {
4265       int i = 0;
4266       for (ClientProtos.Action action: mutations) {
4267         MutationProto m = action.getMutation();
4268         Mutation mutation;
4269         if (m.getMutateType() == MutationType.PUT) {
4270           mutation = ProtobufUtil.toPut(m, cells);
4271           batchContainsPuts = true;
4272         } else {
4273           mutation = ProtobufUtil.toDelete(m, cells);
4274           batchContainsDelete = true;
4275         }
4276         mArray[i++] = mutation;
4277       }
4278 
4279       requestCount.add(mutations.size());
4280       if (!region.getRegionInfo().isMetaTable()) {
4281         cacheFlusher.reclaimMemStoreMemory();
4282       }
4283 
4284       OperationStatus codes[] = region.batchMutate(mArray);
4285       for (i = 0; i < codes.length; i++) {
4286         int index = mutations.get(i).getIndex();
4287         Exception e = null;
4288         switch (codes[i].getOperationStatusCode()) {
4289           case BAD_FAMILY:
4290             e = new NoSuchColumnFamilyException(codes[i].getExceptionMsg());
4291             builder.addResultOrException(getResultOrException(e, index));
4292             break;
4293 
4294           case SANITY_CHECK_FAILURE:
4295             e = new FailedSanityCheckException(codes[i].getExceptionMsg());
4296             builder.addResultOrException(getResultOrException(e, index));
4297             break;
4298 
4299           default:
4300             e = new DoNotRetryIOException(codes[i].getExceptionMsg());
4301             builder.addResultOrException(getResultOrException(e, index));
4302             break;
4303 
4304           case SUCCESS:
4305             builder.addResultOrException(getResultOrException(ClientProtos.Result.getDefaultInstance(), index));
4306             break;
4307         }
4308       }
4309     } catch (IOException ie) {
4310       for (int i = 0; i < mutations.size(); i++) {
4311         builder.addResultOrException(getResultOrException(ie, mutations.get(i).getIndex()));
4312       }
4313     }
4314     long after = EnvironmentEdgeManager.currentTimeMillis();
4315     if (batchContainsPuts) {
4316       metricsRegionServer.updatePut(after - before);
4317     }
4318     if (batchContainsDelete) {
4319       metricsRegionServer.updateDelete(after - before);
4320     }
4321   }
4322   private static ResultOrException getResultOrException(final ClientProtos.Result r,
4323       final int index) {
4324     return getResultOrException(ResponseConverter.buildActionResult(r), index);
4325   }
4326   private static ResultOrException getResultOrException(final Exception e, final int index) {
4327     return getResultOrException(ResponseConverter.buildActionResult(e), index);
4328   }
4329 
4330   private static ResultOrException getResultOrException(final ResultOrException.Builder builder,
4331       final int index) {
4332     return builder.setIndex(index).build();
4333   }
4334 
4335   /**
4336    * Execute a list of Put/Delete mutations. The function returns OperationStatus instead of
4337    * constructing MultiResponse to save a possible loop if caller doesn't need MultiResponse.
4338    * @param region
4339    * @param mutations
4340    * @return an array of OperationStatus which internally contains the OperationStatusCode and the
4341    *         exceptionMessage if any
4342    * @throws IOException
4343    */
4344   protected OperationStatus [] doReplayBatchOp(final HRegion region,
4345       final List<HLogSplitter.MutationReplay> mutations) throws IOException {
4346     HLogSplitter.MutationReplay[] mArray = new HLogSplitter.MutationReplay[mutations.size()];
4347 
4348     long before = EnvironmentEdgeManager.currentTimeMillis();
4349     boolean batchContainsPuts = false, batchContainsDelete = false;
4350     try {
4351       int i = 0;
4352       for (HLogSplitter.MutationReplay m : mutations) {
4353         if (m.type == MutationType.PUT) {
4354           batchContainsPuts = true;
4355         } else {
4356           batchContainsDelete = true;
4357         }
4358         mArray[i++] = m;
4359       }
4360       requestCount.add(mutations.size());
4361       if (!region.getRegionInfo().isMetaTable()) {
4362         cacheFlusher.reclaimMemStoreMemory();
4363       }
4364       return region.batchReplay(mArray);
4365     } finally {
4366       long after = EnvironmentEdgeManager.currentTimeMillis();
4367       if (batchContainsPuts) {
4368         metricsRegionServer.updatePut(after - before);
4369       }
4370       if (batchContainsDelete) {
4371         metricsRegionServer.updateDelete(after - before);
4372       }
4373     }
4374   }
4375 
4376   /**
4377    * Mutate a list of rows atomically.
4378    *
4379    * @param region
4380    * @param actions
4381  * @param cellScanner if non-null, the mutation data -- the Cell content.
4382    * @throws IOException
4383    */
4384   protected void mutateRows(final HRegion region, final List<ClientProtos.Action> actions,
4385       final CellScanner cellScanner)
4386   throws IOException {
4387     if (!region.getRegionInfo().isMetaTable()) {
4388       cacheFlusher.reclaimMemStoreMemory();
4389     }
4390     RowMutations rm = null;
4391     for (ClientProtos.Action action: actions) {
4392       if (action.hasGet()) {
4393         throw new DoNotRetryIOException("Atomic put and/or delete only, not a Get=" +
4394           action.getGet());
4395       }
4396       MutationType type = action.getMutation().getMutateType();
4397       if (rm == null) {
4398         rm = new RowMutations(action.getMutation().getRow().toByteArray());
4399       }
4400       switch (type) {
4401       case PUT:
4402         rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
4403         break;
4404       case DELETE:
4405         rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
4406         break;
4407       default:
4408           throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
4409       }
4410     }
4411     region.mutateRow(rm);
4412   }
4413 
4414   private static class MovedRegionInfo {
4415     private final ServerName serverName;
4416     private final long seqNum;
4417     private final long ts;
4418 
4419     public MovedRegionInfo(ServerName serverName, long closeSeqNum) {
4420       this.serverName = serverName;
4421       this.seqNum = closeSeqNum;
4422       ts = EnvironmentEdgeManager.currentTimeMillis();
4423      }
4424 
4425     public ServerName getServerName() {
4426       return serverName;
4427     }
4428 
4429     public long getSeqNum() {
4430       return seqNum;
4431     }
4432 
4433     public long getMoveTime() {
4434       return ts;
4435     }
4436   }
4437 
4438   // This map will contains all the regions that we closed for a move.
4439   //  We add the time it was moved as we don't want to keep too old information
4440   protected Map<String, MovedRegionInfo> movedRegions =
4441       new ConcurrentHashMap<String, MovedRegionInfo>(3000);
4442 
4443   // We need a timeout. If not there is a risk of giving a wrong information: this would double
4444   //  the number of network calls instead of reducing them.
4445   private static final int TIMEOUT_REGION_MOVED = (2 * 60 * 1000);
4446 
4447   protected void addToMovedRegions(String encodedName, ServerName destination, long closeSeqNum) {
4448     if (ServerName.isSameHostnameAndPort(destination, this.getServerName())) {
4449       LOG.warn("Not adding moved region record: " + encodedName + " to self.");
4450       return;
4451     }
4452     LOG.info("Adding moved region record: " + encodedName + " to "
4453         + destination.getServerName() + ":" + destination.getPort()
4454         + " as of " + closeSeqNum);
4455     movedRegions.put(encodedName, new MovedRegionInfo(destination, closeSeqNum));
4456   }
4457 
4458   private void removeFromMovedRegions(String encodedName) {
4459     movedRegions.remove(encodedName);
4460   }
4461 
4462   private MovedRegionInfo getMovedRegion(final String encodedRegionName) {
4463     MovedRegionInfo dest = movedRegions.get(encodedRegionName);
4464 
4465     long now = EnvironmentEdgeManager.currentTimeMillis();
4466     if (dest != null) {
4467       if (dest.getMoveTime() > (now - TIMEOUT_REGION_MOVED)) {
4468         return dest;
4469       } else {
4470         movedRegions.remove(encodedRegionName);
4471       }
4472     }
4473 
4474     return null;
4475   }
4476 
4477   /**
4478    * Remove the expired entries from the moved regions list.
4479    */
4480   protected void cleanMovedRegions() {
4481     final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED;
4482     Iterator<Entry<String, MovedRegionInfo>> it = movedRegions.entrySet().iterator();
4483 
4484     while (it.hasNext()){
4485       Map.Entry<String, MovedRegionInfo> e = it.next();
4486       if (e.getValue().getMoveTime() < cutOff) {
4487         it.remove();
4488       }
4489     }
4490   }
4491 
4492   /**
4493    * Creates a Chore thread to clean the moved region cache.
4494    */
4495   protected static class MovedRegionsCleaner extends Chore implements Stoppable {
4496     private HRegionServer regionServer;
4497     Stoppable stoppable;
4498 
4499     private MovedRegionsCleaner(
4500       HRegionServer regionServer, Stoppable stoppable){
4501       super("MovedRegionsCleaner for region "+regionServer, TIMEOUT_REGION_MOVED, stoppable);
4502       this.regionServer = regionServer;
4503       this.stoppable = stoppable;
4504     }
4505 
4506     static MovedRegionsCleaner createAndStart(HRegionServer rs){
4507       Stoppable stoppable = new Stoppable() {
4508         private volatile boolean isStopped = false;
4509         @Override public void stop(String why) { isStopped = true;}
4510         @Override public boolean isStopped() {return isStopped;}
4511       };
4512 
4513       return new MovedRegionsCleaner(rs, stoppable);
4514     }
4515 
4516     @Override
4517     protected void chore() {
4518       regionServer.cleanMovedRegions();
4519     }
4520 
4521     @Override
4522     public void stop(String why) {
4523       stoppable.stop(why);
4524     }
4525 
4526     @Override
4527     public boolean isStopped() {
4528       return stoppable.isStopped();
4529     }
4530   }
4531 
4532   private String getMyEphemeralNodePath() {
4533     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
4534   }
4535 
4536   /**
4537    * Holder class which holds the RegionScanner and nextCallSeq together.
4538    */
4539   private static class RegionScannerHolder {
4540     private RegionScanner s;
4541     private long nextCallSeq = 0L;
4542     private HRegion r;
4543 
4544     public RegionScannerHolder(RegionScanner s, HRegion r) {
4545       this.s = s;
4546       this.r = r;
4547     }
4548   }
4549 
4550   private boolean isHealthCheckerConfigured() {
4551     String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
4552     return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
4553   }
4554 
4555   /**
4556    * @return the underlying {@link CompactSplitThread} for the servers
4557    */
4558   public CompactSplitThread getCompactSplitThread() {
4559     return this.compactSplitThread;
4560   }
4561 
4562   /**
4563    * A helper function to store the last flushed sequence Id with the previous failed RS for a
4564    * recovering region. The Id is used to skip wal edits which are flushed. Since the flushed
4565    * sequence id is only valid for each RS, we associate the Id with corresponding failed RS.
4566    * @throws KeeperException
4567    * @throws IOException
4568    */
4569   private void updateRecoveringRegionLastFlushedSequenceId(HRegion r) throws KeeperException,
4570       IOException {
4571     if (!r.isRecovering()) {
4572       // return immdiately for non-recovering regions
4573       return;
4574     }
4575 
4576     HRegionInfo region = r.getRegionInfo();
4577     ZooKeeperWatcher zkw = getZooKeeper();
4578     String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName());
4579     Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqIdForLogReplay();
4580     long minSeqIdForLogReplay = -1;
4581     for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) {
4582       if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) {
4583         minSeqIdForLogReplay = storeSeqIdForReplay;
4584       }
4585     }
4586     long lastRecordedFlushedSequenceId = -1;
4587     String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
4588       region.getEncodedName());
4589     // recovering-region level
4590     byte[] data;
4591     try {
4592       data = ZKUtil.getData(zkw, nodePath);
4593     } catch (InterruptedException e) {
4594       throw new InterruptedIOException();
4595     }
4596     if (data != null) {
4597       lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
4598     }
4599     if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
4600       ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
4601     }
4602     if (previousRSName != null) {
4603       // one level deeper for the failed RS
4604       nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
4605       ZKUtil.setData(zkw, nodePath,
4606         ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
4607       LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for "
4608           + previousRSName);
4609     } else {
4610       LOG.warn("Can't find failed region server for recovering region " + region.getEncodedName());
4611     }
4612   }
4613 
4614   /**
4615    * Return the last failed RS name under /hbase/recovering-regions/encodedRegionName
4616    * @param encodedRegionName
4617    * @throws KeeperException
4618    */
4619   private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
4620     String result = null;
4621     long maxZxid = 0;
4622     ZooKeeperWatcher zkw = this.getZooKeeper();
4623     String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
4624     List<String> failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath);
4625     if (failedServers == null || failedServers.isEmpty()) {
4626       return result;
4627     }
4628     for (String failedServer : failedServers) {
4629       String rsPath = ZKUtil.joinZNode(nodePath, failedServer);
4630       Stat stat = new Stat();
4631       ZKUtil.getDataNoWatch(zkw, rsPath, stat);
4632       if (maxZxid < stat.getCzxid()) {
4633         maxZxid = stat.getCzxid();
4634         result = failedServer;
4635       }
4636     }
4637     return result;
4638   }
4639 
4640   @Override
4641   public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
4642       UpdateFavoredNodesRequest request) throws ServiceException {
4643     List<UpdateFavoredNodesRequest.RegionUpdateInfo> openInfoList = request.getUpdateInfoList();
4644     UpdateFavoredNodesResponse.Builder respBuilder = UpdateFavoredNodesResponse.newBuilder();
4645     for (UpdateFavoredNodesRequest.RegionUpdateInfo regionUpdateInfo : openInfoList) {
4646       HRegionInfo hri = HRegionInfo.convert(regionUpdateInfo.getRegion());
4647       updateRegionFavoredNodesMapping(hri.getEncodedName(),
4648           regionUpdateInfo.getFavoredNodesList());
4649     }
4650     respBuilder.setResponse(openInfoList.size());
4651     return respBuilder.build();
4652   }
4653 }