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