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