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