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