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