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