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