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.hlogFileCount.set(this.hlog.getNumLogFiles());
1597 this.metrics.hlogFileSizeMB.set(this.hlog.getNumLogFileSize() /(1024 * 1024));
1598 this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024)));
1599 this.metrics.mbInMemoryWithoutWAL.set((int) (dataInMemoryWithoutWAL / (1024 * 1024)));
1600 this.metrics.numPutsWithoutWAL.set(numPutsWithoutWAL);
1601 this.metrics.storefileIndexSizeMB.set(
1602 (int) (storefileIndexSize / (1024 * 1024)));
1603 this.metrics.rootIndexSizeKB.set(
1604 (int) (storefileIndexSize / 1024));
1605 this.metrics.totalStaticIndexSizeKB.set(
1606 (int) (totalStaticIndexSize / 1024));
1607 this.metrics.totalStaticBloomSizeKB.set(
1608 (int) (totalStaticBloomSize / 1024));
1609 this.metrics.readRequestsCount.set(readRequestsCount);
1610 this.metrics.writeRequestsCount.set(writeRequestsCount);
1611 this.metrics.compactionQueueSize.set(compactSplitThread
1612 .getCompactionQueueSize());
1613 this.metrics.flushQueueSize.set(cacheFlusher
1614 .getFlushQueueSize());
1615 this.metrics.updatesBlockedSeconds.set(updatesBlockedMs/1000);
1616 final long updatesBlockedMsHigherWater = cacheFlusher.getUpdatesBlockedMsHighWater().get();
1617 this.metrics.updatesBlockedSecondsHighWater.set(updatesBlockedMsHigherWater/1000);
1618
1619 BlockCache blockCache = cacheConfig.getBlockCache();
1620 if (blockCache != null) {
1621 this.metrics.blockCacheCount.set(blockCache.size());
1622 this.metrics.blockCacheFree.set(blockCache.getFreeSize());
1623 this.metrics.blockCacheSize.set(blockCache.getCurrentSize());
1624 CacheStats cacheStats = blockCache.getStats();
1625 this.metrics.blockCacheHitCount.set(cacheStats.getHitCount());
1626 this.metrics.blockCacheMissCount.set(cacheStats.getMissCount());
1627 this.metrics.blockCacheEvictedCount.set(blockCache.getEvictedCount());
1628 double ratio = blockCache.getStats().getHitRatio();
1629 int percent = (int) (ratio * 100);
1630 this.metrics.blockCacheHitRatio.set(percent);
1631 ratio = blockCache.getStats().getHitCachingRatio();
1632 percent = (int) (ratio * 100);
1633 this.metrics.blockCacheHitCachingRatio.set(percent);
1634
1635 cacheStats.rollMetricsPeriod();
1636 ratio = cacheStats.getHitRatioPastNPeriods();
1637 percent = (int) (ratio * 100);
1638 this.metrics.blockCacheHitRatioPastNPeriods.set(percent);
1639 ratio = cacheStats.getHitCachingRatioPastNPeriods();
1640 percent = (int) (ratio * 100);
1641 this.metrics.blockCacheHitCachingRatioPastNPeriods.set(percent);
1642 }
1643 float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
1644 getServerName().getHostname());
1645 int percent = (int) (localityIndex * 100);
1646 this.metrics.hdfsBlocksLocalityIndex.set(percent);
1647
1648 }
1649
1650
1651
1652
1653 public RegionServerMetrics getMetrics() {
1654 return this.metrics;
1655 }
1656
1657
1658
1659
1660 public MasterAddressTracker getMasterAddressManager() {
1661 return this.masterAddressManager;
1662 }
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676 private void startServiceThreads() throws IOException {
1677 String n = Thread.currentThread().getName();
1678
1679 this.service = new ExecutorService(getServerName().toString());
1680 this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1681 conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1682 this.service.startExecutorService(ExecutorType.RS_OPEN_ROOT,
1683 conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
1684 this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1685 conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1686 this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1687 conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1688 this.service.startExecutorService(ExecutorType.RS_CLOSE_ROOT,
1689 conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
1690 this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1691 conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1692
1693 Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), n + ".logRoller",
1694 uncaughtExceptionHandler);
1695 Threads.setDaemonThreadRunning(this.cacheFlusher.getThread(), n + ".cacheFlusher",
1696 uncaughtExceptionHandler);
1697 Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), n +
1698 ".compactionChecker", uncaughtExceptionHandler);
1699 Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), n +
1700 ".periodicFlusher", uncaughtExceptionHandler);
1701 if (this.healthCheckChore != null) {
1702 Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
1703 uncaughtExceptionHandler);
1704 }
1705
1706
1707
1708 this.leases.setName(n + ".leaseChecker");
1709 this.leases.start();
1710
1711
1712
1713 this.webuiport = putUpWebUI();
1714
1715 if (this.replicationSourceHandler == this.replicationSinkHandler &&
1716 this.replicationSourceHandler != null) {
1717 this.replicationSourceHandler.startReplicationService();
1718 } else {
1719 if (this.replicationSourceHandler != null) {
1720 this.replicationSourceHandler.startReplicationService();
1721 }
1722 if (this.replicationSinkHandler != null) {
1723 this.replicationSinkHandler.startReplicationService();
1724 }
1725 }
1726
1727
1728
1729 this.rpcServer.start();
1730
1731
1732 this.splitLogWorker = new SplitLogWorker(this.zooKeeper,
1733 this.getConfiguration(), this.getServerName().toString());
1734 splitLogWorker.start();
1735
1736 }
1737
1738
1739
1740
1741
1742
1743 private int putUpWebUI() throws IOException {
1744 int port = this.conf.getInt("hbase.regionserver.info.port", 60030);
1745
1746 if (port < 0) return port;
1747 String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1748
1749 boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
1750 false);
1751 while (true) {
1752 try {
1753 this.infoServer = new InfoServer("regionserver", addr, port, false, this.conf);
1754 this.infoServer.addServlet("status", "/rs-status", RSStatusServlet.class);
1755 this.infoServer.addServlet("dump", "/dump", RSDumpServlet.class);
1756 this.infoServer.setAttribute(REGIONSERVER, this);
1757 this.infoServer.setAttribute(REGIONSERVER_CONF, conf);
1758 this.infoServer.start();
1759 break;
1760 } catch (BindException e) {
1761 if (!auto) {
1762
1763 throw e;
1764 }
1765
1766 LOG.info("Failed binding http info server to port: " + port);
1767 port++;
1768 }
1769 }
1770 return port;
1771 }
1772
1773
1774
1775
1776 private boolean isHealthy() {
1777 if (!fsOk) {
1778
1779 return false;
1780 }
1781
1782 if (!(leases.isAlive()
1783 && cacheFlusher.isAlive() && hlogRoller.isAlive()
1784 && this.compactionChecker.isAlive())
1785 && this.periodicFlusher.isAlive()) {
1786 stop("One or more threads are no longer alive -- stop");
1787 return false;
1788 }
1789 if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
1790 stop("Meta HLog roller thread is no longer alive -- stop");
1791 return false;
1792 }
1793 return true;
1794 }
1795
1796 public HLog getWAL() {
1797 try {
1798 return getWAL(null);
1799 } catch (IOException e) {
1800 LOG.warn("getWAL threw exception " + e);
1801 return null;
1802 }
1803 }
1804
1805 @Override
1806 public HLog getWAL(HRegionInfo regionInfo) throws IOException {
1807
1808
1809
1810
1811 if (this.separateHLogForMeta &&
1812 regionInfo != null &&
1813 regionInfo.isMetaTable()) {
1814 return getMetaWAL();
1815 }
1816 return this.hlog;
1817 }
1818
1819 @Override
1820 public CatalogTracker getCatalogTracker() {
1821 return this.catalogTracker;
1822 }
1823
1824 @Override
1825 public void stop(final String msg) {
1826 try {
1827 if (this.rsHost != null) {
1828 this.rsHost.preStop(msg);
1829 }
1830 this.stopped = true;
1831 LOG.info("STOPPED: " + msg);
1832
1833 sleeper.skipSleepCycle();
1834 } catch (IOException exp) {
1835 LOG.warn("The region server did not stop", exp);
1836 }
1837 }
1838
1839 public void waitForServerOnline(){
1840 while (!isOnline() && !isStopped()){
1841 sleeper.sleep();
1842 }
1843 }
1844
1845 @Override
1846 public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct,
1847 final boolean daughter)
1848 throws KeeperException, IOException {
1849 checkOpen();
1850 LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString() +
1851 ", daughter=" + daughter);
1852
1853 for (Store s : r.getStores().values()) {
1854 if (s.hasReferences() || s.needsCompaction()) {
1855 getCompactionRequester().requestCompaction(r, s, "Opening Region", null);
1856 }
1857 }
1858
1859 if (r.getRegionInfo().isRootRegion()) {
1860 RootLocationEditor.setRootLocation(getZooKeeper(),
1861 this.serverNameFromMasterPOV);
1862 } else if (r.getRegionInfo().isMetaRegion()) {
1863 MetaEditor.updateMetaLocation(ct, r.getRegionInfo(),
1864 this.serverNameFromMasterPOV);
1865 } else {
1866 if (daughter) {
1867
1868 MetaEditor.addDaughter(ct, r.getRegionInfo(),
1869 this.serverNameFromMasterPOV);
1870 } else {
1871 MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
1872 this.serverNameFromMasterPOV);
1873 }
1874 }
1875 LOG.info("Done with post open deploy task for region=" +
1876 r.getRegionNameAsString() + ", daughter=" + daughter);
1877
1878 }
1879
1880
1881
1882
1883
1884 public HBaseRpcMetrics getRpcMetrics() {
1885 return rpcServer.getRpcMetrics();
1886 }
1887
1888 @Override
1889 public RpcServer getRpcServer() {
1890 return rpcServer;
1891 }
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903 public void abort(String reason, Throwable cause) {
1904 String msg = "ABORTING region server " + this + ": " + reason;
1905 if (cause != null) {
1906 LOG.fatal(msg, cause);
1907 } else {
1908 LOG.fatal(msg);
1909 }
1910 this.abortRequested = true;
1911 this.reservedSpace.clear();
1912
1913
1914
1915 LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1916 CoprocessorHost.getLoadedCoprocessors());
1917 if (this.metrics != null) {
1918 LOG.info("Dump of metrics: " + this.metrics);
1919 }
1920
1921 try {
1922 if (cause != null) {
1923 msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1924 }
1925 if (hbaseMaster != null) {
1926 hbaseMaster.reportRSFatalError(
1927 this.serverNameFromMasterPOV.getVersionedBytes(), msg);
1928 }
1929 } catch (Throwable t) {
1930 LOG.warn("Unable to report fatal error to master", t);
1931 }
1932 stop(reason);
1933 }
1934
1935
1936
1937
1938 public void abort(String reason) {
1939 abort(reason, null);
1940 }
1941
1942 public boolean isAborted() {
1943 return this.abortRequested;
1944 }
1945
1946
1947
1948
1949
1950
1951 protected void kill() {
1952 this.killed = true;
1953 abort("Simulated kill");
1954 }
1955
1956
1957
1958
1959
1960 protected void join() {
1961 Threads.shutdown(this.compactionChecker.getThread());
1962 Threads.shutdown(this.periodicFlusher.getThread());
1963 Threads.shutdown(this.cacheFlusher.getThread());
1964 if (this.healthCheckChore != null) {
1965 Threads.shutdown(this.healthCheckChore.getThread());
1966 }
1967 if (this.hlogRoller != null) {
1968 Threads.shutdown(this.hlogRoller.getThread());
1969 }
1970 if (this.metaHLogRoller != null) {
1971 Threads.shutdown(this.metaHLogRoller.getThread());
1972 }
1973 if (this.compactSplitThread != null) {
1974 this.compactSplitThread.join();
1975 }
1976 if (this.service != null) this.service.shutdown();
1977 if (this.replicationSourceHandler != null &&
1978 this.replicationSourceHandler == this.replicationSinkHandler) {
1979 this.replicationSourceHandler.stopReplicationService();
1980 } else {
1981 if (this.replicationSourceHandler != null) {
1982 this.replicationSourceHandler.stopReplicationService();
1983 }
1984 if (this.replicationSinkHandler != null) {
1985 this.replicationSinkHandler.stopReplicationService();
1986 }
1987 }
1988 }
1989
1990
1991
1992
1993
1994 ReplicationSourceService getReplicationSourceService() {
1995 return replicationSourceHandler;
1996 }
1997
1998
1999
2000
2001
2002 ReplicationSinkService getReplicationSinkService() {
2003 return replicationSinkHandler;
2004 }
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014 private ServerName getMaster() {
2015 ServerName masterServerName = null;
2016 long previousLogTime = 0;
2017 HMasterRegionInterface master = null;
2018 InetSocketAddress masterIsa = null;
2019 while (keepLooping() && master == null) {
2020 masterServerName = this.masterAddressManager.getMasterAddress();
2021 if (masterServerName == null) {
2022 if (!keepLooping()) {
2023
2024 LOG.debug("No master found and cluster is stopped; bailing out");
2025 return null;
2026 }
2027 LOG.debug("No master found; retry");
2028 previousLogTime = System.currentTimeMillis();
2029
2030 sleeper.sleep();
2031 continue;
2032 }
2033
2034 masterIsa =
2035 new InetSocketAddress(masterServerName.getHostname(), masterServerName.getPort());
2036
2037 LOG.info("Attempting connect to Master server at " + masterServerName);
2038 try {
2039
2040
2041 master = HBaseRPC.waitForProxy(this.rpcEngine,
2042 HMasterRegionInterface.class, HMasterRegionInterface.VERSION,
2043 masterIsa, this.conf, -1,
2044 this.rpcTimeout, this.rpcTimeout);
2045 } catch (IOException e) {
2046 e = e instanceof RemoteException ?
2047 ((RemoteException)e).unwrapRemoteException() : e;
2048 if (e instanceof ServerNotRunningYetException) {
2049 if (System.currentTimeMillis() > (previousLogTime+1000)){
2050 LOG.info("Master isn't available yet, retrying");
2051 previousLogTime = System.currentTimeMillis();
2052 }
2053 } else {
2054 if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2055 LOG.warn("Unable to connect to master. Retrying. Error was:", e);
2056 previousLogTime = System.currentTimeMillis();
2057 }
2058 }
2059 try {
2060 Thread.sleep(200);
2061 } catch (InterruptedException ignored) {
2062 }
2063 }
2064 }
2065 LOG.info("Connected to master at " + masterIsa);
2066 this.hbaseMaster = master;
2067 return masterServerName;
2068 }
2069
2070
2071
2072
2073
2074 private boolean keepLooping() {
2075 return !this.stopped && isClusterUp();
2076 }
2077
2078
2079
2080
2081
2082
2083
2084
2085 private MapWritable reportForDuty() throws IOException {
2086 MapWritable result = null;
2087 ServerName masterServerName = getMaster();
2088 if (masterServerName == null) return result;
2089 try {
2090 this.requestCount.set(0);
2091 LOG.info("Telling master at " + masterServerName + " that we are up " +
2092 "with port=" + this.isa.getPort() + ", startcode=" + this.startcode);
2093 long now = EnvironmentEdgeManager.currentTimeMillis();
2094 int port = this.isa.getPort();
2095 result = this.hbaseMaster.regionServerStartup(port, this.startcode, now);
2096 } catch (RemoteException e) {
2097 IOException ioe = e.unwrapRemoteException();
2098 if (ioe instanceof ClockOutOfSyncException) {
2099 LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2100
2101 throw ioe;
2102 } else {
2103 LOG.warn("remote error telling master we are up", e);
2104 }
2105 } catch (IOException e) {
2106 LOG.warn("error telling master we are up", e);
2107 }
2108 return result;
2109 }
2110
2111
2112
2113
2114
2115
2116 protected void closeAllRegions(final boolean abort) {
2117 closeUserRegions(abort);
2118 closeMetaTableRegions(abort);
2119 }
2120
2121
2122
2123
2124
2125 void closeMetaTableRegions(final boolean abort) {
2126 HRegion meta = null;
2127 HRegion root = null;
2128 this.lock.writeLock().lock();
2129 try {
2130 for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2131 HRegionInfo hri = e.getValue().getRegionInfo();
2132 if (hri.isRootRegion()) {
2133 root = e.getValue();
2134 } else if (hri.isMetaRegion()) {
2135 meta = e.getValue();
2136 }
2137 if (meta != null && root != null) break;
2138 }
2139 } finally {
2140 this.lock.writeLock().unlock();
2141 }
2142 if (meta != null) closeRegion(meta.getRegionInfo(), abort, false);
2143 if (root != null) closeRegion(root.getRegionInfo(), abort, false);
2144 }
2145
2146
2147
2148
2149
2150
2151
2152 void closeUserRegions(final boolean abort) {
2153 this.lock.writeLock().lock();
2154 try {
2155 for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2156 HRegion r = e.getValue();
2157 if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2158
2159 closeRegion(r.getRegionInfo(), abort, false);
2160 }
2161 }
2162 } finally {
2163 this.lock.writeLock().unlock();
2164 }
2165 }
2166
2167 @Override
2168 @QosPriority(priority=HConstants.HIGH_QOS)
2169 public HRegionInfo getRegionInfo(final byte[] regionName)
2170 throws NotServingRegionException, IOException {
2171 checkOpen();
2172 requestCount.incrementAndGet();
2173 return getRegion(regionName).getRegionInfo();
2174 }
2175
2176 public Result getClosestRowBefore(final byte[] regionName, final byte[] row,
2177 final byte[] family) throws IOException {
2178 checkOpen();
2179 requestCount.incrementAndGet();
2180 try {
2181
2182 HRegion region = getRegion(regionName);
2183
2184
2185 Result r = region.getClosestRowBefore(row, family);
2186 return r;
2187 } catch (Throwable t) {
2188 throw convertThrowableToIOE(cleanup(t));
2189 }
2190 }
2191
2192
2193 public Result get(byte[] regionName, Get get) throws IOException {
2194 checkOpen();
2195 requestCount.incrementAndGet();
2196 try {
2197 HRegion region = getRegion(regionName);
2198 return region.get(get, getLockFromId(get.getLockId()));
2199 } catch (Throwable t) {
2200 throw convertThrowableToIOE(cleanup(t));
2201 }
2202 }
2203
2204 public boolean exists(byte[] regionName, Get get) throws IOException {
2205 checkOpen();
2206 requestCount.incrementAndGet();
2207 try {
2208 HRegion region = getRegion(regionName);
2209 Integer lock = getLockFromId(get.getLockId());
2210 if (region.getCoprocessorHost() != null) {
2211 Boolean result = region.getCoprocessorHost().preExists(get);
2212 if (result != null) {
2213 return result.booleanValue();
2214 }
2215 }
2216 Result r = region.get(get, lock);
2217 boolean result = r != null && !r.isEmpty();
2218 if (region.getCoprocessorHost() != null) {
2219 result = region.getCoprocessorHost().postExists(get, result);
2220 }
2221 return result;
2222 } catch (Throwable t) {
2223 throw convertThrowableToIOE(cleanup(t));
2224 }
2225 }
2226
2227 public void put(final byte[] regionName, final Put put) throws IOException {
2228 if (put.getRow() == null) {
2229 throw new IllegalArgumentException("update has null row");
2230 }
2231
2232 checkOpen();
2233 this.requestCount.incrementAndGet();
2234 HRegion region = getRegion(regionName);
2235 try {
2236 if (!region.getRegionInfo().isMetaTable()) {
2237 this.cacheFlusher.reclaimMemStoreMemory();
2238 }
2239 boolean writeToWAL = put.getWriteToWAL();
2240 region.put(put, getLockFromId(put.getLockId()), writeToWAL);
2241 } catch (Throwable t) {
2242 throw convertThrowableToIOE(cleanup(t));
2243 }
2244 }
2245
2246 public int put(final byte[] regionName, final List<Put> puts)
2247 throws IOException {
2248 checkOpen();
2249 HRegion region = null;
2250 int i = 0;
2251
2252 try {
2253 region = getRegion(regionName);
2254 if (!region.getRegionInfo().isMetaTable()) {
2255 this.cacheFlusher.reclaimMemStoreMemory();
2256 }
2257
2258 @SuppressWarnings("unchecked")
2259 Pair<Mutation, Integer>[] putsWithLocks = new Pair[puts.size()];
2260
2261 for (Put p : puts) {
2262 Integer lock = getLockFromId(p.getLockId());
2263 putsWithLocks[i++] = new Pair<Mutation, Integer>(p, lock);
2264 }
2265
2266 this.requestCount.addAndGet(puts.size());
2267 OperationStatus codes[] = region.batchMutate(putsWithLocks);
2268 for (i = 0; i < codes.length; i++) {
2269 if (codes[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
2270 return i;
2271 }
2272 }
2273 return -1;
2274 } catch (Throwable t) {
2275 throw convertThrowableToIOE(cleanup(t));
2276 }
2277 }
2278
2279 private boolean checkAndMutate(final byte[] regionName, final byte[] row,
2280 final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2281 final WritableByteArrayComparable comparator, final Writable w,
2282 Integer lock) throws IOException {
2283 checkOpen();
2284 this.requestCount.incrementAndGet();
2285 HRegion region = getRegion(regionName);
2286 try {
2287 if (!region.getRegionInfo().isMetaTable()) {
2288 this.cacheFlusher.reclaimMemStoreMemory();
2289 }
2290 return region.checkAndMutate(row, family, qualifier, compareOp,
2291 comparator, w, lock, true);
2292 } catch (Throwable t) {
2293 throw convertThrowableToIOE(cleanup(t));
2294 }
2295 }
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309 public boolean checkAndPut(final byte[] regionName, final byte[] row,
2310 final byte[] family, final byte[] qualifier, final byte[] value,
2311 final Put put) throws IOException {
2312 checkOpen();
2313 if (regionName == null) {
2314 throw new IOException("Invalid arguments to checkAndPut "
2315 + "regionName is null");
2316 }
2317 HRegion region = getRegion(regionName);
2318 Integer lock = getLockFromId(put.getLockId());
2319 WritableByteArrayComparable comparator = new BinaryComparator(value);
2320 if (region.getCoprocessorHost() != null) {
2321 Boolean result = region.getCoprocessorHost()
2322 .preCheckAndPut(row, family, qualifier, CompareOp.EQUAL, comparator,
2323 put);
2324 if (result != null) {
2325 return result.booleanValue();
2326 }
2327 }
2328 boolean result = checkAndMutate(regionName, row, family, qualifier,
2329 CompareOp.EQUAL, comparator, put,
2330 lock);
2331 if (region.getCoprocessorHost() != null) {
2332 result = region.getCoprocessorHost().postCheckAndPut(row, family,
2333 qualifier, CompareOp.EQUAL, comparator, put, result);
2334 }
2335 return result;
2336 }
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350 public boolean checkAndPut(final byte[] regionName, final byte[] row,
2351 final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2352 final WritableByteArrayComparable comparator, final Put put)
2353 throws IOException {
2354 checkOpen();
2355 if (regionName == null) {
2356 throw new IOException("Invalid arguments to checkAndPut "
2357 + "regionName is null");
2358 }
2359 HRegion region = getRegion(regionName);
2360 Integer lock = getLockFromId(put.getLockId());
2361 if (region.getCoprocessorHost() != null) {
2362 Boolean result = region.getCoprocessorHost()
2363 .preCheckAndPut(row, family, qualifier, compareOp, comparator, put);
2364 if (result != null) {
2365 return result.booleanValue();
2366 }
2367 }
2368 boolean result = checkAndMutate(regionName, row, family, qualifier,
2369 compareOp, comparator, put, lock);
2370 if (region.getCoprocessorHost() != null) {
2371 result = region.getCoprocessorHost().postCheckAndPut(row, family,
2372 qualifier, compareOp, comparator, put, result);
2373 }
2374 return result;
2375 }
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389 public boolean checkAndDelete(final byte[] regionName, final byte[] row,
2390 final byte[] family, final byte[] qualifier, final byte[] value,
2391 final Delete delete) throws IOException {
2392 checkOpen();
2393
2394 if (regionName == null) {
2395 throw new IOException("Invalid arguments to checkAndDelete "
2396 + "regionName is null");
2397 }
2398 HRegion region = getRegion(regionName);
2399 Integer lock = getLockFromId(delete.getLockId());
2400 WritableByteArrayComparable comparator = new BinaryComparator(value);
2401 if (region.getCoprocessorHost() != null) {
2402 Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
2403 family, qualifier, CompareOp.EQUAL, comparator, delete);
2404 if (result != null) {
2405 return result.booleanValue();
2406 }
2407 }
2408 boolean result = checkAndMutate(regionName, row, family, qualifier,
2409 CompareOp.EQUAL, comparator, delete, lock);
2410 if (region.getCoprocessorHost() != null) {
2411 result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2412 qualifier, CompareOp.EQUAL, comparator, delete, result);
2413 }
2414 return result;
2415 }
2416
2417 @Override
2418 public List<String> getStoreFileList(byte[] regionName, byte[] columnFamily)
2419 throws IllegalArgumentException {
2420 return getStoreFileList(regionName, new byte[][]{columnFamily});
2421 }
2422
2423 @Override
2424 public List<String> getStoreFileList(byte[] regionName, byte[][] columnFamilies)
2425 throws IllegalArgumentException {
2426 HRegion region = getOnlineRegion(regionName);
2427 if (region == null) {
2428 throw new IllegalArgumentException("No region: " + new String(regionName)
2429 + " available");
2430 }
2431 return region.getStoreFileList(columnFamilies);
2432 }
2433
2434 public List<String> getStoreFileList(byte[] regionName)
2435 throws IllegalArgumentException {
2436 HRegion region = getOnlineRegion(regionName);
2437 if (region == null) {
2438 throw new IllegalArgumentException("No region: " + new String(regionName)
2439 + " available");
2440 }
2441 Set<byte[]> columnFamilies = region.getStores().keySet();
2442 int nCF = columnFamilies.size();
2443 return region.getStoreFileList(columnFamilies.toArray(new byte[nCF][]));
2444 }
2445
2446
2447
2448
2449 public void flushRegion(byte[] regionName)
2450 throws IllegalArgumentException, IOException {
2451 HRegion region = getOnlineRegion(regionName);
2452 if (region == null) {
2453 throw new IllegalArgumentException("No region : " + new String(regionName)
2454 + " available");
2455 }
2456 boolean needsCompaction = region.flushcache();
2457 if (needsCompaction) {
2458 this.compactSplitThread.requestCompaction(region, "Compaction through user triggered flush");
2459 }
2460 }
2461
2462
2463
2464
2465 public void flushRegion(byte[] regionName, long ifOlderThanTS)
2466 throws IllegalArgumentException, IOException {
2467 HRegion region = getOnlineRegion(regionName);
2468 if (region == null) {
2469 throw new IllegalArgumentException("No region : " + new String(regionName)
2470 + " available");
2471 }
2472 if (region.getLastFlushTime() < ifOlderThanTS) {
2473 boolean needsCompaction = region.flushcache();
2474 if (needsCompaction) {
2475 this.compactSplitThread
2476 .requestCompaction(region, "Compaction through user triggered flush");
2477 }
2478 }
2479 }
2480
2481
2482
2483
2484
2485 public long getLastFlushTime(byte[] regionName) {
2486 HRegion region = getOnlineRegion(regionName);
2487 if (region == null) {
2488 throw new IllegalArgumentException("No region : " + new String(regionName)
2489 + " available");
2490 }
2491 return region.getLastFlushTime();
2492 }
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506 public boolean checkAndDelete(final byte[] regionName, final byte[] row,
2507 final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2508 final WritableByteArrayComparable comparator, final Delete delete)
2509 throws IOException {
2510 checkOpen();
2511
2512 if (regionName == null) {
2513 throw new IOException("Invalid arguments to checkAndDelete "
2514 + "regionName is null");
2515 }
2516 HRegion region = getRegion(regionName);
2517 Integer lock = getLockFromId(delete.getLockId());
2518 if (region.getCoprocessorHost() != null) {
2519 Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
2520 family, qualifier, compareOp, comparator, delete);
2521 if (result != null) {
2522 return result.booleanValue();
2523 }
2524 }
2525 boolean result = checkAndMutate(regionName, row, family, qualifier,
2526 compareOp, comparator, delete, lock);
2527 if (region.getCoprocessorHost() != null) {
2528 result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2529 qualifier, compareOp, comparator, delete, result);
2530 }
2531 return result;
2532 }
2533
2534
2535
2536
2537
2538 public long openScanner(byte[] regionName, Scan scan) throws IOException {
2539 RegionScanner s = internalOpenScanner(regionName, scan);
2540 long scannerId = addScanner(s);
2541 return scannerId;
2542 }
2543
2544 private RegionScanner internalOpenScanner(byte[] regionName, Scan scan)
2545 throws IOException {
2546 checkOpen();
2547 NullPointerException npe = null;
2548 if (regionName == null) {
2549 npe = new NullPointerException("regionName is null");
2550 } else if (scan == null) {
2551 npe = new NullPointerException("scan is null");
2552 }
2553 if (npe != null) {
2554 throw new IOException("Invalid arguments to openScanner", npe);
2555 }
2556 requestCount.incrementAndGet();
2557 try {
2558 HRegion r = getRegion(regionName);
2559 r.checkRow(scan.getStartRow(), "Scan");
2560 scan.setLoadColumnFamiliesOnDemand(r.isLoadingCfsOnDemandDefault()
2561 || scan.doLoadColumnFamiliesOnDemand());
2562 r.prepareScanner(scan);
2563 RegionScanner s = null;
2564 if (r.getCoprocessorHost() != null) {
2565 s = r.getCoprocessorHost().preScannerOpen(scan);
2566 }
2567 if (s == null) {
2568 s = r.getScanner(scan);
2569 }
2570 if (r.getCoprocessorHost() != null) {
2571 RegionScanner savedScanner = r.getCoprocessorHost().postScannerOpen(
2572 scan, s);
2573 if (savedScanner == null) {
2574 LOG.warn("PostScannerOpen impl returning null. "
2575 + "Check the RegionObserver implementation.");
2576 } else {
2577 s = savedScanner;
2578 }
2579 }
2580 return s;
2581 } catch (Throwable t) {
2582 throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
2583 }
2584 }
2585
2586 protected long addScanner(RegionScanner s) throws LeaseStillHeldException {
2587 long scannerId = -1L;
2588 scannerId = rand.nextLong();
2589 String scannerName = String.valueOf(scannerId);
2590 scanners.put(scannerName, s);
2591 this.leases.createLease(scannerName, new ScannerListener(scannerName));
2592 return scannerId;
2593 }
2594
2595 public Result next(final long scannerId) throws IOException {
2596 Result[] res = next(scannerId, 1);
2597 if (res == null || res.length == 0) {
2598 return null;
2599 }
2600 return res[0];
2601 }
2602
2603 public Result[] next(final long scannerId, int nbRows) throws IOException {
2604 String scannerName = String.valueOf(scannerId);
2605 RegionScanner s = this.scanners.get(scannerName);
2606 if (s == null) {
2607 LOG.info("Client tried to access missing scanner " + scannerName);
2608 throw new UnknownScannerException("Name: " + scannerName);
2609 }
2610 return internalNext(s, nbRows, scannerName);
2611 }
2612
2613 private Result[] internalNext(final RegionScanner s, int nbRows,
2614 String scannerName) throws IOException {
2615 try {
2616 checkOpen();
2617 } catch (IOException e) {
2618
2619
2620 if (scannerName != null) {
2621 try {
2622 this.leases.cancelLease(scannerName);
2623 } catch (LeaseException le) {
2624 LOG.info("Server shutting down and client tried to access missing scanner "
2625 + scannerName);
2626 }
2627 }
2628 throw e;
2629 }
2630 Leases.Lease lease = null;
2631 try {
2632
2633
2634 try {
2635 if (scannerName != null) {
2636 lease = this.leases.removeLease(scannerName);
2637 }
2638 } catch (LeaseException le) {
2639
2640 LOG.info("Client tried to access missing scanner " + scannerName + " (no lease)");
2641 throw new UnknownScannerException("No lease for " + scannerName + ": " + le.getMessage());
2642 }
2643 List<Result> results = new ArrayList<Result>(nbRows);
2644 long currentScanResultSize = 0;
2645 List<KeyValue> values = new ArrayList<KeyValue>();
2646
2647
2648 HRegion region = getRegion(s.getRegionInfo().getRegionName());
2649 if (region != null && region.getCoprocessorHost() != null) {
2650 Boolean bypass = region.getCoprocessorHost().preScannerNext(s,
2651 results, nbRows);
2652 if (!results.isEmpty()) {
2653 for (Result r : results) {
2654 if (maxScannerResultSize < Long.MAX_VALUE){
2655 for (KeyValue kv : r.raw()) {
2656 currentScanResultSize += kv.heapSize();
2657 }
2658 }
2659 }
2660 }
2661 if (bypass != null) {
2662 return s.isFilterDone() && results.isEmpty() ? null
2663 : results.toArray(new Result[0]);
2664 }
2665 }
2666
2667 MultiVersionConsistencyControl.setThreadReadPoint(s.getMvccReadPoint());
2668 region.startRegionOperation();
2669 try {
2670 int i = 0;
2671 synchronized(s) {
2672 for (; i < nbRows
2673 && currentScanResultSize < maxScannerResultSize; ) {
2674
2675 boolean moreRows = s.nextRaw(values, SchemaMetrics.METRIC_NEXTSIZE);
2676 if (!values.isEmpty()) {
2677 if (maxScannerResultSize < Long.MAX_VALUE){
2678 for (KeyValue kv : values) {
2679 currentScanResultSize += kv.heapSize();
2680 }
2681 }
2682 results.add(new Result(values));
2683 i++;
2684 }
2685 if (!moreRows) {
2686 break;
2687 }
2688 values.clear();
2689 }
2690 }
2691 requestCount.addAndGet(i);
2692 region.readRequestsCount.add(i);
2693 region.setOpMetricsReadRequestCount(region.readRequestsCount.get());
2694 } finally {
2695 region.closeRegionOperation();
2696 }
2697
2698 if (region != null && region.getCoprocessorHost() != null) {
2699 region.getCoprocessorHost().postScannerNext(s, results, nbRows, true);
2700 }
2701
2702
2703
2704
2705 return s.isFilterDone() && results.isEmpty() ? null
2706 : results.toArray(new Result[0]);
2707 } catch (Throwable t) {
2708 if (t instanceof NotServingRegionException && scannerName != null) {
2709 this.scanners.remove(scannerName);
2710 }
2711 throw convertThrowableToIOE(cleanup(t));
2712 } finally {
2713
2714
2715 if (scannerName != null && this.scanners.containsKey(scannerName)) {
2716 if (lease != null) this.leases.addLease(lease);
2717 }
2718 }
2719 }
2720
2721 public void close(final long scannerId) throws IOException {
2722 String scannerName = String.valueOf(scannerId);
2723 RegionScanner s = scanners.get(scannerName);
2724 internalCloseScanner(s, scannerName);
2725 }
2726
2727 private void internalCloseScanner(final RegionScanner s, String scannerName)
2728 throws IOException {
2729 try {
2730 checkOpen();
2731 requestCount.incrementAndGet();
2732
2733 HRegion region = null;
2734 if (s != null) {
2735
2736 region = getRegion(s.getRegionInfo().getRegionName());
2737 if (region != null && region.getCoprocessorHost() != null) {
2738 if (region.getCoprocessorHost().preScannerClose(s)) {
2739 return;
2740 }
2741 }
2742 }
2743 RegionScanner toCloseScanner = s;
2744 if (scannerName != null) {
2745 toCloseScanner = scanners.remove(scannerName);
2746 }
2747 if (toCloseScanner != null) {
2748 toCloseScanner.close();
2749 if (scannerName != null) {
2750 this.leases.cancelLease(scannerName);
2751 }
2752
2753 if (region != null && region.getCoprocessorHost() != null) {
2754 region.getCoprocessorHost().postScannerClose(toCloseScanner);
2755 }
2756 }
2757 } catch (Throwable t) {
2758 throw convertThrowableToIOE(cleanup(t));
2759 }
2760 }
2761
2762 @Override
2763 public Result[] scan(byte[] regionName, Scan scan, int numberOfRows)
2764 throws IOException {
2765 RegionScanner s = internalOpenScanner(regionName, scan);
2766 try {
2767 Result[] results = internalNext(s, numberOfRows, null);
2768 return results;
2769 } finally {
2770 internalCloseScanner(s, null);
2771 }
2772 }
2773
2774
2775
2776
2777
2778 private class ScannerListener implements LeaseListener {
2779 private final String scannerName;
2780
2781 ScannerListener(final String n) {
2782 this.scannerName = n;
2783 }
2784
2785 public void leaseExpired() {
2786 RegionScanner s = scanners.remove(this.scannerName);
2787 if (s != null) {
2788 LOG.info("Scanner " + this.scannerName + " lease expired on region "
2789 + s.getRegionInfo().getRegionNameAsString());
2790 try {
2791 HRegion region = getRegion(s.getRegionInfo().getRegionName());
2792 if (region != null && region.getCoprocessorHost() != null) {
2793 region.getCoprocessorHost().preScannerClose(s);
2794 }
2795
2796 s.close();
2797 if (region != null && region.getCoprocessorHost() != null) {
2798 region.getCoprocessorHost().postScannerClose(s);
2799 }
2800 } catch (IOException e) {
2801 LOG.error("Closing scanner for "
2802 + s.getRegionInfo().getRegionNameAsString(), e);
2803 }
2804 } else {
2805 LOG.info("Scanner " + this.scannerName + " lease expired");
2806 }
2807 }
2808 }
2809
2810
2811
2812
2813 public void delete(final byte[] regionName, final Delete delete)
2814 throws IOException {
2815 checkOpen();
2816 try {
2817 boolean writeToWAL = delete.getWriteToWAL();
2818 this.requestCount.incrementAndGet();
2819 HRegion region = getRegion(regionName);
2820 if (!region.getRegionInfo().isMetaTable()) {
2821 this.cacheFlusher.reclaimMemStoreMemory();
2822 }
2823 Integer lid = getLockFromId(delete.getLockId());
2824 region.delete(delete, lid, writeToWAL);
2825 } catch (Throwable t) {
2826 throw convertThrowableToIOE(cleanup(t));
2827 }
2828 }
2829
2830 public int delete(final byte[] regionName, final List<Delete> deletes)
2831 throws IOException {
2832 checkOpen();
2833
2834 int i = 0;
2835 HRegion region = null;
2836 try {
2837 region = getRegion(regionName);
2838 if (!region.getRegionInfo().isMetaTable()) {
2839 this.cacheFlusher.reclaimMemStoreMemory();
2840 }
2841 int size = deletes.size();
2842 Integer[] locks = new Integer[size];
2843 for (Delete delete : deletes) {
2844 this.requestCount.incrementAndGet();
2845 locks[i] = getLockFromId(delete.getLockId());
2846 region.delete(delete, locks[i], delete.getWriteToWAL());
2847 i++;
2848 }
2849 } catch (WrongRegionException ex) {
2850 LOG.debug("Batch deletes: " + i, ex);
2851 return i;
2852 } catch (NotServingRegionException ex) {
2853 return i;
2854 } catch (Throwable t) {
2855 throw convertThrowableToIOE(cleanup(t));
2856 }
2857 return -1;
2858 }
2859
2860
2861
2862
2863 public long lockRow(byte[] regionName, byte[] row) throws IOException {
2864 checkOpen();
2865 NullPointerException npe = null;
2866 if (regionName == null) {
2867 npe = new NullPointerException("regionName is null");
2868 } else if (row == null) {
2869 npe = new NullPointerException("row to lock is null");
2870 }
2871 if (npe != null) {
2872 IOException io = new IOException("Invalid arguments to lockRow");
2873 io.initCause(npe);
2874 throw io;
2875 }
2876 requestCount.incrementAndGet();
2877 try {
2878 HRegion region = getRegion(regionName);
2879 if (region.getCoprocessorHost() != null) {
2880 region.getCoprocessorHost().preLockRow(regionName, row);
2881 }
2882 Integer r = region.obtainRowLock(row);
2883 long lockId = addRowLock(r, region);
2884 LOG.debug("Row lock " + lockId + " explicitly acquired by client");
2885 return lockId;
2886 } catch (Throwable t) {
2887 throw convertThrowableToIOE(cleanup(t, "Error obtaining row lock (fsOk: "
2888 + this.fsOk + ")"));
2889 }
2890 }
2891
2892 protected long addRowLock(Integer r, HRegion region)
2893 throws LeaseStillHeldException {
2894 long lockId = -1L;
2895 lockId = rand.nextLong();
2896 String lockName = String.valueOf(lockId);
2897 rowlocks.put(lockName, r);
2898 this.leases.createLease(lockName, new RowLockListener(lockName, region));
2899 return lockId;
2900 }
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912 Integer getLockFromId(long lockId) throws IOException {
2913 if (lockId == -1L) {
2914 return null;
2915 }
2916 String lockName = String.valueOf(lockId);
2917 Integer rl = rowlocks.get(lockName);
2918 if (rl == null) {
2919 throw new UnknownRowLockException("Invalid row lock");
2920 }
2921 this.leases.renewLease(lockName);
2922 return rl;
2923 }
2924
2925
2926
2927
2928 @Override
2929 @QosPriority(priority=HConstants.HIGH_QOS)
2930 public void unlockRow(byte[] regionName, long lockId) throws IOException {
2931 checkOpen();
2932 NullPointerException npe = null;
2933 if (regionName == null) {
2934 npe = new NullPointerException("regionName is null");
2935 } else if (lockId == -1L) {
2936 npe = new NullPointerException("lockId is null");
2937 }
2938 if (npe != null) {
2939 IOException io = new IOException("Invalid arguments to unlockRow");
2940 io.initCause(npe);
2941 throw io;
2942 }
2943 requestCount.incrementAndGet();
2944 try {
2945 HRegion region = getRegion(regionName);
2946 if (region.getCoprocessorHost() != null) {
2947 region.getCoprocessorHost().preUnLockRow(regionName, lockId);
2948 }
2949 String lockName = String.valueOf(lockId);
2950 Integer r = rowlocks.remove(lockName);
2951 if (r == null) {
2952 throw new UnknownRowLockException(lockName);
2953 }
2954 region.releaseRowLock(r);
2955 this.leases.cancelLease(lockName);
2956 LOG.debug("Row lock " + lockId
2957 + " has been explicitly released by client");
2958 } catch (Throwable t) {
2959 throw convertThrowableToIOE(cleanup(t));
2960 }
2961 }
2962
2963
2964
2965
2966
2967
2968 @Override
2969 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
2970 byte[] regionName) throws IOException {
2971 return bulkLoadHFiles(familyPaths, regionName, false);
2972 }
2973
2974
2975
2976
2977
2978
2979 @Override
2980 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
2981 byte[] regionName, boolean assignSeqNum) throws IOException {
2982 checkOpen();
2983 HRegion region = getRegion(regionName);
2984 boolean bypass = false;
2985 if (region.getCoprocessorHost() != null) {
2986 bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
2987 }
2988 boolean loaded = false;
2989 if (!bypass) {
2990 loaded = region.bulkLoadHFiles(familyPaths, assignSeqNum);
2991 }
2992 if (region.getCoprocessorHost() != null) {
2993 loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
2994 }
2995 return loaded;
2996 }
2997
2998 Map<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
2999
3000
3001
3002
3003
3004 private class RowLockListener implements LeaseListener {
3005 private final String lockName;
3006 private final HRegion region;
3007
3008 RowLockListener(final String lockName, final HRegion region) {
3009 this.lockName = lockName;
3010 this.region = region;
3011 }
3012
3013 public void leaseExpired() {
3014 LOG.info("Row Lock " + this.lockName + " lease expired");
3015 Integer r = rowlocks.remove(this.lockName);
3016 if (r != null) {
3017 region.releaseRowLock(r);
3018 }
3019 }
3020 }
3021
3022
3023
3024 @Override
3025 @QosPriority(priority=HConstants.HIGH_QOS)
3026 public RegionOpeningState openRegion(HRegionInfo region)
3027 throws IOException {
3028 return openRegion(region, -1);
3029 }
3030
3031 @Override
3032 @QosPriority(priority = HConstants.HIGH_QOS)
3033 public RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode)
3034 throws IOException {
3035 return openRegion(region, versionOfOfflineNode, null);
3036 }
3037
3038 private RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode,
3039 Map<String, HTableDescriptor> htds) throws IOException {
3040 checkOpen();
3041 HRegion onlineRegion = this.getFromOnlineRegions(region.getEncodedName());
3042 if (null != onlineRegion) {
3043
3044
3045 Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(
3046 this.catalogTracker, region.getRegionName());
3047 if (this.getServerName().equals(p.getSecond())) {
3048 LOG.warn("Attempted open of " + region.getEncodedName()
3049 + " but already online on this server");
3050 return RegionOpeningState.ALREADY_OPENED;
3051 } else {
3052 LOG.warn("The region " + region.getEncodedName()
3053 + " is online on this server but META does not have this server.");
3054 this.removeFromOnlineRegions(region.getEncodedName());
3055 }
3056 }
3057
3058
3059 boolean isNewRit = addRegionsInTransition(region, OPEN);
3060 if (!isNewRit) {
3061
3062 LOG.info("Receiving OPEN for the region:" +
3063 region.getRegionNameAsString() + " , which we are already trying to OPEN" +
3064 " - ignoring this new request for this region.");
3065 return RegionOpeningState.OPENED;
3066 }
3067 try {
3068 LOG.info("Received request to open region: " +
3069 region.getRegionNameAsString());
3070 HTableDescriptor htd = null;
3071 if (htds == null) {
3072 htd = this.tableDescriptors.get(region.getTableName());
3073 } else {
3074 htd = htds.get(region.getTableNameAsString());
3075 if (htd == null) {
3076 htd = this.tableDescriptors.get(region.getTableName());
3077 htds.put(region.getTableNameAsString(), htd);
3078 }
3079 }
3080
3081
3082
3083 int version = transitionZookeeperOfflineToOpening(region, versionOfOfflineNode);
3084
3085 if (region.isRootRegion()) {
3086 this.service.submit(new OpenRootHandler(this, this, region, htd, version));
3087 } else if (region.isMetaRegion()) {
3088 this.service.submit(new OpenMetaHandler(this, this, region, htd, version));
3089 } else {
3090 this.service.submit(new OpenRegionHandler(this, this, region, htd, version));
3091 }
3092 } catch (IOException ie) {
3093
3094 removeFromRegionsInTransition(region);
3095 throw ie;
3096 }
3097 return RegionOpeningState.OPENED;
3098 }
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111 int transitionZookeeperOfflineToOpening(final HRegionInfo hri, int versionOfOfflineNode)
3112 throws IOException {
3113
3114 int version = -1;
3115 try {
3116
3117 version = ZKAssign.transitionNode(this.zooKeeper, hri, this.getServerName(),
3118 EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_OPENING, versionOfOfflineNode);
3119 } catch (KeeperException e) {
3120 LOG.error("Error transition from OFFLINE to OPENING for region=" + hri.getEncodedName(), e);
3121 }
3122 if (version == -1) {
3123
3124
3125
3126
3127 throw new IOException("Failed transition from OFFLINE to OPENING for region="
3128 + hri.getEncodedName());
3129 }
3130 return version;
3131 }
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150 protected boolean addRegionsInTransition(final HRegionInfo region, final String currentAction)
3151 throws RegionAlreadyInTransitionException {
3152 boolean isOpen = currentAction.equals(OPEN);
3153 Boolean action = this.regionsInTransitionInRS.putIfAbsent(
3154 region.getEncodedNameAsBytes(), isOpen);
3155 if (action == null) return true;
3156 if (isOpen && action.booleanValue()) {
3157 return false;
3158 }
3159
3160 throw new RegionAlreadyInTransitionException("Received:" + currentAction
3161 + " for the region:" + region.getRegionNameAsString()
3162 + ", which we are already trying to " + (action ? OPEN : CLOSE) + ".");
3163 }
3164
3165 @Override
3166 @QosPriority(priority=HConstants.HIGH_QOS)
3167 public void openRegions(List<HRegionInfo> regions)
3168 throws IOException {
3169 checkOpen();
3170 LOG.info("Received request to open " + regions.size() + " region(s)");
3171 Map<String, HTableDescriptor> htds = new HashMap<String, HTableDescriptor>(regions.size());
3172 for (HRegionInfo region : regions) openRegion(region, -1, htds);
3173 }
3174
3175 @Override
3176 @QosPriority(priority=HConstants.HIGH_QOS)
3177 public boolean closeRegion(HRegionInfo region)
3178 throws IOException {
3179 return closeRegion(region, true, -1);
3180 }
3181
3182 @Override
3183 @QosPriority(priority=HConstants.HIGH_QOS)
3184 public boolean closeRegion(final HRegionInfo region,
3185 final int versionOfClosingNode)
3186 throws IOException {
3187 return closeRegion(region, true, versionOfClosingNode);
3188 }
3189
3190 @Override
3191 @QosPriority(priority=HConstants.HIGH_QOS)
3192 public boolean closeRegion(HRegionInfo region, final boolean zk)
3193 throws IOException {
3194 return closeRegion(region, zk, -1);
3195 }
3196
3197 @QosPriority(priority=HConstants.HIGH_QOS)
3198 protected boolean closeRegion(HRegionInfo region, final boolean zk,
3199 final int versionOfClosingNode)
3200 throws IOException {
3201 checkOpen();
3202
3203 HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
3204 if (actualRegion != null && actualRegion.getCoprocessorHost() != null) {
3205 actualRegion.getCoprocessorHost().preClose(false);
3206 }
3207 LOG.info("Received close region: " + region.getRegionNameAsString() +
3208 ". Version of ZK closing node:" + versionOfClosingNode);
3209 boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
3210 if (!hasit) {
3211 LOG.warn("Received close for region we are not serving; " +
3212 region.getEncodedName());
3213 throw new NotServingRegionException("Received close for "
3214 + region.getRegionNameAsString() + " but we are not serving it");
3215 }
3216 return closeRegion(region, false, zk, versionOfClosingNode);
3217 }
3218
3219 @Override
3220 @QosPriority(priority=HConstants.HIGH_QOS)
3221 public boolean closeRegion(byte[] encodedRegionName, boolean zk)
3222 throws IOException {
3223 return closeRegion(encodedRegionName, false, zk);
3224 }
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234 protected boolean closeRegion(HRegionInfo region, final boolean abort,
3235 final boolean zk) {
3236 return closeRegion(region, abort, zk, -1);
3237 }
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251 protected boolean closeRegion(HRegionInfo region, final boolean abort,
3252 final boolean zk, final int versionOfClosingNode) {
3253
3254 HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
3255 if (actualRegion != null && actualRegion.getCoprocessorHost() != null) {
3256 try {
3257 actualRegion.getCoprocessorHost().preClose(abort);
3258 } catch (IOException e) {
3259 LOG.warn(e);
3260 return false;
3261 }
3262 }
3263 try {
3264 addRegionsInTransition(region, CLOSE);
3265 } catch (RegionAlreadyInTransitionException rate) {
3266 LOG.warn("Received close for region we are already opening or closing; "
3267 + region.getEncodedName());
3268 return false;
3269 }
3270 boolean success = false;
3271 try {
3272 CloseRegionHandler crh = null;
3273 if (region.isRootRegion()) {
3274 crh = new CloseRootHandler(this, this, region, abort, zk, versionOfClosingNode);
3275 } else if (region.isMetaRegion()) {
3276 crh = new CloseMetaHandler(this, this, region, abort, zk, versionOfClosingNode);
3277 } else {
3278 crh = new CloseRegionHandler(this, this, region, abort, zk, versionOfClosingNode);
3279 }
3280 this.service.submit(crh);
3281 success = true;
3282 } finally {
3283
3284 if (!success) removeFromRegionsInTransition(region);
3285 }
3286 return true;
3287 }
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300 protected boolean closeRegion(byte[] encodedRegionName, final boolean abort,
3301 final boolean zk) throws IOException {
3302 String encodedRegionNameStr = Bytes.toString(encodedRegionName);
3303 HRegion region = this.getFromOnlineRegions(encodedRegionNameStr);
3304 if (null != region) {
3305 return closeRegion(region.getRegionInfo(), abort, zk);
3306 }
3307 LOG.error("The specified region name" + encodedRegionNameStr
3308 + " does not exist to close the region.");
3309 return false;
3310 }
3311
3312
3313
3314 @Override
3315 @QosPriority(priority=HConstants.HIGH_QOS)
3316 public void flushRegion(HRegionInfo regionInfo)
3317 throws NotServingRegionException, IOException {
3318 checkOpen();
3319 LOG.info("Flushing " + regionInfo.getRegionNameAsString());
3320 HRegion region = getRegion(regionInfo.getRegionName());
3321 boolean needsCompaction = region.flushcache();
3322 if (needsCompaction) {
3323 this.compactSplitThread.requestCompaction(region, "Compaction through user triggered flush");
3324 }
3325 }
3326
3327 @Override
3328 @QosPriority(priority=HConstants.HIGH_QOS)
3329 public void splitRegion(HRegionInfo regionInfo)
3330 throws NotServingRegionException, IOException {
3331 splitRegion(regionInfo, null);
3332 }
3333
3334 @Override
3335 public void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
3336 throws NotServingRegionException, IOException {
3337 checkOpen();
3338 HRegion region = getRegion(regionInfo.getRegionName());
3339 region.flushcache();
3340 region.forceSplit(splitPoint);
3341 compactSplitThread.requestSplit(region, region.checkSplit());
3342 }
3343
3344 @Override
3345 @QosPriority(priority=HConstants.HIGH_QOS)
3346 public void compactRegion(HRegionInfo regionInfo, boolean major)
3347 throws NotServingRegionException, IOException {
3348 compactRegion(regionInfo, major, null);
3349 }
3350
3351 @Override
3352 @QosPriority(priority=HConstants.HIGH_QOS)
3353 public void compactRegion(HRegionInfo regionInfo, boolean major, byte[] family)
3354 throws NotServingRegionException, IOException {
3355 checkOpen();
3356 HRegion region = getRegion(regionInfo.getRegionName());
3357 Store store = null;
3358 if (family != null) {
3359 store = region.getStore(family);
3360 if (store == null) {
3361 throw new IOException("column family " + Bytes.toString(family) +
3362 " does not exist in region " + new String(region.getRegionNameAsString()));
3363 }
3364 }
3365
3366 if (major) {
3367 if (family != null) {
3368 store.triggerMajorCompaction();
3369 } else {
3370 region.triggerMajorCompaction();
3371 }
3372 }
3373 String familyLogMsg = (family != null)?" for column family: " + Bytes.toString(family):"";
3374 LOG.trace("User-triggered compaction requested for region " +
3375 region.getRegionNameAsString() + familyLogMsg);
3376 String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
3377 if (family != null) {
3378 compactSplitThread.requestCompaction(region, store, log,
3379 Store.PRIORITY_USER, null);
3380 } else {
3381 compactSplitThread.requestCompaction(region, log,
3382 Store.PRIORITY_USER, null);
3383 }
3384 }
3385
3386
3387 public InfoServer getInfoServer() {
3388 return infoServer;
3389 }
3390
3391
3392
3393
3394 public boolean isStopped() {
3395 return this.stopped;
3396 }
3397
3398 @Override
3399 public boolean isStopping() {
3400 return this.stopping;
3401 }
3402
3403
3404
3405
3406
3407 public Configuration getConfiguration() {
3408 return conf;
3409 }
3410
3411
3412 ReentrantReadWriteLock.WriteLock getWriteLock() {
3413 return lock.writeLock();
3414 }
3415
3416 @Override
3417 @QosPriority(priority=HConstants.HIGH_QOS)
3418 public List<HRegionInfo> getOnlineRegions() throws IOException {
3419 checkOpen();
3420 List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
3421 for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
3422 list.add(e.getValue().getRegionInfo());
3423 }
3424 Collections.sort(list);
3425 return list;
3426 }
3427
3428 public int getNumberOfOnlineRegions() {
3429 return this.onlineRegions.size();
3430 }
3431
3432 boolean isOnlineRegionsEmpty() {
3433 return this.onlineRegions.isEmpty();
3434 }
3435
3436
3437
3438
3439
3440
3441 public byte [] getRegionStats(final String encodedRegionName)
3442 throws IOException {
3443 HRegion r = null;
3444 synchronized (this.onlineRegions) {
3445 r = this.onlineRegions.get(encodedRegionName);
3446 }
3447 if (r == null) return null;
3448 ObjectMapper mapper = new ObjectMapper();
3449 int stores = 0;
3450 int storefiles = 0;
3451 int storefileSizeMB = 0;
3452 int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
3453 int storefileIndexSizeMB = 0;
3454 long totalCompactingKVs = 0;
3455 long currentCompactedKVs = 0;
3456 synchronized (r.stores) {
3457 stores += r.stores.size();
3458 for (Store store : r.stores.values()) {
3459 storefiles += store.getStorefilesCount();
3460 storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
3461 storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
3462 }
3463 }
3464 Map<String, Integer> map = new TreeMap<String, Integer>();
3465 map.put("stores", stores);
3466 map.put("storefiles", storefiles);
3467 map.put("storefileSizeMB", storefileIndexSizeMB);
3468 map.put("memstoreSizeMB", memstoreSizeMB);
3469 StringWriter w = new StringWriter();
3470 mapper.writeValue(w, map);
3471 w.close();
3472 return Bytes.toBytes(w.toString());
3473 }
3474
3475
3476
3477
3478
3479
3480
3481 public Collection<HRegion> getOnlineRegionsLocalContext() {
3482 Collection<HRegion> regions = this.onlineRegions.values();
3483 return Collections.unmodifiableCollection(regions);
3484 }
3485
3486 @Override
3487 public void addToOnlineRegions(HRegion region) {
3488 this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
3489 }
3490
3491 @Override
3492 public boolean removeFromOnlineRegions(final String encodedName) {
3493 HRegion toReturn = null;
3494 toReturn = this.onlineRegions.remove(encodedName);
3495
3496
3497
3498
3499
3500
3501
3502 this.dynamicMetrics.clear();
3503 return toReturn != null;
3504 }
3505
3506
3507
3508
3509
3510 public SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
3511
3512 SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
3513 new Comparator<Long>() {
3514 public int compare(Long a, Long b) {
3515 return -1 * a.compareTo(b);
3516 }
3517 });
3518
3519 for (HRegion region : this.onlineRegions.values()) {
3520 sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
3521 }
3522 return sortedRegions;
3523 }
3524
3525 @Override
3526 public HRegion getFromOnlineRegions(final String encodedRegionName) {
3527 HRegion r = null;
3528 r = this.onlineRegions.get(encodedRegionName);
3529 return r;
3530 }
3531
3532
3533
3534
3535
3536
3537 public HRegion getOnlineRegion(final byte[] regionName) {
3538 return getFromOnlineRegions(HRegionInfo.encodeRegionName(regionName));
3539 }
3540
3541
3542 public AtomicInteger getRequestCount() {
3543 return this.requestCount;
3544 }
3545
3546
3547
3548
3549 public long getStartcode() {
3550 return this.startcode;
3551 }
3552
3553
3554 public FlushRequester getFlushRequester() {
3555 return this.cacheFlusher;
3556 }
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566 protected HRegion getRegion(final byte[] regionName)
3567 throws NotServingRegionException {
3568 HRegion region = null;
3569 region = getOnlineRegion(regionName);
3570 if (region == null) {
3571 throw new NotServingRegionException("Region is not online: " +
3572 Bytes.toStringBinary(regionName));
3573 }
3574 return region;
3575 }
3576
3577
3578
3579
3580
3581
3582
3583 protected HRegionInfo[] getMostLoadedRegions() {
3584 ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
3585 for (HRegion r : onlineRegions.values()) {
3586 if (!r.isAvailable()) {
3587 continue;
3588 }
3589 if (regions.size() < numRegionsToReport) {
3590 regions.add(r.getRegionInfo());
3591 } else {
3592 break;
3593 }
3594 }
3595 return regions.toArray(new HRegionInfo[regions.size()]);
3596 }
3597
3598
3599
3600
3601
3602
3603 protected void checkOpen() throws IOException {
3604 if (this.stopped || this.abortRequested) {
3605 throw new RegionServerStoppedException("Server " + getServerName() +
3606 " not running" + (this.abortRequested ? ", aborting" : ""));
3607 }
3608 if (!fsOk) {
3609 throw new RegionServerStoppedException("File system not available");
3610 }
3611 }
3612
3613 @Override
3614 @QosPriority(priority=HConstants.HIGH_QOS)
3615 public ProtocolSignature getProtocolSignature(
3616 String protocol, long version, int clientMethodsHashCode)
3617 throws IOException {
3618 if (protocol.equals(HRegionInterface.class.getName())) {
3619 return new ProtocolSignature(HRegionInterface.VERSION, null);
3620 }
3621 throw new IOException("Unknown protocol: " + protocol);
3622 }
3623
3624 @Override
3625 @QosPriority(priority=HConstants.HIGH_QOS)
3626 public long getProtocolVersion(final String protocol, final long clientVersion)
3627 throws IOException {
3628 if (protocol.equals(HRegionInterface.class.getName())) {
3629 return HRegionInterface.VERSION;
3630 }
3631 throw new IOException("Unknown protocol: " + protocol);
3632 }
3633
3634 @Override
3635 public Leases getLeases() {
3636 return leases;
3637 }
3638
3639
3640
3641
3642 protected Path getRootDir() {
3643 return rootDir;
3644 }
3645
3646
3647
3648
3649 public FileSystem getFileSystem() {
3650 return fs;
3651 }
3652
3653
3654
3655
3656
3657 public HServerInfo getServerInfo() {
3658 try {
3659 return getHServerInfo();
3660 } catch (IOException e) {
3661 e.printStackTrace();
3662 }
3663 return null;
3664 }
3665
3666 @Override
3667 public void mutateRow(byte[] regionName, RowMutations rm)
3668 throws IOException {
3669 checkOpen();
3670 if (regionName == null) {
3671 throw new IOException("Invalid arguments to mutateRow " +
3672 "regionName is null");
3673 }
3674 requestCount.incrementAndGet();
3675 try {
3676 HRegion region = getRegion(regionName);
3677 if (!region.getRegionInfo().isMetaTable()) {
3678 this.cacheFlusher.reclaimMemStoreMemory();
3679 }
3680 region.mutateRow(rm);
3681 } catch (IOException e) {
3682 checkFileSystem();
3683 throw e;
3684 }
3685 }
3686
3687 @Override
3688 public Result append(byte[] regionName, Append append)
3689 throws IOException {
3690 checkOpen();
3691 if (regionName == null) {
3692 throw new IOException("Invalid arguments to increment " +
3693 "regionName is null");
3694 }
3695 requestCount.incrementAndGet();
3696 try {
3697 HRegion region = getRegion(regionName);
3698 Integer lock = getLockFromId(append.getLockId());
3699 Append appVal = append;
3700 Result resVal;
3701 if (region.getCoprocessorHost() != null) {
3702 resVal = region.getCoprocessorHost().preAppend(appVal);
3703 if (resVal != null) {
3704 return resVal;
3705 }
3706 }
3707 resVal = region.append(appVal, lock, append.getWriteToWAL());
3708 if (region.getCoprocessorHost() != null) {
3709 region.getCoprocessorHost().postAppend(appVal, resVal);
3710 }
3711 return resVal;
3712 } catch (IOException e) {
3713 checkFileSystem();
3714 throw e;
3715 }
3716 }
3717
3718 @Override
3719 public Result increment(byte[] regionName, Increment increment)
3720 throws IOException {
3721 checkOpen();
3722 if (regionName == null) {
3723 throw new IOException("Invalid arguments to increment " +
3724 "regionName is null");
3725 }
3726 requestCount.incrementAndGet();
3727 try {
3728 HRegion region = getRegion(regionName);
3729 Integer lock = getLockFromId(increment.getLockId());
3730 Increment incVal = increment;
3731 Result resVal;
3732 if (region.getCoprocessorHost() != null) {
3733 resVal = region.getCoprocessorHost().preIncrement(incVal);
3734 if (resVal != null) {
3735 return resVal;
3736 }
3737 }
3738 resVal = region.increment(incVal, lock,
3739 increment.getWriteToWAL());
3740 if (region.getCoprocessorHost() != null) {
3741 resVal = region.getCoprocessorHost().postIncrement(incVal, resVal);
3742 }
3743 return resVal;
3744 } catch (IOException e) {
3745 checkFileSystem();
3746 throw e;
3747 }
3748 }
3749
3750
3751 public long incrementColumnValue(byte[] regionName, byte[] row,
3752 byte[] family, byte[] qualifier, long amount, boolean writeToWAL)
3753 throws IOException {
3754 checkOpen();
3755
3756 if (regionName == null) {
3757 throw new IOException("Invalid arguments to incrementColumnValue "
3758 + "regionName is null");
3759 }
3760 requestCount.incrementAndGet();
3761 try {
3762 HRegion region = getRegion(regionName);
3763 if (region.getCoprocessorHost() != null) {
3764 Long amountVal = region.getCoprocessorHost().preIncrementColumnValue(row,
3765 family, qualifier, amount, writeToWAL);
3766 if (amountVal != null) {
3767 return amountVal.longValue();
3768 }
3769 }
3770 long retval = region.incrementColumnValue(row, family, qualifier, amount,
3771 writeToWAL);
3772 if (region.getCoprocessorHost() != null) {
3773 retval = region.getCoprocessorHost().postIncrementColumnValue(row,
3774 family, qualifier, amount, writeToWAL, retval);
3775 }
3776 return retval;
3777 } catch (IOException e) {
3778 checkFileSystem();
3779 throw e;
3780 }
3781 }
3782
3783
3784
3785
3786 @Override
3787 @QosPriority(priority=HConstants.HIGH_QOS)
3788 public HServerInfo getHServerInfo() throws IOException {
3789 checkOpen();
3790 return new HServerInfo(new HServerAddress(this.isa),
3791 this.startcode, this.webuiport);
3792 }
3793
3794 @SuppressWarnings("unchecked")
3795 @Override
3796 public <R> MultiResponse multi(MultiAction<R> multi) throws IOException {
3797 checkOpen();
3798 MultiResponse response = new MultiResponse();
3799 for (Map.Entry<byte[], List<Action<R>>> e : multi.actions.entrySet()) {
3800 byte[] regionName = e.getKey();
3801 List<Action<R>> actionsForRegion = e.getValue();
3802
3803
3804
3805 Collections.sort(actionsForRegion);
3806 Row action;
3807 List<Action<R>> mutations = new ArrayList<Action<R>>();
3808 for (Action<R> a : actionsForRegion) {
3809 action = a.getAction();
3810 int originalIndex = a.getOriginalIndex();
3811
3812 try {
3813 if (action instanceof Delete || action instanceof Put) {
3814 mutations.add(a);
3815 } else if (action instanceof Get) {
3816 response.add(regionName, originalIndex,
3817 get(regionName, (Get)action));
3818 } else if (action instanceof Exec) {
3819 ExecResult result = execCoprocessor(regionName, (Exec)action);
3820 response.add(regionName, new Pair<Integer, Object>(
3821 a.getOriginalIndex(), result.getValue()
3822 ));
3823 } else if (action instanceof Increment) {
3824 response.add(regionName, originalIndex,
3825 increment(regionName, (Increment)action));
3826 } else if (action instanceof Append) {
3827 response.add(regionName, originalIndex,
3828 append(regionName, (Append)action));
3829 } else if (action instanceof RowMutations) {
3830 mutateRow(regionName, (RowMutations)action);
3831 response.add(regionName, originalIndex, new Result());
3832 } else {
3833 LOG.debug("Error: invalid Action, row must be a Get, Delete, " +
3834 "Put, Exec, Increment, or Append.");
3835 throw new DoNotRetryIOException("Invalid Action, row must be a " +
3836 "Get, Delete, Put, Exec, Increment, or Append.");
3837 }
3838 } catch (IOException ex) {
3839 response.add(regionName, originalIndex, ex);
3840 }
3841 }
3842
3843
3844
3845
3846 if (!mutations.isEmpty()) {
3847 try {
3848 HRegion region = getRegion(regionName);
3849
3850 if (!region.getRegionInfo().isMetaTable()) {
3851 this.cacheFlusher.reclaimMemStoreMemory();
3852 }
3853
3854 List<Pair<Mutation,Integer>> mutationsWithLocks =
3855 Lists.newArrayListWithCapacity(mutations.size());
3856 for (Action<R> a : mutations) {
3857 Mutation m = (Mutation) a.getAction();
3858
3859 Integer lock;
3860 try {
3861 lock = getLockFromId(m.getLockId());
3862 } catch (UnknownRowLockException ex) {
3863 response.add(regionName, a.getOriginalIndex(), ex);
3864 continue;
3865 }
3866 mutationsWithLocks.add(new Pair<Mutation, Integer>(m, lock));
3867 }
3868
3869 this.requestCount.addAndGet(mutations.size());
3870
3871 OperationStatus[] codes =
3872 region.batchMutate(mutationsWithLocks.toArray(new Pair[]{}));
3873
3874 for( int i = 0 ; i < codes.length ; i++) {
3875 OperationStatus code = codes[i];
3876
3877 Action<R> theAction = mutations.get(i);
3878 Object result = null;
3879
3880 if (code.getOperationStatusCode() == OperationStatusCode.SUCCESS) {
3881 result = new Result();
3882 } else if (code.getOperationStatusCode()
3883 == OperationStatusCode.SANITY_CHECK_FAILURE) {
3884
3885
3886
3887 result = new DoNotRetryIOException(code.getExceptionMsg());
3888 } else if (code.getOperationStatusCode() == OperationStatusCode.BAD_FAMILY) {
3889 result = new NoSuchColumnFamilyException(code.getExceptionMsg());
3890 }
3891
3892
3893 response.add(regionName, theAction.getOriginalIndex(), result);
3894 }
3895 } catch (IOException ioe) {
3896
3897 for (Action<R> a: mutations) {
3898 response.add(regionName, a.getOriginalIndex(), ioe);
3899 }
3900 }
3901 }
3902 }
3903 return response;
3904 }
3905
3906
3907
3908
3909
3910
3911
3912
3913
3914
3915
3916
3917
3918
3919
3920
3921
3922
3923 @Override
3924 public ExecResult execCoprocessor(byte[] regionName, Exec call)
3925 throws IOException {
3926 checkOpen();
3927 requestCount.incrementAndGet();
3928 try {
3929 HRegion region = getRegion(regionName);
3930 return region.exec(call);
3931 } catch (Throwable t) {
3932 throw convertThrowableToIOE(cleanup(t));
3933 }
3934 }
3935
3936 public String toString() {
3937 return getServerName().toString();
3938 }
3939
3940
3941
3942
3943
3944
3945 public int getThreadWakeFrequency() {
3946 return threadWakeFrequency;
3947 }
3948
3949 @Override
3950 public ZooKeeperWatcher getZooKeeper() {
3951 return zooKeeper;
3952 }
3953
3954 @Override
3955 public ServerName getServerName() {
3956
3957 return this.serverNameFromMasterPOV == null?
3958 new ServerName(this.isa.getHostName(), this.isa.getPort(), this.startcode):
3959 this.serverNameFromMasterPOV;
3960 }
3961
3962 @Override
3963 public CompactionRequestor getCompactionRequester() {
3964 return this.compactSplitThread;
3965 }
3966
3967 public ZooKeeperWatcher getZooKeeperWatcher() {
3968 return this.zooKeeper;
3969 }
3970
3971 public RegionServerCoprocessorHost getCoprocessorHost(){
3972 return this.rsHost;
3973 }
3974
3975 @Override
3976 public boolean removeFromRegionsInTransition(final HRegionInfo hri) {
3977 return this.regionsInTransitionInRS.remove(hri.getEncodedNameAsBytes());
3978 }
3979
3980 @Override
3981 public boolean containsKeyInRegionsInTransition(final HRegionInfo hri) {
3982 return this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes());
3983 }
3984
3985 public ExecutorService getExecutorService() {
3986 return service;
3987 }
3988
3989
3990
3991
3992
3993
3994
3995
3996 static private void createNewReplicationInstance(Configuration conf,
3997 HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
3998
3999
4000 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
4001 return;
4002 }
4003
4004
4005 String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
4006 HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
4007
4008
4009 String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
4010 HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
4011
4012
4013
4014 if (sourceClassname.equals(sinkClassname)) {
4015 server.replicationSourceHandler = (ReplicationSourceService)
4016 newReplicationInstance(sourceClassname,
4017 conf, server, fs, logDir, oldLogDir);
4018 server.replicationSinkHandler = (ReplicationSinkService)
4019 server.replicationSourceHandler;
4020 }
4021 else {
4022 server.replicationSourceHandler = (ReplicationSourceService)
4023 newReplicationInstance(sourceClassname,
4024 conf, server, fs, logDir, oldLogDir);
4025 server.replicationSinkHandler = (ReplicationSinkService)
4026 newReplicationInstance(sinkClassname,
4027 conf, server, fs, logDir, oldLogDir);
4028 }
4029 }
4030
4031 static private ReplicationService newReplicationInstance(String classname,
4032 Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
4033 Path oldLogDir) throws IOException{
4034
4035 Class<?> clazz = null;
4036 try {
4037 ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
4038 clazz = Class.forName(classname, true, classLoader);
4039 } catch (java.lang.ClassNotFoundException nfe) {
4040 throw new IOException("Cound not find class for " + classname);
4041 }
4042
4043
4044 ReplicationService service = (ReplicationService)
4045 ReflectionUtils.newInstance(clazz, conf);
4046 service.initialize(server, fs, logDir, oldLogDir);
4047 return service;
4048 }
4049
4050
4051
4052
4053
4054
4055 public static Thread startRegionServer(final HRegionServer hrs)
4056 throws IOException {
4057 return startRegionServer(hrs, "regionserver" + hrs.isa.getPort());
4058 }
4059
4060
4061
4062
4063
4064
4065
4066 public static Thread startRegionServer(final HRegionServer hrs,
4067 final String name) throws IOException {
4068 Thread t = new Thread(hrs);
4069 t.setName(name);
4070 t.start();
4071
4072
4073 ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
4074 .getConfiguration()), hrs, t);
4075 return t;
4076 }
4077
4078
4079
4080
4081
4082
4083
4084
4085 public static HRegionServer constructRegionServer(
4086 Class<? extends HRegionServer> regionServerClass,
4087 final Configuration conf2) {
4088 try {
4089 Constructor<? extends HRegionServer> c = regionServerClass
4090 .getConstructor(Configuration.class);
4091 return c.newInstance(conf2);
4092 } catch (Exception e) {
4093 throw new RuntimeException("Failed construction of " + "Regionserver: "
4094 + regionServerClass.toString(), e);
4095 }
4096 }
4097
4098 @Override
4099 @QosPriority(priority=HConstants.REPLICATION_QOS)
4100 public void replicateLogEntries(final HLog.Entry[] entries)
4101 throws IOException {
4102 checkOpen();
4103 if (this.replicationSinkHandler == null) return;
4104 this.replicationSinkHandler.replicateLogEntries(entries);
4105 }
4106
4107
4108
4109
4110 public static void main(String[] args) throws Exception {
4111 VersionInfo.logVersion();
4112 Configuration conf = HBaseConfiguration.create();
4113 @SuppressWarnings("unchecked")
4114 Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
4115 .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
4116
4117 new HRegionServerCommandLine(regionServerClass).doMain(args);
4118 }
4119
4120 @Override
4121 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries() throws IOException {
4122 BlockCache c = new CacheConfig(this.conf).getBlockCache();
4123 return c.getBlockCacheColumnFamilySummaries(this.conf);
4124 }
4125
4126 @Override
4127 public byte[][] rollHLogWriter() throws IOException, FailedLogCloseException {
4128 HLog wal = this.getWAL();
4129 return wal.rollWriter(true);
4130 }
4131
4132
4133
4134
4135
4136
4137
4138
4139
4140
4141
4142 public List<HRegion> getOnlineRegions(byte[] tableName) {
4143 List<HRegion> tableRegions = new ArrayList<HRegion>();
4144 synchronized (this.onlineRegions) {
4145 for (HRegion region: this.onlineRegions.values()) {
4146 HRegionInfo regionInfo = region.getRegionInfo();
4147 if(Bytes.equals(regionInfo.getTableName(), tableName)) {
4148 tableRegions.add(region);
4149 }
4150 }
4151 }
4152 return tableRegions;
4153 }
4154
4155
4156 public String[] getCoprocessors() {
4157 TreeSet<String> coprocessors = new TreeSet<String>(
4158 this.hlog.getCoprocessorHost().getCoprocessors());
4159 Collection<HRegion> regions = getOnlineRegionsLocalContext();
4160 for (HRegion region: regions) {
4161 coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
4162 }
4163 return coprocessors.toArray(new String[0]);
4164 }
4165
4166
4167
4168
4169 @SuppressWarnings("deprecation")
4170 void registerMBean() {
4171 MXBeanImpl mxBeanInfo = MXBeanImpl.init(this);
4172 mxBean = MBeanUtil.registerMBean("RegionServer", "RegionServer",
4173 mxBeanInfo);
4174 LOG.info("Registered RegionServer MXBean");
4175 }
4176
4177
4178
4179
4180
4181
4182
4183
4184 public String getCompactionState(final byte[] regionName) throws IOException {
4185 checkOpen();
4186 requestCount.incrementAndGet();
4187 HRegion region = getRegion(regionName);
4188 HRegionInfo info = region.getRegionInfo();
4189 return CompactionRequest.getCompactionState(info.getRegionId()).name();
4190 }
4191
4192 public long getResponseQueueSize(){
4193 if (server != null) {
4194 return server.getResponseQueueSize();
4195 }
4196 return 0;
4197 }
4198
4199 private boolean isHealthCheckerConfigured() {
4200 String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
4201 return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
4202 }
4203
4204
4205
4206
4207 public CompactSplitThread getCompactSplitThread() {
4208 return this.compactSplitThread;
4209 }
4210 }