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