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