View Javadoc

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