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