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