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