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