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