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