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