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