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