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