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