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