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