1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.master;
20
21 import java.io.IOException;
22 import java.lang.reflect.Constructor;
23 import java.lang.reflect.InvocationTargetException;
24 import java.net.InetAddress;
25 import java.net.InetSocketAddress;
26 import java.net.UnknownHostException;
27 import java.util.ArrayList;
28 import java.util.Collection;
29 import java.util.Collections;
30 import java.util.Comparator;
31 import java.util.HashSet;
32 import java.util.List;
33 import java.util.Map;
34 import java.util.Set;
35 import java.util.concurrent.Callable;
36 import java.util.concurrent.ExecutionException;
37 import java.util.concurrent.Executors;
38 import java.util.concurrent.Future;
39 import java.util.concurrent.TimeUnit;
40 import java.util.concurrent.atomic.AtomicReference;
41
42 import javax.management.ObjectName;
43
44 import org.apache.commons.logging.Log;
45 import org.apache.commons.logging.LogFactory;
46 import org.apache.hadoop.classification.InterfaceAudience;
47 import org.apache.hadoop.conf.Configuration;
48 import org.apache.hadoop.fs.Path;
49 import org.apache.hadoop.hbase.Abortable;
50 import org.apache.hadoop.hbase.Chore;
51 import org.apache.hadoop.hbase.ClusterId;
52 import org.apache.hadoop.hbase.ClusterStatus;
53 import org.apache.hadoop.hbase.HBaseIOException;
54 import org.apache.hadoop.hbase.HColumnDescriptor;
55 import org.apache.hadoop.hbase.HConstants;
56 import org.apache.hadoop.hbase.HRegionInfo;
57 import org.apache.hadoop.hbase.HTableDescriptor;
58 import org.apache.hadoop.hbase.HealthCheckChore;
59 import org.apache.hadoop.hbase.MasterNotRunningException;
60 import org.apache.hadoop.hbase.NamespaceDescriptor;
61 import org.apache.hadoop.hbase.NamespaceNotFoundException;
62 import org.apache.hadoop.hbase.PleaseHoldException;
63 import org.apache.hadoop.hbase.Server;
64 import org.apache.hadoop.hbase.ServerLoad;
65 import org.apache.hadoop.hbase.ServerName;
66 import org.apache.hadoop.hbase.TableDescriptors;
67 import org.apache.hadoop.hbase.TableName;
68 import org.apache.hadoop.hbase.TableNotDisabledException;
69 import org.apache.hadoop.hbase.TableNotFoundException;
70 import org.apache.hadoop.hbase.UnknownRegionException;
71 import org.apache.hadoop.hbase.catalog.CatalogTracker;
72 import org.apache.hadoop.hbase.catalog.MetaReader;
73 import org.apache.hadoop.hbase.client.HConnectionManager;
74 import org.apache.hadoop.hbase.client.MetaScanner;
75 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
76 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
77 import org.apache.hadoop.hbase.client.Result;
78 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
79 import org.apache.hadoop.hbase.exceptions.DeserializationException;
80 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
81 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
82 import org.apache.hadoop.hbase.executor.ExecutorService;
83 import org.apache.hadoop.hbase.executor.ExecutorType;
84 import org.apache.hadoop.hbase.ipc.RpcServer;
85 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
86 import org.apache.hadoop.hbase.ipc.RequestContext;
87 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
88 import org.apache.hadoop.hbase.ipc.ServerRpcController;
89 import org.apache.hadoop.hbase.master.RegionState.State;
90 import org.apache.hadoop.hbase.master.balancer.BalancerChore;
91 import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
92 import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
93 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
94 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
95 import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
96 import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
97 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
98 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
99 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
100 import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
101 import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
102 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
103 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
104 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
105 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
106 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
107 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
108 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
109 import org.apache.hadoop.hbase.protobuf.RequestConverter;
110 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
111 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
112 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
113 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
114 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
115 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
116 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
117 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
118 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
119 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
121 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
122 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
123 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
124 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
125 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
126 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
127 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
128 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
129 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
130 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
131 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
132 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
133 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
134 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
135 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
136 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
137 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
138 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
139 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
140 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
141 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
142 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
143 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
144 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
145 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
146 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
147 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
148 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
149 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
150 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
151 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
152 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
153 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
154 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
155 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
156 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesResponse;
157 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
158 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
159 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
160 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
161 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
162 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
163 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
164 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
165 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
166 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
167 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
168 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
169 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
170 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
171 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
172 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
173 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
174 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
175 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
176 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
177 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
178 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
179 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
180 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
181 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
182 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
183 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
184 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
185 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
186 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
187 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
188 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
189 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
190 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
191 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
192 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
193 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
194 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
195 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
196 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
197 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
198 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
199 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse;
200 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
201 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
202 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
203 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
204 import org.apache.hadoop.hbase.replication.regionserver.Replication;
205 import org.apache.hadoop.hbase.security.UserProvider;
206 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
207 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
208 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
209 import org.apache.hadoop.hbase.util.Bytes;
210 import org.apache.hadoop.hbase.util.CompressionTest;
211 import org.apache.hadoop.hbase.util.FSTableDescriptors;
212 import org.apache.hadoop.hbase.util.FSUtils;
213 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
214 import org.apache.hadoop.hbase.util.HasThread;
215 import org.apache.hadoop.hbase.util.InfoServer;
216 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
217 import org.apache.hadoop.hbase.util.Pair;
218 import org.apache.hadoop.hbase.util.Sleeper;
219 import org.apache.hadoop.hbase.util.Strings;
220 import org.apache.hadoop.hbase.util.Threads;
221 import org.apache.hadoop.hbase.util.VersionInfo;
222 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
223 import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
224 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
225 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
226 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
227 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
228 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
229 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
230 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
231 import org.apache.hadoop.metrics.util.MBeanUtil;
232 import org.apache.hadoop.net.DNS;
233 import org.apache.zookeeper.KeeperException;
234 import org.apache.zookeeper.Watcher;
235
236 import com.google.common.collect.Lists;
237 import com.google.common.collect.Maps;
238 import com.google.protobuf.Descriptors;
239 import com.google.protobuf.Message;
240 import com.google.protobuf.RpcCallback;
241 import com.google.protobuf.RpcController;
242 import com.google.protobuf.Service;
243 import com.google.protobuf.ServiceException;
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260 @InterfaceAudience.Private
261 @SuppressWarnings("deprecation")
262 public class HMaster extends HasThread implements MasterProtos.MasterService.BlockingInterface,
263 RegionServerStatusProtos.RegionServerStatusService.BlockingInterface,
264 MasterServices, Server {
265 private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
266
267
268
269 public static final String MASTER = "master";
270
271
272 private final Configuration conf;
273
274 private InfoServer infoServer;
275
276
277 private ZooKeeperWatcher zooKeeper;
278
279 private ActiveMasterManager activeMasterManager;
280
281 RegionServerTracker regionServerTracker;
282
283 private DrainingServerTracker drainingServerTracker;
284
285 private LoadBalancerTracker loadBalancerTracker;
286
287 private MasterAddressTracker masterAddressTracker;
288
289
290 private final RpcServerInterface rpcServer;
291 private JvmPauseMonitor pauseMonitor;
292
293
294 private volatile boolean rpcServerOpen = false;
295
296
297 private TableNamespaceManager tableNamespaceManager;
298 private NamespaceJanitor namespaceJanitorChore;
299
300
301
302
303 private final InetSocketAddress isa;
304
305
306 private final MetricsMaster metricsMaster;
307
308 private MasterFileSystem fileSystemManager;
309
310
311 ServerManager serverManager;
312
313
314 AssignmentManager assignmentManager;
315
316 private CatalogTracker catalogTracker;
317
318 private ClusterStatusTracker clusterStatusTracker;
319
320
321
322
323 private MemoryBoundedLogMessageBuffer rsFatals;
324
325
326
327 private volatile boolean stopped = false;
328
329 private volatile boolean abort = false;
330
331 private volatile boolean isActiveMaster = false;
332
333
334
335 volatile boolean initialized = false;
336
337
338 private volatile boolean serverShutdownHandlerEnabled = false;
339
340
341 ExecutorService executorService;
342
343 private LoadBalancer balancer;
344 private Thread balancerChore;
345 private Thread clusterStatusChore;
346 private ClusterStatusPublisher clusterStatusPublisherChore = null;
347
348 private CatalogJanitor catalogJanitorChore;
349 private LogCleaner logCleaner;
350 private HFileCleaner hfileCleaner;
351
352 private MasterCoprocessorHost cpHost;
353 private final ServerName serverName;
354
355 private TableDescriptors tableDescriptors;
356
357
358 private TableLockManager tableLockManager;
359
360
361 private long masterStartTime;
362 private long masterActiveTime;
363
364
365 private final int msgInterval;
366
367
368
369 private ObjectName mxBean = null;
370
371
372 private final boolean masterCheckCompression;
373
374 private SpanReceiverHost spanReceiverHost;
375
376 private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
377
378
379 private SnapshotManager snapshotManager;
380
381
382 private HealthCheckChore healthCheckChore;
383
384
385
386
387
388 private final boolean distributedLogReplay;
389
390
391 private volatile boolean initializationBeforeMetaAssignment = false;
392
393
394 private List<ZooKeeperListener> registeredZKListenersBeforeRecovery;
395
396
397
398
399
400
401
402
403
404
405
406
407
408 public HMaster(final Configuration conf)
409 throws IOException, KeeperException, InterruptedException {
410 this.conf = new Configuration(conf);
411
412 this.conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
413 FSUtils.setupShortCircuitRead(conf);
414
415 String hostname = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
416 conf.get("hbase.master.dns.interface", "default"),
417 conf.get("hbase.master.dns.nameserver", "default")));
418 int port = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT);
419
420 InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
421 if (initialIsa.getAddress() == null) {
422 throw new IllegalArgumentException("Failed resolve of hostname " + initialIsa);
423 }
424
425 String bindAddress = conf.get("hbase.master.ipc.address");
426 if (bindAddress != null) {
427 initialIsa = new InetSocketAddress(bindAddress, port);
428 if (initialIsa.getAddress() == null) {
429 throw new IllegalArgumentException("Failed resolve of bind address " + initialIsa);
430 }
431 }
432 String name = "master/" + initialIsa.toString();
433
434 HConnectionManager.setServerSideHConnectionRetries(this.conf, name, LOG);
435 int numHandlers = conf.getInt("hbase.master.handler.count",
436 conf.getInt("hbase.regionserver.handler.count", 25));
437 this.rpcServer = new RpcServer(this, name, getServices(),
438 initialIsa,
439 numHandlers,
440 0,
441 conf,
442 0);
443
444 this.isa = this.rpcServer.getListenerAddress();
445
446 this.serverName = ServerName.valueOf(hostname, this.isa.getPort(), System.currentTimeMillis());
447 this.rsFatals = new MemoryBoundedLogMessageBuffer(
448 conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
449
450
451 ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
452 "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
453
454
455 UserProvider provider = UserProvider.instantiate(conf);
456 provider.login("hbase.master.keytab.file",
457 "hbase.master.kerberos.principal", this.isa.getHostName());
458
459 LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) +
460 ", hbase.cluster.distributed=" + this.conf.getBoolean("hbase.cluster.distributed", false));
461
462
463 setName(MASTER + ":" + this.serverName.toShortString());
464
465 Replication.decorateMasterConfiguration(this.conf);
466
467
468
469 if (this.conf.get("mapred.task.id") == null) {
470 this.conf.set("mapred.task.id", "hb_m_" + this.serverName.toString());
471 }
472
473 this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" + isa.getPort(), this, true);
474 this.rpcServer.startThreads();
475 this.pauseMonitor = new JvmPauseMonitor(conf);
476 this.pauseMonitor.start();
477
478
479 this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
480
481
482 this.masterCheckCompression = conf.getBoolean("hbase.master.check.compression", true);
483
484 this.metricsMaster = new MetricsMaster( new MetricsMasterWrapperImpl(this));
485
486
487 int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
488 HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
489 if (isHealthCheckerConfigured()) {
490 healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
491 }
492
493
494 boolean shouldPublish = conf.getBoolean(HConstants.STATUS_PUBLISHED,
495 HConstants.STATUS_PUBLISHED_DEFAULT);
496 Class<? extends ClusterStatusPublisher.Publisher> publisherClass =
497 conf.getClass(ClusterStatusPublisher.STATUS_PUBLISHER_CLASS,
498 ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS,
499 ClusterStatusPublisher.Publisher.class);
500
501 if (shouldPublish) {
502 if (publisherClass == null) {
503 LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " +
504 ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS +
505 " is not set - not publishing status");
506 } else {
507 clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
508 Threads.setDaemonThreadRunning(clusterStatusPublisherChore.getThread());
509 }
510 }
511
512 distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
513 HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
514 }
515
516
517
518
519 private List<BlockingServiceAndInterface> getServices() {
520 List<BlockingServiceAndInterface> bssi = new ArrayList<BlockingServiceAndInterface>(3);
521 bssi.add(new BlockingServiceAndInterface(
522 MasterProtos.MasterService.newReflectiveBlockingService(this),
523 MasterProtos.MasterService.BlockingInterface.class));
524 bssi.add(new BlockingServiceAndInterface(
525 RegionServerStatusProtos.RegionServerStatusService.newReflectiveBlockingService(this),
526 RegionServerStatusProtos.RegionServerStatusService.BlockingInterface.class));
527 return bssi;
528 }
529
530
531
532
533
534
535
536
537 private static void stallIfBackupMaster(final Configuration c,
538 final ActiveMasterManager amm)
539 throws InterruptedException {
540
541 if (!c.getBoolean(HConstants.MASTER_TYPE_BACKUP,
542 HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
543 return;
544 }
545 LOG.debug("HMaster started in backup mode. " +
546 "Stalling until master znode is written.");
547
548
549 while (!amm.isActiveMaster()) {
550 LOG.debug("Waiting for master address ZNode to be written " +
551 "(Also watching cluster state node)");
552 Thread.sleep(
553 c.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT));
554 }
555
556 }
557
558 MetricsMaster getMetrics() {
559 return metricsMaster;
560 }
561
562
563
564
565
566
567
568
569
570
571 @Override
572 public void run() {
573 MonitoredTask startupStatus =
574 TaskMonitor.get().createStatus("Master startup");
575 startupStatus.setDescription("Master startup");
576 masterStartTime = System.currentTimeMillis();
577 try {
578 this.masterAddressTracker = new MasterAddressTracker(getZooKeeperWatcher(), this);
579 this.masterAddressTracker.start();
580
581
582 int port = this.conf.getInt("hbase.master.info.port", 60010);
583 if (port >= 0) {
584 String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
585 this.infoServer = new InfoServer(MASTER, a, port, false, this.conf);
586 this.infoServer.addServlet("status", "/master-status", MasterStatusServlet.class);
587 this.infoServer.addServlet("dump", "/dump", MasterDumpServlet.class);
588 this.infoServer.setAttribute(MASTER, this);
589 this.infoServer.start();
590 }
591
592 this.registeredZKListenersBeforeRecovery = this.zooKeeper.getListeners();
593
594
595
596
597
598
599
600
601
602
603 becomeActiveMaster(startupStatus);
604
605
606 if (!this.stopped) {
607 finishInitialization(startupStatus, false);
608 loop();
609 }
610 } catch (Throwable t) {
611
612 if (t instanceof NoClassDefFoundError &&
613 t.getMessage().contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
614
615 abort("HBase is having a problem with its Hadoop jars. You may need to "
616 + "recompile HBase against Hadoop version "
617 + org.apache.hadoop.util.VersionInfo.getVersion()
618 + " or change your hadoop jars to start properly", t);
619 } else {
620 abort("Unhandled exception. Starting shutdown.", t);
621 }
622 } finally {
623 startupStatus.cleanup();
624
625 stopChores();
626
627
628 if (!this.abort && this.serverManager != null &&
629 this.serverManager.isClusterShutdown()) {
630 this.serverManager.letRegionServersShutdown();
631 }
632 stopServiceThreads();
633
634 if (this.activeMasterManager != null) this.activeMasterManager.stop();
635 if (this.catalogTracker != null) this.catalogTracker.stop();
636 if (this.serverManager != null) this.serverManager.stop();
637 if (this.assignmentManager != null) this.assignmentManager.stop();
638 if (this.fileSystemManager != null) this.fileSystemManager.stop();
639 if (this.snapshotManager != null) this.snapshotManager.stop("server shutting down.");
640 this.zooKeeper.close();
641 }
642 LOG.info("HMaster main thread exiting");
643 }
644
645
646
647
648
649
650
651 private boolean becomeActiveMaster(MonitoredTask startupStatus)
652 throws InterruptedException {
653
654
655 this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName,
656 this);
657 this.zooKeeper.registerListener(activeMasterManager);
658 stallIfBackupMaster(this.conf, this.activeMasterManager);
659
660
661
662
663 this.clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this);
664 this.clusterStatusTracker.start();
665 return this.activeMasterManager.blockUntilBecomingActiveMaster(startupStatus);
666 }
667
668
669
670
671
672
673 void initializeZKBasedSystemTrackers() throws IOException,
674 InterruptedException, KeeperException {
675 this.catalogTracker = createCatalogTracker(this.zooKeeper, this.conf, this);
676 this.catalogTracker.start();
677
678 this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
679 this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
680 this.loadBalancerTracker.start();
681 this.assignmentManager = new AssignmentManager(this, serverManager,
682 this.catalogTracker, this.balancer, this.executorService, this.metricsMaster,
683 this.tableLockManager);
684 zooKeeper.registerListenerFirst(assignmentManager);
685
686 this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
687 this.serverManager);
688 this.regionServerTracker.start();
689
690 this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this,
691 this.serverManager);
692 this.drainingServerTracker.start();
693
694
695
696 boolean wasUp = this.clusterStatusTracker.isClusterUp();
697 if (!wasUp) this.clusterStatusTracker.setClusterUp();
698
699 LOG.info("Server active/primary master=" + this.serverName +
700 ", sessionid=0x" +
701 Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
702 ", setting cluster-up flag (Was=" + wasUp + ")");
703
704
705 this.snapshotManager = new SnapshotManager(this, this.metricsMaster);
706 }
707
708
709
710
711
712
713
714
715
716
717
718
719
720 CatalogTracker createCatalogTracker(final ZooKeeperWatcher zk,
721 final Configuration conf, Abortable abortable)
722 throws IOException {
723 return new CatalogTracker(zk, conf, abortable);
724 }
725
726
727 private Sleeper stopSleeper = new Sleeper(100, this);
728
729 private void loop() {
730 long lastMsgTs = 0l;
731 long now = 0l;
732 while (!this.stopped) {
733 now = System.currentTimeMillis();
734 if ((now - lastMsgTs) >= this.msgInterval) {
735 doMetrics();
736 lastMsgTs = System.currentTimeMillis();
737 }
738 stopSleeper.sleep();
739 }
740 }
741
742
743
744
745
746 private void doMetrics() {
747 try {
748 this.assignmentManager.updateRegionsInTransitionMetrics();
749 } catch (Throwable e) {
750 LOG.error("Couldn't update metrics: " + e.getMessage());
751 }
752 }
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775 private void finishInitialization(MonitoredTask status, boolean masterRecovery)
776 throws IOException, InterruptedException, KeeperException {
777
778 isActiveMaster = true;
779
780
781
782
783
784
785
786 status.setStatus("Initializing Master file system");
787
788 this.masterActiveTime = System.currentTimeMillis();
789
790 this.fileSystemManager = new MasterFileSystem(this, this, masterRecovery);
791
792 this.tableDescriptors =
793 new FSTableDescriptors(this.fileSystemManager.getFileSystem(),
794 this.fileSystemManager.getRootDir());
795
796
797 status.setStatus("Publishing Cluster ID in ZooKeeper");
798 ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
799
800 if (!masterRecovery) {
801 this.executorService = new ExecutorService(getServerName().toShortString());
802 this.serverManager = createServerManager(this, this);
803 }
804
805
806
807 this.tableLockManager = TableLockManager.createTableLockManager(conf, zooKeeper, serverName);
808 if (!masterRecovery) {
809 this.tableLockManager.reapWriteLocks();
810 }
811
812 status.setStatus("Initializing ZK system trackers");
813 initializeZKBasedSystemTrackers();
814
815 if (!masterRecovery) {
816
817 status.setStatus("Initializing master coprocessors");
818 this.cpHost = new MasterCoprocessorHost(this, this.conf);
819
820 spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
821
822
823 status.setStatus("Initializing master service threads");
824 startServiceThreads();
825 }
826
827
828 this.serverManager.waitForRegionServers(status);
829
830 for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
831 if (!this.serverManager.isServerOnline(sn)
832 && serverManager.checkAlreadySameHostPortAndRecordNewServer(
833 sn, ServerLoad.EMPTY_SERVERLOAD)) {
834 LOG.info("Registered server found up in zk but who has not yet "
835 + "reported in: " + sn);
836 }
837 }
838
839 if (!masterRecovery) {
840 this.assignmentManager.startTimeOutMonitor();
841 }
842
843
844
845
846 Set<ServerName> previouslyFailedServers = this.fileSystemManager
847 .getFailedServersFromLogFolders();
848
849
850 this.fileSystemManager.removeStaleRecoveringRegionsFromZK(previouslyFailedServers);
851
852
853 ServerName oldMetaServerLocation = this.catalogTracker.getMetaLocation();
854 if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) {
855 splitMetaLogBeforeAssignment(oldMetaServerLocation);
856
857
858 }
859 Set<ServerName> previouslyFailedMetaRSs = getPreviouselyFailedMetaServersFromZK();
860
861 this.initializationBeforeMetaAssignment = true;
862
863
864 this.balancer.setClusterStatus(getClusterStatus());
865 this.balancer.setMasterServices(this);
866 this.balancer.initialize();
867
868
869 status.setStatus("Assigning Meta Region");
870 assignMeta(status);
871
872
873 if(this.stopped) return;
874
875 if (this.distributedLogReplay && (!previouslyFailedMetaRSs.isEmpty())) {
876
877 status.setStatus("replaying log for Meta Region");
878
879
880
881
882
883
884
885
886 previouslyFailedMetaRSs.addAll(previouslyFailedServers);
887 this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs);
888 }
889
890 status.setStatus("Submitting log splitting work for previously failed region servers");
891
892
893 for (ServerName tmpServer : previouslyFailedServers) {
894 this.serverManager.processDeadServer(tmpServer, true);
895 }
896
897
898
899
900 org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB
901 .updateMetaIfNecessary(this);
902
903
904 status.setStatus("Starting assignment manager");
905 this.assignmentManager.joinCluster();
906
907
908 this.balancer.setClusterStatus(getClusterStatus());
909
910 if (!masterRecovery) {
911
912
913 status.setStatus("Starting balancer and catalog janitor");
914 this.clusterStatusChore = getAndStartClusterStatusChore(this);
915 this.balancerChore = getAndStartBalancerChore(this);
916 this.catalogJanitorChore = new CatalogJanitor(this, this);
917 startCatalogJanitorChore();
918 }
919
920 status.setStatus("Starting namespace manager");
921 initNamespace();
922
923 if (this.cpHost != null) {
924 try {
925 this.cpHost.preMasterInitialization();
926 } catch (IOException e) {
927 LOG.error("Coprocessor preMasterInitialization() hook failed", e);
928 }
929 }
930
931 status.markComplete("Initialization successful");
932 LOG.info("Master has completed initialization");
933 initialized = true;
934
935
936
937 this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
938
939 if (!masterRecovery) {
940 if (this.cpHost != null) {
941
942 try {
943 this.cpHost.postStartMaster();
944 } catch (IOException ioe) {
945 LOG.error("Coprocessor postStartMaster() hook failed", ioe);
946 }
947 }
948 }
949 }
950
951
952
953
954
955 protected void startCatalogJanitorChore() {
956 Threads.setDaemonThreadRunning(catalogJanitorChore.getThread());
957 }
958
959
960
961
962
963 protected void startNamespaceJanitorChore() {
964 Threads.setDaemonThreadRunning(namespaceJanitorChore.getThread());
965 }
966
967
968
969
970
971
972
973
974
975 ServerManager createServerManager(final Server master,
976 final MasterServices services)
977 throws IOException {
978
979
980 return new ServerManager(master, services);
981 }
982
983
984
985
986
987
988
989
990 void assignMeta(MonitoredTask status)
991 throws InterruptedException, IOException, KeeperException {
992
993 int assigned = 0;
994 long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
995 status.setStatus("Assigning hbase:meta region");
996 ServerName logReplayFailedMetaServer = null;
997
998 RegionStates regionStates = assignmentManager.getRegionStates();
999 regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
1000 boolean rit = this.assignmentManager
1001 .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
1002 boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
1003 ServerName currentMetaServer = this.catalogTracker.getMetaLocation();
1004 if (!metaRegionLocation) {
1005
1006
1007 assigned++;
1008 if (!rit) {
1009
1010 if (currentMetaServer != null) {
1011
1012
1013
1014
1015
1016
1017
1018 if (serverManager.isServerOnline(currentMetaServer)) {
1019 LOG.info("Forcing expire of " + currentMetaServer);
1020 serverManager.expireServer(currentMetaServer);
1021 }
1022 splitMetaLogBeforeAssignment(currentMetaServer);
1023 if (this.distributedLogReplay) {
1024 logReplayFailedMetaServer = currentMetaServer;
1025 }
1026 }
1027 assignmentManager.assignMeta();
1028 }
1029 } else {
1030
1031 regionStates.updateRegionState(
1032 HRegionInfo.FIRST_META_REGIONINFO, State.OPEN, currentMetaServer);
1033 this.assignmentManager.regionOnline(
1034 HRegionInfo.FIRST_META_REGIONINFO, currentMetaServer);
1035 }
1036
1037 enableMeta(TableName.META_TABLE_NAME);
1038
1039
1040
1041
1042
1043 enableServerShutdownHandler(assigned != 0);
1044
1045
1046
1047 if (logReplayFailedMetaServer != null) {
1048
1049 this.fileSystemManager.splitMetaLog(logReplayFailedMetaServer);
1050 }
1051
1052 LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
1053 ", location=" + catalogTracker.getMetaLocation());
1054 status.setStatus("META assigned.");
1055 }
1056
1057 void initNamespace() throws IOException {
1058
1059 tableNamespaceManager = new TableNamespaceManager(this);
1060 tableNamespaceManager.start();
1061 }
1062
1063 private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
1064 if (this.distributedLogReplay) {
1065
1066 Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
1067 regions.add(HRegionInfo.FIRST_META_REGIONINFO);
1068 this.fileSystemManager.prepareLogReplay(currentMetaServer, regions);
1069 } else {
1070
1071 this.fileSystemManager.splitMetaLog(currentMetaServer);
1072 }
1073 }
1074
1075 private void enableServerShutdownHandler(
1076 final boolean waitForMeta) throws IOException, InterruptedException {
1077
1078
1079
1080
1081
1082 if (!serverShutdownHandlerEnabled) {
1083 serverShutdownHandlerEnabled = true;
1084 this.serverManager.processQueuedDeadServers();
1085 }
1086
1087 if (waitForMeta) {
1088 this.catalogTracker.waitForMeta();
1089
1090
1091 this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
1092 }
1093 }
1094
1095 private void enableMeta(TableName metaTableName) {
1096 if (!this.assignmentManager.getZKTable().isEnabledTable(metaTableName)) {
1097 this.assignmentManager.setEnabledTable(metaTableName);
1098 }
1099 }
1100
1101
1102
1103
1104
1105
1106 private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
1107 Set<ServerName> result = new HashSet<ServerName>();
1108 String metaRecoveringZNode = ZKUtil.joinZNode(zooKeeper.recoveringRegionsZNode,
1109 HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
1110 List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(zooKeeper, metaRecoveringZNode);
1111 if (regionFailedServers == null) return result;
1112
1113 for(String failedServer : regionFailedServers) {
1114 ServerName server = ServerName.parseServerName(failedServer);
1115 result.add(server);
1116 }
1117 return result;
1118 }
1119
1120 @Override
1121 public TableDescriptors getTableDescriptors() {
1122 return this.tableDescriptors;
1123 }
1124
1125
1126 public InfoServer getInfoServer() {
1127 return this.infoServer;
1128 }
1129
1130 @Override
1131 public Configuration getConfiguration() {
1132 return this.conf;
1133 }
1134
1135 @Override
1136 public ServerManager getServerManager() {
1137 return this.serverManager;
1138 }
1139
1140 @Override
1141 public ExecutorService getExecutorService() {
1142 return this.executorService;
1143 }
1144
1145 @Override
1146 public MasterFileSystem getMasterFileSystem() {
1147 return this.fileSystemManager;
1148 }
1149
1150
1151
1152
1153
1154 public ZooKeeperWatcher getZooKeeperWatcher() {
1155 return this.zooKeeper;
1156 }
1157
1158 public ActiveMasterManager getActiveMasterManager() {
1159 return this.activeMasterManager;
1160 }
1161
1162 public MasterAddressTracker getMasterAddressTracker() {
1163 return this.masterAddressTracker;
1164 }
1165
1166
1167
1168
1169
1170
1171
1172
1173 void startServiceThreads() throws IOException{
1174
1175 this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
1176 conf.getInt("hbase.master.executor.openregion.threads", 5));
1177 this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
1178 conf.getInt("hbase.master.executor.closeregion.threads", 5));
1179 this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
1180 conf.getInt("hbase.master.executor.serverops.threads", 5));
1181 this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
1182 conf.getInt("hbase.master.executor.serverops.threads", 5));
1183 this.executorService.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS,
1184 conf.getInt("hbase.master.executor.logreplayops.threads", 10));
1185
1186
1187
1188
1189 this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
1190
1191
1192 String n = Thread.currentThread().getName();
1193 int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
1194 this.logCleaner =
1195 new LogCleaner(cleanerInterval,
1196 this, conf, getMasterFileSystem().getFileSystem(),
1197 getMasterFileSystem().getOldLogDir());
1198 Threads.setDaemonThreadRunning(logCleaner.getThread(), n + ".oldLogCleaner");
1199
1200
1201 Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
1202 this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem()
1203 .getFileSystem(), archiveDir);
1204 Threads.setDaemonThreadRunning(hfileCleaner.getThread(), n + ".archivedHFileCleaner");
1205
1206
1207 if (this.healthCheckChore != null) {
1208 Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker");
1209 }
1210
1211
1212 this.rpcServer.openServer();
1213 this.rpcServerOpen = true;
1214 if (LOG.isTraceEnabled()) {
1215 LOG.trace("Started service threads");
1216 }
1217 }
1218
1219
1220
1221
1222
1223 boolean isRpcServerOpen() {
1224 return this.rpcServerOpen;
1225 }
1226
1227 private void stopServiceThreads() {
1228 if (LOG.isDebugEnabled()) {
1229 LOG.debug("Stopping service threads");
1230 }
1231 if (this.rpcServer != null) this.rpcServer.stop();
1232 this.rpcServerOpen = false;
1233
1234 if (this.logCleaner!= null) this.logCleaner.interrupt();
1235 if (this.hfileCleaner != null) this.hfileCleaner.interrupt();
1236
1237 if (this.infoServer != null) {
1238 LOG.info("Stopping infoServer");
1239 try {
1240 this.infoServer.stop();
1241 } catch (Exception ex) {
1242 ex.printStackTrace();
1243 }
1244 }
1245 if (this.executorService != null) this.executorService.shutdown();
1246 if (this.healthCheckChore != null) {
1247 this.healthCheckChore.interrupt();
1248 }
1249 if (this.pauseMonitor != null) {
1250 this.pauseMonitor.stop();
1251 }
1252 }
1253
1254 private static Thread getAndStartClusterStatusChore(HMaster master) {
1255 if (master == null || master.balancer == null) {
1256 return null;
1257 }
1258 Chore chore = new ClusterStatusChore(master, master.balancer);
1259 return Threads.setDaemonThreadRunning(chore.getThread());
1260 }
1261
1262 private static Thread getAndStartBalancerChore(final HMaster master) {
1263
1264 Chore chore = new BalancerChore(master);
1265 return Threads.setDaemonThreadRunning(chore.getThread());
1266 }
1267
1268 private void stopChores() {
1269 if (this.balancerChore != null) {
1270 this.balancerChore.interrupt();
1271 }
1272 if (this.clusterStatusChore != null) {
1273 this.clusterStatusChore.interrupt();
1274 }
1275 if (this.catalogJanitorChore != null) {
1276 this.catalogJanitorChore.interrupt();
1277 }
1278 if (this.clusterStatusPublisherChore != null){
1279 clusterStatusPublisherChore.interrupt();
1280 }
1281 if (this.namespaceJanitorChore != null){
1282 namespaceJanitorChore.interrupt();
1283 }
1284 }
1285
1286 @Override
1287 public RegionServerStartupResponse regionServerStartup(
1288 RpcController controller, RegionServerStartupRequest request) throws ServiceException {
1289
1290 try {
1291 InetAddress ia = getRemoteInetAddress(request.getPort(), request.getServerStartCode());
1292 ServerName rs = this.serverManager.regionServerStartup(ia, request.getPort(),
1293 request.getServerStartCode(), request.getServerCurrentTime());
1294
1295
1296 RegionServerStartupResponse.Builder resp = createConfigurationSubset();
1297 NameStringPair.Builder entry = NameStringPair.newBuilder()
1298 .setName(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)
1299 .setValue(rs.getHostname());
1300 resp.addMapEntries(entry.build());
1301
1302 return resp.build();
1303 } catch (IOException ioe) {
1304 throw new ServiceException(ioe);
1305 }
1306 }
1307
1308
1309
1310
1311
1312 InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
1313 throws UnknownHostException {
1314
1315
1316 return RpcServer.getRemoteIp();
1317 }
1318
1319
1320
1321
1322
1323 protected RegionServerStartupResponse.Builder createConfigurationSubset() {
1324 RegionServerStartupResponse.Builder resp = addConfig(
1325 RegionServerStartupResponse.newBuilder(), HConstants.HBASE_DIR);
1326 return addConfig(resp, "fs.default.name");
1327 }
1328
1329 private RegionServerStartupResponse.Builder addConfig(
1330 final RegionServerStartupResponse.Builder resp, final String key) {
1331 NameStringPair.Builder entry = NameStringPair.newBuilder()
1332 .setName(key)
1333 .setValue(this.conf.get(key));
1334 resp.addMapEntries(entry.build());
1335 return resp;
1336 }
1337
1338 @Override
1339 public GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(RpcController controller,
1340 GetLastFlushedSequenceIdRequest request) throws ServiceException {
1341 byte[] regionName = request.getRegionName().toByteArray();
1342 long seqId = serverManager.getLastFlushedSequenceId(regionName);
1343 return ResponseConverter.buildGetLastFlushedSequenceIdResponse(seqId);
1344 }
1345
1346 @Override
1347 public RegionServerReportResponse regionServerReport(
1348 RpcController controller, RegionServerReportRequest request) throws ServiceException {
1349 try {
1350 ClusterStatusProtos.ServerLoad sl = request.getLoad();
1351 ServerName serverName = ProtobufUtil.toServerName(request.getServer());
1352 ServerLoad oldLoad = serverManager.getLoad(serverName);
1353 this.serverManager.regionServerReport(serverName, new ServerLoad(sl));
1354 if (sl != null && this.metricsMaster != null) {
1355
1356 this.metricsMaster.incrementRequests(sl.getTotalNumberOfRequests()
1357 - (oldLoad != null ? oldLoad.getTotalNumberOfRequests() : 0));
1358 }
1359 } catch (IOException ioe) {
1360 throw new ServiceException(ioe);
1361 }
1362
1363 return RegionServerReportResponse.newBuilder().build();
1364 }
1365
1366 @Override
1367 public ReportRSFatalErrorResponse reportRSFatalError(
1368 RpcController controller, ReportRSFatalErrorRequest request) throws ServiceException {
1369 String errorText = request.getErrorMessage();
1370 ServerName sn = ProtobufUtil.toServerName(request.getServer());
1371 String msg = "Region server " + sn +
1372 " reported a fatal error:\n" + errorText;
1373 LOG.error(msg);
1374 rsFatals.add(msg);
1375
1376 return ReportRSFatalErrorResponse.newBuilder().build();
1377 }
1378
1379 public boolean isMasterRunning() {
1380 return !isStopped();
1381 }
1382
1383 @Override
1384 public IsMasterRunningResponse isMasterRunning(RpcController c, IsMasterRunningRequest req)
1385 throws ServiceException {
1386 return IsMasterRunningResponse.newBuilder().setIsMasterRunning(isMasterRunning()).build();
1387 }
1388
1389 @Override
1390 public RunCatalogScanResponse runCatalogScan(RpcController c,
1391 RunCatalogScanRequest req) throws ServiceException {
1392 try {
1393 return ResponseConverter.buildRunCatalogScanResponse(catalogJanitorChore.scan());
1394 } catch (IOException ioe) {
1395 throw new ServiceException(ioe);
1396 }
1397 }
1398
1399 @Override
1400 public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
1401 EnableCatalogJanitorRequest req) throws ServiceException {
1402 return EnableCatalogJanitorResponse.newBuilder().
1403 setPrevValue(catalogJanitorChore.setEnabled(req.getEnable())).build();
1404 }
1405
1406 @Override
1407 public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
1408 IsCatalogJanitorEnabledRequest req) throws ServiceException {
1409 boolean isEnabled = catalogJanitorChore != null ? catalogJanitorChore.getEnabled() : false;
1410 return IsCatalogJanitorEnabledResponse.newBuilder().setValue(isEnabled).build();
1411 }
1412
1413
1414
1415
1416 private int getBalancerCutoffTime() {
1417 int balancerCutoffTime =
1418 getConfiguration().getInt("hbase.balancer.max.balancing", -1);
1419 if (balancerCutoffTime == -1) {
1420
1421 int balancerPeriod =
1422 getConfiguration().getInt("hbase.balancer.period", 300000);
1423 balancerCutoffTime = balancerPeriod;
1424
1425 if (balancerCutoffTime <= 0) balancerCutoffTime = balancerPeriod;
1426 }
1427 return balancerCutoffTime;
1428 }
1429
1430 public boolean balance() throws HBaseIOException {
1431
1432 if (!this.initialized) {
1433 LOG.debug("Master has not been initialized, don't run balancer.");
1434 return false;
1435 }
1436
1437 if (!this.loadBalancerTracker.isBalancerOn()) return false;
1438
1439 int maximumBalanceTime = getBalancerCutoffTime();
1440 boolean balancerRan;
1441 synchronized (this.balancer) {
1442
1443 if (this.assignmentManager.getRegionStates().isRegionsInTransition()) {
1444 Map<String, RegionState> regionsInTransition =
1445 this.assignmentManager.getRegionStates().getRegionsInTransition();
1446 LOG.debug("Not running balancer because " + regionsInTransition.size() +
1447 " region(s) in transition: " + org.apache.commons.lang.StringUtils.
1448 abbreviate(regionsInTransition.toString(), 256));
1449 return false;
1450 }
1451 if (this.serverManager.areDeadServersInProgress()) {
1452 LOG.debug("Not running balancer because processing dead regionserver(s): " +
1453 this.serverManager.getDeadServers());
1454 return false;
1455 }
1456
1457 if (this.cpHost != null) {
1458 try {
1459 if (this.cpHost.preBalance()) {
1460 LOG.debug("Coprocessor bypassing balancer request");
1461 return false;
1462 }
1463 } catch (IOException ioe) {
1464 LOG.error("Error invoking master coprocessor preBalance()", ioe);
1465 return false;
1466 }
1467 }
1468
1469 Map<TableName, Map<ServerName, List<HRegionInfo>>> assignmentsByTable =
1470 this.assignmentManager.getRegionStates().getAssignmentsByTable();
1471
1472 List<RegionPlan> plans = new ArrayList<RegionPlan>();
1473
1474 this.balancer.setClusterStatus(getClusterStatus());
1475 for (Map<ServerName, List<HRegionInfo>> assignments : assignmentsByTable.values()) {
1476 List<RegionPlan> partialPlans = this.balancer.balanceCluster(assignments);
1477 if (partialPlans != null) plans.addAll(partialPlans);
1478 }
1479 long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
1480 int rpCount = 0;
1481 long totalRegPlanExecTime = 0;
1482 balancerRan = plans != null;
1483 if (plans != null && !plans.isEmpty()) {
1484 for (RegionPlan plan: plans) {
1485 LOG.info("balance " + plan);
1486 long balStartTime = System.currentTimeMillis();
1487
1488 this.assignmentManager.balance(plan);
1489 totalRegPlanExecTime += System.currentTimeMillis()-balStartTime;
1490 rpCount++;
1491 if (rpCount < plans.size() &&
1492
1493 (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
1494
1495 LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
1496 maximumBalanceTime);
1497 break;
1498 }
1499 }
1500 }
1501 if (this.cpHost != null) {
1502 try {
1503 this.cpHost.postBalance(rpCount < plans.size() ? plans.subList(0, rpCount) : plans);
1504 } catch (IOException ioe) {
1505
1506 LOG.error("Error invoking master coprocessor postBalance()", ioe);
1507 }
1508 }
1509 }
1510 return balancerRan;
1511 }
1512
1513 @Override
1514 public BalanceResponse balance(RpcController c, BalanceRequest request) throws ServiceException {
1515 try {
1516 return BalanceResponse.newBuilder().setBalancerRan(balance()).build();
1517 } catch (HBaseIOException ex) {
1518 throw new ServiceException(ex);
1519 }
1520 }
1521
1522 enum BalanceSwitchMode {
1523 SYNC,
1524 ASYNC
1525 }
1526
1527
1528
1529
1530
1531
1532
1533 public boolean switchBalancer(final boolean b, BalanceSwitchMode mode) throws IOException {
1534 boolean oldValue = this.loadBalancerTracker.isBalancerOn();
1535 boolean newValue = b;
1536 try {
1537 if (this.cpHost != null) {
1538 newValue = this.cpHost.preBalanceSwitch(newValue);
1539 }
1540 try {
1541 if (mode == BalanceSwitchMode.SYNC) {
1542 synchronized (this.balancer) {
1543 this.loadBalancerTracker.setBalancerOn(newValue);
1544 }
1545 } else {
1546 this.loadBalancerTracker.setBalancerOn(newValue);
1547 }
1548 } catch (KeeperException ke) {
1549 throw new IOException(ke);
1550 }
1551 LOG.info(getClientIdAuditPrefix() + " set balanceSwitch=" + newValue);
1552 if (this.cpHost != null) {
1553 this.cpHost.postBalanceSwitch(oldValue, newValue);
1554 }
1555 } catch (IOException ioe) {
1556 LOG.warn("Error flipping balance switch", ioe);
1557 }
1558 return oldValue;
1559 }
1560
1561
1562
1563
1564 String getClientIdAuditPrefix() {
1565 return "Client=" + RequestContext.getRequestUserName() + "/" +
1566 RequestContext.get().getRemoteAddress();
1567 }
1568
1569 public boolean synchronousBalanceSwitch(final boolean b) throws IOException {
1570 return switchBalancer(b, BalanceSwitchMode.SYNC);
1571 }
1572
1573 public boolean balanceSwitch(final boolean b) throws IOException {
1574 return switchBalancer(b, BalanceSwitchMode.ASYNC);
1575 }
1576
1577 @Override
1578 public SetBalancerRunningResponse setBalancerRunning(
1579 RpcController controller, SetBalancerRunningRequest req) throws ServiceException {
1580 try {
1581 boolean prevValue = (req.getSynchronous())?
1582 synchronousBalanceSwitch(req.getOn()):balanceSwitch(req.getOn());
1583 return SetBalancerRunningResponse.newBuilder().setPrevBalanceValue(prevValue).build();
1584 } catch (IOException ioe) {
1585 throw new ServiceException(ioe);
1586 }
1587 }
1588
1589
1590
1591
1592
1593
1594
1595 public void setCatalogJanitorEnabled(final boolean b) {
1596 this.catalogJanitorChore.setEnabled(b);
1597 }
1598
1599 @Override
1600 public DispatchMergingRegionsResponse dispatchMergingRegions(
1601 RpcController controller, DispatchMergingRegionsRequest request)
1602 throws ServiceException {
1603 final byte[] encodedNameOfRegionA = request.getRegionA().getValue()
1604 .toByteArray();
1605 final byte[] encodedNameOfRegionB = request.getRegionB().getValue()
1606 .toByteArray();
1607 final boolean forcible = request.getForcible();
1608 if (request.getRegionA().getType() != RegionSpecifierType.ENCODED_REGION_NAME
1609 || request.getRegionB().getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
1610 LOG.warn("mergeRegions specifier type: expected: "
1611 + RegionSpecifierType.ENCODED_REGION_NAME + " actual: region_a="
1612 + request.getRegionA().getType() + ", region_b="
1613 + request.getRegionB().getType());
1614 }
1615 RegionState regionStateA = assignmentManager.getRegionStates()
1616 .getRegionState(Bytes.toString(encodedNameOfRegionA));
1617 RegionState regionStateB = assignmentManager.getRegionStates()
1618 .getRegionState(Bytes.toString(encodedNameOfRegionB));
1619 if (regionStateA == null || regionStateB == null) {
1620 throw new ServiceException(new UnknownRegionException(
1621 Bytes.toStringBinary(regionStateA == null ? encodedNameOfRegionA
1622 : encodedNameOfRegionB)));
1623 }
1624
1625 if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
1626 throw new ServiceException(new MergeRegionException(
1627 "Unable to merge regions not online " + regionStateA + ", " + regionStateB));
1628 }
1629
1630 HRegionInfo regionInfoA = regionStateA.getRegion();
1631 HRegionInfo regionInfoB = regionStateB.getRegion();
1632 if (regionInfoA.compareTo(regionInfoB) == 0) {
1633 throw new ServiceException(new MergeRegionException(
1634 "Unable to merge a region to itself " + regionInfoA + ", " + regionInfoB));
1635 }
1636
1637 if (!forcible && !HRegionInfo.areAdjacent(regionInfoA, regionInfoB)) {
1638 throw new ServiceException(new MergeRegionException(
1639 "Unable to merge not adjacent regions "
1640 + regionInfoA.getRegionNameAsString() + ", "
1641 + regionInfoB.getRegionNameAsString()
1642 + " where forcible = " + forcible));
1643 }
1644
1645 try {
1646 dispatchMergingRegions(regionInfoA, regionInfoB, forcible);
1647 } catch (IOException ioe) {
1648 throw new ServiceException(ioe);
1649 }
1650
1651 return DispatchMergingRegionsResponse.newBuilder().build();
1652 }
1653
1654 @Override
1655 public void dispatchMergingRegions(final HRegionInfo region_a,
1656 final HRegionInfo region_b, final boolean forcible) throws IOException {
1657 checkInitialized();
1658 this.executorService.submit(new DispatchMergingRegionHandler(this,
1659 this.catalogJanitorChore, region_a, region_b, forcible));
1660 }
1661
1662 @Override
1663 public MoveRegionResponse moveRegion(RpcController controller, MoveRegionRequest req)
1664 throws ServiceException {
1665 final byte [] encodedRegionName = req.getRegion().getValue().toByteArray();
1666 RegionSpecifierType type = req.getRegion().getType();
1667 final byte [] destServerName = (req.hasDestServerName())?
1668 Bytes.toBytes(ProtobufUtil.toServerName(req.getDestServerName()).getServerName()):null;
1669 MoveRegionResponse mrr = MoveRegionResponse.newBuilder().build();
1670
1671 if (type != RegionSpecifierType.ENCODED_REGION_NAME) {
1672 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME
1673 + " actual: " + type);
1674 }
1675
1676 try {
1677 move(encodedRegionName, destServerName);
1678 } catch (HBaseIOException ioe) {
1679 throw new ServiceException(ioe);
1680 }
1681 return mrr;
1682 }
1683
1684 void move(final byte[] encodedRegionName,
1685 final byte[] destServerName) throws HBaseIOException {
1686 RegionState regionState = assignmentManager.getRegionStates().
1687 getRegionState(Bytes.toString(encodedRegionName));
1688 if (regionState == null) {
1689 throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
1690 }
1691
1692 HRegionInfo hri = regionState.getRegion();
1693 ServerName dest;
1694 if (destServerName == null || destServerName.length == 0) {
1695 LOG.info("Passed destination servername is null/empty so " +
1696 "choosing a server at random");
1697 final List<ServerName> destServers = this.serverManager.createDestinationServersList(
1698 regionState.getServerName());
1699 dest = balancer.randomAssignment(hri, destServers);
1700 } else {
1701 dest = ServerName.valueOf(Bytes.toString(destServerName));
1702 if (dest.equals(regionState.getServerName())) {
1703 LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
1704 + " because region already assigned to the same server " + dest + ".");
1705 return;
1706 }
1707 }
1708
1709
1710 RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
1711
1712 try {
1713 checkInitialized();
1714 if (this.cpHost != null) {
1715 if (this.cpHost.preMove(hri, rp.getSource(), rp.getDestination())) {
1716 return;
1717 }
1718 }
1719 LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
1720 this.assignmentManager.balance(rp);
1721 if (this.cpHost != null) {
1722 this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
1723 }
1724 } catch (IOException ioe) {
1725 if (ioe instanceof HBaseIOException) {
1726 throw (HBaseIOException)ioe;
1727 }
1728 throw new HBaseIOException(ioe);
1729 }
1730 }
1731
1732 @Override
1733 public void createTable(HTableDescriptor hTableDescriptor,
1734 byte [][] splitKeys)
1735 throws IOException {
1736 if (!isMasterRunning()) {
1737 throw new MasterNotRunningException();
1738 }
1739
1740 String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
1741 getNamespaceDescriptor(namespace);
1742
1743 HRegionInfo[] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
1744 checkInitialized();
1745 checkCompression(hTableDescriptor);
1746 if (cpHost != null) {
1747 cpHost.preCreateTable(hTableDescriptor, newRegions);
1748 }
1749 LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
1750 this.executorService.submit(new CreateTableHandler(this,
1751 this.fileSystemManager, hTableDescriptor, conf,
1752 newRegions, this).prepare());
1753 if (cpHost != null) {
1754 cpHost.postCreateTable(hTableDescriptor, newRegions);
1755 }
1756
1757 }
1758
1759 private void checkCompression(final HTableDescriptor htd)
1760 throws IOException {
1761 if (!this.masterCheckCompression) return;
1762 for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1763 checkCompression(hcd);
1764 }
1765 }
1766
1767 private void checkCompression(final HColumnDescriptor hcd)
1768 throws IOException {
1769 if (!this.masterCheckCompression) return;
1770 CompressionTest.testCompression(hcd.getCompression());
1771 CompressionTest.testCompression(hcd.getCompactionCompression());
1772 }
1773
1774 @Override
1775 public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
1776 throws ServiceException {
1777 HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema());
1778 byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
1779 try {
1780 createTable(hTableDescriptor,splitKeys);
1781 } catch (IOException ioe) {
1782 throw new ServiceException(ioe);
1783 }
1784 return CreateTableResponse.newBuilder().build();
1785 }
1786
1787 private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
1788 byte[][] splitKeys) {
1789 HRegionInfo[] hRegionInfos = null;
1790 if (splitKeys == null || splitKeys.length == 0) {
1791 hRegionInfos = new HRegionInfo[]{
1792 new HRegionInfo(hTableDescriptor.getTableName(), null, null)};
1793 } else {
1794 int numRegions = splitKeys.length + 1;
1795 hRegionInfos = new HRegionInfo[numRegions];
1796 byte[] startKey = null;
1797 byte[] endKey = null;
1798 for (int i = 0; i < numRegions; i++) {
1799 endKey = (i == splitKeys.length) ? null : splitKeys[i];
1800 hRegionInfos[i] =
1801 new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey);
1802 startKey = endKey;
1803 }
1804 }
1805 return hRegionInfos;
1806 }
1807
1808 private static boolean isCatalogTable(final TableName tableName) {
1809 return tableName.equals(TableName.META_TABLE_NAME);
1810 }
1811
1812 @Override
1813 public void deleteTable(final TableName tableName) throws IOException {
1814 checkInitialized();
1815 if (cpHost != null) {
1816 cpHost.preDeleteTable(tableName);
1817 }
1818 LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
1819 this.executorService.submit(new DeleteTableHandler(tableName, this, this).prepare());
1820 if (cpHost != null) {
1821 cpHost.postDeleteTable(tableName);
1822 }
1823 }
1824
1825 @Override
1826 public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest request)
1827 throws ServiceException {
1828 try {
1829 deleteTable(ProtobufUtil.toTableName(request.getTableName()));
1830 } catch (IOException ioe) {
1831 throw new ServiceException(ioe);
1832 }
1833 return DeleteTableResponse.newBuilder().build();
1834 }
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844 @Override
1845 public GetSchemaAlterStatusResponse getSchemaAlterStatus(
1846 RpcController controller, GetSchemaAlterStatusRequest req) throws ServiceException {
1847
1848
1849
1850
1851 TableName tableName = ProtobufUtil.toTableName(req.getTableName());
1852
1853 try {
1854 Pair<Integer,Integer> pair = this.assignmentManager.getReopenStatus(tableName);
1855 GetSchemaAlterStatusResponse.Builder ret = GetSchemaAlterStatusResponse.newBuilder();
1856 ret.setYetToUpdateRegions(pair.getFirst());
1857 ret.setTotalRegions(pair.getSecond());
1858 return ret.build();
1859 } catch (IOException ioe) {
1860 throw new ServiceException(ioe);
1861 }
1862 }
1863
1864 @Override
1865 public void addColumn(final TableName tableName, final HColumnDescriptor column)
1866 throws IOException {
1867 checkInitialized();
1868 if (cpHost != null) {
1869 if (cpHost.preAddColumn(tableName, column)) {
1870 return;
1871 }
1872 }
1873
1874 new TableAddFamilyHandler(tableName, column, this, this).prepare().process();
1875 if (cpHost != null) {
1876 cpHost.postAddColumn(tableName, column);
1877 }
1878 }
1879
1880 @Override
1881 public AddColumnResponse addColumn(RpcController controller, AddColumnRequest req)
1882 throws ServiceException {
1883 try {
1884 addColumn(ProtobufUtil.toTableName(req.getTableName()),
1885 HColumnDescriptor.convert(req.getColumnFamilies()));
1886 } catch (IOException ioe) {
1887 throw new ServiceException(ioe);
1888 }
1889 return AddColumnResponse.newBuilder().build();
1890 }
1891
1892 @Override
1893 public void modifyColumn(TableName tableName, HColumnDescriptor descriptor)
1894 throws IOException {
1895 checkInitialized();
1896 checkCompression(descriptor);
1897 if (cpHost != null) {
1898 if (cpHost.preModifyColumn(tableName, descriptor)) {
1899 return;
1900 }
1901 }
1902 LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
1903 new TableModifyFamilyHandler(tableName, descriptor, this, this)
1904 .prepare().process();
1905 if (cpHost != null) {
1906 cpHost.postModifyColumn(tableName, descriptor);
1907 }
1908 }
1909
1910 @Override
1911 public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest req)
1912 throws ServiceException {
1913 try {
1914 modifyColumn(ProtobufUtil.toTableName(req.getTableName()),
1915 HColumnDescriptor.convert(req.getColumnFamilies()));
1916 } catch (IOException ioe) {
1917 throw new ServiceException(ioe);
1918 }
1919 return ModifyColumnResponse.newBuilder().build();
1920 }
1921
1922 @Override
1923 public void deleteColumn(final TableName tableName, final byte[] columnName)
1924 throws IOException {
1925 checkInitialized();
1926 if (cpHost != null) {
1927 if (cpHost.preDeleteColumn(tableName, columnName)) {
1928 return;
1929 }
1930 }
1931 LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
1932 new TableDeleteFamilyHandler(tableName, columnName, this, this).prepare().process();
1933 if (cpHost != null) {
1934 cpHost.postDeleteColumn(tableName, columnName);
1935 }
1936 }
1937
1938 @Override
1939 public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest req)
1940 throws ServiceException {
1941 try {
1942 deleteColumn(ProtobufUtil.toTableName(req.getTableName()),
1943 req.getColumnName().toByteArray());
1944 } catch (IOException ioe) {
1945 throw new ServiceException(ioe);
1946 }
1947 return DeleteColumnResponse.newBuilder().build();
1948 }
1949
1950 @Override
1951 public void enableTable(final TableName tableName) throws IOException {
1952 checkInitialized();
1953 if (cpHost != null) {
1954 cpHost.preEnableTable(tableName);
1955 }
1956 LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
1957 this.executorService.submit(new EnableTableHandler(this, tableName,
1958 catalogTracker, assignmentManager, tableLockManager, false).prepare());
1959 if (cpHost != null) {
1960 cpHost.postEnableTable(tableName);
1961 }
1962 }
1963
1964 @Override
1965 public EnableTableResponse enableTable(RpcController controller, EnableTableRequest request)
1966 throws ServiceException {
1967 try {
1968 enableTable(ProtobufUtil.toTableName(request.getTableName()));
1969 } catch (IOException ioe) {
1970 throw new ServiceException(ioe);
1971 }
1972 return EnableTableResponse.newBuilder().build();
1973 }
1974
1975 @Override
1976 public void disableTable(final TableName tableName) throws IOException {
1977 checkInitialized();
1978 if (cpHost != null) {
1979 cpHost.preDisableTable(tableName);
1980 }
1981 LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
1982 this.executorService.submit(new DisableTableHandler(this, tableName,
1983 catalogTracker, assignmentManager, tableLockManager, false).prepare());
1984 if (cpHost != null) {
1985 cpHost.postDisableTable(tableName);
1986 }
1987 }
1988
1989 @Override
1990 public DisableTableResponse disableTable(RpcController controller, DisableTableRequest request)
1991 throws ServiceException {
1992 try {
1993 disableTable(ProtobufUtil.toTableName(request.getTableName()));
1994 } catch (IOException ioe) {
1995 throw new ServiceException(ioe);
1996 }
1997 return DisableTableResponse.newBuilder().build();
1998 }
1999
2000
2001
2002
2003
2004
2005
2006 Pair<HRegionInfo, ServerName> getTableRegionForRow(
2007 final TableName tableName, final byte [] rowKey)
2008 throws IOException {
2009 final AtomicReference<Pair<HRegionInfo, ServerName>> result =
2010 new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
2011
2012 MetaScannerVisitor visitor =
2013 new MetaScannerVisitorBase() {
2014 @Override
2015 public boolean processRow(Result data) throws IOException {
2016 if (data == null || data.size() <= 0) {
2017 return true;
2018 }
2019 Pair<HRegionInfo, ServerName> pair = HRegionInfo.getHRegionInfoAndServerName(data);
2020 if (pair == null) {
2021 return false;
2022 }
2023 if (!pair.getFirst().getTable().equals(tableName)) {
2024 return false;
2025 }
2026 result.set(pair);
2027 return true;
2028 }
2029 };
2030
2031 MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1);
2032 return result.get();
2033 }
2034
2035 @Override
2036 public void modifyTable(final TableName tableName, final HTableDescriptor descriptor)
2037 throws IOException {
2038 checkInitialized();
2039 checkCompression(descriptor);
2040 if (cpHost != null) {
2041 cpHost.preModifyTable(tableName, descriptor);
2042 }
2043 LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
2044 new ModifyTableHandler(tableName, descriptor, this, this).prepare().process();
2045 if (cpHost != null) {
2046 cpHost.postModifyTable(tableName, descriptor);
2047 }
2048 }
2049
2050 @Override
2051 public ModifyTableResponse modifyTable(RpcController controller, ModifyTableRequest req)
2052 throws ServiceException {
2053 try {
2054 modifyTable(ProtobufUtil.toTableName(req.getTableName()),
2055 HTableDescriptor.convert(req.getTableSchema()));
2056 } catch (IOException ioe) {
2057 throw new ServiceException(ioe);
2058 }
2059 return ModifyTableResponse.newBuilder().build();
2060 }
2061
2062 @Override
2063 public void checkTableModifiable(final TableName tableName)
2064 throws IOException, TableNotFoundException, TableNotDisabledException {
2065 if (isCatalogTable(tableName)) {
2066 throw new IOException("Can't modify catalog tables");
2067 }
2068 if (!MetaReader.tableExists(getCatalogTracker(), tableName)) {
2069 throw new TableNotFoundException(tableName);
2070 }
2071 if (!getAssignmentManager().getZKTable().
2072 isDisabledTable(tableName)) {
2073 throw new TableNotDisabledException(tableName);
2074 }
2075 }
2076
2077 @Override
2078 public GetClusterStatusResponse getClusterStatus(RpcController controller,
2079 GetClusterStatusRequest req)
2080 throws ServiceException {
2081 GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
2082 response.setClusterStatus(getClusterStatus().convert());
2083 return response.build();
2084 }
2085
2086
2087
2088
2089 public ClusterStatus getClusterStatus() {
2090
2091 List<String> backupMasterStrings;
2092 try {
2093 backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper,
2094 this.zooKeeper.backupMasterAddressesZNode);
2095 } catch (KeeperException e) {
2096 LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e);
2097 backupMasterStrings = new ArrayList<String>(0);
2098 }
2099 List<ServerName> backupMasters = new ArrayList<ServerName>(
2100 backupMasterStrings.size());
2101 for (String s: backupMasterStrings) {
2102 try {
2103 byte [] bytes =
2104 ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(
2105 this.zooKeeper.backupMasterAddressesZNode, s));
2106 if (bytes != null) {
2107 ServerName sn;
2108 try {
2109 sn = ServerName.parseFrom(bytes);
2110 } catch (DeserializationException e) {
2111 LOG.warn("Failed parse, skipping registering backup server", e);
2112 continue;
2113 }
2114 backupMasters.add(sn);
2115 }
2116 } catch (KeeperException e) {
2117 LOG.warn(this.zooKeeper.prefix("Unable to get information about " +
2118 "backup servers"), e);
2119 }
2120 }
2121 Collections.sort(backupMasters, new Comparator<ServerName>() {
2122 @Override
2123 public int compare(ServerName s1, ServerName s2) {
2124 return s1.getServerName().compareTo(s2.getServerName());
2125 }});
2126
2127 return new ClusterStatus(VersionInfo.getVersion(),
2128 this.fileSystemManager.getClusterId().toString(),
2129 this.serverManager.getOnlineServers(),
2130 this.serverManager.getDeadServers().copyServerNames(),
2131 this.serverName,
2132 backupMasters,
2133 this.assignmentManager.getRegionStates().getRegionsInTransition(),
2134 this.getCoprocessors(), this.loadBalancerTracker.isBalancerOn());
2135 }
2136
2137 public String getClusterId() {
2138 if (fileSystemManager == null) {
2139 return "";
2140 }
2141 ClusterId id = fileSystemManager.getClusterId();
2142 if (id == null) {
2143 return "";
2144 }
2145 return id.toString();
2146 }
2147
2148
2149
2150
2151
2152
2153
2154
2155 public static String getLoadedCoprocessors() {
2156 return CoprocessorHost.getLoadedCoprocessors().toString();
2157 }
2158
2159
2160
2161
2162 public long getMasterStartTime() {
2163 return masterStartTime;
2164 }
2165
2166
2167
2168
2169 public long getMasterActiveTime() {
2170 return masterActiveTime;
2171 }
2172
2173 public int getRegionServerInfoPort(final ServerName sn) {
2174 RegionServerInfo info = this.regionServerTracker.getRegionServerInfo(sn);
2175 if (info == null || info.getInfoPort() == 0) {
2176 return conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
2177 HConstants.DEFAULT_REGIONSERVER_INFOPORT);
2178 }
2179 return info.getInfoPort();
2180 }
2181
2182
2183
2184
2185 public String[] getCoprocessors() {
2186 Set<String> masterCoprocessors =
2187 getCoprocessorHost().getCoprocessors();
2188 return masterCoprocessors.toArray(new String[masterCoprocessors.size()]);
2189 }
2190
2191 @Override
2192 public void abort(final String msg, final Throwable t) {
2193 if (cpHost != null) {
2194
2195 LOG.fatal("Master server abort: loaded coprocessors are: " +
2196 getLoadedCoprocessors());
2197 }
2198
2199 if (abortNow(msg, t)) {
2200 if (t != null) LOG.fatal(msg, t);
2201 else LOG.fatal(msg);
2202 this.abort = true;
2203 stop("Aborting");
2204 }
2205 }
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224 private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
2225 IOException, KeeperException, ExecutionException {
2226
2227 this.zooKeeper.unregisterAllListeners();
2228
2229
2230 if (this.registeredZKListenersBeforeRecovery != null) {
2231 for (ZooKeeperListener curListener : this.registeredZKListenersBeforeRecovery) {
2232 this.zooKeeper.registerListener(curListener);
2233 }
2234 }
2235
2236 this.zooKeeper.reconnectAfterExpiration();
2237
2238 Callable<Boolean> callable = new Callable<Boolean> () {
2239 @Override
2240 public Boolean call() throws InterruptedException,
2241 IOException, KeeperException {
2242 MonitoredTask status =
2243 TaskMonitor.get().createStatus("Recovering expired ZK session");
2244 try {
2245 if (!becomeActiveMaster(status)) {
2246 return Boolean.FALSE;
2247 }
2248 serverShutdownHandlerEnabled = false;
2249 initialized = false;
2250 finishInitialization(status, true);
2251 return !stopped;
2252 } finally {
2253 status.cleanup();
2254 }
2255 }
2256 };
2257
2258 long timeout =
2259 conf.getLong("hbase.master.zksession.recover.timeout", 300000);
2260 java.util.concurrent.ExecutorService executor =
2261 Executors.newSingleThreadExecutor();
2262 Future<Boolean> result = executor.submit(callable);
2263 executor.shutdown();
2264 if (executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)
2265 && result.isDone()) {
2266 Boolean recovered = result.get();
2267 if (recovered != null) {
2268 return recovered.booleanValue();
2269 }
2270 }
2271 executor.shutdownNow();
2272 return false;
2273 }
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283 private boolean abortNow(final String msg, final Throwable t) {
2284 if (!this.isActiveMaster || this.stopped) {
2285 return true;
2286 }
2287
2288 boolean failFast = conf.getBoolean("fail.fast.expired.active.master", false);
2289 if (t != null && t instanceof KeeperException.SessionExpiredException
2290 && !failFast) {
2291 try {
2292 LOG.info("Primary Master trying to recover from ZooKeeper session " +
2293 "expiry.");
2294 return !tryRecoveringExpiredZKSession();
2295 } catch (Throwable newT) {
2296 LOG.error("Primary master encountered unexpected exception while " +
2297 "trying to recover from ZooKeeper session" +
2298 " expiry. Proceeding with server abort.", newT);
2299 }
2300 }
2301 return true;
2302 }
2303
2304 @Override
2305 public ZooKeeperWatcher getZooKeeper() {
2306 return zooKeeper;
2307 }
2308
2309 @Override
2310 public MasterCoprocessorHost getCoprocessorHost() {
2311 return cpHost;
2312 }
2313
2314 @Override
2315 public ServerName getServerName() {
2316 return this.serverName;
2317 }
2318
2319 @Override
2320 public CatalogTracker getCatalogTracker() {
2321 return catalogTracker;
2322 }
2323
2324 @Override
2325 public AssignmentManager getAssignmentManager() {
2326 return this.assignmentManager;
2327 }
2328
2329 @Override
2330 public TableLockManager getTableLockManager() {
2331 return this.tableLockManager;
2332 }
2333
2334 public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
2335 return rsFatals;
2336 }
2337
2338 public void shutdown() {
2339 if (spanReceiverHost != null) {
2340 spanReceiverHost.closeReceivers();
2341 }
2342 if (cpHost != null) {
2343 try {
2344 cpHost.preShutdown();
2345 } catch (IOException ioe) {
2346 LOG.error("Error call master coprocessor preShutdown()", ioe);
2347 }
2348 }
2349 if (mxBean != null) {
2350 MBeanUtil.unregisterMBean(mxBean);
2351 mxBean = null;
2352 }
2353 if (this.assignmentManager != null) this.assignmentManager.shutdown();
2354 if (this.serverManager != null) this.serverManager.shutdownCluster();
2355 try {
2356 if (this.clusterStatusTracker != null){
2357 this.clusterStatusTracker.setClusterDown();
2358 }
2359 } catch (KeeperException e) {
2360 LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
2361 }
2362 }
2363
2364 @Override
2365 public ShutdownResponse shutdown(RpcController controller, ShutdownRequest request)
2366 throws ServiceException {
2367 LOG.info(getClientIdAuditPrefix() + " shutdown");
2368 shutdown();
2369 return ShutdownResponse.newBuilder().build();
2370 }
2371
2372 public void stopMaster() {
2373 if (cpHost != null) {
2374 try {
2375 cpHost.preStopMaster();
2376 } catch (IOException ioe) {
2377 LOG.error("Error call master coprocessor preStopMaster()", ioe);
2378 }
2379 }
2380 stop("Stopped by " + Thread.currentThread().getName());
2381 }
2382
2383 @Override
2384 public StopMasterResponse stopMaster(RpcController controller, StopMasterRequest request)
2385 throws ServiceException {
2386 LOG.info(getClientIdAuditPrefix() + " stop");
2387 stopMaster();
2388 return StopMasterResponse.newBuilder().build();
2389 }
2390
2391 @Override
2392 public void stop(final String why) {
2393 LOG.info(why);
2394 this.stopped = true;
2395
2396 stopSleeper.skipSleepCycle();
2397
2398 if (this.activeMasterManager != null) {
2399 synchronized (this.activeMasterManager.clusterHasActiveMaster) {
2400 this.activeMasterManager.clusterHasActiveMaster.notifyAll();
2401 }
2402 }
2403
2404
2405 if (this.catalogTracker != null && this.serverManager.getOnlineServers().isEmpty()) {
2406 this.catalogTracker.stop();
2407 }
2408 }
2409
2410 @Override
2411 public boolean isStopped() {
2412 return this.stopped;
2413 }
2414
2415 @Override
2416 public boolean isAborted() {
2417 return this.abort;
2418 }
2419
2420 void checkInitialized() throws PleaseHoldException {
2421 if (!this.initialized) {
2422 throw new PleaseHoldException("Master is initializing");
2423 }
2424 }
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434 public boolean isActiveMaster() {
2435 return isActiveMaster;
2436 }
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447 @Override
2448 public boolean isInitialized() {
2449 return initialized;
2450 }
2451
2452
2453
2454
2455
2456
2457 @Override
2458 public boolean isServerShutdownHandlerEnabled() {
2459 return this.serverShutdownHandlerEnabled;
2460 }
2461
2462
2463
2464
2465
2466 public boolean isInitializationStartsMetaRegionAssignment() {
2467 return this.initializationBeforeMetaAssignment;
2468 }
2469
2470 @Override
2471 public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest req)
2472 throws ServiceException {
2473 try {
2474 final byte [] regionName = req.getRegion().getValue().toByteArray();
2475 RegionSpecifierType type = req.getRegion().getType();
2476 AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
2477
2478 checkInitialized();
2479 if (type != RegionSpecifierType.REGION_NAME) {
2480 LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
2481 + " actual: " + type);
2482 }
2483 HRegionInfo regionInfo = assignmentManager.getRegionStates().getRegionInfo(regionName);
2484 if (regionInfo == null) throw new UnknownRegionException(Bytes.toString(regionName));
2485 if (cpHost != null) {
2486 if (cpHost.preAssign(regionInfo)) {
2487 return arr;
2488 }
2489 }
2490 LOG.info(getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString());
2491 assignmentManager.assign(regionInfo, true, true);
2492 if (cpHost != null) {
2493 cpHost.postAssign(regionInfo);
2494 }
2495
2496 return arr;
2497 } catch (IOException ioe) {
2498 throw new ServiceException(ioe);
2499 }
2500 }
2501
2502 public void assignRegion(HRegionInfo hri) {
2503 assignmentManager.assign(hri, true);
2504 }
2505
2506 @Override
2507 public UnassignRegionResponse unassignRegion(RpcController controller, UnassignRegionRequest req)
2508 throws ServiceException {
2509 try {
2510 final byte [] regionName = req.getRegion().getValue().toByteArray();
2511 RegionSpecifierType type = req.getRegion().getType();
2512 final boolean force = req.getForce();
2513 UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build();
2514
2515 checkInitialized();
2516 if (type != RegionSpecifierType.REGION_NAME) {
2517 LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
2518 + " actual: " + type);
2519 }
2520 Pair<HRegionInfo, ServerName> pair =
2521 MetaReader.getRegion(this.catalogTracker, regionName);
2522 if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
2523 HRegionInfo hri = pair.getFirst();
2524 if (cpHost != null) {
2525 if (cpHost.preUnassign(hri, force)) {
2526 return urr;
2527 }
2528 }
2529 LOG.debug(getClientIdAuditPrefix() + " unassign " + hri.getRegionNameAsString()
2530 + " in current location if it is online and reassign.force=" + force);
2531 this.assignmentManager.unassign(hri, force);
2532 if (this.assignmentManager.getRegionStates().isRegionOffline(hri)) {
2533 LOG.debug("Region " + hri.getRegionNameAsString()
2534 + " is not online on any region server, reassigning it.");
2535 assignRegion(hri);
2536 }
2537 if (cpHost != null) {
2538 cpHost.postUnassign(hri, force);
2539 }
2540
2541 return urr;
2542 } catch (IOException ioe) {
2543 throw new ServiceException(ioe);
2544 }
2545 }
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555 @Override
2556 public GetTableDescriptorsResponse getTableDescriptors(
2557 RpcController controller, GetTableDescriptorsRequest req) throws ServiceException {
2558 List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
2559 List<TableName> tableNameList = new ArrayList<TableName>();
2560 for(HBaseProtos.TableName tableNamePB: req.getTableNamesList()) {
2561 tableNameList.add(ProtobufUtil.toTableName(tableNamePB));
2562 }
2563 boolean bypass = false;
2564 if (this.cpHost != null) {
2565 try {
2566 bypass = this.cpHost.preGetTableDescriptors(tableNameList, descriptors);
2567 } catch (IOException ioe) {
2568 throw new ServiceException(ioe);
2569 }
2570 }
2571
2572 if (!bypass) {
2573 if (req.getTableNamesCount() == 0) {
2574
2575 Map<String, HTableDescriptor> descriptorMap = null;
2576 try {
2577 descriptorMap = this.tableDescriptors.getAll();
2578 } catch (IOException e) {
2579 LOG.warn("Failed getting all descriptors", e);
2580 }
2581 if (descriptorMap != null) {
2582 for(HTableDescriptor desc: descriptorMap.values()) {
2583 if(!desc.getTableName().isSystemTable()) {
2584 descriptors.add(desc);
2585 }
2586 }
2587 }
2588 } else {
2589 for (TableName s: tableNameList) {
2590 try {
2591 HTableDescriptor desc = this.tableDescriptors.get(s);
2592 if (desc != null) {
2593 descriptors.add(desc);
2594 }
2595 } catch (IOException e) {
2596 LOG.warn("Failed getting descriptor for " + s, e);
2597 }
2598 }
2599 }
2600
2601 if (this.cpHost != null) {
2602 try {
2603 this.cpHost.postGetTableDescriptors(descriptors);
2604 } catch (IOException ioe) {
2605 throw new ServiceException(ioe);
2606 }
2607 }
2608 }
2609
2610 GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder();
2611 for (HTableDescriptor htd: descriptors) {
2612 builder.addTableSchema(htd.convert());
2613 }
2614 return builder.build();
2615 }
2616
2617
2618
2619
2620
2621
2622
2623
2624 @Override
2625 public GetTableNamesResponse getTableNames(
2626 RpcController controller, GetTableNamesRequest req) throws ServiceException {
2627 try {
2628 Collection<HTableDescriptor> descriptors = this.tableDescriptors.getAll().values();
2629 GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
2630 for (HTableDescriptor descriptor: descriptors) {
2631 if (descriptor.getTableName().isSystemTable()) {
2632 continue;
2633 }
2634 builder.addTableNames(ProtobufUtil.toProtoTableName(descriptor.getTableName()));
2635 }
2636 return builder.build();
2637 } catch (IOException e) {
2638 throw new ServiceException(e);
2639 }
2640 }
2641
2642
2643
2644
2645
2646
2647
2648 public double getAverageLoad() {
2649 if (this.assignmentManager == null) {
2650 return 0;
2651 }
2652
2653 RegionStates regionStates = this.assignmentManager.getRegionStates();
2654 if (regionStates == null) {
2655 return 0;
2656 }
2657 return regionStates.getAverageLoad();
2658 }
2659
2660
2661
2662
2663
2664
2665
2666
2667 @Override
2668 public OfflineRegionResponse offlineRegion(RpcController controller, OfflineRegionRequest request)
2669 throws ServiceException {
2670 final byte [] regionName = request.getRegion().getValue().toByteArray();
2671 RegionSpecifierType type = request.getRegion().getType();
2672 if (type != RegionSpecifierType.REGION_NAME) {
2673 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
2674 + " actual: " + type);
2675 }
2676
2677 try {
2678 Pair<HRegionInfo, ServerName> pair =
2679 MetaReader.getRegion(this.catalogTracker, regionName);
2680 if (pair == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
2681 HRegionInfo hri = pair.getFirst();
2682 if (cpHost != null) {
2683 cpHost.preRegionOffline(hri);
2684 }
2685 LOG.info(getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString());
2686 this.assignmentManager.regionOffline(hri);
2687 if (cpHost != null) {
2688 cpHost.postRegionOffline(hri);
2689 }
2690 } catch (IOException ioe) {
2691 throw new ServiceException(ioe);
2692 }
2693 return OfflineRegionResponse.newBuilder().build();
2694 }
2695
2696 @Override
2697 public boolean registerService(Service instance) {
2698
2699
2700
2701 Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
2702 if (coprocessorServiceHandlers.containsKey(serviceDesc.getFullName())) {
2703 LOG.error("Coprocessor service "+serviceDesc.getFullName()+
2704 " already registered, rejecting request from "+instance
2705 );
2706 return false;
2707 }
2708
2709 coprocessorServiceHandlers.put(serviceDesc.getFullName(), instance);
2710 if (LOG.isDebugEnabled()) {
2711 LOG.debug("Registered master coprocessor service: service="+serviceDesc.getFullName());
2712 }
2713 return true;
2714 }
2715
2716 @Override
2717 public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
2718 final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
2719 try {
2720 ServerRpcController execController = new ServerRpcController();
2721
2722 ClientProtos.CoprocessorServiceCall call = request.getCall();
2723 String serviceName = call.getServiceName();
2724 String methodName = call.getMethodName();
2725 if (!coprocessorServiceHandlers.containsKey(serviceName)) {
2726 throw new UnknownProtocolException(null,
2727 "No registered master coprocessor service found for name "+serviceName);
2728 }
2729
2730 Service service = coprocessorServiceHandlers.get(serviceName);
2731 Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
2732 Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
2733 if (methodDesc == null) {
2734 throw new UnknownProtocolException(service.getClass(),
2735 "Unknown method "+methodName+" called on master service "+serviceName);
2736 }
2737
2738
2739 Message execRequest = service.getRequestPrototype(methodDesc).newBuilderForType()
2740 .mergeFrom(call.getRequest()).build();
2741 final Message.Builder responseBuilder =
2742 service.getResponsePrototype(methodDesc).newBuilderForType();
2743 service.callMethod(methodDesc, execController, execRequest, new RpcCallback<Message>() {
2744 @Override
2745 public void run(Message message) {
2746 if (message != null) {
2747 responseBuilder.mergeFrom(message);
2748 }
2749 }
2750 });
2751 Message execResult = responseBuilder.build();
2752
2753 if (execController.getFailedOn() != null) {
2754 throw execController.getFailedOn();
2755 }
2756 ClientProtos.CoprocessorServiceResponse.Builder builder =
2757 ClientProtos.CoprocessorServiceResponse.newBuilder();
2758 builder.setRegion(RequestConverter.buildRegionSpecifier(
2759 RegionSpecifierType.REGION_NAME, HConstants.EMPTY_BYTE_ARRAY));
2760 builder.setValue(
2761 builder.getValueBuilder().setName(execResult.getClass().getName())
2762 .setValue(execResult.toByteString()));
2763 return builder.build();
2764 } catch (IOException ie) {
2765 throw new ServiceException(ie);
2766 }
2767 }
2768
2769
2770
2771
2772
2773
2774
2775 public static HMaster constructMaster(Class<? extends HMaster> masterClass,
2776 final Configuration conf) {
2777 try {
2778 Constructor<? extends HMaster> c =
2779 masterClass.getConstructor(Configuration.class);
2780 return c.newInstance(conf);
2781 } catch (InvocationTargetException ite) {
2782 Throwable target = ite.getTargetException() != null?
2783 ite.getTargetException(): ite;
2784 if (target.getCause() != null) target = target.getCause();
2785 throw new RuntimeException("Failed construction of Master: " +
2786 masterClass.toString(), target);
2787 } catch (Exception e) {
2788 throw new RuntimeException("Failed construction of Master: " +
2789 masterClass.toString() + ((e.getCause() != null)?
2790 e.getCause().getMessage(): ""), e);
2791 }
2792 }
2793
2794
2795
2796
2797 public static void main(String [] args) {
2798 VersionInfo.logVersion();
2799 new HMasterCommandLine(HMaster.class).doMain(args);
2800 }
2801
2802 public HFileCleaner getHFileCleaner() {
2803 return this.hfileCleaner;
2804 }
2805
2806
2807
2808
2809
2810 public SnapshotManager getSnapshotManagerForTesting() {
2811 return this.snapshotManager;
2812 }
2813
2814
2815
2816
2817
2818 @Override
2819 public SnapshotResponse snapshot(RpcController controller, SnapshotRequest request)
2820 throws ServiceException {
2821 try {
2822 this.snapshotManager.checkSnapshotSupport();
2823 } catch (UnsupportedOperationException e) {
2824 throw new ServiceException(e);
2825 }
2826
2827 LOG.info(getClientIdAuditPrefix() + " snapshot request for:" +
2828 ClientSnapshotDescriptionUtils.toString(request.getSnapshot()));
2829
2830 SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(request.getSnapshot(),
2831 this.conf);
2832 try {
2833 snapshotManager.takeSnapshot(snapshot);
2834 } catch (IOException e) {
2835 throw new ServiceException(e);
2836 }
2837
2838
2839 long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
2840 SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
2841 return SnapshotResponse.newBuilder().setExpectedTimeout(waitTime).build();
2842 }
2843
2844
2845
2846
2847 @Override
2848 public GetCompletedSnapshotsResponse getCompletedSnapshots(RpcController controller,
2849 GetCompletedSnapshotsRequest request) throws ServiceException {
2850 try {
2851 GetCompletedSnapshotsResponse.Builder builder = GetCompletedSnapshotsResponse.newBuilder();
2852 List<SnapshotDescription> snapshots = snapshotManager.getCompletedSnapshots();
2853
2854
2855 for (SnapshotDescription snapshot : snapshots) {
2856 builder.addSnapshots(snapshot);
2857 }
2858 return builder.build();
2859 } catch (IOException e) {
2860 throw new ServiceException(e);
2861 }
2862 }
2863
2864
2865
2866
2867
2868
2869
2870
2871 @Override
2872 public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
2873 DeleteSnapshotRequest request) throws ServiceException {
2874 try {
2875 this.snapshotManager.checkSnapshotSupport();
2876 } catch (UnsupportedOperationException e) {
2877 throw new ServiceException(e);
2878 }
2879
2880 try {
2881 LOG.info(getClientIdAuditPrefix() + " delete " + request.getSnapshot());
2882 snapshotManager.deleteSnapshot(request.getSnapshot());
2883 return DeleteSnapshotResponse.newBuilder().build();
2884 } catch (IOException e) {
2885 throw new ServiceException(e);
2886 }
2887 }
2888
2889
2890
2891
2892
2893
2894
2895
2896 @Override
2897 public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
2898 IsSnapshotDoneRequest request) throws ServiceException {
2899 LOG.debug("Checking to see if snapshot from request:" +
2900 ClientSnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done");
2901 try {
2902 IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
2903 boolean done = snapshotManager.isSnapshotDone(request.getSnapshot());
2904 builder.setDone(done);
2905 return builder.build();
2906 } catch (IOException e) {
2907 throw new ServiceException(e);
2908 }
2909 }
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924 @Override
2925 public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
2926 RestoreSnapshotRequest request) throws ServiceException {
2927 try {
2928 this.snapshotManager.checkSnapshotSupport();
2929 } catch (UnsupportedOperationException e) {
2930 throw new ServiceException(e);
2931 }
2932
2933
2934 try {
2935 TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
2936 getNamespaceDescriptor(dstTable.getNamespaceAsString());
2937 } catch (IOException ioe) {
2938 throw new ServiceException(ioe);
2939 }
2940
2941 try {
2942 SnapshotDescription reqSnapshot = request.getSnapshot();
2943 snapshotManager.restoreSnapshot(reqSnapshot);
2944 return RestoreSnapshotResponse.newBuilder().build();
2945 } catch (IOException e) {
2946 throw new ServiceException(e);
2947 }
2948 }
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960 @Override
2961 public IsRestoreSnapshotDoneResponse isRestoreSnapshotDone(RpcController controller,
2962 IsRestoreSnapshotDoneRequest request) throws ServiceException {
2963 try {
2964 SnapshotDescription snapshot = request.getSnapshot();
2965 IsRestoreSnapshotDoneResponse.Builder builder = IsRestoreSnapshotDoneResponse.newBuilder();
2966 boolean done = snapshotManager.isRestoreDone(snapshot);
2967 builder.setDone(done);
2968 return builder.build();
2969 } catch (IOException e) {
2970 throw new ServiceException(e);
2971 }
2972 }
2973
2974 @Override
2975 public ModifyNamespaceResponse modifyNamespace(RpcController controller,
2976 ModifyNamespaceRequest request) throws ServiceException {
2977 try {
2978 modifyNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
2979 return ModifyNamespaceResponse.getDefaultInstance();
2980 } catch (IOException e) {
2981 throw new ServiceException(e);
2982 }
2983 }
2984
2985 @Override
2986 public CreateNamespaceResponse createNamespace(RpcController controller,
2987 CreateNamespaceRequest request) throws ServiceException {
2988 try {
2989 createNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
2990 return CreateNamespaceResponse.getDefaultInstance();
2991 } catch (IOException e) {
2992 throw new ServiceException(e);
2993 }
2994 }
2995
2996 @Override
2997 public DeleteNamespaceResponse deleteNamespace(RpcController controller,
2998 DeleteNamespaceRequest request) throws ServiceException {
2999 try {
3000 deleteNamespace(request.getNamespaceName());
3001 return DeleteNamespaceResponse.getDefaultInstance();
3002 } catch (IOException e) {
3003 throw new ServiceException(e);
3004 }
3005 }
3006
3007 @Override
3008 public GetNamespaceDescriptorResponse getNamespaceDescriptor(
3009 RpcController controller, GetNamespaceDescriptorRequest request)
3010 throws ServiceException {
3011 try {
3012 return GetNamespaceDescriptorResponse.newBuilder()
3013 .setNamespaceDescriptor(
3014 ProtobufUtil.toProtoNamespaceDescriptor(getNamespaceDescriptor(request.getNamespaceName())))
3015 .build();
3016 } catch (IOException e) {
3017 throw new ServiceException(e);
3018 }
3019 }
3020
3021 @Override
3022 public ListNamespaceDescriptorsResponse listNamespaceDescriptors(
3023 RpcController controller, ListNamespaceDescriptorsRequest request)
3024 throws ServiceException {
3025 try {
3026 ListNamespaceDescriptorsResponse.Builder response =
3027 ListNamespaceDescriptorsResponse.newBuilder();
3028 for(NamespaceDescriptor ns: listNamespaceDescriptors()) {
3029 response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
3030 }
3031 return response.build();
3032 } catch (IOException e) {
3033 throw new ServiceException(e);
3034 }
3035 }
3036
3037 @Override
3038 public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(
3039 RpcController controller, ListTableDescriptorsByNamespaceRequest request)
3040 throws ServiceException {
3041 try {
3042 ListTableDescriptorsByNamespaceResponse.Builder b =
3043 ListTableDescriptorsByNamespaceResponse.newBuilder();
3044 for(HTableDescriptor htd: listTableDescriptorsByNamespace(request.getNamespaceName())) {
3045 b.addTableSchema(htd.convert());
3046 }
3047 return b.build();
3048 } catch (IOException e) {
3049 throw new ServiceException(e);
3050 }
3051 }
3052
3053 @Override
3054 public ListTableNamesByNamespaceResponse listTableNamesByNamespace(
3055 RpcController controller, ListTableNamesByNamespaceRequest request)
3056 throws ServiceException {
3057 try {
3058 ListTableNamesByNamespaceResponse.Builder b =
3059 ListTableNamesByNamespaceResponse.newBuilder();
3060 for (TableName tableName: listTableNamesByNamespace(request.getNamespaceName())) {
3061 b.addTableName(ProtobufUtil.toProtoTableName(tableName));
3062 }
3063 return b.build();
3064 } catch (IOException e) {
3065 throw new ServiceException(e);
3066 }
3067 }
3068
3069 private boolean isHealthCheckerConfigured() {
3070 String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
3071 return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
3072 }
3073
3074 @Override
3075 public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
3076 TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
3077 if (cpHost != null) {
3078 if (cpHost.preCreateNamespace(descriptor)) {
3079 return;
3080 }
3081 }
3082 LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
3083 tableNamespaceManager.create(descriptor);
3084 if (cpHost != null) {
3085 cpHost.postCreateNamespace(descriptor);
3086 }
3087 }
3088
3089 @Override
3090 public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
3091 TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
3092 if (cpHost != null) {
3093 if (cpHost.preModifyNamespace(descriptor)) {
3094 return;
3095 }
3096 }
3097 LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
3098 tableNamespaceManager.update(descriptor);
3099 if (cpHost != null) {
3100 cpHost.postModifyNamespace(descriptor);
3101 }
3102 }
3103
3104 @Override
3105 public void deleteNamespace(String name) throws IOException {
3106 if (cpHost != null) {
3107 if (cpHost.preDeleteNamespace(name)) {
3108 return;
3109 }
3110 }
3111 LOG.info(getClientIdAuditPrefix() + " delete " + name);
3112 tableNamespaceManager.remove(name);
3113 if (cpHost != null) {
3114 cpHost.postDeleteNamespace(name);
3115 }
3116 }
3117
3118 @Override
3119 public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
3120 if (!isTableNamespaceManagerReady()) {
3121 throw new IOException("Table Namespace Manager not ready yet, try again later");
3122 }
3123 NamespaceDescriptor nsd = tableNamespaceManager.get(name);
3124 if (nsd == null) {
3125 throw new NamespaceNotFoundException(name);
3126 }
3127 return nsd;
3128 }
3129
3130 @Override
3131 public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
3132 if (!isTableNamespaceManagerReady()) {
3133 return Lists.newArrayList();
3134 }
3135 return Lists.newArrayList(tableNamespaceManager.list());
3136 }
3137
3138 @Override
3139 public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
3140 if (!isTableNamespaceManagerReady()) {
3141 return Lists.newArrayList();
3142 }
3143 getNamespaceDescriptor(name);
3144 return Lists.newArrayList(tableDescriptors.getByNamespace(name).values());
3145 }
3146
3147 @Override
3148 public List<TableName> listTableNamesByNamespace(String name) throws IOException {
3149 List<TableName> tableNames = Lists.newArrayList();
3150 if (!isTableNamespaceManagerReady()) {
3151 return tableNames;
3152 }
3153 getNamespaceDescriptor(name);
3154 for (HTableDescriptor descriptor: tableDescriptors.getByNamespace(name).values()) {
3155 tableNames.add(descriptor.getTableName());
3156 }
3157 return tableNames;
3158 }
3159
3160 private boolean isTableNamespaceManagerReady() throws IOException {
3161 boolean ready = tableNamespaceManager != null &&
3162 tableNamespaceManager.isTableAvailableAndInitialized();
3163 if (!ready) {
3164 LOG.warn("Table Namespace Manager not ready yet");
3165 }
3166 return ready;
3167 }
3168
3169 }