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