1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import com.google.common.base.Preconditions;
22 import com.google.common.collect.Maps;
23 import com.google.protobuf.BlockingRpcChannel;
24 import com.google.protobuf.Descriptors;
25 import com.google.protobuf.Message;
26 import com.google.protobuf.RpcCallback;
27 import com.google.protobuf.RpcController;
28 import com.google.protobuf.Service;
29 import com.google.protobuf.ServiceException;
30
31 import java.io.IOException;
32 import java.io.InterruptedIOException;
33 import java.lang.Thread.UncaughtExceptionHandler;
34 import java.lang.management.MemoryUsage;
35 import java.lang.reflect.Constructor;
36 import java.net.BindException;
37 import java.net.InetAddress;
38 import java.net.InetSocketAddress;
39 import java.net.UnknownHostException;
40 import java.security.PrivilegedExceptionAction;
41 import java.text.MessageFormat;
42 import java.util.ArrayList;
43 import java.util.Collection;
44 import java.util.Collections;
45 import java.util.Comparator;
46 import java.util.HashMap;
47 import java.util.HashSet;
48 import java.util.Iterator;
49 import java.util.List;
50 import java.util.Map;
51 import java.util.Map.Entry;
52 import java.util.Set;
53 import java.util.SortedMap;
54 import java.util.Timer;
55 import java.util.TimerTask;
56 import java.util.TreeMap;
57 import java.util.TreeSet;
58 import java.util.concurrent.ConcurrentHashMap;
59 import java.util.concurrent.ConcurrentMap;
60 import java.util.concurrent.ConcurrentSkipListMap;
61 import java.util.concurrent.CountDownLatch;
62 import java.util.concurrent.TimeUnit;
63 import java.util.concurrent.atomic.AtomicBoolean;
64 import java.util.concurrent.atomic.AtomicReference;
65 import java.util.concurrent.locks.ReentrantReadWriteLock;
66
67 import javax.management.MalformedObjectNameException;
68 import javax.management.ObjectName;
69 import javax.servlet.http.HttpServlet;
70
71 import org.apache.commons.lang.SystemUtils;
72 import org.apache.commons.lang.math.RandomUtils;
73 import org.apache.commons.logging.Log;
74 import org.apache.commons.logging.LogFactory;
75 import org.apache.hadoop.conf.Configuration;
76 import org.apache.hadoop.fs.FileSystem;
77 import org.apache.hadoop.fs.Path;
78 import org.apache.hadoop.hbase.ChoreService;
79 import org.apache.hadoop.hbase.ClockOutOfSyncException;
80 import org.apache.hadoop.hbase.CoordinatedStateManager;
81 import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
82 import org.apache.hadoop.hbase.ExecutorStatusChore;
83 import org.apache.hadoop.hbase.HBaseConfiguration;
84 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
85 import org.apache.hadoop.hbase.HConstants;
86 import org.apache.hadoop.hbase.HRegionInfo;
87 import org.apache.hadoop.hbase.HealthCheckChore;
88 import org.apache.hadoop.hbase.MetaTableAccessor;
89 import org.apache.hadoop.hbase.NotServingRegionException;
90 import org.apache.hadoop.hbase.RemoteExceptionHandler;
91 import org.apache.hadoop.hbase.ScheduledChore;
92 import org.apache.hadoop.hbase.ServerName;
93 import org.apache.hadoop.hbase.Stoppable;
94 import org.apache.hadoop.hbase.TableDescriptors;
95 import org.apache.hadoop.hbase.TableName;
96 import org.apache.hadoop.hbase.YouAreDeadException;
97 import org.apache.hadoop.hbase.ZNodeClearer;
98 import org.apache.hadoop.hbase.classification.InterfaceAudience;
99 import org.apache.hadoop.hbase.client.ClusterConnection;
100 import org.apache.hadoop.hbase.client.ConnectionUtils;
101 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
102 import org.apache.hadoop.hbase.conf.ConfigurationManager;
103 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
104 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
105 import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
106 import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
107 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
108 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
109 import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
110 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
111 import org.apache.hadoop.hbase.executor.ExecutorService;
112 import org.apache.hadoop.hbase.executor.ExecutorType;
113 import org.apache.hadoop.hbase.fs.HFileSystem;
114 import org.apache.hadoop.hbase.http.InfoServer;
115 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
116 import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
117 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
118 import org.apache.hadoop.hbase.ipc.RpcClient;
119 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
120 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
121 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
122 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
123 import org.apache.hadoop.hbase.ipc.ServerRpcController;
124 import org.apache.hadoop.hbase.master.HMaster;
125 import org.apache.hadoop.hbase.master.RegionState.State;
126 import org.apache.hadoop.hbase.master.TableLockManager;
127 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
128 import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
129 import org.apache.hadoop.hbase.namequeues.SlowLogTableOpsChore;
130 import org.apache.hadoop.hbase.net.Address;
131 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
132 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
133 import org.apache.hadoop.hbase.protobuf.RequestConverter;
134 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
135 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
136 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
137 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
138 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
139 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
140 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
141 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
142 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
143 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.Builder;
144 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
145 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
146 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
147 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
148 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
149 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
150 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
151 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
152 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
153 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
154 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
155 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
156 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
157 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
158 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
159 import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
160 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
161 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
162 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
163 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
164 import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
165 import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
166 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
167 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
168 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
169 import org.apache.hadoop.hbase.replication.regionserver.Replication;
170 import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
171 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
172 import org.apache.hadoop.hbase.replication.regionserver.ReplicationStatus;
173 import org.apache.hadoop.hbase.security.Superusers;
174 import org.apache.hadoop.hbase.security.User;
175 import org.apache.hadoop.hbase.security.UserProvider;
176 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
177 import org.apache.hadoop.hbase.util.Addressing;
178 import org.apache.hadoop.hbase.util.ByteStringer;
179 import org.apache.hadoop.hbase.util.Bytes;
180 import org.apache.hadoop.hbase.util.CompressionTest;
181 import org.apache.hadoop.hbase.util.ConfigUtil;
182 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
183 import org.apache.hadoop.hbase.util.FSTableDescriptors;
184 import org.apache.hadoop.hbase.util.FSUtils;
185 import org.apache.hadoop.hbase.util.HasThread;
186 import org.apache.hadoop.hbase.util.JSONBean;
187 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
188 import org.apache.hadoop.hbase.util.MBeanUtil;
189 import org.apache.hadoop.hbase.util.RetryCounter;
190 import org.apache.hadoop.hbase.util.RetryCounterFactory;
191 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
192 import org.apache.hadoop.hbase.util.Sleeper;
193 import org.apache.hadoop.hbase.util.Threads;
194 import org.apache.hadoop.hbase.util.VersionInfo;
195 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
196 import org.apache.hadoop.hbase.wal.WAL;
197 import org.apache.hadoop.hbase.wal.WALFactory;
198 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
199 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
200 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
201 import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
202 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
203 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
204 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
205 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
206 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
207 import org.apache.hadoop.ipc.RemoteException;
208 import org.apache.hadoop.util.ReflectionUtils;
209 import org.apache.hadoop.util.StringUtils;
210 import org.apache.zookeeper.KeeperException;
211 import org.apache.zookeeper.KeeperException.NoNodeException;
212 import org.apache.zookeeper.data.Stat;
213
214 import sun.misc.Signal;
215 import sun.misc.SignalHandler;
216
217
218
219
220
221 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
222 @SuppressWarnings("deprecation")
223 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="JLM_JSR166_UTILCONCURRENT_MONITORENTER",
224 justification="Use of an atomic type both as monitor and condition variable is intended")
225 public class HRegionServer extends HasThread implements
226 RegionServerServices, LastSequenceId, ConfigurationObserver {
227
228 private static final Log LOG = LogFactory.getLog(HRegionServer.class);
229
230
231
232
233
234 protected static final String OPEN = "OPEN";
235 protected static final String CLOSE = "CLOSE";
236
237
238
239
240 protected final ConcurrentMap<byte[], Boolean> regionsInTransitionInRS =
241 new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
242
243
244 protected MemStoreFlusher cacheFlusher;
245
246 protected HeapMemoryManager hMemManager;
247 protected CountDownLatch initLatch = null;
248
249
250
251
252
253
254 protected ClusterConnection clusterConnection;
255
256
257
258
259
260
261
262 protected MetaTableLocator metaTableLocator;
263
264
265 @SuppressWarnings("unused")
266 private RecoveringRegionWatcher recoveringRegionWatcher;
267
268
269
270
271 protected TableDescriptors tableDescriptors;
272
273
274 protected ReplicationSourceService replicationSourceHandler;
275 protected ReplicationSinkService replicationSinkHandler;
276
277
278 public CompactSplitThread compactSplitThread;
279
280
281
282
283
284 protected final Map<String, Region> onlineRegions = new ConcurrentHashMap<String, Region>();
285
286
287
288
289
290
291
292
293
294
295
296
297 protected final Map<String, Address[]> regionFavoredNodesMap =
298 new ConcurrentHashMap<String, Address[]>();
299
300
301
302
303
304 protected final Map<String, Region> recoveringRegions = Collections
305 .synchronizedMap(new HashMap<String, Region>());
306
307
308 protected Leases leases;
309
310
311 protected ExecutorService service;
312
313
314 protected volatile boolean fsOk;
315 protected HFileSystem fs;
316 protected HFileSystem walFs;
317
318
319
320
321 private volatile boolean stopped = false;
322
323
324
325 private AtomicBoolean abortRequested;
326 public static final String ABORT_TIMEOUT = "hbase.regionserver.abort.timeout";
327
328 private static final long DEFAULT_ABORT_TIMEOUT = 1200000;
329
330 public static final String ABORT_TIMEOUT_TASK = "hbase.regionserver.abort.timeout.task";
331
332 ConcurrentMap<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
333
334
335
336 private boolean stopping = false;
337
338 volatile boolean killed = false;
339
340 protected final Configuration conf;
341
342 private Path rootDir;
343 private Path walRootDir;
344
345 protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
346
347 final int numRetries;
348 protected final int threadWakeFrequency;
349 protected final int msgInterval;
350
351 private static final String PERIOD_COMPACTION = "hbase.regionserver.compaction.check.period";
352 private final int compactionCheckFrequency;
353 private static final String PERIOD_FLUSH = "hbase.regionserver.flush.check.period";
354 private final int flushCheckFrequency;
355
356 protected final int numRegionsToReport;
357
358
359 private volatile RegionServerStatusService.BlockingInterface rssStub;
360
361 RpcClient rpcClient;
362
363 private RpcRetryingCallerFactory rpcRetryingCallerFactory;
364 private RpcControllerFactory rpcControllerFactory;
365
366 private UncaughtExceptionHandler uncaughtExceptionHandler;
367
368
369
370
371 protected InfoServer infoServer;
372 private JvmPauseMonitor pauseMonitor;
373
374
375 public static final String REGIONSERVER = "regionserver";
376
377 MetricsRegionServer metricsRegionServer;
378 MetricsRegionServerWrapperImpl metricsRegionServerImpl;
379 MetricsTable metricsTable;
380 private SpanReceiverHost spanReceiverHost;
381
382
383
384
385 private final ChoreService choreService;
386
387
388
389
390 ScheduledChore compactionChecker;
391
392
393
394
395 ScheduledChore periodicFlusher;
396
397 protected volatile WALFactory walFactory;
398
399
400
401 final LogRoller walRoller;
402
403 final AtomicReference<LogRoller> metawalRoller = new AtomicReference<LogRoller>();
404
405
406 final AtomicBoolean online = new AtomicBoolean(false);
407
408
409 protected ZooKeeperWatcher zooKeeper;
410
411
412 private MasterAddressTracker masterAddressTracker;
413
414
415 protected ClusterStatusTracker clusterStatusTracker;
416
417
418 private SplitLogWorker splitLogWorker;
419
420
421 protected final Sleeper sleeper;
422
423 private final int operationTimeout;
424 private final int shortOperationTimeout;
425
426 private SlowLogTableOpsChore slowLogTableOpsChore = null;
427
428 private final RegionServerAccounting regionServerAccounting;
429
430
431 protected CacheConfig cacheConfig;
432
433
434 private HealthCheckChore healthCheckChore;
435
436
437 private ExecutorStatusChore executorStatusChore;
438
439
440 private ScheduledChore nonceManagerChore;
441
442 private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
443
444
445
446
447
448
449 protected ServerName serverName;
450
451
452
453
454 protected String useThisHostnameInstead;
455
456
457
458
459 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
460 final static String RS_HOSTNAME_KEY = "hbase.regionserver.hostname";
461 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
462 protected final static String MASTER_HOSTNAME_KEY = "hbase.master.hostname";
463
464
465
466 final static String RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY =
467 "hbase.regionserver.hostname.disable.master.reversedns";
468
469
470
471
472 protected final long startcode;
473
474
475
476
477 String clusterId;
478
479
480
481
482 private ObjectName mxBean = null;
483
484
485
486
487 private MovedRegionsCleaner movedRegionsCleaner;
488
489
490 private StorefileRefresherChore storefileRefresher;
491
492 private RegionServerCoprocessorHost rsHost;
493
494 private RegionServerProcedureManagerHost rspmHost;
495
496 private RegionServerQuotaManager rsQuotaManager;
497
498
499 protected TableLockManager tableLockManager;
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519 final ServerNonceManager nonceManager;
520
521 private UserProvider userProvider;
522
523 protected final RSRpcServices rpcServices;
524
525 protected BaseCoordinatedStateManager csm;
526
527 private final boolean useZKForAssignment;
528
529
530
531
532
533 protected final ConfigurationManager configurationManager;
534
535 private CompactedHFilesDischarger compactedFileDischarger;
536
537 private volatile ThroughputController flushThroughputController;
538
539
540
541
542 private NamedQueueRecorder namedQueueRecorder = null;
543
544
545
546
547 public HRegionServer(Configuration conf) throws IOException, InterruptedException {
548 this(conf, CoordinatedStateManagerFactory.getCoordinatedStateManager(conf));
549 }
550
551
552
553
554
555 public HRegionServer(Configuration conf, CoordinatedStateManager csm)
556 throws IOException, InterruptedException {
557 super("RegionServer");
558 this.startcode = System.currentTimeMillis();
559 this.fsOk = true;
560 this.conf = conf;
561 checkCodecs(this.conf);
562 this.userProvider = UserProvider.instantiate(conf);
563 FSUtils.setupShortCircuitRead(this.conf);
564
565 Replication.decorateRegionServerConfiguration(this.conf);
566
567
568 this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
569
570
571 this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
572 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
573 this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
574 this.compactionCheckFrequency = conf.getInt(PERIOD_COMPACTION, this.threadWakeFrequency);
575 this.flushCheckFrequency = conf.getInt(PERIOD_FLUSH, this.threadWakeFrequency);
576 this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
577
578 this.sleeper = new Sleeper(this.msgInterval, this);
579
580 boolean isNoncesEnabled = conf.getBoolean(HConstants.HBASE_RS_NONCES_ENABLED, true);
581 this.nonceManager = isNoncesEnabled ? new ServerNonceManager(this.conf) : null;
582
583 this.numRegionsToReport = conf.getInt(
584 "hbase.regionserver.numregionstoreport", 10);
585
586 this.operationTimeout = conf.getInt(
587 HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
588 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
589
590 this.shortOperationTimeout = conf.getInt(
591 HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
592 HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
593
594 this.abortRequested = new AtomicBoolean(false);
595 this.stopped = false;
596
597 initNamedQueueRecorder(conf);
598
599 rpcServices = createRpcServices();
600 if (this instanceof HMaster) {
601 useThisHostnameInstead = conf.get(MASTER_HOSTNAME_KEY);
602 } else {
603 useThisHostnameInstead = conf.get(RS_HOSTNAME_KEY);
604 if (conf.getBoolean(RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY, false)) {
605 if (shouldUseThisHostnameInstead()) {
606 String msg = RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY + " and " + RS_HOSTNAME_KEY +
607 " are mutually exclusive. Do not set " + RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY +
608 " to true while " + RS_HOSTNAME_KEY + " is used";
609 throw new IOException(msg);
610 } else {
611 useThisHostnameInstead = rpcServices.isa.getHostName();
612 }
613 }
614 }
615 String hostName = shouldUseThisHostnameInstead() ? useThisHostnameInstead :
616 rpcServices.isa.getHostName();
617 serverName = ServerName.valueOf(hostName, rpcServices.isa.getPort(), startcode);
618
619 rpcControllerFactory = RpcControllerFactory.instantiate(this.conf);
620 rpcRetryingCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
621
622
623 ZKUtil.loginClient(this.conf, HConstants.ZK_CLIENT_KEYTAB_FILE,
624 HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL, hostName);
625
626 login(userProvider, hostName);
627
628
629 Superusers.initialize(conf);
630
631 regionServerAccounting = new RegionServerAccounting();
632 uncaughtExceptionHandler = new UncaughtExceptionHandler() {
633 @Override
634 public void uncaughtException(Thread t, Throwable e) {
635 abort("Uncaught exception in service thread " + t.getName(), e);
636 }
637 };
638
639 useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
640
641 initializeFileSystem();
642
643 service = new ExecutorService(getServerName().toShortString());
644 spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
645
646
647 if (!conf.getBoolean("hbase.testing.nocluster", false)) {
648
649 zooKeeper = new ZooKeeperWatcher(conf, getProcessName() + ":" +
650 rpcServices.isa.getPort(), this, canCreateBaseZNode());
651
652 this.csm = (BaseCoordinatedStateManager) csm;
653 this.csm.initialize(this);
654 this.csm.start();
655
656 tableLockManager = TableLockManager.createTableLockManager(
657 conf, zooKeeper, serverName);
658
659 masterAddressTracker = new MasterAddressTracker(getZooKeeper(), this);
660 masterAddressTracker.start();
661
662 clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this);
663 clusterStatusTracker.start();
664 }
665 this.configurationManager = new ConfigurationManager();
666
667 rpcServices.start(zooKeeper);
668 putUpWebUI();
669 this.walRoller = new LogRoller(this, this);
670 this.choreService = new ChoreService(getServerName().toString(), true);
671 this.flushThroughputController = FlushThroughputControllerFactory.create(this, conf);
672
673 if (!SystemUtils.IS_OS_WINDOWS) {
674 Signal.handle(new Signal("HUP"), new SignalHandler() {
675 @Override
676 public void handle(Signal signal) {
677 getConfiguration().reloadConfiguration();
678 configurationManager.notifyAllObservers(getConfiguration());
679 }
680 });
681 }
682
683
684
685
686
687 int cleanerInterval = conf
688 .getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_INTERVAL, 2 * 60 * 1000);
689 this.compactedFileDischarger =
690 new CompactedHFilesDischarger(cleanerInterval, (Stoppable)this, (RegionServerServices)this);
691 choreService.scheduleChore(compactedFileDischarger);
692 }
693
694 private void initializeFileSystem() throws IOException {
695
696
697 boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
698 FSUtils.setFsDefault(this.conf, FSUtils.getWALRootDir(this.conf));
699 this.walFs = new HFileSystem(this.conf, useHBaseChecksum);
700 this.walRootDir = FSUtils.getWALRootDir(this.conf);
701
702
703
704 FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
705 this.fs = new HFileSystem(this.conf, useHBaseChecksum);
706 this.rootDir = FSUtils.getRootDir(this.conf);
707 this.tableDescriptors = new FSTableDescriptors(
708 this.conf, this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
709 }
710
711 protected void setInitLatch(CountDownLatch latch) {
712 this.initLatch = latch;
713 }
714
715 private void initNamedQueueRecorder(Configuration conf) {
716 if (!(this instanceof HMaster)) {
717 final boolean isOnlineLogProviderEnabled = conf.getBoolean(
718 HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
719 HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
720 if (isOnlineLogProviderEnabled) {
721 this.namedQueueRecorder = NamedQueueRecorder.getInstance(this.conf);
722 }
723 } else {
724 final boolean isBalancerDecisionRecording = conf
725 .getBoolean(BaseLoadBalancer.BALANCER_DECISION_BUFFER_ENABLED,
726 BaseLoadBalancer.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED);
727 if (isBalancerDecisionRecording) {
728 this.namedQueueRecorder = NamedQueueRecorder.getInstance(this.conf);
729 }
730 }
731 }
732
733
734
735
736 protected boolean shouldUseThisHostnameInstead() {
737 return useThisHostnameInstead != null && !useThisHostnameInstead.isEmpty();
738 }
739
740 protected void login(UserProvider user, String host) throws IOException {
741 user.login("hbase.regionserver.keytab.file",
742 "hbase.regionserver.kerberos.principal", host);
743 }
744
745 protected void waitForMasterActive(){
746 }
747
748 protected String getProcessName() {
749 return REGIONSERVER;
750 }
751
752 protected boolean canCreateBaseZNode() {
753 return false;
754 }
755
756 protected boolean canUpdateTableDescriptor() {
757 return false;
758 }
759
760 protected RSRpcServices createRpcServices() throws IOException {
761 return new RSRpcServices(this);
762 }
763
764 protected void configureInfoServer() {
765 infoServer.addServlet("rs-status", "/rs-status", RSStatusServlet.class);
766 infoServer.setAttribute(REGIONSERVER, this);
767 }
768
769 protected Class<? extends HttpServlet> getDumpServlet() {
770 return RSDumpServlet.class;
771 }
772
773 @Override
774 public boolean registerService(Service instance) {
775
776
777
778 Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
779 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
780 if (coprocessorServiceHandlers.containsKey(serviceName)) {
781 LOG.error("Coprocessor service " + serviceName
782 + " already registered, rejecting request from " + instance);
783 return false;
784 }
785
786 coprocessorServiceHandlers.put(serviceName, instance);
787 if (LOG.isDebugEnabled()) {
788 LOG.debug("Registered regionserver coprocessor service: service=" + serviceName);
789 }
790 return true;
791 }
792
793
794
795
796
797
798
799 @InterfaceAudience.Private
800 protected ClusterConnection createClusterConnection() throws IOException {
801
802
803
804 return ConnectionUtils.createShortCircuitConnection(conf, null, userProvider.getCurrent(),
805 serverName, rpcServices, rpcServices);
806 }
807
808
809
810
811
812
813 private static void checkCodecs(final Configuration c) throws IOException {
814
815 String [] codecs = c.getStrings("hbase.regionserver.codecs", (String[])null);
816 if (codecs == null) return;
817 for (String codec : codecs) {
818 if (!CompressionTest.testCompression(codec)) {
819 throw new IOException("Compression codec " + codec +
820 " not supported, aborting RS construction");
821 }
822 }
823 }
824
825 public String getClusterId() {
826 return this.clusterId;
827 }
828
829
830
831
832
833 protected synchronized void setupClusterConnection() throws IOException {
834 if (clusterConnection == null) {
835 clusterConnection = createClusterConnection();
836 metaTableLocator = new MetaTableLocator();
837 }
838 }
839
840
841
842
843
844
845
846 private void preRegistrationInitialization(){
847 try {
848 setupClusterConnection();
849
850
851 if (isHealthCheckerConfigured()) {
852 int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
853 HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
854 healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
855 }
856
857 initializeZooKeeper();
858 if (!isStopped() && !isAborted()) {
859 initializeThreads();
860 }
861 } catch (Throwable t) {
862
863
864 this.rpcServices.stop();
865 abort("Initialization of RS failed. Hence aborting RS.", t);
866 }
867 }
868
869
870
871
872
873
874
875
876
877 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
878 value={"RV_RETURN_VALUE_IGNORED_BAD_PRACTICE", "RV_RETURN_VALUE_IGNORED"},
879 justification="cluster Id znode read would give us correct response")
880 private void initializeZooKeeper() throws IOException, InterruptedException {
881
882
883
884 blockAndCheckIfStopped(this.masterAddressTracker);
885
886
887
888 blockAndCheckIfStopped(this.clusterStatusTracker);
889
890 if (this.initLatch != null) {
891 this.initLatch.await(20, TimeUnit.SECONDS);
892 }
893
894
895
896 try {
897 clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
898 if (clusterId == null) {
899 this.abort("Cluster ID has not been set");
900 }
901 LOG.info("ClusterId : "+clusterId);
902 } catch (KeeperException e) {
903 this.abort("Failed to retrieve Cluster ID",e);
904 }
905
906
907
908
909
910 waitForMasterActive();
911 if (isStopped() || isAborted()) {
912 return;
913 }
914
915
916 try {
917 rspmHost = new RegionServerProcedureManagerHost();
918 rspmHost.loadProcedures(conf);
919 rspmHost.initialize(this);
920 } catch (KeeperException e) {
921 this.abort("Failed to reach zk cluster when creating procedure handler.", e);
922 }
923
924 this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);
925 }
926
927
928
929
930
931
932
933
934 private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
935 throws IOException, InterruptedException {
936 while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
937 if (this.stopped) {
938 throw new IOException("Received the shutdown message while waiting.");
939 }
940 }
941 }
942
943
944
945
946 private boolean isClusterUp() {
947 return clusterStatusTracker != null && clusterStatusTracker.isClusterUp();
948 }
949
950 private void initializeThreads() throws IOException {
951
952 this.cacheFlusher = new MemStoreFlusher(conf, this);
953
954
955 this.compactSplitThread = new CompactSplitThread(this);
956
957
958
959 this.compactionChecker = new CompactionChecker(this, this.compactionCheckFrequency, this);
960 this.periodicFlusher = new PeriodicMemstoreFlusher(this.flushCheckFrequency, this);
961 this.leases = new Leases(this.threadWakeFrequency);
962
963 final boolean isSlowLogTableEnabled = conf.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
964 HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
965 if (isSlowLogTableEnabled) {
966
967 final int duration = conf.getInt("hbase.slowlog.systable.chore.duration", 10 * 60 * 1000);
968 slowLogTableOpsChore = new SlowLogTableOpsChore(this, duration, this.namedQueueRecorder);
969 }
970
971
972 movedRegionsCleaner = MovedRegionsCleaner.create(this);
973
974 if (this.nonceManager != null) {
975
976 nonceManagerChore = this.nonceManager.createCleanupScheduledChore(this);
977 }
978
979
980 rsQuotaManager = new RegionServerQuotaManager(this);
981
982
983 rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
984 rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
985
986 boolean onlyMetaRefresh = false;
987 int storefileRefreshPeriod = conf.getInt(
988 StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD
989 , StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
990 if (storefileRefreshPeriod == 0) {
991 storefileRefreshPeriod = conf.getInt(
992 StorefileRefresherChore.REGIONSERVER_META_STOREFILE_REFRESH_PERIOD,
993 StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
994 onlyMetaRefresh = true;
995 }
996 if (storefileRefreshPeriod > 0) {
997 this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
998 onlyMetaRefresh, this, this);
999 }
1000 registerConfigurationObservers();
1001 }
1002
1003 private void registerConfigurationObservers() {
1004
1005 configurationManager.registerObserver(this.compactSplitThread);
1006 configurationManager.registerObserver(this.rpcServices);
1007 configurationManager.registerObserver(this);
1008 }
1009
1010
1011
1012
1013 @Override
1014 public void run() {
1015 try {
1016
1017 preRegistrationInitialization();
1018 } catch (Throwable e) {
1019 abort("Fatal exception during initialization", e);
1020 }
1021
1022 try {
1023 if (!isStopped() && !isAborted()) {
1024 ShutdownHook.install(conf, fs, this, Thread.currentThread());
1025
1026
1027 this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
1028 }
1029
1030
1031
1032
1033 RetryCounterFactory rcf = new RetryCounterFactory(Integer.MAX_VALUE,
1034 this.sleeper.getPeriod(), 1000 * 60 * 5);
1035 RetryCounter rc = rcf.create();
1036 while (keepLooping()) {
1037 RegionServerStartupResponse w = reportForDuty();
1038 if (w == null) {
1039 long sleepTime = rc.getBackoffTimeAndIncrementAttempts();
1040 LOG.warn("reportForDuty failed; sleeping " + sleepTime + " ms and then retrying");
1041 this.sleeper.sleep(sleepTime);
1042 } else {
1043 handleReportForDutyResponse(w);
1044 break;
1045 }
1046 }
1047
1048 if (!isStopped() && isHealthy()){
1049
1050
1051 rspmHost.start();
1052 }
1053
1054
1055 if (this.rsQuotaManager != null) {
1056 rsQuotaManager.start(getRpcServer().getScheduler());
1057 }
1058
1059
1060 long lastMsg = System.currentTimeMillis();
1061 long oldRequestCount = -1;
1062
1063 while (!isStopped() && isHealthy()) {
1064 if (!isClusterUp()) {
1065 if (isOnlineRegionsEmpty()) {
1066 stop("Exiting; cluster shutdown set and not carrying any regions");
1067 } else if (!this.stopping) {
1068 this.stopping = true;
1069 LOG.info("Closing user regions");
1070 closeUserRegions(this.abortRequested.get());
1071 } else if (this.stopping) {
1072 boolean allUserRegionsOffline = areAllUserRegionsOffline();
1073 if (allUserRegionsOffline) {
1074
1075
1076
1077 if (oldRequestCount == getWriteRequestCount()) {
1078 stop("Stopped; only catalog regions remaining online");
1079 break;
1080 }
1081 oldRequestCount = getWriteRequestCount();
1082 } else {
1083
1084
1085
1086 closeUserRegions(this.abortRequested.get());
1087 }
1088 LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
1089 }
1090 }
1091 long now = System.currentTimeMillis();
1092 if ((now - lastMsg) >= msgInterval) {
1093 tryRegionServerReport(lastMsg, now);
1094 lastMsg = System.currentTimeMillis();
1095 }
1096 if (!isStopped() && !isAborted()) {
1097 this.sleeper.sleep();
1098 }
1099 }
1100 } catch (Throwable t) {
1101 if (!rpcServices.checkOOME(t)) {
1102 String prefix = t instanceof YouAreDeadException? "": "Unhandled: ";
1103 abort(prefix + t.getMessage(), t);
1104 }
1105 }
1106
1107
1108 if (mxBean != null) {
1109 MBeanUtil.unregisterMBean(mxBean);
1110 mxBean = null;
1111 }
1112
1113 if (abortRequested.get()) {
1114 Timer abortMonitor = new Timer("Abort regionserver monitor", true);
1115 TimerTask abortTimeoutTask = null;
1116 try {
1117 Constructor<? extends TimerTask> timerTaskCtor =
1118 Class.forName(conf.get(ABORT_TIMEOUT_TASK, SystemExitWhenAbortTimeout.class.getName()))
1119 .asSubclass(TimerTask.class).getDeclaredConstructor();
1120 timerTaskCtor.setAccessible(true);
1121 abortTimeoutTask = timerTaskCtor.newInstance();
1122 } catch (Exception e) {
1123 LOG.warn("Initialize abort timeout task failed", e);
1124 }
1125 if (abortTimeoutTask != null) {
1126 abortMonitor.schedule(abortTimeoutTask, conf.getLong(ABORT_TIMEOUT, DEFAULT_ABORT_TIMEOUT));
1127 }
1128 }
1129
1130 if (this.leases != null) {
1131 this.leases.closeAfterLeasesExpire();
1132 }
1133 if (this.splitLogWorker != null) {
1134 splitLogWorker.stop();
1135 }
1136 if (this.infoServer != null) {
1137 LOG.info("Stopping infoServer");
1138 try {
1139 this.infoServer.stop();
1140 } catch (Exception e) {
1141 LOG.error("Failed to stop infoServer", e);
1142 }
1143 }
1144
1145 if (cacheConfig != null && cacheConfig.isBlockCacheEnabled()) {
1146 cacheConfig.getBlockCache().shutdown();
1147 }
1148
1149 if (movedRegionsCleaner != null) {
1150 movedRegionsCleaner.stop("Region Server stopping");
1151 }
1152
1153
1154
1155 if (this.hMemManager != null) this.hMemManager.stop();
1156 if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
1157 if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
1158 sendShutdownInterrupt();
1159
1160
1161 if (rsQuotaManager != null) {
1162 rsQuotaManager.stop();
1163 }
1164
1165
1166 if (rspmHost != null) {
1167 rspmHost.stop(this.abortRequested.get() || this.killed);
1168 }
1169
1170 if (this.killed) {
1171
1172 } else if (abortRequested.get()) {
1173 if (this.fsOk) {
1174 closeUserRegions(abortRequested.get());
1175 }
1176 LOG.info("aborting server " + this.serverName);
1177 } else {
1178 closeUserRegions(abortRequested.get());
1179 LOG.info("stopping server " + this.serverName);
1180 }
1181
1182
1183 if (this.metaTableLocator != null) this.metaTableLocator.stop();
1184 if (this.clusterConnection != null && !clusterConnection.isClosed()) {
1185 try {
1186 this.clusterConnection.close();
1187 } catch (IOException e) {
1188
1189
1190 LOG.warn("Attempt to close server's short circuit HConnection failed.", e);
1191 }
1192 }
1193
1194
1195 if (!this.killed && containsMetaTableRegions()) {
1196 if (!abortRequested.get() || this.fsOk) {
1197 if (this.compactSplitThread != null) {
1198 this.compactSplitThread.join();
1199 this.compactSplitThread = null;
1200 }
1201 closeMetaTableRegions(abortRequested.get());
1202 }
1203 }
1204
1205 if (!this.killed && this.fsOk) {
1206 waitOnAllRegionsToClose(abortRequested.get());
1207 LOG.info("stopping server " + this.serverName +
1208 "; all regions closed.");
1209 }
1210
1211
1212 if (this.fsOk) {
1213 shutdownWAL(!abortRequested.get());
1214 }
1215
1216
1217 if (this.rssStub != null) {
1218 this.rssStub = null;
1219 }
1220 if (this.rpcClient != null) {
1221 this.rpcClient.close();
1222 }
1223 if (this.leases != null) {
1224 this.leases.close();
1225 }
1226 if (this.pauseMonitor != null) {
1227 this.pauseMonitor.stop();
1228 }
1229
1230 if (!killed) {
1231 stopServiceThreads();
1232 }
1233
1234 if (this.rpcServices != null) {
1235 this.rpcServices.stop();
1236 }
1237
1238 try {
1239 deleteMyEphemeralNode();
1240 } catch (KeeperException.NoNodeException nn) {
1241 } catch (KeeperException e) {
1242 LOG.warn("Failed deleting my ephemeral node", e);
1243 }
1244
1245
1246 ZNodeClearer.deleteMyEphemeralNodeOnDisk();
1247
1248 if (this.zooKeeper != null) {
1249 this.zooKeeper.close();
1250 }
1251 LOG.info("stopping server " + this.serverName +
1252 "; zookeeper connection closed.");
1253
1254 LOG.info(Thread.currentThread().getName() + " exiting");
1255 }
1256
1257 private boolean containsMetaTableRegions() {
1258 return onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
1259 }
1260
1261 private boolean areAllUserRegionsOffline() {
1262 if (getNumberOfOnlineRegions() > 2) return false;
1263 boolean allUserRegionsOffline = true;
1264 for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
1265 if (!e.getValue().getRegionInfo().isMetaTable()) {
1266 allUserRegionsOffline = false;
1267 break;
1268 }
1269 }
1270 return allUserRegionsOffline;
1271 }
1272
1273
1274
1275
1276 private long getWriteRequestCount() {
1277 long writeCount = 0;
1278 for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
1279 writeCount += e.getValue().getWriteRequestsCount();
1280 }
1281 return writeCount;
1282 }
1283
1284 @InterfaceAudience.Private
1285 protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
1286 throws IOException {
1287 RegionServerStatusService.BlockingInterface rss = rssStub;
1288 if (rss == null) {
1289
1290 return;
1291 }
1292 ClusterStatusProtos.ServerLoad sl = buildServerLoad(reportStartTime, reportEndTime);
1293 try {
1294 RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();
1295 request.setServer(ProtobufUtil.toServerName(this.serverName));
1296 request.setLoad(sl);
1297 rss.regionServerReport(null, request.build());
1298 } catch (ServiceException se) {
1299 IOException ioe = ProtobufUtil.getRemoteException(se);
1300 if (ioe instanceof YouAreDeadException) {
1301
1302 throw ioe;
1303 }
1304 if (rssStub == rss) {
1305 rssStub = null;
1306 }
1307
1308
1309 createRegionServerStatusStub(true);
1310 }
1311 }
1312
1313 ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime)
1314 throws IOException {
1315
1316
1317
1318
1319
1320
1321
1322 MetricsRegionServerWrapper regionServerWrapper = metricsRegionServer.getRegionServerWrapper();
1323 Collection<Region> regions = getOnlineRegionsLocalContext();
1324 long usedMemory = -1L;
1325 long maxMemory = -1L;
1326 final MemoryUsage usage = HeapMemorySizeUtil.safeGetHeapMemoryUsage();
1327 if (usage != null) {
1328 usedMemory = usage.getUsed();
1329 maxMemory = usage.getMax();
1330 }
1331
1332 ClusterStatusProtos.ServerLoad.Builder serverLoad =
1333 ClusterStatusProtos.ServerLoad.newBuilder();
1334 serverLoad.setNumberOfRequests((int) regionServerWrapper.getRequestsPerSecond());
1335 serverLoad.setTotalNumberOfRequests(regionServerWrapper.getTotalRequestCount());
1336 serverLoad.setUsedHeapMB((int)(usedMemory / 1024 / 1024));
1337 serverLoad.setMaxHeapMB((int) (maxMemory / 1024 / 1024));
1338 Set<String> coprocessors = getWAL(null).getCoprocessorHost().getCoprocessors();
1339 Builder coprocessorBuilder = Coprocessor.newBuilder();
1340 for (String coprocessor : coprocessors) {
1341 serverLoad.addCoprocessors(coprocessorBuilder.setName(coprocessor).build());
1342 }
1343 RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder();
1344 RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
1345 for (Region region : regions) {
1346 if (region.getCoprocessorHost() != null) {
1347 Set<String> regionCoprocessors = region.getCoprocessorHost().getCoprocessors();
1348 Iterator<String> iterator = regionCoprocessors.iterator();
1349 while (iterator.hasNext()) {
1350 serverLoad.addCoprocessors(coprocessorBuilder.setName(iterator.next()).build());
1351 }
1352 }
1353 serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier));
1354 for (String coprocessor : getWAL(region.getRegionInfo()).getCoprocessorHost()
1355 .getCoprocessors()) {
1356 serverLoad.addCoprocessors(coprocessorBuilder.setName(coprocessor).build());
1357 }
1358 }
1359 serverLoad.setReportStartTime(reportStartTime);
1360 serverLoad.setReportEndTime(reportEndTime);
1361 if (this.infoServer != null) {
1362 serverLoad.setInfoServerPort(this.infoServer.getPort());
1363 } else {
1364 serverLoad.setInfoServerPort(-1);
1365 }
1366
1367
1368
1369 ReplicationSourceService rsources = getReplicationSourceService();
1370
1371 if (rsources != null) {
1372
1373 ReplicationLoad rLoad = rsources.refreshAndGetReplicationLoad();
1374 if (rLoad != null) {
1375 serverLoad.setReplLoadSink(rLoad.getReplicationLoadSink());
1376 for (ClusterStatusProtos.ReplicationLoadSource rLS : rLoad.getReplicationLoadSourceList()) {
1377 serverLoad.addReplLoadSource(rLS);
1378 }
1379 }
1380 }
1381
1382 return serverLoad.build();
1383 }
1384
1385 String getOnlineRegionsAsPrintableString() {
1386 StringBuilder sb = new StringBuilder();
1387 for (Region r: this.onlineRegions.values()) {
1388 if (sb.length() > 0) sb.append(", ");
1389 sb.append(r.getRegionInfo().getEncodedName());
1390 }
1391 return sb.toString();
1392 }
1393
1394
1395
1396
1397 private void waitOnAllRegionsToClose(final boolean abort) {
1398
1399 int lastCount = -1;
1400 long previousLogTime = 0;
1401 Set<String> closedRegions = new HashSet<String>();
1402 boolean interrupted = false;
1403 try {
1404 while (!isOnlineRegionsEmpty()) {
1405 int count = getNumberOfOnlineRegions();
1406
1407 if (count != lastCount) {
1408
1409 if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1410 previousLogTime = System.currentTimeMillis();
1411 lastCount = count;
1412 LOG.info("Waiting on " + count + " regions to close");
1413
1414
1415 if (count < 10 && LOG.isDebugEnabled()) {
1416 LOG.debug(this.onlineRegions);
1417 }
1418 }
1419 }
1420
1421
1422
1423 for (Map.Entry<String, Region> e : this.onlineRegions.entrySet()) {
1424 HRegionInfo hri = e.getValue().getRegionInfo();
1425 if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
1426 && !closedRegions.contains(hri.getEncodedName())) {
1427 closedRegions.add(hri.getEncodedName());
1428
1429 closeRegionIgnoreErrors(hri, abort);
1430 }
1431 }
1432
1433 if (this.regionsInTransitionInRS.isEmpty()) {
1434 if (!isOnlineRegionsEmpty()) {
1435 LOG.info("We were exiting though online regions are not empty," +
1436 " because some regions failed closing");
1437 }
1438 break;
1439 }
1440 if (sleep(200)) {
1441 interrupted = true;
1442 }
1443 }
1444 } finally {
1445 if (interrupted) {
1446 Thread.currentThread().interrupt();
1447 }
1448 }
1449 }
1450
1451 private boolean sleep(long millis) {
1452 boolean interrupted = false;
1453 try {
1454 Thread.sleep(millis);
1455 } catch (InterruptedException e) {
1456 LOG.warn("Interrupted while sleeping");
1457 interrupted = true;
1458 }
1459 return interrupted;
1460 }
1461
1462 private void shutdownWAL(final boolean close) {
1463 if (this.walFactory != null) {
1464 try {
1465 if (close) {
1466 walFactory.close();
1467 } else {
1468 walFactory.shutdown();
1469 }
1470 } catch (Throwable e) {
1471 e = RemoteExceptionHandler.checkThrowable(e);
1472 LOG.error("Shutdown / close of WAL failed: " + e);
1473 LOG.debug("Shutdown / close exception details:", e);
1474 }
1475 }
1476 }
1477
1478
1479
1480
1481
1482
1483 public NamedQueueRecorder getNamedQueueRecorder() {
1484 return this.namedQueueRecorder;
1485 }
1486
1487
1488
1489
1490
1491
1492 protected void handleReportForDutyResponse(final RegionServerStartupResponse c)
1493 throws IOException {
1494 try {
1495 boolean updateRootDir = false;
1496 for (NameStringPair e : c.getMapEntriesList()) {
1497 String key = e.getName();
1498
1499 if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1500 String hostnameFromMasterPOV = e.getValue();
1501 this.serverName = ServerName.valueOf(hostnameFromMasterPOV,
1502 rpcServices.isa.getPort(), this.startcode);
1503 if (shouldUseThisHostnameInstead() &&
1504 !hostnameFromMasterPOV.equals(useThisHostnameInstead)) {
1505 String msg = "Master passed us a different hostname to use; was=" +
1506 this.useThisHostnameInstead + ", but now=" + hostnameFromMasterPOV;
1507 LOG.error(msg);
1508 throw new IOException(msg);
1509 }
1510 if (!shouldUseThisHostnameInstead() &&
1511 !hostnameFromMasterPOV.equals(rpcServices.isa.getHostName())) {
1512 String msg = "Master passed us a different hostname to use; was=" +
1513 rpcServices.isa.getHostName() + ", but now=" + hostnameFromMasterPOV;
1514 LOG.error(msg);
1515 }
1516 continue;
1517 }
1518 String value = e.getValue();
1519 if (key.equals(HConstants.HBASE_DIR)) {
1520 if (value != null && !value.equals(conf.get(HConstants.HBASE_DIR))) {
1521 updateRootDir = true;
1522 }
1523 }
1524 if (LOG.isDebugEnabled()) {
1525 LOG.info("Config from master: " + key + "=" + value);
1526 }
1527 this.conf.set(key, value);
1528 }
1529
1530 createMyEphemeralNode();
1531
1532 if (updateRootDir) {
1533
1534 initializeFileSystem();
1535 }
1536
1537
1538
1539 if (this.conf.get("mapreduce.task.attempt.id") == null) {
1540 this.conf.set("mapreduce.task.attempt.id", "hb_rs_" +
1541 this.serverName.toString());
1542 }
1543
1544
1545 ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
1546
1547 this.cacheConfig = new CacheConfig(conf);
1548 this.walFactory = setupWALAndReplication();
1549
1550 this.metricsRegionServerImpl = new MetricsRegionServerWrapperImpl(this);
1551 this.metricsRegionServer = new MetricsRegionServer(metricsRegionServerImpl, conf);
1552 this.metricsTable = new MetricsTable(new MetricsTableWrapperAggregateImpl(this));
1553
1554 this.pauseMonitor = new JvmPauseMonitor(conf, getMetrics().getMetricsSource());
1555 pauseMonitor.start();
1556
1557 startServiceThreads();
1558
1559
1560 if (this.conf.getBoolean(HConstants.EXECUTOR_STATUS_COLLECT_ENABLED,
1561 HConstants.DEFAULT_EXECUTOR_STATUS_COLLECT_ENABLED)) {
1562 int sleepTime = this.conf.getInt(ExecutorStatusChore.WAKE_FREQ,
1563 ExecutorStatusChore.DEFAULT_WAKE_FREQ);
1564 executorStatusChore = new ExecutorStatusChore(sleepTime, this, this.getExecutorService(),
1565 this.getRegionServerMetrics().getMetricsSource());
1566 }
1567
1568 startHeapMemoryManager();
1569 LOG.info("Serving as " + this.serverName +
1570 ", RpcServer on " + rpcServices.isa +
1571 ", sessionid=0x" +
1572 Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1573
1574
1575 synchronized (online) {
1576 online.set(true);
1577 online.notifyAll();
1578 }
1579 } catch (Throwable e) {
1580 stop("Failed initialization");
1581 throw convertThrowableToIOE(cleanup(e, "Failed init"),
1582 "Region server startup failed");
1583 } finally {
1584 sleeper.skipSleepCycle();
1585 }
1586 }
1587
1588 private void startHeapMemoryManager() {
1589 this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher,
1590 this, this.regionServerAccounting);
1591 if (this.hMemManager != null) {
1592 this.hMemManager.start(getChoreService());
1593 }
1594 }
1595
1596 private void createMyEphemeralNode() throws KeeperException, IOException {
1597 RegionServerInfo.Builder rsInfo = RegionServerInfo.newBuilder();
1598 rsInfo.setInfoPort(infoServer != null ? infoServer.getPort() : -1);
1599 rsInfo.setVersionInfo(ProtobufUtil.getVersionInfo());
1600 byte[] data = ProtobufUtil.prependPBMagic(rsInfo.build().toByteArray());
1601 ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper,
1602 getMyEphemeralNodePath(), data);
1603 }
1604
1605 private void deleteMyEphemeralNode() throws KeeperException {
1606 ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1607 }
1608
1609 @Override
1610 public RegionServerAccounting getRegionServerAccounting() {
1611 return regionServerAccounting;
1612 }
1613
1614 @Override
1615 public TableLockManager getTableLockManager() {
1616 return tableLockManager;
1617 }
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627 private RegionLoad createRegionLoad(final Region r, RegionLoad.Builder regionLoadBldr,
1628 RegionSpecifier.Builder regionSpecifier) throws IOException {
1629 byte[] name = r.getRegionInfo().getRegionName();
1630 int stores = 0;
1631 int storefiles = 0;
1632 int storeRefCount = 0;
1633 int maxCompactedStoreFileRefCount = 0;
1634 int storeUncompressedSizeMB = 0;
1635 int storefileSizeMB = 0;
1636 int memstoreSizeMB = (int) (r.getMemstoreSize() / 1024 / 1024);
1637 int storefileIndexSizeMB = 0;
1638 int rootIndexSizeKB = 0;
1639 int totalStaticIndexSizeKB = 0;
1640 int totalStaticBloomSizeKB = 0;
1641 long totalCompactingKVs = 0;
1642 long currentCompactedKVs = 0;
1643 List<Store> storeList = r.getStores();
1644 stores += storeList.size();
1645 for (Store store : storeList) {
1646 storefiles += store.getStorefilesCount();
1647 if (store instanceof HStore) {
1648 HStore hStore = (HStore) store;
1649 int currentStoreRefCount = hStore.getStoreRefCount();
1650 storeRefCount += currentStoreRefCount;
1651 int currentMaxCompactedStoreFileRefCount = hStore.getMaxCompactedStoreFileRefCount();
1652 maxCompactedStoreFileRefCount = Math.max(maxCompactedStoreFileRefCount,
1653 currentMaxCompactedStoreFileRefCount);
1654 }
1655 storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024);
1656 storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1657 storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
1658 CompactionProgress progress = store.getCompactionProgress();
1659 if (progress != null) {
1660 totalCompactingKVs += progress.totalCompactingKVs;
1661 currentCompactedKVs += progress.currentCompactedKVs;
1662 }
1663 rootIndexSizeKB += (int) (store.getStorefilesIndexSize() / 1024);
1664 totalStaticIndexSizeKB += (int) (store.getTotalStaticIndexSize() / 1024);
1665 totalStaticBloomSizeKB += (int) (store.getTotalStaticBloomSize() / 1024);
1666 }
1667
1668 float dataLocality =
1669 r.getHDFSBlocksDistribution().getBlockLocalityIndex(serverName.getHostname());
1670 if (regionLoadBldr == null) {
1671 regionLoadBldr = RegionLoad.newBuilder();
1672 }
1673 if (regionSpecifier == null) {
1674 regionSpecifier = RegionSpecifier.newBuilder();
1675 }
1676 regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
1677 regionSpecifier.setValue(ByteStringer.wrap(name));
1678 regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
1679 .setStores(stores)
1680 .setStorefiles(storefiles)
1681 .setStoreRefCount(storeRefCount)
1682 .setMaxCompactedStoreFileRefCount(maxCompactedStoreFileRefCount)
1683 .setStoreUncompressedSizeMB(storeUncompressedSizeMB)
1684 .setStorefileSizeMB(storefileSizeMB)
1685 .setMemstoreSizeMB(memstoreSizeMB)
1686 .setStorefileIndexSizeMB(storefileIndexSizeMB)
1687 .setRootIndexSizeKB(rootIndexSizeKB)
1688 .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
1689 .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
1690 .setReadRequestsCount(r.getReadRequestsCount())
1691 .setWriteRequestsCount(r.getWriteRequestsCount())
1692 .setTotalCompactingKVs(totalCompactingKVs)
1693 .setCurrentCompactedKVs(currentCompactedKVs)
1694 .setDataLocality(dataLocality)
1695 .setLastMajorCompactionTs(r.getOldestHfileTs(true));
1696 ((HRegion)r).setCompleteSequenceId(regionLoadBldr);
1697
1698 return regionLoadBldr.build();
1699 }
1700
1701
1702
1703
1704
1705 public RegionLoad createRegionLoad(final String encodedRegionName) throws IOException {
1706 Region r = onlineRegions.get(encodedRegionName);
1707 return r != null ? createRegionLoad(r, null, null) : null;
1708 }
1709
1710
1711
1712
1713 private static class CompactionChecker extends ScheduledChore {
1714 private final HRegionServer instance;
1715 private final int majorCompactPriority;
1716 private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1717
1718
1719 private long iteration = 1;
1720
1721 CompactionChecker(final HRegionServer h, final int sleepTime,
1722 final Stoppable stopper) {
1723 super("CompactionChecker", stopper, sleepTime);
1724 this.instance = h;
1725 LOG.info(this.getName() + " runs every " + StringUtils.formatTime(sleepTime));
1726
1727
1728
1729
1730 this.majorCompactPriority = this.instance.conf.
1731 getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1732 DEFAULT_PRIORITY);
1733 }
1734
1735 @Override
1736 protected void chore() {
1737 for (Region r : this.instance.onlineRegions.values()) {
1738 if (r == null)
1739 continue;
1740 for (Store s : r.getStores()) {
1741 try {
1742 long multiplier = s.getCompactionCheckMultiplier();
1743 assert multiplier > 0;
1744 if (iteration % multiplier != 0) continue;
1745 if (s.needsCompaction()) {
1746
1747 this.instance.compactSplitThread.requestSystemCompaction(r, s, getName()
1748 + " requests compaction");
1749 } else if (s.isMajorCompaction()) {
1750 s.triggerMajorCompaction();
1751 if (majorCompactPriority == DEFAULT_PRIORITY
1752 || majorCompactPriority > ((HRegion)r).getCompactPriority()) {
1753 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1754 + " requests major compaction; use default priority", null);
1755 } else {
1756 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1757 + " requests major compaction; use configured priority",
1758 this.majorCompactPriority, null, null);
1759 }
1760 }
1761 } catch (IOException e) {
1762 LOG.warn("Failed major compaction check on " + r, e);
1763 }
1764 }
1765 }
1766 iteration = (iteration == Long.MAX_VALUE) ? 0 : (iteration + 1);
1767 }
1768 }
1769
1770 static class PeriodicMemstoreFlusher extends ScheduledChore {
1771 final HRegionServer server;
1772 final static int RANGE_OF_DELAY = 5 * 60;
1773 final static int MIN_DELAY_TIME = 0;
1774 final int rangeOfDelay;
1775 public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1776 super(server.getServerName() + "-MemstoreFlusherChore", server, cacheFlushInterval);
1777 this.server = server;
1778
1779 this.rangeOfDelay = this.server.conf.getInt("hbase.regionserver.periodicmemstoreflusher.rangeofdelayseconds",
1780 RANGE_OF_DELAY)*1000;
1781 }
1782
1783 @Override
1784 protected void chore() {
1785 final StringBuffer whyFlush = new StringBuffer();
1786 for (Region r : this.server.onlineRegions.values()) {
1787 if (r == null) continue;
1788 if (((HRegion) r).shouldFlush(whyFlush)) {
1789 FlushRequester requester = server.getFlushRequester();
1790 if (requester != null) {
1791 long randomDelay = (long) RandomUtils.nextInt(rangeOfDelay) + MIN_DELAY_TIME;
1792
1793
1794
1795 if (requester.requestDelayedFlush(r, randomDelay, false)) {
1796 LOG.info(MessageFormat.format("{0} requesting flush of {1} because {2} " +
1797 "after random delay {3} ms", getName(),
1798 r.getRegionInfo().getRegionNameAsString(), whyFlush.toString(), randomDelay));
1799 }
1800 }
1801 }
1802 }
1803 }
1804 }
1805
1806
1807
1808
1809
1810
1811
1812
1813 public boolean isOnline() {
1814 return online.get();
1815 }
1816
1817
1818
1819
1820
1821
1822
1823 private WALFactory setupWALAndReplication() throws IOException {
1824
1825 final Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1826 final String logName = DefaultWALProvider.getWALDirectoryName(this.serverName.toString());
1827
1828 Path logDir = new Path(walRootDir, logName);
1829 if (LOG.isDebugEnabled()) LOG.debug("logDir=" + logDir);
1830 if (this.walFs.exists(logDir)) {
1831 throw new RegionServerRunningException("Region server has already " +
1832 "created directory at " + this.serverName.toString());
1833 }
1834
1835
1836
1837 createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir);
1838
1839
1840 final List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1841 listeners.add(new MetricsWAL());
1842 if (this.replicationSourceHandler != null &&
1843 this.replicationSourceHandler.getWALActionsListener() != null) {
1844
1845 listeners.add(this.replicationSourceHandler.getWALActionsListener());
1846 }
1847
1848 return new WALFactory(conf, listeners, serverName.toString());
1849 }
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859 protected LogRoller ensureMetaWALRoller() {
1860
1861
1862 LogRoller roller = metawalRoller.get();
1863 if (null == roller) {
1864 LogRoller tmpLogRoller = new LogRoller(this, this);
1865 String n = Thread.currentThread().getName();
1866 Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
1867 n + "-MetaLogRoller", uncaughtExceptionHandler);
1868 if (metawalRoller.compareAndSet(null, tmpLogRoller)) {
1869 roller = tmpLogRoller;
1870 } else {
1871
1872 Threads.shutdown(tmpLogRoller.getThread());
1873 roller = metawalRoller.get();
1874 }
1875 }
1876 return roller;
1877 }
1878
1879 public MetricsRegionServer getRegionServerMetrics() {
1880 return this.metricsRegionServer;
1881 }
1882
1883
1884
1885
1886 public MasterAddressTracker getMasterAddressTracker() {
1887 return this.masterAddressTracker;
1888 }
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902 private void startServiceThreads() throws IOException {
1903
1904 this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1905 conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1906 this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1907 conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1908 this.service.startExecutorService(ExecutorType.RS_OPEN_PRIORITY_REGION,
1909 conf.getInt("hbase.regionserver.executor.openpriorityregion.threads", 3));
1910 this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1911 conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1912 this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1913 conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1914 if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
1915 this.service.startExecutorService(ExecutorType.RS_PARALLEL_SEEK,
1916 conf.getInt("hbase.storescanner.parallel.seek.threads", 10));
1917 }
1918 this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, conf.getInt(
1919 "hbase.regionserver.wal.max.splitters", SplitLogWorkerCoordination.DEFAULT_MAX_SPLITTERS));
1920
1921 this.service.startExecutorService(ExecutorType.RS_COMPACTED_FILES_DISCHARGER,
1922 conf.getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT, 10));
1923 if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) {
1924 this.service.startExecutorService(ExecutorType.RS_REGION_REPLICA_FLUSH_OPS,
1925 conf.getInt("hbase.regionserver.region.replica.flusher.threads",
1926 conf.getInt("hbase.regionserver.executor.openregion.threads", 3)));
1927 }
1928
1929 Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
1930 uncaughtExceptionHandler);
1931 if (this.cacheFlusher != null) {
1932 this.cacheFlusher.start(uncaughtExceptionHandler);
1933 }
1934 if (this.compactionChecker != null) {
1935 choreService.scheduleChore(compactionChecker);
1936 }
1937 if (this.periodicFlusher != null) {
1938 choreService.scheduleChore(periodicFlusher);
1939 }
1940 if (this.healthCheckChore != null) {
1941 choreService.scheduleChore(healthCheckChore);
1942 }
1943 if (this.executorStatusChore != null) {
1944 choreService.scheduleChore(executorStatusChore);
1945 }
1946 if (this.nonceManagerChore != null) {
1947 choreService.scheduleChore(nonceManagerChore);
1948 }
1949 if (this.storefileRefresher != null) {
1950 choreService.scheduleChore(storefileRefresher);
1951 }
1952 if (this.movedRegionsCleaner != null) {
1953 choreService.scheduleChore(movedRegionsCleaner);
1954 }
1955 if (this.slowLogTableOpsChore != null) {
1956 choreService.scheduleChore(slowLogTableOpsChore);
1957 }
1958
1959
1960
1961 Threads.setDaemonThreadRunning(this.leases.getThread(), getName() + ".leaseChecker",
1962 uncaughtExceptionHandler);
1963
1964 if (this.replicationSourceHandler == this.replicationSinkHandler &&
1965 this.replicationSourceHandler != null) {
1966 this.replicationSourceHandler.startReplicationService();
1967 } else {
1968 if (this.replicationSourceHandler != null) {
1969 this.replicationSourceHandler.startReplicationService();
1970 }
1971 if (this.replicationSinkHandler != null) {
1972 this.replicationSinkHandler.startReplicationService();
1973 }
1974 }
1975
1976
1977
1978
1979
1980 Configuration sinkConf = HBaseConfiguration.create(conf);
1981 sinkConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
1982 conf.getInt("hbase.log.replay.retries.number", 8));
1983 sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
1984 conf.getInt("hbase.log.replay.rpc.timeout", 30000));
1985 sinkConf.setInt("hbase.client.serverside.retries.multiplier", 1);
1986 this.splitLogWorker = new SplitLogWorker(this, sinkConf, this, this, walFactory);
1987 splitLogWorker.start();
1988 }
1989
1990
1991
1992
1993
1994
1995 private int putUpWebUI() throws IOException {
1996 int port = this.conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
1997 HConstants.DEFAULT_REGIONSERVER_INFOPORT);
1998 String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1999
2000 if(this instanceof HMaster) {
2001 port = conf.getInt(HConstants.MASTER_INFO_PORT,
2002 HConstants.DEFAULT_MASTER_INFOPORT);
2003 addr = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
2004 }
2005
2006 if (port < 0) return port;
2007
2008 if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
2009 String msg =
2010 "Failed to start http info server. Address " + addr
2011 + " does not belong to this host. Correct configuration parameter: "
2012 + "hbase.regionserver.info.bindAddress";
2013 LOG.error(msg);
2014 throw new IOException(msg);
2015 }
2016
2017 boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
2018 false);
2019 while (true) {
2020 try {
2021 this.infoServer = new InfoServer(getProcessName(), addr, port, false, this.conf);
2022 infoServer.addServlet("dump", "/dump", getDumpServlet());
2023 configureInfoServer();
2024 this.infoServer.start();
2025 break;
2026 } catch (BindException e) {
2027 if (!auto) {
2028
2029 LOG.error("Failed binding http info server to port: " + port);
2030 throw e;
2031 }
2032
2033 LOG.info("Failed binding http info server to port: " + port);
2034 port++;
2035 LOG.info("Retry starting http info server with port: " + port);
2036 }
2037 }
2038 port = this.infoServer.getPort();
2039 conf.setInt(HConstants.REGIONSERVER_INFO_PORT, port);
2040 int masterInfoPort = conf.getInt(HConstants.MASTER_INFO_PORT,
2041 HConstants.DEFAULT_MASTER_INFOPORT);
2042 conf.setInt("hbase.master.info.port.orig", masterInfoPort);
2043 conf.setInt(HConstants.MASTER_INFO_PORT, port);
2044 return port;
2045 }
2046
2047
2048
2049
2050 private boolean isHealthy() {
2051 if (!fsOk) {
2052
2053 return false;
2054 }
2055
2056 if (!(leases.isAlive()
2057 && cacheFlusher.isAlive() && walRoller.isAlive()
2058 && this.compactionChecker.isScheduled()
2059 && this.periodicFlusher.isScheduled())) {
2060 stop("One or more threads are no longer alive -- stop");
2061 return false;
2062 }
2063 final LogRoller metawalRoller = this.metawalRoller.get();
2064 if (metawalRoller != null && !metawalRoller.isAlive()) {
2065 stop("Meta WAL roller thread is no longer alive -- stop");
2066 return false;
2067 }
2068 return true;
2069 }
2070
2071 private static final byte[] UNSPECIFIED_REGION = new byte[]{};
2072
2073 public List<WAL> getWALs() throws IOException {
2074 return walFactory.getWALs();
2075 }
2076
2077 @Override
2078 public WAL getWAL(HRegionInfo regionInfo) throws IOException {
2079 WAL wal;
2080 LogRoller roller = walRoller;
2081
2082 if (regionInfo != null && regionInfo.isMetaTable() &&
2083 regionInfo.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
2084 roller = ensureMetaWALRoller();
2085 wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
2086 } else if (regionInfo == null) {
2087 wal = walFactory.getWAL(UNSPECIFIED_REGION, null);
2088 } else {
2089 byte[] namespace = regionInfo.getTable().getNamespace();
2090 wal = walFactory.getWAL(regionInfo.getEncodedNameAsBytes(), namespace);
2091 }
2092 roller.addWAL(wal);
2093 return wal;
2094 }
2095
2096 @Override
2097 public ClusterConnection getConnection() {
2098 return this.clusterConnection;
2099 }
2100
2101 @Override
2102 public MetaTableLocator getMetaTableLocator() {
2103 return this.metaTableLocator;
2104 }
2105
2106 @Override
2107 public void stop(final String msg) {
2108 stop(msg, false);
2109 }
2110
2111
2112
2113
2114
2115
2116 public void stop(final String msg, final boolean force) {
2117 if (!this.stopped) {
2118 if (this.rsHost != null) {
2119
2120 try {
2121 this.rsHost.preStop(msg);
2122 } catch (IOException ioe) {
2123 if (!force) {
2124 LOG.warn("The region server did not stop", ioe);
2125 return;
2126 }
2127 LOG.warn("Skipping coprocessor exception on preStop() due to forced shutdown", ioe);
2128 }
2129 }
2130 this.stopped = true;
2131 LOG.info("STOPPED: " + msg);
2132
2133 sleeper.skipSleepCycle();
2134 }
2135 }
2136
2137 public void waitForServerOnline(){
2138 while (!isStopped() && !isOnline()) {
2139 synchronized (online) {
2140 try {
2141 online.wait(msgInterval);
2142 } catch (InterruptedException ie) {
2143 Thread.currentThread().interrupt();
2144 break;
2145 }
2146 }
2147 }
2148 }
2149
2150 @Override
2151 public void postOpenDeployTasks(final Region r) throws KeeperException, IOException {
2152 postOpenDeployTasks(new PostOpenDeployContext(r, -1));
2153 }
2154
2155 @Override
2156 public void postOpenDeployTasks(final PostOpenDeployContext context)
2157 throws KeeperException, IOException {
2158 Region r = context.getRegion();
2159 long masterSystemTime = context.getMasterSystemTime();
2160 Preconditions.checkArgument(r instanceof HRegion, "r must be an HRegion");
2161 rpcServices.checkOpen();
2162 LOG.info("Post open deploy tasks for " + r.getRegionInfo().getRegionNameAsString());
2163
2164 for (Store s : r.getStores()) {
2165 if (s.hasReferences() || s.needsCompaction()) {
2166 this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
2167 }
2168 }
2169 long openSeqNum = r.getOpenSeqNum();
2170 if (openSeqNum == HConstants.NO_SEQNUM) {
2171
2172 LOG.error("No sequence number found when opening " +
2173 r.getRegionInfo().getRegionNameAsString());
2174 openSeqNum = 0;
2175 }
2176
2177
2178 updateRecoveringRegionLastFlushedSequenceId(r);
2179
2180
2181 if (r.getRegionInfo().isMetaRegion()) {
2182 MetaTableLocator.setMetaLocation(getZooKeeper(), serverName, r.getRegionInfo().getReplicaId(),
2183 State.OPEN);
2184 } else if (useZKForAssignment) {
2185 MetaTableAccessor.updateRegionLocation(getConnection(), r.getRegionInfo(),
2186 this.serverName, openSeqNum, masterSystemTime);
2187 }
2188 if (!useZKForAssignment && !reportRegionStateTransition(new RegionStateTransitionContext(
2189 TransitionCode.OPENED, openSeqNum, masterSystemTime, r.getRegionInfo()))) {
2190 throw new IOException("Failed to report opened region to master: "
2191 + r.getRegionInfo().getRegionNameAsString());
2192 }
2193
2194 triggerFlushInPrimaryRegion((HRegion)r);
2195
2196 LOG.debug("Finished post open deploy task for " + r.getRegionInfo().getRegionNameAsString());
2197 }
2198
2199 @Override
2200 public boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris) {
2201 return reportRegionStateTransition(code, HConstants.NO_SEQNUM, hris);
2202 }
2203
2204 @Override
2205 public boolean reportRegionStateTransition(
2206 TransitionCode code, long openSeqNum, HRegionInfo... hris) {
2207 return reportRegionStateTransition(
2208 new RegionStateTransitionContext(code, HConstants.NO_SEQNUM, -1, hris));
2209 }
2210
2211 @Override
2212 public boolean reportRegionStateTransition(final RegionStateTransitionContext context) {
2213 TransitionCode code = context.getCode();
2214 long openSeqNum = context.getOpenSeqNum();
2215 HRegionInfo[] hris = context.getHris();
2216
2217 ReportRegionStateTransitionRequest.Builder builder =
2218 ReportRegionStateTransitionRequest.newBuilder();
2219 builder.setServer(ProtobufUtil.toServerName(serverName));
2220 RegionStateTransition.Builder transition = builder.addTransitionBuilder();
2221 transition.setTransitionCode(code);
2222 if (code == TransitionCode.OPENED && openSeqNum >= 0) {
2223 transition.setOpenSeqNum(openSeqNum);
2224 }
2225 for (HRegionInfo hri: hris) {
2226 transition.addRegionInfo(HRegionInfo.convert(hri));
2227 }
2228 ReportRegionStateTransitionRequest request = builder.build();
2229 while (keepLooping()) {
2230 RegionServerStatusService.BlockingInterface rss = rssStub;
2231 try {
2232 if (rss == null) {
2233 createRegionServerStatusStub();
2234 continue;
2235 }
2236 ReportRegionStateTransitionResponse response =
2237 rss.reportRegionStateTransition(null, request);
2238 if (response.hasErrorMessage()) {
2239 LOG.info("Failed to transition " + hris[0]
2240 + " to " + code + ": " + response.getErrorMessage());
2241 return false;
2242 }
2243 return true;
2244 } catch (ServiceException se) {
2245 IOException ioe = ProtobufUtil.getRemoteException(se);
2246 LOG.info("Failed to report region transition, will retry", ioe);
2247 if (rssStub == rss) {
2248 rssStub = null;
2249 }
2250 }
2251 }
2252 return false;
2253 }
2254
2255
2256
2257
2258
2259 void triggerFlushInPrimaryRegion(final HRegion region) {
2260 if (ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())) {
2261 return;
2262 }
2263 if (!ServerRegionReplicaUtil.isRegionReplicaReplicationEnabled(region.conf) ||
2264 !ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(
2265 region.conf)) {
2266 region.setReadsEnabled(true);
2267 return;
2268 }
2269
2270 region.setReadsEnabled(false);
2271
2272
2273
2274 this.service.submit(
2275 new RegionReplicaFlushHandler(this, clusterConnection,
2276 rpcRetryingCallerFactory, rpcControllerFactory, operationTimeout, region));
2277 }
2278
2279 @Override
2280 public RpcServerInterface getRpcServer() {
2281 return rpcServices.rpcServer;
2282 }
2283
2284 @InterfaceAudience.Private
2285 public RSRpcServices getRSRpcServices() {
2286 return rpcServices;
2287 }
2288
2289
2290
2291
2292
2293
2294 protected boolean setAbortRequested() {
2295 return abortRequested.compareAndSet(false, true);
2296 }
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308 @Override
2309 public void abort(final String reason, Throwable cause) {
2310 if (!setAbortRequested()) {
2311
2312 LOG.debug(String.format(
2313 "Abort already in progress. Ignoring the current request with reason: %s", reason));
2314 return;
2315 }
2316 String msg = "ABORTING region server " + this + ": " + reason;
2317 if (cause != null) {
2318 LOG.fatal(msg, cause);
2319 } else {
2320 LOG.fatal(msg);
2321 }
2322
2323
2324
2325 LOG.fatal("RegionServer abort: loaded coprocessors are: " +
2326 CoprocessorHost.getLoadedCoprocessors());
2327
2328 try {
2329 LOG.info("Dump of metrics as JSON on abort: " + JSONBean.dumpRegionServerMetrics());
2330 } catch (MalformedObjectNameException | IOException e) {
2331 LOG.warn("Failed dumping metrics", e);
2332 }
2333
2334
2335 try {
2336 if (cause != null) {
2337 msg += "\nCause:\n" + StringUtils.stringifyException(cause);
2338 }
2339
2340 if (rssStub != null && this.serverName != null) {
2341 ReportRSFatalErrorRequest.Builder builder =
2342 ReportRSFatalErrorRequest.newBuilder();
2343 builder.setServer(ProtobufUtil.toServerName(this.serverName));
2344 builder.setErrorMessage(msg);
2345 rssStub.reportRSFatalError(null, builder.build());
2346 }
2347 } catch (Throwable t) {
2348 LOG.warn("Unable to report fatal error to master", t);
2349 }
2350
2351 if (User.isHBaseSecurityEnabled(conf)) {
2352 try {
2353 User.runAsLoginUser(new PrivilegedExceptionAction<Object>() {
2354 @Override
2355 public Object run() throws Exception {
2356 stop(reason, true);
2357 return null;
2358 }
2359 });
2360 } catch (IOException neverThrown) {
2361 }
2362 } else {
2363 stop(reason, true);
2364 }
2365 }
2366
2367
2368
2369
2370 public void abort(String reason) {
2371 abort(reason, null);
2372 }
2373
2374 @Override
2375 public boolean isAborted() {
2376 return this.abortRequested.get();
2377 }
2378
2379
2380
2381
2382
2383
2384 @InterfaceAudience.Private
2385 protected void kill() {
2386 this.killed = true;
2387 abort("Simulated kill");
2388 }
2389
2390
2391
2392
2393 protected void sendShutdownInterrupt() {
2394 }
2395
2396
2397
2398
2399
2400 protected void stopServiceThreads() {
2401
2402 if (this.choreService != null) {
2403 choreService.cancelChore(nonceManagerChore);
2404 choreService.cancelChore(compactionChecker);
2405 choreService.cancelChore(periodicFlusher);
2406 choreService.cancelChore(healthCheckChore);
2407 choreService.cancelChore(executorStatusChore);
2408 choreService.cancelChore(storefileRefresher);
2409 choreService.cancelChore(movedRegionsCleaner);
2410 choreService.cancelChore(slowLogTableOpsChore);
2411
2412 choreService.shutdown();
2413 }
2414
2415 if (this.cacheFlusher != null) {
2416 this.cacheFlusher.join();
2417 }
2418
2419 if (this.spanReceiverHost != null) {
2420 this.spanReceiverHost.closeReceivers();
2421 }
2422 if (this.walRoller != null) {
2423 Threads.shutdown(this.walRoller.getThread());
2424 }
2425 final LogRoller metawalRoller = this.metawalRoller.get();
2426 if (metawalRoller != null) {
2427 Threads.shutdown(metawalRoller.getThread());
2428 }
2429 if (this.compactSplitThread != null) {
2430 this.compactSplitThread.join();
2431 }
2432 if (this.service != null) this.service.shutdown();
2433 if (this.replicationSourceHandler != null &&
2434 this.replicationSourceHandler == this.replicationSinkHandler) {
2435 this.replicationSourceHandler.stopReplicationService();
2436 } else {
2437 if (this.replicationSourceHandler != null) {
2438 this.replicationSourceHandler.stopReplicationService();
2439 }
2440 if (this.replicationSinkHandler != null) {
2441 this.replicationSinkHandler.stopReplicationService();
2442 }
2443 }
2444 }
2445
2446
2447
2448
2449
2450 @InterfaceAudience.Private
2451 public ReplicationSourceService getReplicationSourceService() {
2452 return replicationSourceHandler;
2453 }
2454
2455
2456
2457
2458
2459 ReplicationSinkService getReplicationSinkService() {
2460 return replicationSinkHandler;
2461 }
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471 @InterfaceAudience.Private
2472 protected synchronized ServerName createRegionServerStatusStub() {
2473
2474 return createRegionServerStatusStub(false);
2475 }
2476
2477
2478
2479
2480
2481
2482
2483
2484 @InterfaceAudience.Private
2485 protected synchronized ServerName createRegionServerStatusStub(boolean refresh) {
2486 if (rssStub != null) {
2487 return masterAddressTracker.getMasterAddress();
2488 }
2489 ServerName sn = null;
2490 long previousLogTime = 0;
2491 RegionServerStatusService.BlockingInterface intf = null;
2492 boolean interrupted = false;
2493 try {
2494 while (keepLooping()) {
2495 sn = this.masterAddressTracker.getMasterAddress(refresh);
2496 if (sn == null) {
2497 if (!keepLooping()) {
2498
2499 LOG.debug("No master found and cluster is stopped; bailing out");
2500 return null;
2501 }
2502 if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2503 LOG.debug("No master found; retry");
2504 previousLogTime = System.currentTimeMillis();
2505 }
2506 refresh = true;
2507 if (sleep(200)) {
2508 interrupted = true;
2509 }
2510 continue;
2511 }
2512
2513
2514 if (this instanceof HMaster && sn.equals(getServerName())) {
2515 intf = ((HMaster)this).getMasterRpcServices();
2516 break;
2517 }
2518 try {
2519 BlockingRpcChannel channel =
2520 this.rpcClient.createBlockingRpcChannel(sn, userProvider.getCurrent(),
2521 shortOperationTimeout);
2522 intf = RegionServerStatusService.newBlockingStub(channel);
2523 break;
2524 } catch (IOException e) {
2525 if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2526 e = e instanceof RemoteException ?
2527 ((RemoteException)e).unwrapRemoteException() : e;
2528 if (e instanceof ServerNotRunningYetException) {
2529 LOG.info("Master isn't available yet, retrying");
2530 } else {
2531 LOG.warn("Unable to connect to master. Retrying. Error was:", e);
2532 }
2533 previousLogTime = System.currentTimeMillis();
2534 }
2535 if (sleep(200)) {
2536 interrupted = true;
2537 }
2538 }
2539 }
2540 } finally {
2541 if (interrupted) {
2542 Thread.currentThread().interrupt();
2543 }
2544 }
2545 rssStub = intf;
2546 return sn;
2547 }
2548
2549
2550
2551
2552
2553 private boolean keepLooping() {
2554 return !this.stopped && isClusterUp();
2555 }
2556
2557
2558
2559
2560
2561
2562
2563
2564 private RegionServerStartupResponse reportForDuty() throws IOException {
2565 ServerName masterServerName = createRegionServerStatusStub(true);
2566 if (masterServerName == null) return null;
2567 RegionServerStartupResponse result = null;
2568 try {
2569 rpcServices.requestCount.set(0);
2570 rpcServices.rpcGetRequestCount.set(0);
2571 rpcServices.rpcScanRequestCount.set(0);
2572 rpcServices.rpcMultiRequestCount.set(0);
2573 rpcServices.rpcMutateRequestCount.set(0);
2574 LOG.info("reportForDuty to master=" + masterServerName + " with port="
2575 + rpcServices.isa.getPort() + ", startcode=" + this.startcode);
2576 long now = EnvironmentEdgeManager.currentTime();
2577 int port = rpcServices.isa.getPort();
2578 RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
2579 if (shouldUseThisHostnameInstead()) {
2580 request.setUseThisHostnameInstead(useThisHostnameInstead);
2581 }
2582 request.setPort(port);
2583 request.setServerStartCode(this.startcode);
2584 request.setServerCurrentTime(now);
2585 result = this.rssStub.regionServerStartup(null, request.build());
2586 } catch (ServiceException se) {
2587 IOException ioe = ProtobufUtil.getRemoteException(se);
2588 if (ioe instanceof ClockOutOfSyncException) {
2589 LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2590
2591 throw ioe;
2592 } else if (ioe instanceof ServerNotRunningYetException) {
2593 LOG.debug("Master is not running yet");
2594 } else {
2595 LOG.warn("error telling master we are up", se);
2596 }
2597 rssStub = null;
2598 }
2599 return result;
2600 }
2601
2602 @Override
2603 public RegionStoreSequenceIds getLastSequenceId(byte[] encodedRegionName) {
2604 try {
2605 GetLastFlushedSequenceIdRequest req =
2606 RequestConverter.buildGetLastFlushedSequenceIdRequest(encodedRegionName);
2607 RegionServerStatusService.BlockingInterface rss = rssStub;
2608 if (rss == null) {
2609 createRegionServerStatusStub();
2610 rss = rssStub;
2611 if (rss == null) {
2612
2613 LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id");
2614 return RegionStoreSequenceIds.newBuilder().setLastFlushedSequenceId(HConstants.NO_SEQNUM)
2615 .build();
2616 }
2617 }
2618 GetLastFlushedSequenceIdResponse resp = rss.getLastFlushedSequenceId(null, req);
2619 return RegionStoreSequenceIds.newBuilder()
2620 .setLastFlushedSequenceId(resp.getLastFlushedSequenceId())
2621 .addAllStoreSequenceId(resp.getStoreLastFlushedSequenceIdList()).build();
2622 } catch (ServiceException e) {
2623 LOG.warn("Unable to connect to the master to check the last flushed sequence id", e);
2624 return RegionStoreSequenceIds.newBuilder().setLastFlushedSequenceId(HConstants.NO_SEQNUM)
2625 .build();
2626 }
2627 }
2628
2629
2630
2631
2632
2633
2634 protected void closeAllRegions(final boolean abort) {
2635 closeUserRegions(abort);
2636 closeMetaTableRegions(abort);
2637 }
2638
2639
2640
2641
2642
2643 void closeMetaTableRegions(final boolean abort) {
2644 Region meta = null;
2645 this.lock.writeLock().lock();
2646 try {
2647 for (Map.Entry<String, Region> e: onlineRegions.entrySet()) {
2648 HRegionInfo hri = e.getValue().getRegionInfo();
2649 if (hri.isMetaRegion()) {
2650 meta = e.getValue();
2651 }
2652 if (meta != null) break;
2653 }
2654 } finally {
2655 this.lock.writeLock().unlock();
2656 }
2657 if (meta != null) closeRegionIgnoreErrors(meta.getRegionInfo(), abort);
2658 }
2659
2660
2661
2662
2663
2664
2665
2666 void closeUserRegions(final boolean abort) {
2667 this.lock.writeLock().lock();
2668 try {
2669 for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
2670 Region r = e.getValue();
2671 if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2672
2673 closeRegionIgnoreErrors(r.getRegionInfo(), abort);
2674 }
2675 }
2676 } finally {
2677 this.lock.writeLock().unlock();
2678 }
2679 }
2680
2681
2682 public InfoServer getInfoServer() {
2683 return infoServer;
2684 }
2685
2686
2687
2688
2689 @Override
2690 public boolean isStopped() {
2691 return this.stopped;
2692 }
2693
2694 @Override
2695 public boolean isStopping() {
2696 return this.stopping;
2697 }
2698
2699 @Override
2700 public Map<String, Region> getRecoveringRegions() {
2701 return this.recoveringRegions;
2702 }
2703
2704
2705
2706
2707
2708 @Override
2709 public Configuration getConfiguration() {
2710 return conf;
2711 }
2712
2713
2714 ReentrantReadWriteLock.WriteLock getWriteLock() {
2715 return lock.writeLock();
2716 }
2717
2718 public int getNumberOfOnlineRegions() {
2719 return this.onlineRegions.size();
2720 }
2721
2722 boolean isOnlineRegionsEmpty() {
2723 return this.onlineRegions.isEmpty();
2724 }
2725
2726
2727
2728
2729
2730
2731 public Collection<Region> getOnlineRegionsLocalContext() {
2732 Collection<Region> regions = this.onlineRegions.values();
2733 return Collections.unmodifiableCollection(regions);
2734 }
2735
2736 @Override
2737 public void addToOnlineRegions(Region region) {
2738 this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2739 configurationManager.registerObserver(region);
2740 }
2741
2742
2743
2744
2745
2746
2747 SortedMap<Long, Region> getCopyOfOnlineRegionsSortedBySize() {
2748
2749 SortedMap<Long, Region> sortedRegions = new TreeMap<Long, Region>(
2750 new Comparator<Long>() {
2751 @Override
2752 public int compare(Long a, Long b) {
2753 return -1 * a.compareTo(b);
2754 }
2755 });
2756
2757 for (Region region : this.onlineRegions.values()) {
2758 sortedRegions.put(region.getMemstoreSize(), region);
2759 }
2760 return sortedRegions;
2761 }
2762
2763
2764
2765
2766 public long getStartcode() {
2767 return this.startcode;
2768 }
2769
2770
2771 @Override
2772 public FlushRequester getFlushRequester() {
2773 return this.cacheFlusher;
2774 }
2775
2776 @Override
2777 public Leases getLeases() {
2778 return leases;
2779 }
2780
2781
2782
2783
2784 protected Path getRootDir() {
2785 return rootDir;
2786 }
2787
2788
2789
2790
2791 @Override
2792 public FileSystem getFileSystem() {
2793 return fs;
2794 }
2795
2796
2797
2798
2799 protected Path getWALRootDir() {
2800 return walRootDir;
2801 }
2802
2803
2804
2805
2806 protected FileSystem getWALFileSystem() {
2807 return walFs;
2808 }
2809
2810 @Override
2811 public String toString() {
2812 return getServerName().toString();
2813 }
2814
2815
2816
2817
2818
2819
2820 public int getThreadWakeFrequency() {
2821 return threadWakeFrequency;
2822 }
2823
2824 @Override
2825 public ZooKeeperWatcher getZooKeeper() {
2826 return zooKeeper;
2827 }
2828
2829 @Override
2830 public BaseCoordinatedStateManager getCoordinatedStateManager() {
2831 return csm;
2832 }
2833
2834 @Override
2835 public ServerName getServerName() {
2836 return serverName;
2837 }
2838
2839 @Override
2840 public CompactionRequestor getCompactionRequester() {
2841 return this.compactSplitThread;
2842 }
2843
2844 public RegionServerCoprocessorHost getRegionServerCoprocessorHost(){
2845 return this.rsHost;
2846 }
2847
2848 @Override
2849 public ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS() {
2850 return this.regionsInTransitionInRS;
2851 }
2852
2853 @Override
2854 public ExecutorService getExecutorService() {
2855 return service;
2856 }
2857
2858 @Override
2859 public ChoreService getChoreService() {
2860 return choreService;
2861 }
2862
2863 @Override
2864 public RegionServerQuotaManager getRegionServerQuotaManager() {
2865 return rsQuotaManager;
2866 }
2867
2868
2869
2870
2871
2872
2873
2874
2875 static private void createNewReplicationInstance(Configuration conf,
2876 HRegionServer server, FileSystem walFs, Path walDir, Path oldWALDir) throws IOException{
2877
2878
2879 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
2880 HConstants.REPLICATION_ENABLE_DEFAULT)) {
2881 return;
2882 }
2883
2884 if ((server instanceof HMaster) &&
2885 (!BaseLoadBalancer.userTablesOnMaster(conf))) {
2886 return;
2887 }
2888
2889
2890 String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
2891 HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2892
2893
2894 String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
2895 HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2896
2897
2898
2899 if (sourceClassname.equals(sinkClassname)) {
2900 server.replicationSourceHandler = (ReplicationSourceService)
2901 newReplicationInstance(sourceClassname,
2902 conf, server, walFs, walDir, oldWALDir);
2903 server.replicationSinkHandler = (ReplicationSinkService)
2904 server.replicationSourceHandler;
2905 } else {
2906 server.replicationSourceHandler = (ReplicationSourceService)
2907 newReplicationInstance(sourceClassname,
2908 conf, server, walFs, walDir, oldWALDir);
2909 server.replicationSinkHandler = (ReplicationSinkService)
2910 newReplicationInstance(sinkClassname,
2911 conf, server, walFs, walDir, oldWALDir);
2912 }
2913 }
2914
2915 static private ReplicationService newReplicationInstance(String classname,
2916 Configuration conf, HRegionServer server, FileSystem walFs, Path walDir,
2917 Path oldLogDir) throws IOException{
2918
2919 Class<?> clazz = null;
2920 try {
2921 ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
2922 clazz = Class.forName(classname, true, classLoader);
2923 } catch (java.lang.ClassNotFoundException nfe) {
2924 throw new IOException("Could not find class for " + classname);
2925 }
2926
2927
2928 ReplicationService service = (ReplicationService)
2929 ReflectionUtils.newInstance(clazz, conf);
2930 service.initialize(server, walFs, walDir, oldLogDir);
2931 return service;
2932 }
2933
2934 public Map<String, ReplicationStatus> getWalGroupsReplicationStatus(){
2935 Map<String, ReplicationStatus> walGroupsReplicationStatus = new TreeMap<>();
2936 if(!this.isOnline()){
2937 return walGroupsReplicationStatus;
2938 }
2939 List<ReplicationSourceInterface> allSources = new ArrayList<>();
2940 allSources.addAll(replicationSourceHandler.getReplicationManager().getSources());
2941 allSources.addAll(replicationSourceHandler.getReplicationManager().getOldSources());
2942 for(ReplicationSourceInterface source: allSources){
2943 walGroupsReplicationStatus.putAll(source.getWalGroupStatus());
2944 }
2945 return walGroupsReplicationStatus;
2946 }
2947
2948
2949
2950
2951
2952
2953
2954
2955 public static HRegionServer constructRegionServer(
2956 Class<? extends HRegionServer> regionServerClass,
2957 final Configuration conf2, CoordinatedStateManager cp) {
2958 try {
2959 Constructor<? extends HRegionServer> c = regionServerClass
2960 .getConstructor(Configuration.class, CoordinatedStateManager.class);
2961 return c.newInstance(conf2, cp);
2962 } catch (Exception e) {
2963 throw new RuntimeException("Failed construction of " + "Regionserver: "
2964 + regionServerClass.toString(), e);
2965 }
2966 }
2967
2968
2969
2970
2971 public static void main(String[] args) throws Exception {
2972 VersionInfo.logVersion();
2973 Configuration conf = HBaseConfiguration.create();
2974 @SuppressWarnings("unchecked")
2975 Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
2976 .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
2977
2978 new HRegionServerCommandLine(regionServerClass).doMain(args);
2979 }
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991 @Override
2992 public List<Region> getOnlineRegions(TableName tableName) {
2993 List<Region> tableRegions = new ArrayList<Region>();
2994 synchronized (this.onlineRegions) {
2995 for (Region region: this.onlineRegions.values()) {
2996 HRegionInfo regionInfo = region.getRegionInfo();
2997 if(regionInfo.getTable().equals(tableName)) {
2998 tableRegions.add(region);
2999 }
3000 }
3001 }
3002 return tableRegions;
3003 }
3004
3005
3006
3007
3008
3009
3010 @Override
3011 public Set<TableName> getOnlineTables() {
3012 Set<TableName> tables = new HashSet<TableName>();
3013 synchronized (this.onlineRegions) {
3014 for (Region region: this.onlineRegions.values()) {
3015 tables.add(region.getTableDesc().getTableName());
3016 }
3017 }
3018 return tables;
3019 }
3020
3021
3022 public String[] getRegionServerCoprocessors() {
3023 TreeSet<String> coprocessors = new TreeSet<String>();
3024 try {
3025 coprocessors.addAll(getWAL(null).getCoprocessorHost().getCoprocessors());
3026 } catch (IOException exception) {
3027 LOG.warn("Exception attempting to fetch wal coprocessor information for the common wal; " +
3028 "skipping.");
3029 LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
3030 }
3031 Collection<Region> regions = getOnlineRegionsLocalContext();
3032 for (Region region: regions) {
3033 coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
3034 try {
3035 coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors());
3036 } catch (IOException exception) {
3037 LOG.warn("Exception attempting to fetch wal coprocessor information for region " + region +
3038 "; skipping.");
3039 LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
3040 }
3041 }
3042 coprocessors.addAll(rsHost.getCoprocessors());
3043 return coprocessors.toArray(new String[coprocessors.size()]);
3044 }
3045
3046 @Override
3047 public List<Region> getOnlineRegions() {
3048 List<Region> allRegions = new ArrayList<Region>();
3049 synchronized (this.onlineRegions) {
3050
3051 allRegions.addAll(onlineRegions.values());
3052 }
3053 return allRegions;
3054 }
3055
3056
3057
3058
3059 private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
3060 try {
3061 CloseRegionCoordination.CloseRegionDetails details =
3062 csm.getCloseRegionCoordination().getDetaultDetails();
3063 if (!closeRegion(region.getEncodedName(), abort, details, null)) {
3064 LOG.warn("Failed to close " + region.getRegionNameAsString() +
3065 " - ignoring and continuing");
3066 }
3067 } catch (IOException e) {
3068 LOG.warn("Failed to close " + region.getRegionNameAsString() +
3069 " - ignoring and continuing", e);
3070 }
3071 }
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093 protected boolean closeRegion(String encodedName, final boolean abort,
3094 CloseRegionCoordination.CloseRegionDetails crd, final ServerName sn)
3095 throws NotServingRegionException, RegionAlreadyInTransitionException {
3096
3097 Region actualRegion = this.getFromOnlineRegions(encodedName);
3098
3099 if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
3100 try {
3101 actualRegion.getCoprocessorHost().preClose(false);
3102 } catch (IOException exp) {
3103 LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
3104 return false;
3105 }
3106 }
3107
3108 final Boolean previous = this.regionsInTransitionInRS
3109 .putIfAbsent(Bytes.toBytes(encodedName), Boolean.FALSE);
3110
3111 if (Boolean.TRUE.equals(previous)) {
3112 LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
3113 "trying to OPEN. Cancelling OPENING.");
3114 if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
3115
3116
3117 LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
3118 " Doing a standard close now");
3119 return closeRegion(encodedName, abort, crd, sn);
3120 }
3121
3122 actualRegion = this.getFromOnlineRegions(encodedName);
3123 if (actualRegion == null) {
3124 LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
3125
3126 throw new RegionAlreadyInTransitionException("The region " + encodedName +
3127 " was opening but not yet served. Opening is cancelled.");
3128 }
3129 } else if (Boolean.FALSE.equals(previous)) {
3130 LOG.info("Received CLOSE for the region: " + encodedName +
3131 ", which we are already trying to CLOSE, but not completed yet");
3132
3133
3134
3135
3136
3137
3138 throw new RegionAlreadyInTransitionException("The region " + encodedName +
3139 " was already closing. New CLOSE request is ignored.");
3140 }
3141
3142 if (actualRegion == null) {
3143 LOG.error("Received CLOSE for a region which is not online, and we're not opening.");
3144 this.regionsInTransitionInRS.remove(Bytes.toBytes(encodedName));
3145
3146 throw new NotServingRegionException("The region " + encodedName +
3147 " is not online, and is not opening.");
3148 }
3149
3150 CloseRegionHandler crh;
3151 final HRegionInfo hri = actualRegion.getRegionInfo();
3152 if (hri.isMetaRegion()) {
3153 crh = new CloseMetaHandler(this, this, hri, abort,
3154 csm.getCloseRegionCoordination(), crd);
3155 } else {
3156 crh = new CloseRegionHandler(this, this, hri, abort,
3157 csm.getCloseRegionCoordination(), crd, sn);
3158 }
3159 this.service.submit(crh);
3160 return true;
3161 }
3162
3163
3164
3165
3166
3167
3168 public Region getOnlineRegion(final byte[] regionName) {
3169 String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
3170 return this.onlineRegions.get(encodedRegionName);
3171 }
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181 public InetSocketAddress[] getRegionBlockLocations(final String encodedRegionName)
3182 throws UnknownHostException {
3183 return Address.toSocketAddress(regionFavoredNodesMap.get(encodedRegionName));
3184 }
3185
3186 @Override
3187 public Region getFromOnlineRegions(final String encodedRegionName) {
3188 return this.onlineRegions.get(encodedRegionName);
3189 }
3190
3191 @Override
3192 public boolean removeFromOnlineRegions(final Region r, ServerName destination) {
3193 Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
3194 metricsRegionServerImpl.requestsCountCache.remove(r.getRegionInfo().getEncodedName());
3195 if (destination != null) {
3196 long closeSeqNum = r.getMaxFlushedSeqId();
3197 if (closeSeqNum == HConstants.NO_SEQNUM) {
3198
3199 closeSeqNum = r.getOpenSeqNum();
3200 if (closeSeqNum == HConstants.NO_SEQNUM) closeSeqNum = 0;
3201 }
3202 addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
3203 }
3204 this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
3205 return toReturn != null;
3206 }
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216 protected Region getRegion(final byte[] regionName)
3217 throws NotServingRegionException {
3218 String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
3219 return getRegionByEncodedName(regionName, encodedRegionName);
3220 }
3221
3222 public Region getRegionByEncodedName(String encodedRegionName)
3223 throws NotServingRegionException {
3224 return getRegionByEncodedName(null, encodedRegionName);
3225 }
3226
3227 protected Region getRegionByEncodedName(byte[] regionName, String encodedRegionName)
3228 throws NotServingRegionException {
3229 Region region = this.onlineRegions.get(encodedRegionName);
3230 if (region == null) {
3231 MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
3232 if (moveInfo != null) {
3233 throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
3234 }
3235 Boolean isOpening = this.regionsInTransitionInRS.get(Bytes.toBytes(encodedRegionName));
3236 String regionNameStr = regionName == null?
3237 encodedRegionName: Bytes.toStringBinary(regionName);
3238 if (isOpening != null && isOpening.booleanValue()) {
3239 throw new RegionOpeningException("Region " + regionNameStr +
3240 " is opening on " + this.serverName);
3241 }
3242 throw new NotServingRegionException("Region " + regionNameStr +
3243 " is not online on " + this.serverName);
3244 }
3245 return region;
3246 }
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258 private Throwable cleanup(final Throwable t, final String msg) {
3259
3260 if (t instanceof NotServingRegionException) {
3261 LOG.debug("NotServingRegionException; " + t.getMessage());
3262 return t;
3263 }
3264 if (msg == null) {
3265 LOG.error("", RemoteExceptionHandler.checkThrowable(t));
3266 } else {
3267 LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
3268 }
3269 if (!rpcServices.checkOOME(t)) {
3270 checkFileSystem();
3271 }
3272 return t;
3273 }
3274
3275
3276
3277
3278
3279
3280
3281
3282 protected IOException convertThrowableToIOE(final Throwable t, final String msg) {
3283 return (t instanceof IOException ? (IOException) t : msg == null
3284 || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
3285 }
3286
3287
3288
3289
3290
3291
3292
3293 public boolean checkFileSystem() {
3294 if (this.fsOk && this.fs != null) {
3295 try {
3296 FSUtils.checkFileSystemAvailable(this.fs);
3297 } catch (IOException e) {
3298 abort("File System not available", e);
3299 this.fsOk = false;
3300 }
3301 }
3302 return this.fsOk;
3303 }
3304
3305 @Override
3306 public void updateRegionFavoredNodesMapping(String encodedRegionName,
3307 List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName> favoredNodes) {
3308 Address[] addr = new Address[favoredNodes.size()];
3309
3310
3311 for (int i = 0; i < favoredNodes.size(); i++) {
3312 addr[i] = Address.fromParts(favoredNodes.get(i).getHostName(),
3313 favoredNodes.get(i).getPort());
3314 }
3315 regionFavoredNodesMap.put(encodedRegionName, addr);
3316 }
3317
3318
3319
3320
3321
3322
3323
3324
3325 @Override
3326 public InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName) {
3327 return Address.toSocketAddress(regionFavoredNodesMap.get(encodedRegionName));
3328 }
3329
3330 @Override
3331 public ServerNonceManager getNonceManager() {
3332 return this.nonceManager;
3333 }
3334
3335 private static class MovedRegionInfo {
3336 private final ServerName serverName;
3337 private final long seqNum;
3338 private final long ts;
3339
3340 public MovedRegionInfo(ServerName serverName, long closeSeqNum) {
3341 this.serverName = serverName;
3342 this.seqNum = closeSeqNum;
3343 ts = EnvironmentEdgeManager.currentTime();
3344 }
3345
3346 public ServerName getServerName() {
3347 return serverName;
3348 }
3349
3350 public long getSeqNum() {
3351 return seqNum;
3352 }
3353
3354 public long getMoveTime() {
3355 return ts;
3356 }
3357 }
3358
3359
3360
3361 protected Map<String, MovedRegionInfo> movedRegions =
3362 new ConcurrentHashMap<String, MovedRegionInfo>(3000);
3363
3364
3365
3366 private static final int TIMEOUT_REGION_MOVED = (2 * 60 * 1000);
3367
3368 protected void addToMovedRegions(String encodedName, ServerName destination, long closeSeqNum) {
3369 if (ServerName.isSameHostnameAndPort(destination, this.getServerName())) {
3370 LOG.warn("Not adding moved region record: " + encodedName + " to self.");
3371 return;
3372 }
3373 LOG.info("Adding moved region record: "
3374 + encodedName + " to " + destination + " as of " + closeSeqNum);
3375 movedRegions.put(encodedName, new MovedRegionInfo(destination, closeSeqNum));
3376 }
3377
3378 void removeFromMovedRegions(String encodedName) {
3379 movedRegions.remove(encodedName);
3380 }
3381
3382 private MovedRegionInfo getMovedRegion(final String encodedRegionName) {
3383 MovedRegionInfo dest = movedRegions.get(encodedRegionName);
3384
3385 long now = EnvironmentEdgeManager.currentTime();
3386 if (dest != null) {
3387 if (dest.getMoveTime() > (now - TIMEOUT_REGION_MOVED)) {
3388 return dest;
3389 } else {
3390 movedRegions.remove(encodedRegionName);
3391 }
3392 }
3393
3394 return null;
3395 }
3396
3397
3398
3399
3400 protected void cleanMovedRegions() {
3401 final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED;
3402 Iterator<Entry<String, MovedRegionInfo>> it = movedRegions.entrySet().iterator();
3403
3404 while (it.hasNext()){
3405 Map.Entry<String, MovedRegionInfo> e = it.next();
3406 if (e.getValue().getMoveTime() < cutOff) {
3407 it.remove();
3408 }
3409 }
3410 }
3411
3412
3413
3414
3415
3416 protected int movedRegionCleanerPeriod() {
3417 return TIMEOUT_REGION_MOVED;
3418 }
3419
3420
3421
3422
3423
3424 protected final static class MovedRegionsCleaner extends ScheduledChore implements Stoppable {
3425 private HRegionServer regionServer;
3426 Stoppable stoppable;
3427
3428 private MovedRegionsCleaner(
3429 HRegionServer regionServer, Stoppable stoppable){
3430 super("MovedRegionsCleaner for region " + regionServer, stoppable,
3431 regionServer.movedRegionCleanerPeriod());
3432 this.regionServer = regionServer;
3433 this.stoppable = stoppable;
3434 }
3435
3436 static MovedRegionsCleaner create(HRegionServer rs){
3437 Stoppable stoppable = new Stoppable() {
3438 private volatile boolean isStopped = false;
3439 @Override public void stop(String why) { isStopped = true;}
3440 @Override public boolean isStopped() {return isStopped;}
3441 };
3442
3443 return new MovedRegionsCleaner(rs, stoppable);
3444 }
3445
3446 @Override
3447 protected void chore() {
3448 regionServer.cleanMovedRegions();
3449 }
3450
3451 @Override
3452 public void stop(String why) {
3453 stoppable.stop(why);
3454 }
3455
3456 @Override
3457 public boolean isStopped() {
3458 return stoppable.isStopped();
3459 }
3460 }
3461
3462 private String getMyEphemeralNodePath() {
3463 return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
3464 }
3465
3466 private boolean isHealthCheckerConfigured() {
3467 String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
3468 return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
3469 }
3470
3471
3472
3473
3474 public CompactSplitThread getCompactSplitThread() {
3475 return this.compactSplitThread;
3476 }
3477
3478
3479
3480
3481
3482
3483
3484
3485 private void updateRecoveringRegionLastFlushedSequenceId(Region r) throws KeeperException,
3486 IOException {
3487 if (!r.isRecovering()) {
3488
3489 return;
3490 }
3491
3492 HRegionInfo regionInfo = r.getRegionInfo();
3493 ZooKeeperWatcher zkw = getZooKeeper();
3494 String previousRSName = this.getLastFailedRSFromZK(regionInfo.getEncodedName());
3495 Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqId();
3496 long minSeqIdForLogReplay = -1;
3497 for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) {
3498 if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) {
3499 minSeqIdForLogReplay = storeSeqIdForReplay;
3500 }
3501 }
3502
3503 try {
3504 long lastRecordedFlushedSequenceId = -1;
3505 String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
3506 regionInfo.getEncodedName());
3507
3508 byte[] data;
3509 try {
3510 data = ZKUtil.getData(zkw, nodePath);
3511 } catch (InterruptedException e) {
3512 throw new InterruptedIOException();
3513 }
3514 if (data != null) {
3515 lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data);
3516 }
3517 if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
3518 ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
3519 }
3520 if (previousRSName != null) {
3521
3522 nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
3523 ZKUtil.setData(zkw, nodePath,
3524 ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
3525 LOG.debug("Update last flushed sequence id of region " + regionInfo.getEncodedName() +
3526 " for " + previousRSName);
3527 } else {
3528 LOG.warn("Can't find failed region server for recovering region " +
3529 regionInfo.getEncodedName());
3530 }
3531 } catch (NoNodeException ignore) {
3532 LOG.debug("Region " + regionInfo.getEncodedName() +
3533 " must have completed recovery because its recovery znode has been removed", ignore);
3534 }
3535 }
3536
3537
3538
3539
3540
3541
3542 private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
3543 String result = null;
3544 long maxZxid = 0;
3545 ZooKeeperWatcher zkw = this.getZooKeeper();
3546 String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
3547 List<String> failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath);
3548 if (failedServers == null || failedServers.isEmpty()) {
3549 return result;
3550 }
3551 for (String failedServer : failedServers) {
3552 String rsPath = ZKUtil.joinZNode(nodePath, failedServer);
3553 Stat stat = new Stat();
3554 ZKUtil.getDataNoWatch(zkw, rsPath, stat);
3555 if (maxZxid < stat.getCzxid()) {
3556 maxZxid = stat.getCzxid();
3557 result = failedServer;
3558 }
3559 }
3560 return result;
3561 }
3562
3563 public CoprocessorServiceResponse execRegionServerService(
3564 @SuppressWarnings("UnusedParameters") final RpcController controller,
3565 final CoprocessorServiceRequest serviceRequest) throws ServiceException {
3566 try {
3567 ServerRpcController serviceController = new ServerRpcController();
3568 CoprocessorServiceCall call = serviceRequest.getCall();
3569 String serviceName = call.getServiceName();
3570 String methodName = call.getMethodName();
3571 if (!coprocessorServiceHandlers.containsKey(serviceName)) {
3572 throw new UnknownProtocolException(null,
3573 "No registered coprocessor service found for name " + serviceName);
3574 }
3575 Service service = coprocessorServiceHandlers.get(serviceName);
3576 Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
3577 Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
3578 if (methodDesc == null) {
3579 throw new UnknownProtocolException(service.getClass(), "Unknown method " + methodName
3580 + " called on service " + serviceName);
3581 }
3582 Message.Builder builderForType = service.getRequestPrototype(methodDesc).newBuilderForType();
3583 ProtobufUtil.mergeFrom(builderForType, call.getRequest());
3584 Message request = builderForType.build();
3585 final Message.Builder responseBuilder =
3586 service.getResponsePrototype(methodDesc).newBuilderForType();
3587 service.callMethod(methodDesc, serviceController, request, new RpcCallback<Message>() {
3588 @Override
3589 public void run(Message message) {
3590 if (message != null) {
3591 responseBuilder.mergeFrom(message);
3592 }
3593 }
3594 });
3595 IOException exception = ResponseConverter.getControllerException(serviceController);
3596 if (exception != null) {
3597 throw exception;
3598 }
3599 Message execResult = responseBuilder.build();
3600 ClientProtos.CoprocessorServiceResponse.Builder builder =
3601 ClientProtos.CoprocessorServiceResponse.newBuilder();
3602 builder.setRegion(RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME,
3603 HConstants.EMPTY_BYTE_ARRAY));
3604 builder.setValue(builder.getValueBuilder().setName(execResult.getClass().getName())
3605 .setValue(execResult.toByteString()));
3606 return builder.build();
3607 } catch (IOException ie) {
3608 throw new ServiceException(ie);
3609 }
3610 }
3611
3612
3613
3614
3615 public CacheConfig getCacheConfig() {
3616 return this.cacheConfig;
3617 }
3618
3619
3620
3621
3622 protected ConfigurationManager getConfigurationManager() {
3623 return configurationManager;
3624 }
3625
3626
3627
3628
3629 public TableDescriptors getTableDescriptors() {
3630 return this.tableDescriptors;
3631 }
3632
3633
3634
3635
3636 public void updateConfiguration() {
3637 LOG.info("Reloading the configuration from disk.");
3638
3639 conf.reloadConfiguration();
3640 configurationManager.notifyAllObservers(conf);
3641 }
3642
3643 @Override
3644 public HeapMemoryManager getHeapMemoryManager() {
3645 return hMemManager;
3646 }
3647
3648 @Override
3649 public double getCompactionPressure() {
3650 double max = 0;
3651 for (Region region : onlineRegions.values()) {
3652 for (Store store : region.getStores()) {
3653 double normCount = store.getCompactionPressure();
3654 if (normCount > max) {
3655 max = normCount;
3656 }
3657 }
3658 }
3659 return max;
3660 }
3661
3662
3663
3664
3665
3666 @InterfaceAudience.Private
3667 public boolean walRollRequestFinished() {
3668 return this.walRoller.walRollFinished();
3669 }
3670
3671 @Override
3672 public ThroughputController getFlushThroughputController() {
3673 return flushThroughputController;
3674 }
3675
3676 @Override
3677 public double getFlushPressure() {
3678 if (getRegionServerAccounting() == null || cacheFlusher == null) {
3679
3680 return 0.0;
3681 }
3682 return getRegionServerAccounting().getGlobalMemstoreSize() * 1.0
3683 / cacheFlusher.globalMemStoreLimitLowMark;
3684 }
3685
3686 @Override
3687 public void onConfigurationChange(Configuration newConf) {
3688 ThroughputController old = this.flushThroughputController;
3689 if (old != null) {
3690 old.stop("configuration change");
3691 }
3692 this.flushThroughputController = FlushThroughputControllerFactory.create(this, newConf);
3693 }
3694
3695 @Override
3696 public MetricsRegionServer getMetrics() {
3697 return metricsRegionServer;
3698 }
3699
3700 @Override
3701 public void unassign(byte[] regionName) throws IOException {
3702 clusterConnection.getAdmin().unassign(regionName, false);
3703 }
3704
3705
3706
3707
3708 private static class SystemExitWhenAbortTimeout extends TimerTask {
3709 @Override
3710 public void run() {
3711 LOG.warn("Aborting region server timed out, terminating forcibly" +
3712 " and does not wait for any running shutdown hooks or finalizers to finish their work." +
3713 " Thread dump to stdout.");
3714 Threads.printThreadInfo(System.out, "Zombie HRegionServer");
3715 Runtime.getRuntime().halt(1);
3716 }
3717 }
3718 }