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