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