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