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