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