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.io.InterruptedIOException;
23 import java.io.PrintWriter;
24 import java.lang.reflect.Constructor;
25 import java.lang.reflect.InvocationTargetException;
26 import java.net.InetAddress;
27 import java.net.InetSocketAddress;
28 import java.net.UnknownHostException;
29 import java.util.ArrayList;
30 import java.util.Arrays;
31 import java.util.Collection;
32 import java.util.Collections;
33 import java.util.Comparator;
34 import java.util.HashSet;
35 import java.util.List;
36 import java.util.Map;
37 import java.util.Set;
38 import java.util.concurrent.Callable;
39 import java.util.concurrent.ExecutionException;
40 import java.util.concurrent.Executors;
41 import java.util.concurrent.Future;
42 import java.util.concurrent.TimeUnit;
43 import java.util.concurrent.atomic.AtomicReference;
44
45 import javax.management.ObjectName;
46
47 import org.apache.commons.logging.Log;
48 import org.apache.commons.logging.LogFactory;
49 import org.apache.hadoop.hbase.classification.InterfaceAudience;
50 import org.apache.hadoop.conf.Configuration;
51 import org.apache.hadoop.fs.Path;
52 import org.apache.hadoop.hbase.Abortable;
53 import org.apache.hadoop.hbase.Chore;
54 import org.apache.hadoop.hbase.ClusterId;
55 import org.apache.hadoop.hbase.ClusterStatus;
56 import org.apache.hadoop.hbase.DoNotRetryIOException;
57 import org.apache.hadoop.hbase.HBaseIOException;
58 import org.apache.hadoop.hbase.HColumnDescriptor;
59 import org.apache.hadoop.hbase.HConstants;
60 import org.apache.hadoop.hbase.HRegionInfo;
61 import org.apache.hadoop.hbase.HTableDescriptor;
62 import org.apache.hadoop.hbase.HealthCheckChore;
63 import org.apache.hadoop.hbase.MasterNotRunningException;
64 import org.apache.hadoop.hbase.NamespaceDescriptor;
65 import org.apache.hadoop.hbase.NamespaceNotFoundException;
66 import org.apache.hadoop.hbase.PleaseHoldException;
67 import org.apache.hadoop.hbase.Server;
68 import org.apache.hadoop.hbase.ServerLoad;
69 import org.apache.hadoop.hbase.ServerName;
70 import org.apache.hadoop.hbase.TableDescriptors;
71 import org.apache.hadoop.hbase.TableName;
72 import org.apache.hadoop.hbase.TableNotDisabledException;
73 import org.apache.hadoop.hbase.TableNotFoundException;
74 import org.apache.hadoop.hbase.UnknownRegionException;
75 import org.apache.hadoop.hbase.catalog.CatalogTracker;
76 import org.apache.hadoop.hbase.catalog.MetaReader;
77 import org.apache.hadoop.hbase.client.HConnectionManager;
78 import org.apache.hadoop.hbase.client.MetaScanner;
79 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
80 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
81 import org.apache.hadoop.hbase.client.Result;
82 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
83 import org.apache.hadoop.hbase.exceptions.DeserializationException;
84 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
85 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
86 import org.apache.hadoop.hbase.executor.ExecutorService;
87 import org.apache.hadoop.hbase.executor.ExecutorType;
88 import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
89 import org.apache.hadoop.hbase.ipc.RequestContext;
90 import org.apache.hadoop.hbase.ipc.RpcServer;
91 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
92 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
93 import org.apache.hadoop.hbase.ipc.ServerRpcController;
94 import org.apache.hadoop.hbase.master.RegionState.State;
95 import org.apache.hadoop.hbase.master.balancer.BalancerChore;
96 import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
97 import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
98 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
99 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
100 import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
101 import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
102 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
103 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
104 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
105 import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
106 import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
107 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
108 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
109 import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
110 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
111 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
112 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
113 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
114 import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
115 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
116 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
117 import org.apache.hadoop.hbase.protobuf.RequestConverter;
118 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
119 import org.apache.hadoop.hbase.protobuf.generated.*;
120 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
121 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
122 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
123 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
124 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
125 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
126 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
127 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
128 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
129 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
130 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
131 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
132 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
133 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
134 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
135 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
136 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
137 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
138 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
139 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
140 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
141 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
142 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
143 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
144 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
145 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
146 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
147 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
148 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
149 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
150 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
151 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
152 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
153 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
154 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
155 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
156 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
157 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
158 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
159 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
160 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
161 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
162 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
163 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
164 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
165 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesResponse;
166 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
167 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
168 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
169 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
170 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
171 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
172 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
173 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
174 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
175 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
176 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
177 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
178 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
179 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
180 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
181 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
182 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
183 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
184 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
185 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
186 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
187 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
188 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
189 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
190 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
191 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
192 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
193 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
194 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
195 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
196 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
197 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
198 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
199 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
200 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
201 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
202 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
203 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
204 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
205 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
206 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
207 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
208 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
209 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
210 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
211 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
212 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
213 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
214 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
215 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportResponse;
216 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
217 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
218 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
219 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
220 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
221 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
222 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
223 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
224 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
225 import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
226 import org.apache.hadoop.hbase.replication.regionserver.Replication;
227 import org.apache.hadoop.hbase.security.UserProvider;
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 if (!masterRecovery) {
1034 if (this.cpHost != null) {
1035
1036 try {
1037 this.cpHost.postStartMaster();
1038 } catch (IOException ioe) {
1039 LOG.error("Coprocessor postStartMaster() hook failed", ioe);
1040 }
1041 }
1042 }
1043
1044 zombieDetector.interrupt();
1045 }
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055 ServerManager createServerManager(final Server master,
1056 final MasterServices services)
1057 throws IOException {
1058
1059
1060 return new ServerManager(master, services);
1061 }
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071 void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs)
1072 throws InterruptedException, IOException, KeeperException {
1073
1074 int assigned = 0;
1075 long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
1076 status.setStatus("Assigning hbase:meta region");
1077
1078 RegionStates regionStates = assignmentManager.getRegionStates();
1079
1080 RegionState regionState = this.catalogTracker.getMetaRegionState();
1081 ServerName currentMetaServer = regionState.getServerName();
1082
1083 if (!ConfigUtil.useZKForAssignment(conf)) {
1084 regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO, regionState.getState(),
1085 currentMetaServer);
1086 } else {
1087 regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
1088 }
1089 boolean rit =
1090 this.assignmentManager
1091 .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
1092 boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
1093 if (!metaRegionLocation || !regionState.isOpened()) {
1094
1095
1096 assigned++;
1097 if (!ConfigUtil.useZKForAssignment(conf)) {
1098 assignMetaZkLess(regionStates, regionState, timeout, previouslyFailedMetaRSs);
1099 } else if (!rit) {
1100
1101 if (currentMetaServer != null) {
1102
1103
1104
1105
1106
1107
1108
1109 if (serverManager.isServerOnline(currentMetaServer)) {
1110 LOG.info("Forcing expire of " + currentMetaServer);
1111 serverManager.expireServer(currentMetaServer);
1112 }
1113 splitMetaLogBeforeAssignment(currentMetaServer);
1114 previouslyFailedMetaRSs.add(currentMetaServer);
1115 }
1116 assignmentManager.assignMeta();
1117 }
1118 } else {
1119
1120 regionStates.updateRegionState(
1121 HRegionInfo.FIRST_META_REGIONINFO, State.OPEN, currentMetaServer);
1122 this.assignmentManager.regionOnline(
1123 HRegionInfo.FIRST_META_REGIONINFO, currentMetaServer);
1124 }
1125
1126 enableMeta(TableName.META_TABLE_NAME);
1127
1128 if ((RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode())
1129 && (!previouslyFailedMetaRSs.isEmpty())) {
1130
1131 status.setStatus("replaying log for Meta Region");
1132 this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs);
1133 }
1134
1135
1136
1137
1138
1139 enableServerShutdownHandler(assigned != 0);
1140
1141 LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
1142 ", location=" + catalogTracker.getMetaLocation());
1143 status.setStatus("META assigned.");
1144 }
1145
1146 private void assignMetaZkLess(RegionStates regionStates, RegionState regionState, long timeout,
1147 Set<ServerName> previouslyFailedRs) throws IOException, KeeperException {
1148 ServerName currentServer = regionState.getServerName();
1149 if (serverManager.isServerOnline(currentServer)) {
1150 LOG.info("Meta was in transition on " + currentServer);
1151 assignmentManager.processRegionInTransitionZkLess();
1152 } else {
1153 if (currentServer != null) {
1154 splitMetaLogBeforeAssignment(currentServer);
1155 regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
1156 previouslyFailedRs.add(currentServer);
1157 }
1158 LOG.info("Re-assigning hbase:meta, it was on " + currentServer);
1159 regionStates.updateRegionState(HRegionInfo.FIRST_META_REGIONINFO, State.OFFLINE);
1160 assignmentManager.assignMeta();
1161 }
1162 }
1163
1164 void initNamespace() throws IOException {
1165
1166 tableNamespaceManager = new TableNamespaceManager(this);
1167 tableNamespaceManager.start();
1168 }
1169
1170 private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
1171 if (RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode()) {
1172
1173 Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
1174 regions.add(HRegionInfo.FIRST_META_REGIONINFO);
1175 this.fileSystemManager.prepareLogReplay(currentMetaServer, regions);
1176 } else {
1177
1178 this.fileSystemManager.splitMetaLog(currentMetaServer);
1179 }
1180 }
1181
1182 private void enableServerShutdownHandler(
1183 final boolean waitForMeta) throws IOException, InterruptedException {
1184
1185
1186
1187
1188
1189 if (!serverShutdownHandlerEnabled) {
1190 serverShutdownHandlerEnabled = true;
1191 this.serverManager.processQueuedDeadServers();
1192 }
1193
1194 if (waitForMeta) {
1195 this.catalogTracker.waitForMeta();
1196
1197
1198 this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
1199 }
1200 }
1201
1202 private void enableMeta(TableName metaTableName) {
1203 if (!this.assignmentManager.getZKTable().isEnabledTable(metaTableName)) {
1204 this.assignmentManager.setEnabledTable(metaTableName);
1205 }
1206 }
1207
1208
1209
1210
1211
1212
1213 private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
1214 Set<ServerName> result = new HashSet<ServerName>();
1215 String metaRecoveringZNode = ZKUtil.joinZNode(zooKeeper.recoveringRegionsZNode,
1216 HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
1217 List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(zooKeeper, metaRecoveringZNode);
1218 if (regionFailedServers == null) return result;
1219
1220 for(String failedServer : regionFailedServers) {
1221 ServerName server = ServerName.parseServerName(failedServer);
1222 result.add(server);
1223 }
1224 return result;
1225 }
1226
1227 @Override
1228 public TableDescriptors getTableDescriptors() {
1229 return this.tableDescriptors;
1230 }
1231
1232
1233 public InfoServer getInfoServer() {
1234 return this.infoServer;
1235 }
1236
1237 @Override
1238 public Configuration getConfiguration() {
1239 return this.conf;
1240 }
1241
1242 @Override
1243 public ServerManager getServerManager() {
1244 return this.serverManager;
1245 }
1246
1247 @Override
1248 public ExecutorService getExecutorService() {
1249 return this.executorService;
1250 }
1251
1252 @Override
1253 public MasterFileSystem getMasterFileSystem() {
1254 return this.fileSystemManager;
1255 }
1256
1257
1258
1259
1260
1261 public ZooKeeperWatcher getZooKeeperWatcher() {
1262 return this.zooKeeper;
1263 }
1264
1265 public ActiveMasterManager getActiveMasterManager() {
1266 return this.activeMasterManager;
1267 }
1268
1269 public MasterAddressTracker getMasterAddressTracker() {
1270 return this.masterAddressTracker;
1271 }
1272
1273
1274
1275
1276
1277
1278
1279
1280 void startServiceThreads() throws IOException{
1281
1282 this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
1283 conf.getInt("hbase.master.executor.openregion.threads", 5));
1284 this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
1285 conf.getInt("hbase.master.executor.closeregion.threads", 5));
1286 this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
1287 conf.getInt("hbase.master.executor.serverops.threads", 5));
1288 this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
1289 conf.getInt("hbase.master.executor.serverops.threads", 5));
1290 this.executorService.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS,
1291 conf.getInt("hbase.master.executor.logreplayops.threads", 10));
1292
1293
1294
1295
1296
1297
1298 this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
1299
1300
1301 String n = Thread.currentThread().getName();
1302 int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
1303 this.logCleaner =
1304 new LogCleaner(cleanerInterval,
1305 this, conf, getMasterFileSystem().getFileSystem(),
1306 getMasterFileSystem().getOldLogDir());
1307 Threads.setDaemonThreadRunning(logCleaner.getThread(), n + ".oldLogCleaner");
1308
1309
1310 Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
1311 this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem()
1312 .getFileSystem(), archiveDir);
1313 Threads.setDaemonThreadRunning(hfileCleaner.getThread(), n + ".archivedHFileCleaner");
1314
1315
1316 if (this.healthCheckChore != null) {
1317 Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker");
1318 }
1319
1320
1321 this.rpcServer.openServer();
1322 this.rpcServerOpen = true;
1323 if (LOG.isTraceEnabled()) {
1324 LOG.trace("Started service threads");
1325 }
1326 }
1327
1328
1329
1330
1331
1332 boolean isRpcServerOpen() {
1333 return this.rpcServerOpen;
1334 }
1335
1336 private void stopServiceThreads() {
1337 if (LOG.isDebugEnabled()) {
1338 LOG.debug("Stopping service threads");
1339 }
1340 if (this.rpcServer != null) this.rpcServer.stop();
1341 this.rpcServerOpen = false;
1342
1343 if (this.logCleaner!= null) this.logCleaner.interrupt();
1344 if (this.hfileCleaner != null) this.hfileCleaner.interrupt();
1345
1346 if (this.infoServer != null) {
1347 LOG.info("Stopping infoServer");
1348 try {
1349 this.infoServer.stop();
1350 } catch (Exception ex) {
1351 ex.printStackTrace();
1352 }
1353 }
1354 if (this.executorService != null) this.executorService.shutdown();
1355 if (this.healthCheckChore != null) {
1356 this.healthCheckChore.interrupt();
1357 }
1358 if (this.pauseMonitor != null) {
1359 this.pauseMonitor.stop();
1360 }
1361 }
1362
1363 private static Thread getAndStartClusterStatusChore(HMaster master) {
1364 if (master == null || master.balancer == null) {
1365 return null;
1366 }
1367 Chore chore = new ClusterStatusChore(master, master.balancer);
1368 return Threads.setDaemonThreadRunning(chore.getThread());
1369 }
1370
1371 private static Thread getAndStartBalancerChore(final HMaster master) {
1372
1373 Chore chore = new BalancerChore(master);
1374 return Threads.setDaemonThreadRunning(chore.getThread());
1375 }
1376
1377 private void stopChores() {
1378 if (this.balancerChore != null) {
1379 this.balancerChore.interrupt();
1380 }
1381 if (this.clusterStatusChore != null) {
1382 this.clusterStatusChore.interrupt();
1383 }
1384 if (this.catalogJanitorChore != null) {
1385 this.catalogJanitorChore.interrupt();
1386 }
1387 if (this.clusterStatusPublisherChore != null){
1388 clusterStatusPublisherChore.interrupt();
1389 }
1390 }
1391
1392 @Override
1393 public RegionServerStartupResponse regionServerStartup(
1394 RpcController controller, RegionServerStartupRequest request) throws ServiceException {
1395
1396 try {
1397 InetAddress ia = getRemoteInetAddress(request.getPort(), request.getServerStartCode());
1398 ServerName rs = this.serverManager.regionServerStartup(ia, request.getPort(),
1399 request.getServerStartCode(), request.getServerCurrentTime());
1400
1401
1402 RegionServerStartupResponse.Builder resp = createConfigurationSubset();
1403 NameStringPair.Builder entry = NameStringPair.newBuilder()
1404 .setName(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)
1405 .setValue(rs.getHostname());
1406 resp.addMapEntries(entry.build());
1407
1408 return resp.build();
1409 } catch (IOException ioe) {
1410 throw new ServiceException(ioe);
1411 }
1412 }
1413
1414
1415
1416
1417
1418 InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
1419 throws UnknownHostException {
1420
1421
1422 return RpcServer.getRemoteIp();
1423 }
1424
1425
1426
1427
1428
1429 protected RegionServerStartupResponse.Builder createConfigurationSubset() {
1430 RegionServerStartupResponse.Builder resp = addConfig(
1431 RegionServerStartupResponse.newBuilder(), HConstants.HBASE_DIR);
1432 resp = addConfig(resp, "fs.default.name");
1433 return addConfig(resp, "hbase.master.info.port");
1434 }
1435
1436 private RegionServerStartupResponse.Builder addConfig(
1437 final RegionServerStartupResponse.Builder resp, final String key) {
1438 NameStringPair.Builder entry = NameStringPair.newBuilder()
1439 .setName(key)
1440 .setValue(this.conf.get(key));
1441 resp.addMapEntries(entry.build());
1442 return resp;
1443 }
1444
1445 @Override
1446 public GetLastFlushedSequenceIdResponse getLastFlushedSequenceId(RpcController controller,
1447 GetLastFlushedSequenceIdRequest request) throws ServiceException {
1448 byte[] regionName = request.getRegionName().toByteArray();
1449 long seqId = serverManager.getLastFlushedSequenceId(regionName);
1450 return ResponseConverter.buildGetLastFlushedSequenceIdResponse(seqId);
1451 }
1452
1453 @Override
1454 public RegionServerReportResponse regionServerReport(
1455 RpcController controller, RegionServerReportRequest request) throws ServiceException {
1456 try {
1457 ClusterStatusProtos.ServerLoad sl = request.getLoad();
1458 ServerName serverName = ProtobufUtil.toServerName(request.getServer());
1459 ServerLoad oldLoad = serverManager.getLoad(serverName);
1460 this.serverManager.regionServerReport(serverName, new ServerLoad(sl));
1461 if (sl != null && this.metricsMaster != null) {
1462
1463 this.metricsMaster.incrementRequests(sl.getTotalNumberOfRequests()
1464 - (oldLoad != null ? oldLoad.getTotalNumberOfRequests() : 0));
1465 }
1466 } catch (IOException ioe) {
1467 throw new ServiceException(ioe);
1468 }
1469
1470 return RegionServerReportResponse.newBuilder().build();
1471 }
1472
1473 @Override
1474 public ReportRSFatalErrorResponse reportRSFatalError(
1475 RpcController controller, ReportRSFatalErrorRequest request) throws ServiceException {
1476 String errorText = request.getErrorMessage();
1477 ServerName sn = ProtobufUtil.toServerName(request.getServer());
1478 String msg = "Region server " + sn +
1479 " reported a fatal error:\n" + errorText;
1480 LOG.error(msg);
1481 rsFatals.add(msg);
1482
1483 return ReportRSFatalErrorResponse.newBuilder().build();
1484 }
1485
1486 public boolean isMasterRunning() {
1487 return !isStopped();
1488 }
1489
1490 @Override
1491 public IsMasterRunningResponse isMasterRunning(RpcController c, IsMasterRunningRequest req)
1492 throws ServiceException {
1493 return IsMasterRunningResponse.newBuilder().setIsMasterRunning(isMasterRunning()).build();
1494 }
1495
1496 @Override
1497 public RunCatalogScanResponse runCatalogScan(RpcController c,
1498 RunCatalogScanRequest req) throws ServiceException {
1499 try {
1500 return ResponseConverter.buildRunCatalogScanResponse(catalogJanitorChore.scan());
1501 } catch (IOException ioe) {
1502 throw new ServiceException(ioe);
1503 }
1504 }
1505
1506 @Override
1507 public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
1508 EnableCatalogJanitorRequest req) throws ServiceException {
1509 return EnableCatalogJanitorResponse.newBuilder().
1510 setPrevValue(catalogJanitorChore.setEnabled(req.getEnable())).build();
1511 }
1512
1513 @Override
1514 public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController c,
1515 IsCatalogJanitorEnabledRequest req) throws ServiceException {
1516 boolean isEnabled = catalogJanitorChore != null ? catalogJanitorChore.getEnabled() : false;
1517 return IsCatalogJanitorEnabledResponse.newBuilder().setValue(isEnabled).build();
1518 }
1519
1520
1521
1522
1523 private int getBalancerCutoffTime() {
1524 int balancerCutoffTime =
1525 getConfiguration().getInt("hbase.balancer.max.balancing", -1);
1526 if (balancerCutoffTime == -1) {
1527
1528 int balancerPeriod =
1529 getConfiguration().getInt("hbase.balancer.period", 300000);
1530 balancerCutoffTime = balancerPeriod;
1531
1532 if (balancerCutoffTime <= 0) balancerCutoffTime = balancerPeriod;
1533 }
1534 return balancerCutoffTime;
1535 }
1536
1537 public boolean balance() throws HBaseIOException {
1538
1539 if (!this.initialized) {
1540 LOG.debug("Master has not been initialized, don't run balancer.");
1541 return false;
1542 }
1543
1544 int maximumBalanceTime = getBalancerCutoffTime();
1545 boolean balancerRan;
1546 synchronized (this.balancer) {
1547
1548 if (!this.loadBalancerTracker.isBalancerOn()) return false;
1549
1550 if (this.assignmentManager.getRegionStates().isRegionsInTransition()) {
1551 Map<String, RegionState> regionsInTransition =
1552 this.assignmentManager.getRegionStates().getRegionsInTransition();
1553 LOG.debug("Not running balancer because " + regionsInTransition.size() +
1554 " region(s) in transition: " + org.apache.commons.lang.StringUtils.
1555 abbreviate(regionsInTransition.toString(), 256));
1556 return false;
1557 }
1558 if (this.serverManager.areDeadServersInProgress()) {
1559 LOG.debug("Not running balancer because processing dead regionserver(s): " +
1560 this.serverManager.getDeadServers());
1561 return false;
1562 }
1563
1564 if (this.cpHost != null) {
1565 try {
1566 if (this.cpHost.preBalance()) {
1567 LOG.debug("Coprocessor bypassing balancer request");
1568 return false;
1569 }
1570 } catch (IOException ioe) {
1571 LOG.error("Error invoking master coprocessor preBalance()", ioe);
1572 return false;
1573 }
1574 }
1575
1576 Map<TableName, Map<ServerName, List<HRegionInfo>>> assignmentsByTable =
1577 this.assignmentManager.getRegionStates().getAssignmentsByTable();
1578
1579 List<RegionPlan> plans = new ArrayList<RegionPlan>();
1580
1581 this.balancer.setClusterStatus(getClusterStatus());
1582 for (Map<ServerName, List<HRegionInfo>> assignments : assignmentsByTable.values()) {
1583 List<RegionPlan> partialPlans = this.balancer.balanceCluster(assignments);
1584 if (partialPlans != null) plans.addAll(partialPlans);
1585 }
1586 long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
1587 int rpCount = 0;
1588 long totalRegPlanExecTime = 0;
1589 balancerRan = plans != null;
1590 if (plans != null && !plans.isEmpty()) {
1591 for (RegionPlan plan: plans) {
1592 LOG.info("balance " + plan);
1593 long balStartTime = System.currentTimeMillis();
1594
1595 this.assignmentManager.balance(plan);
1596 totalRegPlanExecTime += System.currentTimeMillis()-balStartTime;
1597 rpCount++;
1598 if (rpCount < plans.size() &&
1599
1600 (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
1601
1602 LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
1603 maximumBalanceTime);
1604 break;
1605 }
1606 }
1607 }
1608 if (this.cpHost != null) {
1609 try {
1610 this.cpHost.postBalance(rpCount < plans.size() ? plans.subList(0, rpCount) : plans);
1611 } catch (IOException ioe) {
1612
1613 LOG.error("Error invoking master coprocessor postBalance()", ioe);
1614 }
1615 }
1616 }
1617 return balancerRan;
1618 }
1619
1620 @Override
1621 public BalanceResponse balance(RpcController c, BalanceRequest request) throws ServiceException {
1622 try {
1623 return BalanceResponse.newBuilder().setBalancerRan(balance()).build();
1624 } catch (HBaseIOException ex) {
1625 throw new ServiceException(ex);
1626 }
1627 }
1628
1629 enum BalanceSwitchMode {
1630 SYNC,
1631 ASYNC
1632 }
1633
1634
1635
1636
1637
1638
1639
1640 public boolean switchBalancer(final boolean b, BalanceSwitchMode mode) throws IOException {
1641 boolean oldValue = this.loadBalancerTracker.isBalancerOn();
1642 boolean newValue = b;
1643 try {
1644 if (this.cpHost != null) {
1645 newValue = this.cpHost.preBalanceSwitch(newValue);
1646 }
1647 try {
1648 if (mode == BalanceSwitchMode.SYNC) {
1649 synchronized (this.balancer) {
1650 this.loadBalancerTracker.setBalancerOn(newValue);
1651 }
1652 } else {
1653 this.loadBalancerTracker.setBalancerOn(newValue);
1654 }
1655 } catch (KeeperException ke) {
1656 throw new IOException(ke);
1657 }
1658 LOG.info(getClientIdAuditPrefix() + " set balanceSwitch=" + newValue);
1659 if (this.cpHost != null) {
1660 this.cpHost.postBalanceSwitch(oldValue, newValue);
1661 }
1662 } catch (IOException ioe) {
1663 LOG.warn("Error flipping balance switch", ioe);
1664 }
1665 return oldValue;
1666 }
1667
1668
1669
1670
1671 String getClientIdAuditPrefix() {
1672 return "Client=" + RequestContext.getRequestUserName() + "/" +
1673 RequestContext.get().getRemoteAddress();
1674 }
1675
1676 public boolean synchronousBalanceSwitch(final boolean b) throws IOException {
1677 return switchBalancer(b, BalanceSwitchMode.SYNC);
1678 }
1679
1680 public boolean balanceSwitch(final boolean b) throws IOException {
1681 return switchBalancer(b, BalanceSwitchMode.ASYNC);
1682 }
1683
1684 @Override
1685 public SetBalancerRunningResponse setBalancerRunning(
1686 RpcController controller, SetBalancerRunningRequest req) throws ServiceException {
1687 try {
1688 boolean prevValue = (req.getSynchronous())?
1689 synchronousBalanceSwitch(req.getOn()):balanceSwitch(req.getOn());
1690 return SetBalancerRunningResponse.newBuilder().setPrevBalanceValue(prevValue).build();
1691 } catch (IOException ioe) {
1692 throw new ServiceException(ioe);
1693 }
1694 }
1695
1696
1697
1698
1699
1700
1701
1702 public void setCatalogJanitorEnabled(final boolean b) {
1703 this.catalogJanitorChore.setEnabled(b);
1704 }
1705
1706 @Override
1707 public DispatchMergingRegionsResponse dispatchMergingRegions(
1708 RpcController controller, DispatchMergingRegionsRequest request)
1709 throws ServiceException {
1710 final byte[] encodedNameOfRegionA = request.getRegionA().getValue()
1711 .toByteArray();
1712 final byte[] encodedNameOfRegionB = request.getRegionB().getValue()
1713 .toByteArray();
1714 final boolean forcible = request.getForcible();
1715 if (request.getRegionA().getType() != RegionSpecifierType.ENCODED_REGION_NAME
1716 || request.getRegionB().getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
1717 LOG.warn("mergeRegions specifier type: expected: "
1718 + RegionSpecifierType.ENCODED_REGION_NAME + " actual: region_a="
1719 + request.getRegionA().getType() + ", region_b="
1720 + request.getRegionB().getType());
1721 }
1722 RegionState regionStateA = assignmentManager.getRegionStates()
1723 .getRegionState(Bytes.toString(encodedNameOfRegionA));
1724 RegionState regionStateB = assignmentManager.getRegionStates()
1725 .getRegionState(Bytes.toString(encodedNameOfRegionB));
1726 if (regionStateA == null || regionStateB == null) {
1727 throw new ServiceException(new UnknownRegionException(
1728 Bytes.toStringBinary(regionStateA == null ? encodedNameOfRegionA
1729 : encodedNameOfRegionB)));
1730 }
1731
1732 if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
1733 throw new ServiceException(new MergeRegionException(
1734 "Unable to merge regions not online " + regionStateA + ", " + regionStateB));
1735 }
1736
1737 HRegionInfo regionInfoA = regionStateA.getRegion();
1738 HRegionInfo regionInfoB = regionStateB.getRegion();
1739 if (regionInfoA.compareTo(regionInfoB) == 0) {
1740 throw new ServiceException(new MergeRegionException(
1741 "Unable to merge a region to itself " + regionInfoA + ", " + regionInfoB));
1742 }
1743
1744 if (!forcible && !HRegionInfo.areAdjacent(regionInfoA, regionInfoB)) {
1745 throw new ServiceException(new MergeRegionException(
1746 "Unable to merge not adjacent regions "
1747 + regionInfoA.getRegionNameAsString() + ", "
1748 + regionInfoB.getRegionNameAsString()
1749 + " where forcible = " + forcible));
1750 }
1751
1752 try {
1753 dispatchMergingRegions(regionInfoA, regionInfoB, forcible);
1754 } catch (IOException ioe) {
1755 throw new ServiceException(ioe);
1756 }
1757
1758 return DispatchMergingRegionsResponse.newBuilder().build();
1759 }
1760
1761 @Override
1762 public void dispatchMergingRegions(final HRegionInfo region_a,
1763 final HRegionInfo region_b, final boolean forcible) throws IOException {
1764 checkInitialized();
1765 this.executorService.submit(new DispatchMergingRegionHandler(this,
1766 this.catalogJanitorChore, region_a, region_b, forcible));
1767 }
1768
1769 @Override
1770 public MoveRegionResponse moveRegion(RpcController controller, MoveRegionRequest req)
1771 throws ServiceException {
1772 final byte [] encodedRegionName = req.getRegion().getValue().toByteArray();
1773 RegionSpecifierType type = req.getRegion().getType();
1774 final byte [] destServerName = (req.hasDestServerName())?
1775 Bytes.toBytes(ProtobufUtil.toServerName(req.getDestServerName()).getServerName()):null;
1776 MoveRegionResponse mrr = MoveRegionResponse.newBuilder().build();
1777
1778 if (type != RegionSpecifierType.ENCODED_REGION_NAME) {
1779 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME
1780 + " actual: " + type);
1781 }
1782
1783 try {
1784 move(encodedRegionName, destServerName);
1785 } catch (HBaseIOException ioe) {
1786 throw new ServiceException(ioe);
1787 }
1788 return mrr;
1789 }
1790
1791 void move(final byte[] encodedRegionName,
1792 final byte[] destServerName) throws HBaseIOException {
1793 RegionState regionState = assignmentManager.getRegionStates().
1794 getRegionState(Bytes.toString(encodedRegionName));
1795 if (regionState == null) {
1796 throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
1797 }
1798
1799 HRegionInfo hri = regionState.getRegion();
1800 ServerName dest;
1801 if (destServerName == null || destServerName.length == 0) {
1802 LOG.info("Passed destination servername is null/empty so " +
1803 "choosing a server at random");
1804 final List<ServerName> destServers = this.serverManager.createDestinationServersList(
1805 regionState.getServerName());
1806 dest = balancer.randomAssignment(hri, destServers);
1807 } else {
1808 dest = ServerName.valueOf(Bytes.toString(destServerName));
1809 if (dest.equals(regionState.getServerName())) {
1810 LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
1811 + " because region already assigned to the same server " + dest + ".");
1812 return;
1813 }
1814 }
1815
1816
1817 RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
1818
1819 try {
1820 checkInitialized();
1821 if (this.cpHost != null) {
1822 if (this.cpHost.preMove(hri, rp.getSource(), rp.getDestination())) {
1823 return;
1824 }
1825 }
1826 LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
1827 this.assignmentManager.balance(rp);
1828 if (this.cpHost != null) {
1829 this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
1830 }
1831 } catch (IOException ioe) {
1832 if (ioe instanceof HBaseIOException) {
1833 throw (HBaseIOException)ioe;
1834 }
1835 throw new HBaseIOException(ioe);
1836 }
1837 }
1838
1839 @Override
1840 public void createTable(HTableDescriptor hTableDescriptor,
1841 byte [][] splitKeys)
1842 throws IOException {
1843 if (!isMasterRunning()) {
1844 throw new MasterNotRunningException();
1845 }
1846
1847 String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
1848 getNamespaceDescriptor(namespace);
1849
1850 HRegionInfo[] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
1851 checkInitialized();
1852 sanityCheckTableDescriptor(hTableDescriptor);
1853 if (cpHost != null) {
1854 cpHost.preCreateTable(hTableDescriptor, newRegions);
1855 }
1856 LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
1857 this.executorService.submit(new CreateTableHandler(this,
1858 this.fileSystemManager, hTableDescriptor, conf,
1859 newRegions, this).prepare());
1860 if (cpHost != null) {
1861 cpHost.postCreateTable(hTableDescriptor, newRegions);
1862 }
1863
1864 }
1865
1866
1867
1868
1869
1870
1871 private void sanityCheckTableDescriptor(final HTableDescriptor htd) throws IOException {
1872 final String CONF_KEY = "hbase.table.sanity.checks";
1873 if (!conf.getBoolean(CONF_KEY, true)) {
1874 return;
1875 }
1876 String tableVal = htd.getConfigurationValue(CONF_KEY);
1877 if (tableVal != null && !Boolean.valueOf(tableVal)) {
1878 return;
1879 }
1880
1881
1882 long maxFileSizeLowerLimit = 2 * 1024 * 1024L;
1883 long maxFileSize = htd.getMaxFileSize();
1884 if (maxFileSize < 0) {
1885 maxFileSize = conf.getLong(HConstants.HREGION_MAX_FILESIZE, maxFileSizeLowerLimit);
1886 }
1887 if (maxFileSize < conf.getLong("hbase.hregion.max.filesize.limit", maxFileSizeLowerLimit)) {
1888 throw new DoNotRetryIOException("MAX_FILESIZE for table descriptor or "
1889 + "\"hbase.hregion.max.filesize\" (" + maxFileSize
1890 + ") is too small, which might cause over splitting into unmanageable "
1891 + "number of regions. Set " + CONF_KEY + " to false at conf or table descriptor "
1892 + "if you want to bypass sanity checks");
1893 }
1894
1895
1896 long flushSizeLowerLimit = 1024 * 1024L;
1897 long flushSize = htd.getMemStoreFlushSize();
1898 if (flushSize < 0) {
1899 flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeLowerLimit);
1900 }
1901 if (flushSize < conf.getLong("hbase.hregion.memstore.flush.size.limit", flushSizeLowerLimit)) {
1902 throw new DoNotRetryIOException("MEMSTORE_FLUSHSIZE for table descriptor or "
1903 + "\"hbase.hregion.memstore.flush.size\" ("+flushSize+") is too small, which might cause"
1904 + " very frequent flushing. Set " + CONF_KEY + " to false at conf or table descriptor "
1905 + "if you want to bypass sanity checks");
1906 }
1907
1908
1909 try {
1910 checkClassLoading(conf, htd);
1911 } catch (Exception ex) {
1912 throw new DoNotRetryIOException(ex);
1913 }
1914
1915
1916 try {
1917 checkCompression(htd);
1918 } catch (IOException e) {
1919 throw new DoNotRetryIOException(e.getMessage(), e);
1920 }
1921
1922
1923 try {
1924 checkEncryption(conf, htd);
1925 } catch (IOException e) {
1926 throw new DoNotRetryIOException(e.getMessage(), e);
1927 }
1928
1929
1930 if (htd.getColumnFamilies().length == 0) {
1931 throw new DoNotRetryIOException("Table should have at least one column family "
1932 + "Set "+CONF_KEY+" at conf or table descriptor if you want to bypass sanity checks");
1933 }
1934
1935 for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1936 if (hcd.getTimeToLive() <= 0) {
1937 throw new DoNotRetryIOException("TTL for column family " + hcd.getNameAsString()
1938 + " must be positive. Set " + CONF_KEY + " to false at conf or table descriptor "
1939 + "if you want to bypass sanity checks");
1940 }
1941
1942
1943 if (hcd.getBlocksize() < 1024 || hcd.getBlocksize() > 16 * 1024 * 1024) {
1944 throw new DoNotRetryIOException("Block size for column family " + hcd.getNameAsString()
1945 + " must be between 1K and 16MB Set "+CONF_KEY+" to false at conf or table descriptor "
1946 + "if you want to bypass sanity checks");
1947 }
1948
1949
1950 if (hcd.getMinVersions() < 0) {
1951 throw new DoNotRetryIOException("Min versions for column family " + hcd.getNameAsString()
1952 + " must be positive. Set " + CONF_KEY + " to false at conf or table descriptor "
1953 + "if you want to bypass sanity checks");
1954 }
1955
1956
1957
1958 if (hcd.getScope() < 0) {
1959 throw new DoNotRetryIOException("Replication scope for column family "
1960 + hcd.getNameAsString() + " must be positive. Set " + CONF_KEY + " to false at conf "
1961 + "or table descriptor if you want to bypass sanity checks");
1962 }
1963
1964
1965 }
1966 }
1967
1968 private void checkCompression(final HTableDescriptor htd)
1969 throws IOException {
1970 if (!this.masterCheckCompression) return;
1971 for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1972 checkCompression(hcd);
1973 }
1974 }
1975
1976 private void checkCompression(final HColumnDescriptor hcd)
1977 throws IOException {
1978 if (!this.masterCheckCompression) return;
1979 CompressionTest.testCompression(hcd.getCompression());
1980 CompressionTest.testCompression(hcd.getCompactionCompression());
1981 }
1982
1983 private void checkEncryption(final Configuration conf, final HTableDescriptor htd)
1984 throws IOException {
1985 if (!this.masterCheckEncryption) return;
1986 for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1987 checkEncryption(conf, hcd);
1988 }
1989 }
1990
1991 private void checkEncryption(final Configuration conf, final HColumnDescriptor hcd)
1992 throws IOException {
1993 if (!this.masterCheckEncryption) return;
1994 EncryptionTest.testEncryption(conf, hcd.getEncryptionType(), hcd.getEncryptionKey());
1995 }
1996
1997 private void checkClassLoading(final Configuration conf, final HTableDescriptor htd)
1998 throws IOException {
1999 RegionSplitPolicy.getSplitPolicyClass(htd, conf);
2000 RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd);
2001 }
2002
2003 @Override
2004 public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
2005 throws ServiceException {
2006 HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema());
2007 byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
2008 try {
2009 createTable(hTableDescriptor,splitKeys);
2010 } catch (IOException ioe) {
2011 throw new ServiceException(ioe);
2012 }
2013 return CreateTableResponse.newBuilder().build();
2014 }
2015
2016 private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
2017 byte[][] splitKeys) {
2018 HRegionInfo[] hRegionInfos = null;
2019 if (splitKeys == null || splitKeys.length == 0) {
2020 hRegionInfos = new HRegionInfo[]{
2021 new HRegionInfo(hTableDescriptor.getTableName(), null, null)};
2022 } else {
2023 int numRegions = splitKeys.length + 1;
2024 hRegionInfos = new HRegionInfo[numRegions];
2025 byte[] startKey = null;
2026 byte[] endKey = null;
2027 for (int i = 0; i < numRegions; i++) {
2028 endKey = (i == splitKeys.length) ? null : splitKeys[i];
2029 hRegionInfos[i] =
2030 new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey);
2031 startKey = endKey;
2032 }
2033 }
2034 return hRegionInfos;
2035 }
2036
2037 private static boolean isCatalogTable(final TableName tableName) {
2038 return tableName.equals(TableName.META_TABLE_NAME);
2039 }
2040
2041 @Override
2042 public void deleteTable(final TableName tableName) throws IOException {
2043 checkInitialized();
2044 if (cpHost != null) {
2045 cpHost.preDeleteTable(tableName);
2046 }
2047 LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
2048 this.executorService.submit(new DeleteTableHandler(tableName, this, this).prepare());
2049 if (cpHost != null) {
2050 cpHost.postDeleteTable(tableName);
2051 }
2052 }
2053
2054 @Override
2055 public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest request)
2056 throws ServiceException {
2057 try {
2058 deleteTable(ProtobufUtil.toTableName(request.getTableName()));
2059 } catch (IOException ioe) {
2060 throw new ServiceException(ioe);
2061 }
2062 return DeleteTableResponse.newBuilder().build();
2063 }
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073 @Override
2074 public GetSchemaAlterStatusResponse getSchemaAlterStatus(
2075 RpcController controller, GetSchemaAlterStatusRequest req) throws ServiceException {
2076
2077
2078
2079
2080 TableName tableName = ProtobufUtil.toTableName(req.getTableName());
2081
2082 try {
2083 Pair<Integer,Integer> pair = this.assignmentManager.getReopenStatus(tableName);
2084 GetSchemaAlterStatusResponse.Builder ret = GetSchemaAlterStatusResponse.newBuilder();
2085 ret.setYetToUpdateRegions(pair.getFirst());
2086 ret.setTotalRegions(pair.getSecond());
2087 return ret.build();
2088 } catch (IOException ioe) {
2089 throw new ServiceException(ioe);
2090 }
2091 }
2092
2093 @Override
2094 public void addColumn(final TableName tableName, final HColumnDescriptor columnDescriptor)
2095 throws IOException {
2096 checkInitialized();
2097 checkCompression(columnDescriptor);
2098 checkEncryption(conf, columnDescriptor);
2099 if (cpHost != null) {
2100 if (cpHost.preAddColumn(tableName, columnDescriptor)) {
2101 return;
2102 }
2103 }
2104
2105 new TableAddFamilyHandler(tableName, columnDescriptor, this, this).prepare().process();
2106 if (cpHost != null) {
2107 cpHost.postAddColumn(tableName, columnDescriptor);
2108 }
2109 }
2110
2111 @Override
2112 public AddColumnResponse addColumn(RpcController controller, AddColumnRequest req)
2113 throws ServiceException {
2114 try {
2115 addColumn(ProtobufUtil.toTableName(req.getTableName()),
2116 HColumnDescriptor.convert(req.getColumnFamilies()));
2117 } catch (IOException ioe) {
2118 throw new ServiceException(ioe);
2119 }
2120 return AddColumnResponse.newBuilder().build();
2121 }
2122
2123 @Override
2124 public void modifyColumn(TableName tableName, HColumnDescriptor descriptor)
2125 throws IOException {
2126 checkInitialized();
2127 checkCompression(descriptor);
2128 checkEncryption(conf, descriptor);
2129 if (cpHost != null) {
2130 if (cpHost.preModifyColumn(tableName, descriptor)) {
2131 return;
2132 }
2133 }
2134 LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
2135 new TableModifyFamilyHandler(tableName, descriptor, this, this)
2136 .prepare().process();
2137 if (cpHost != null) {
2138 cpHost.postModifyColumn(tableName, descriptor);
2139 }
2140 }
2141
2142 @Override
2143 public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest req)
2144 throws ServiceException {
2145 try {
2146 modifyColumn(ProtobufUtil.toTableName(req.getTableName()),
2147 HColumnDescriptor.convert(req.getColumnFamilies()));
2148 } catch (IOException ioe) {
2149 throw new ServiceException(ioe);
2150 }
2151 return ModifyColumnResponse.newBuilder().build();
2152 }
2153
2154 @Override
2155 public void deleteColumn(final TableName tableName, final byte[] columnName)
2156 throws IOException {
2157 checkInitialized();
2158 if (cpHost != null) {
2159 if (cpHost.preDeleteColumn(tableName, columnName)) {
2160 return;
2161 }
2162 }
2163 LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
2164 new TableDeleteFamilyHandler(tableName, columnName, this, this).prepare().process();
2165 if (cpHost != null) {
2166 cpHost.postDeleteColumn(tableName, columnName);
2167 }
2168 }
2169
2170 @Override
2171 public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest req)
2172 throws ServiceException {
2173 try {
2174 deleteColumn(ProtobufUtil.toTableName(req.getTableName()),
2175 req.getColumnName().toByteArray());
2176 } catch (IOException ioe) {
2177 throw new ServiceException(ioe);
2178 }
2179 return DeleteColumnResponse.newBuilder().build();
2180 }
2181
2182 @Override
2183 public void enableTable(final TableName tableName) throws IOException {
2184 checkInitialized();
2185 if (cpHost != null) {
2186 cpHost.preEnableTable(tableName);
2187 }
2188 LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
2189 this.executorService.submit(new EnableTableHandler(this, tableName,
2190 catalogTracker, assignmentManager, tableLockManager, false).prepare());
2191 if (cpHost != null) {
2192 cpHost.postEnableTable(tableName);
2193 }
2194 }
2195
2196 @Override
2197 public EnableTableResponse enableTable(RpcController controller, EnableTableRequest request)
2198 throws ServiceException {
2199 try {
2200 enableTable(ProtobufUtil.toTableName(request.getTableName()));
2201 } catch (IOException ioe) {
2202 throw new ServiceException(ioe);
2203 }
2204 return EnableTableResponse.newBuilder().build();
2205 }
2206
2207 @Override
2208 public void disableTable(final TableName tableName) throws IOException {
2209 checkInitialized();
2210 if (cpHost != null) {
2211 cpHost.preDisableTable(tableName);
2212 }
2213 LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
2214 this.executorService.submit(new DisableTableHandler(this, tableName,
2215 catalogTracker, assignmentManager, tableLockManager, false).prepare());
2216 if (cpHost != null) {
2217 cpHost.postDisableTable(tableName);
2218 }
2219 }
2220
2221 @Override
2222 public DisableTableResponse disableTable(RpcController controller, DisableTableRequest request)
2223 throws ServiceException {
2224 try {
2225 disableTable(ProtobufUtil.toTableName(request.getTableName()));
2226 } catch (IOException ioe) {
2227 throw new ServiceException(ioe);
2228 }
2229 return DisableTableResponse.newBuilder().build();
2230 }
2231
2232
2233
2234
2235
2236
2237
2238 Pair<HRegionInfo, ServerName> getTableRegionForRow(
2239 final TableName tableName, final byte [] rowKey)
2240 throws IOException {
2241 final AtomicReference<Pair<HRegionInfo, ServerName>> result =
2242 new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
2243
2244 MetaScannerVisitor visitor =
2245 new MetaScannerVisitorBase() {
2246 @Override
2247 public boolean processRow(Result data) throws IOException {
2248 if (data == null || data.size() <= 0) {
2249 return true;
2250 }
2251 Pair<HRegionInfo, ServerName> pair = HRegionInfo.getHRegionInfoAndServerName(data);
2252 if (pair == null) {
2253 return false;
2254 }
2255 if (!pair.getFirst().getTable().equals(tableName)) {
2256 return false;
2257 }
2258 result.set(pair);
2259 return true;
2260 }
2261 };
2262
2263 MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1);
2264 return result.get();
2265 }
2266
2267 @Override
2268 public void modifyTable(final TableName tableName, final HTableDescriptor descriptor)
2269 throws IOException {
2270 checkInitialized();
2271 sanityCheckTableDescriptor(descriptor);
2272 if (cpHost != null) {
2273 cpHost.preModifyTable(tableName, descriptor);
2274 }
2275 LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
2276 new ModifyTableHandler(tableName, descriptor, this, this).prepare().process();
2277 if (cpHost != null) {
2278 cpHost.postModifyTable(tableName, descriptor);
2279 }
2280 }
2281
2282 @Override
2283 public ModifyTableResponse modifyTable(RpcController controller, ModifyTableRequest req)
2284 throws ServiceException {
2285 try {
2286 modifyTable(ProtobufUtil.toTableName(req.getTableName()),
2287 HTableDescriptor.convert(req.getTableSchema()));
2288 } catch (IOException ioe) {
2289 throw new ServiceException(ioe);
2290 }
2291 return ModifyTableResponse.newBuilder().build();
2292 }
2293
2294 @Override
2295 public void checkTableModifiable(final TableName tableName)
2296 throws IOException, TableNotFoundException, TableNotDisabledException {
2297 if (isCatalogTable(tableName)) {
2298 throw new IOException("Can't modify catalog tables");
2299 }
2300 if (!MetaReader.tableExists(getCatalogTracker(), tableName)) {
2301 throw new TableNotFoundException(tableName);
2302 }
2303 if (!getAssignmentManager().getZKTable().
2304 isDisabledTable(tableName)) {
2305 throw new TableNotDisabledException(tableName);
2306 }
2307 }
2308
2309 @Override
2310 public GetClusterStatusResponse getClusterStatus(RpcController controller,
2311 GetClusterStatusRequest req)
2312 throws ServiceException {
2313 GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
2314 response.setClusterStatus(getClusterStatus().convert());
2315 return response.build();
2316 }
2317
2318
2319
2320
2321 public ClusterStatus getClusterStatus() {
2322
2323 List<String> backupMasterStrings;
2324 try {
2325 backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper,
2326 this.zooKeeper.backupMasterAddressesZNode);
2327 } catch (KeeperException e) {
2328 LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e);
2329 backupMasterStrings = new ArrayList<String>(0);
2330 }
2331 List<ServerName> backupMasters = new ArrayList<ServerName>(
2332 backupMasterStrings.size());
2333 for (String s: backupMasterStrings) {
2334 try {
2335 byte [] bytes =
2336 ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(
2337 this.zooKeeper.backupMasterAddressesZNode, s));
2338 if (bytes != null) {
2339 ServerName sn;
2340 try {
2341 sn = ServerName.parseFrom(bytes);
2342 } catch (DeserializationException e) {
2343 LOG.warn("Failed parse, skipping registering backup server", e);
2344 continue;
2345 }
2346 backupMasters.add(sn);
2347 }
2348 } catch (KeeperException e) {
2349 LOG.warn(this.zooKeeper.prefix("Unable to get information about " +
2350 "backup servers"), e);
2351 }
2352 }
2353 Collections.sort(backupMasters, new Comparator<ServerName>() {
2354 @Override
2355 public int compare(ServerName s1, ServerName s2) {
2356 return s1.getServerName().compareTo(s2.getServerName());
2357 }});
2358
2359 return new ClusterStatus(VersionInfo.getVersion(),
2360 this.fileSystemManager.getClusterId().toString(),
2361 this.serverManager.getOnlineServers(),
2362 this.serverManager.getDeadServers().copyServerNames(),
2363 this.serverName,
2364 backupMasters,
2365 this.assignmentManager.getRegionStates().getRegionsInTransition(),
2366 this.getCoprocessors(), this.loadBalancerTracker.isBalancerOn());
2367 }
2368
2369 public String getClusterId() {
2370 if (fileSystemManager == null) {
2371 return "";
2372 }
2373 ClusterId id = fileSystemManager.getClusterId();
2374 if (id == null) {
2375 return "";
2376 }
2377 return id.toString();
2378 }
2379
2380
2381
2382
2383
2384
2385
2386
2387 public static String getLoadedCoprocessors() {
2388 return CoprocessorHost.getLoadedCoprocessors().toString();
2389 }
2390
2391
2392
2393
2394 public long getMasterStartTime() {
2395 return masterStartTime;
2396 }
2397
2398
2399
2400
2401 public long getMasterActiveTime() {
2402 return masterActiveTime;
2403 }
2404
2405 public int getRegionServerInfoPort(final ServerName sn) {
2406 RegionServerInfo info = this.regionServerTracker.getRegionServerInfo(sn);
2407 if (info == null || info.getInfoPort() == 0) {
2408 return conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
2409 HConstants.DEFAULT_REGIONSERVER_INFOPORT);
2410 }
2411 return info.getInfoPort();
2412 }
2413
2414
2415
2416
2417 public String[] getCoprocessors() {
2418 Set<String> masterCoprocessors =
2419 getCoprocessorHost().getCoprocessors();
2420 return masterCoprocessors.toArray(new String[masterCoprocessors.size()]);
2421 }
2422
2423 @Override
2424 public void abort(final String msg, final Throwable t) {
2425 if (cpHost != null) {
2426
2427 LOG.fatal("Master server abort: loaded coprocessors are: " +
2428 getLoadedCoprocessors());
2429 }
2430
2431 if (abortNow(msg, t)) {
2432 if (t != null) LOG.fatal(msg, t);
2433 else LOG.fatal(msg);
2434 this.abort = true;
2435 stop("Aborting");
2436 }
2437 }
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456 private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
2457 IOException, KeeperException, ExecutionException {
2458
2459 this.zooKeeper.unregisterAllListeners();
2460
2461
2462 if (this.registeredZKListenersBeforeRecovery != null) {
2463 for (ZooKeeperListener curListener : this.registeredZKListenersBeforeRecovery) {
2464 this.zooKeeper.registerListener(curListener);
2465 }
2466 }
2467
2468 this.zooKeeper.reconnectAfterExpiration();
2469
2470 Callable<Boolean> callable = new Callable<Boolean> () {
2471 @Override
2472 public Boolean call() throws InterruptedException,
2473 IOException, KeeperException {
2474 MonitoredTask status =
2475 TaskMonitor.get().createStatus("Recovering expired ZK session");
2476 try {
2477 if (!becomeActiveMaster(status)) {
2478 return Boolean.FALSE;
2479 }
2480 serverShutdownHandlerEnabled = false;
2481 initialized = false;
2482 finishInitialization(status, true);
2483 return !stopped;
2484 } finally {
2485 status.cleanup();
2486 }
2487 }
2488 };
2489
2490 long timeout =
2491 conf.getLong("hbase.master.zksession.recover.timeout", 300000);
2492 java.util.concurrent.ExecutorService executor =
2493 Executors.newSingleThreadExecutor();
2494 Future<Boolean> result = executor.submit(callable);
2495 executor.shutdown();
2496 if (executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)
2497 && result.isDone()) {
2498 Boolean recovered = result.get();
2499 if (recovered != null) {
2500 return recovered.booleanValue();
2501 }
2502 }
2503 executor.shutdownNow();
2504 return false;
2505 }
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515 private boolean abortNow(final String msg, final Throwable t) {
2516 if (!this.isActiveMaster || this.stopped) {
2517 return true;
2518 }
2519
2520 boolean failFast = conf.getBoolean("fail.fast.expired.active.master", false);
2521 if (t != null && t instanceof KeeperException.SessionExpiredException
2522 && !failFast) {
2523 try {
2524 LOG.info("Primary Master trying to recover from ZooKeeper session " +
2525 "expiry.");
2526 return !tryRecoveringExpiredZKSession();
2527 } catch (Throwable newT) {
2528 LOG.error("Primary master encountered unexpected exception while " +
2529 "trying to recover from ZooKeeper session" +
2530 " expiry. Proceeding with server abort.", newT);
2531 }
2532 }
2533 return true;
2534 }
2535
2536 @Override
2537 public ZooKeeperWatcher getZooKeeper() {
2538 return zooKeeper;
2539 }
2540
2541 @Override
2542 public MasterCoprocessorHost getCoprocessorHost() {
2543 return cpHost;
2544 }
2545
2546 @Override
2547 public ServerName getServerName() {
2548 return this.serverName;
2549 }
2550
2551 @Override
2552 public CatalogTracker getCatalogTracker() {
2553 return catalogTracker;
2554 }
2555
2556 @Override
2557 public AssignmentManager getAssignmentManager() {
2558 return this.assignmentManager;
2559 }
2560
2561 @Override
2562 public TableLockManager getTableLockManager() {
2563 return this.tableLockManager;
2564 }
2565
2566 public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
2567 return rsFatals;
2568 }
2569
2570 public void shutdown() {
2571 if (spanReceiverHost != null) {
2572 spanReceiverHost.closeReceivers();
2573 }
2574 if (cpHost != null) {
2575 try {
2576 cpHost.preShutdown();
2577 } catch (IOException ioe) {
2578 LOG.error("Error call master coprocessor preShutdown()", ioe);
2579 }
2580 }
2581 if (mxBean != null) {
2582 MBeanUtil.unregisterMBean(mxBean);
2583 mxBean = null;
2584 }
2585 if (this.assignmentManager != null) this.assignmentManager.shutdown();
2586 if (this.serverManager != null) this.serverManager.shutdownCluster();
2587 try {
2588 if (this.clusterStatusTracker != null){
2589 this.clusterStatusTracker.setClusterDown();
2590 }
2591 } catch (KeeperException e) {
2592 LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
2593 }
2594 }
2595
2596 @Override
2597 public ShutdownResponse shutdown(RpcController controller, ShutdownRequest request)
2598 throws ServiceException {
2599 LOG.info(getClientIdAuditPrefix() + " shutdown");
2600 shutdown();
2601 return ShutdownResponse.newBuilder().build();
2602 }
2603
2604 public void stopMaster() {
2605 if (cpHost != null) {
2606 try {
2607 cpHost.preStopMaster();
2608 } catch (IOException ioe) {
2609 LOG.error("Error call master coprocessor preStopMaster()", ioe);
2610 }
2611 }
2612 stop("Stopped by " + Thread.currentThread().getName());
2613 }
2614
2615 @Override
2616 public StopMasterResponse stopMaster(RpcController controller, StopMasterRequest request)
2617 throws ServiceException {
2618 LOG.info(getClientIdAuditPrefix() + " stop");
2619 stopMaster();
2620 return StopMasterResponse.newBuilder().build();
2621 }
2622
2623 @Override
2624 public void stop(final String why) {
2625 LOG.info(why);
2626 this.stopped = true;
2627
2628 stopSleeper.skipSleepCycle();
2629
2630 if (this.activeMasterManager != null) {
2631 synchronized (this.activeMasterManager.clusterHasActiveMaster) {
2632 this.activeMasterManager.clusterHasActiveMaster.notifyAll();
2633 }
2634 }
2635
2636
2637 if (this.catalogTracker != null && this.serverManager.getOnlineServers().isEmpty()) {
2638 this.catalogTracker.stop();
2639 }
2640 }
2641
2642 @Override
2643 public boolean isStopped() {
2644 return this.stopped;
2645 }
2646
2647 @Override
2648 public boolean isAborted() {
2649 return this.abort;
2650 }
2651
2652 void checkInitialized() throws PleaseHoldException {
2653 if (!this.initialized) {
2654 throw new PleaseHoldException("Master is initializing");
2655 }
2656 }
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666 public boolean isActiveMaster() {
2667 return isActiveMaster;
2668 }
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679 @Override
2680 public boolean isInitialized() {
2681 return initialized;
2682 }
2683
2684
2685
2686
2687
2688
2689 @Override
2690 public boolean isServerShutdownHandlerEnabled() {
2691 return this.serverShutdownHandlerEnabled;
2692 }
2693
2694
2695
2696
2697
2698 public boolean isInitializationStartsMetaRegionAssignment() {
2699 return this.initializationBeforeMetaAssignment;
2700 }
2701
2702 @Override
2703 public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest req)
2704 throws ServiceException {
2705 try {
2706 final byte [] regionName = req.getRegion().getValue().toByteArray();
2707 RegionSpecifierType type = req.getRegion().getType();
2708 AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
2709
2710 checkInitialized();
2711 if (type != RegionSpecifierType.REGION_NAME) {
2712 LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
2713 + " actual: " + type);
2714 }
2715 HRegionInfo regionInfo = assignmentManager.getRegionStates().getRegionInfo(regionName);
2716 if (regionInfo == null) throw new UnknownRegionException(Bytes.toString(regionName));
2717 if (cpHost != null) {
2718 if (cpHost.preAssign(regionInfo)) {
2719 return arr;
2720 }
2721 }
2722 LOG.info(getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString());
2723 assignmentManager.assign(regionInfo, true, true);
2724 if (cpHost != null) {
2725 cpHost.postAssign(regionInfo);
2726 }
2727
2728 return arr;
2729 } catch (IOException ioe) {
2730 throw new ServiceException(ioe);
2731 }
2732 }
2733
2734 public void assignRegion(HRegionInfo hri) {
2735 assignmentManager.assign(hri, true);
2736 }
2737
2738 @Override
2739 public UnassignRegionResponse unassignRegion(RpcController controller, UnassignRegionRequest req)
2740 throws ServiceException {
2741 try {
2742 final byte [] regionName = req.getRegion().getValue().toByteArray();
2743 RegionSpecifierType type = req.getRegion().getType();
2744 final boolean force = req.getForce();
2745 UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build();
2746
2747 checkInitialized();
2748 if (type != RegionSpecifierType.REGION_NAME) {
2749 LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
2750 + " actual: " + type);
2751 }
2752 Pair<HRegionInfo, ServerName> pair =
2753 MetaReader.getRegion(this.catalogTracker, regionName);
2754 if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
2755 HRegionInfo hri = pair.getFirst();
2756 if (cpHost != null) {
2757 if (cpHost.preUnassign(hri, force)) {
2758 return urr;
2759 }
2760 }
2761 LOG.debug(getClientIdAuditPrefix() + " unassign " + hri.getRegionNameAsString()
2762 + " in current location if it is online and reassign.force=" + force);
2763 this.assignmentManager.unassign(hri, force);
2764 if (this.assignmentManager.getRegionStates().isRegionOffline(hri)) {
2765 LOG.debug("Region " + hri.getRegionNameAsString()
2766 + " is not online on any region server, reassigning it.");
2767 assignRegion(hri);
2768 }
2769 if (cpHost != null) {
2770 cpHost.postUnassign(hri, force);
2771 }
2772
2773 return urr;
2774 } catch (IOException ioe) {
2775 throw new ServiceException(ioe);
2776 }
2777 }
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787 @Override
2788 public GetTableDescriptorsResponse getTableDescriptors(
2789 RpcController controller, GetTableDescriptorsRequest req) throws ServiceException {
2790 List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
2791 List<TableName> tableNameList = new ArrayList<TableName>();
2792 for(HBaseProtos.TableName tableNamePB: req.getTableNamesList()) {
2793 tableNameList.add(ProtobufUtil.toTableName(tableNamePB));
2794 }
2795 boolean bypass = false;
2796 if (this.cpHost != null) {
2797 try {
2798 bypass = this.cpHost.preGetTableDescriptors(tableNameList, descriptors);
2799 } catch (IOException ioe) {
2800 throw new ServiceException(ioe);
2801 }
2802 }
2803
2804 if (!bypass) {
2805 if (req.getTableNamesCount() == 0) {
2806
2807 Map<String, HTableDescriptor> descriptorMap = null;
2808 try {
2809 descriptorMap = this.tableDescriptors.getAll();
2810 } catch (IOException e) {
2811 LOG.warn("Failed getting all descriptors", e);
2812 }
2813 if (descriptorMap != null) {
2814 for(HTableDescriptor desc: descriptorMap.values()) {
2815 if(!desc.getTableName().isSystemTable()) {
2816 descriptors.add(desc);
2817 }
2818 }
2819 }
2820 } else {
2821 for (TableName s: tableNameList) {
2822 try {
2823 HTableDescriptor desc = this.tableDescriptors.get(s);
2824 if (desc != null) {
2825 descriptors.add(desc);
2826 }
2827 } catch (IOException e) {
2828 LOG.warn("Failed getting descriptor for " + s, e);
2829 }
2830 }
2831 }
2832
2833 if (this.cpHost != null) {
2834 try {
2835 this.cpHost.postGetTableDescriptors(descriptors);
2836 } catch (IOException ioe) {
2837 throw new ServiceException(ioe);
2838 }
2839 }
2840 }
2841
2842 GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder();
2843 for (HTableDescriptor htd: descriptors) {
2844 builder.addTableSchema(htd.convert());
2845 }
2846 return builder.build();
2847 }
2848
2849
2850
2851
2852
2853
2854
2855
2856 @Override
2857 public GetTableNamesResponse getTableNames(
2858 RpcController controller, GetTableNamesRequest req) throws ServiceException {
2859 try {
2860 Collection<HTableDescriptor> descriptors = this.tableDescriptors.getAll().values();
2861 GetTableNamesResponse.Builder builder = GetTableNamesResponse.newBuilder();
2862 for (HTableDescriptor descriptor: descriptors) {
2863 if (descriptor.getTableName().isSystemTable()) {
2864 continue;
2865 }
2866 builder.addTableNames(ProtobufUtil.toProtoTableName(descriptor.getTableName()));
2867 }
2868 return builder.build();
2869 } catch (IOException e) {
2870 throw new ServiceException(e);
2871 }
2872 }
2873
2874
2875
2876
2877
2878
2879
2880 public double getAverageLoad() {
2881 if (this.assignmentManager == null) {
2882 return 0;
2883 }
2884
2885 RegionStates regionStates = this.assignmentManager.getRegionStates();
2886 if (regionStates == null) {
2887 return 0;
2888 }
2889 return regionStates.getAverageLoad();
2890 }
2891
2892
2893
2894
2895
2896
2897
2898
2899 @Override
2900 public OfflineRegionResponse offlineRegion(RpcController controller, OfflineRegionRequest request)
2901 throws ServiceException {
2902 final byte [] regionName = request.getRegion().getValue().toByteArray();
2903 RegionSpecifierType type = request.getRegion().getType();
2904 if (type != RegionSpecifierType.REGION_NAME) {
2905 LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
2906 + " actual: " + type);
2907 }
2908
2909 try {
2910 Pair<HRegionInfo, ServerName> pair =
2911 MetaReader.getRegion(this.catalogTracker, regionName);
2912 if (pair == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
2913 HRegionInfo hri = pair.getFirst();
2914 if (cpHost != null) {
2915 cpHost.preRegionOffline(hri);
2916 }
2917 LOG.info(getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString());
2918 this.assignmentManager.regionOffline(hri);
2919 if (cpHost != null) {
2920 cpHost.postRegionOffline(hri);
2921 }
2922 } catch (IOException ioe) {
2923 throw new ServiceException(ioe);
2924 }
2925 return OfflineRegionResponse.newBuilder().build();
2926 }
2927
2928 @Override
2929 public boolean registerService(Service instance) {
2930
2931
2932
2933 Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
2934 if (coprocessorServiceHandlers.containsKey(serviceDesc.getFullName())) {
2935 LOG.error("Coprocessor service "+serviceDesc.getFullName()+
2936 " already registered, rejecting request from "+instance
2937 );
2938 return false;
2939 }
2940
2941 coprocessorServiceHandlers.put(serviceDesc.getFullName(), instance);
2942 if (LOG.isDebugEnabled()) {
2943 LOG.debug("Registered master coprocessor service: service="+serviceDesc.getFullName());
2944 }
2945 return true;
2946 }
2947
2948 @Override
2949 public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
2950 final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
2951 try {
2952 ServerRpcController execController = new ServerRpcController();
2953
2954 ClientProtos.CoprocessorServiceCall call = request.getCall();
2955 String serviceName = call.getServiceName();
2956 String methodName = call.getMethodName();
2957 if (!coprocessorServiceHandlers.containsKey(serviceName)) {
2958 throw new UnknownProtocolException(null,
2959 "No registered master coprocessor service found for name "+serviceName);
2960 }
2961
2962 Service service = coprocessorServiceHandlers.get(serviceName);
2963 Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
2964 Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
2965 if (methodDesc == null) {
2966 throw new UnknownProtocolException(service.getClass(),
2967 "Unknown method "+methodName+" called on master service "+serviceName);
2968 }
2969
2970
2971 Message execRequest = service.getRequestPrototype(methodDesc).newBuilderForType()
2972 .mergeFrom(call.getRequest()).build();
2973 final Message.Builder responseBuilder =
2974 service.getResponsePrototype(methodDesc).newBuilderForType();
2975 service.callMethod(methodDesc, execController, execRequest, new RpcCallback<Message>() {
2976 @Override
2977 public void run(Message message) {
2978 if (message != null) {
2979 responseBuilder.mergeFrom(message);
2980 }
2981 }
2982 });
2983 Message execResult = responseBuilder.build();
2984
2985 if (execController.getFailedOn() != null) {
2986 throw execController.getFailedOn();
2987 }
2988 ClientProtos.CoprocessorServiceResponse.Builder builder =
2989 ClientProtos.CoprocessorServiceResponse.newBuilder();
2990 builder.setRegion(RequestConverter.buildRegionSpecifier(
2991 RegionSpecifierType.REGION_NAME, HConstants.EMPTY_BYTE_ARRAY));
2992 builder.setValue(
2993 builder.getValueBuilder().setName(execResult.getClass().getName())
2994 .setValue(execResult.toByteString()));
2995 return builder.build();
2996 } catch (IOException ie) {
2997 throw new ServiceException(ie);
2998 }
2999 }
3000
3001
3002
3003
3004
3005
3006
3007 public static HMaster constructMaster(Class<? extends HMaster> masterClass,
3008 final Configuration conf) {
3009 try {
3010 Constructor<? extends HMaster> c =
3011 masterClass.getConstructor(Configuration.class);
3012 return c.newInstance(conf);
3013 } catch (InvocationTargetException ite) {
3014 Throwable target = ite.getTargetException() != null?
3015 ite.getTargetException(): ite;
3016 if (target.getCause() != null) target = target.getCause();
3017 throw new RuntimeException("Failed construction of Master: " +
3018 masterClass.toString(), target);
3019 } catch (Exception e) {
3020 throw new RuntimeException("Failed construction of Master: " +
3021 masterClass.toString() + ((e.getCause() != null)?
3022 e.getCause().getMessage(): ""), e);
3023 }
3024 }
3025
3026
3027
3028
3029 public static void main(String [] args) {
3030 VersionInfo.logVersion();
3031 new HMasterCommandLine(HMaster.class).doMain(args);
3032 }
3033
3034 public HFileCleaner getHFileCleaner() {
3035 return this.hfileCleaner;
3036 }
3037
3038
3039
3040
3041
3042 public SnapshotManager getSnapshotManagerForTesting() {
3043 return this.snapshotManager;
3044 }
3045
3046
3047
3048
3049
3050 @Override
3051 public SnapshotResponse snapshot(RpcController controller, SnapshotRequest request)
3052 throws ServiceException {
3053 try {
3054 this.snapshotManager.checkSnapshotSupport();
3055 } catch (UnsupportedOperationException e) {
3056 throw new ServiceException(e);
3057 }
3058
3059 LOG.info(getClientIdAuditPrefix() + " snapshot request for:" +
3060 ClientSnapshotDescriptionUtils.toString(request.getSnapshot()));
3061
3062 SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(request.getSnapshot(),
3063 this.conf);
3064 try {
3065 snapshotManager.takeSnapshot(snapshot);
3066 } catch (IOException e) {
3067 throw new ServiceException(e);
3068 }
3069
3070
3071 long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
3072 SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
3073 return SnapshotResponse.newBuilder().setExpectedTimeout(waitTime).build();
3074 }
3075
3076
3077
3078
3079 @Override
3080 public GetCompletedSnapshotsResponse getCompletedSnapshots(RpcController controller,
3081 GetCompletedSnapshotsRequest request) throws ServiceException {
3082 try {
3083 GetCompletedSnapshotsResponse.Builder builder = GetCompletedSnapshotsResponse.newBuilder();
3084 List<SnapshotDescription> snapshots = snapshotManager.getCompletedSnapshots();
3085
3086
3087 for (SnapshotDescription snapshot : snapshots) {
3088 builder.addSnapshots(snapshot);
3089 }
3090 return builder.build();
3091 } catch (IOException e) {
3092 throw new ServiceException(e);
3093 }
3094 }
3095
3096
3097
3098
3099
3100
3101
3102
3103 @Override
3104 public DeleteSnapshotResponse deleteSnapshot(RpcController controller,
3105 DeleteSnapshotRequest request) throws ServiceException {
3106 try {
3107 this.snapshotManager.checkSnapshotSupport();
3108 } catch (UnsupportedOperationException e) {
3109 throw new ServiceException(e);
3110 }
3111
3112 try {
3113 LOG.info(getClientIdAuditPrefix() + " delete " + request.getSnapshot());
3114 snapshotManager.deleteSnapshot(request.getSnapshot());
3115 return DeleteSnapshotResponse.newBuilder().build();
3116 } catch (IOException e) {
3117 throw new ServiceException(e);
3118 }
3119 }
3120
3121
3122
3123
3124
3125
3126
3127
3128 @Override
3129 public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
3130 IsSnapshotDoneRequest request) throws ServiceException {
3131 LOG.debug("Checking to see if snapshot from request:" +
3132 ClientSnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done");
3133 try {
3134 IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
3135 boolean done = snapshotManager.isSnapshotDone(request.getSnapshot());
3136 builder.setDone(done);
3137 return builder.build();
3138 } catch (IOException e) {
3139 throw new ServiceException(e);
3140 }
3141 }
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156 @Override
3157 public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
3158 RestoreSnapshotRequest request) throws ServiceException {
3159 try {
3160 this.snapshotManager.checkSnapshotSupport();
3161 } catch (UnsupportedOperationException e) {
3162 throw new ServiceException(e);
3163 }
3164
3165
3166 try {
3167 TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
3168 getNamespaceDescriptor(dstTable.getNamespaceAsString());
3169 } catch (IOException ioe) {
3170 throw new ServiceException(ioe);
3171 }
3172
3173 try {
3174 SnapshotDescription reqSnapshot = request.getSnapshot();
3175 snapshotManager.restoreSnapshot(reqSnapshot);
3176 return RestoreSnapshotResponse.newBuilder().build();
3177 } catch (IOException e) {
3178 throw new ServiceException(e);
3179 }
3180 }
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192 @Override
3193 public IsRestoreSnapshotDoneResponse isRestoreSnapshotDone(RpcController controller,
3194 IsRestoreSnapshotDoneRequest request) throws ServiceException {
3195 try {
3196 SnapshotDescription snapshot = request.getSnapshot();
3197 IsRestoreSnapshotDoneResponse.Builder builder = IsRestoreSnapshotDoneResponse.newBuilder();
3198 boolean done = snapshotManager.isRestoreDone(snapshot);
3199 builder.setDone(done);
3200 return builder.build();
3201 } catch (IOException e) {
3202 throw new ServiceException(e);
3203 }
3204 }
3205
3206
3207
3208
3209
3210 @Override
3211 public ExecProcedureResponse execProcedure(RpcController controller,
3212 ExecProcedureRequest request) throws ServiceException {
3213 ProcedureDescription desc = request.getProcedure();
3214 MasterProcedureManager mpm = this.mpmHost.getProcedureManager(desc
3215 .getSignature());
3216 if (mpm == null) {
3217 throw new ServiceException("The procedure is not registered: "
3218 + desc.getSignature());
3219 }
3220
3221 LOG.info(getClientIdAuditPrefix() + " procedure request for: "
3222 + desc.getSignature());
3223
3224 try {
3225 mpm.execProcedure(desc);
3226 } catch (IOException e) {
3227 throw new ServiceException(e);
3228 }
3229
3230
3231
3232 long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
3233 return ExecProcedureResponse.newBuilder().setExpectedTimeout(waitTime)
3234 .build();
3235 }
3236
3237
3238
3239
3240
3241
3242
3243
3244 @Override
3245 public IsProcedureDoneResponse isProcedureDone(RpcController controller,
3246 IsProcedureDoneRequest request) throws ServiceException {
3247 ProcedureDescription desc = request.getProcedure();
3248 MasterProcedureManager mpm = this.mpmHost.getProcedureManager(desc
3249 .getSignature());
3250 if (mpm == null) {
3251 throw new ServiceException("The procedure is not registered: "
3252 + desc.getSignature());
3253 }
3254 LOG.debug("Checking to see if procedure from request:"
3255 + desc.getSignature() + " is done");
3256
3257 try {
3258 IsProcedureDoneResponse.Builder builder = IsProcedureDoneResponse
3259 .newBuilder();
3260 boolean done = mpm.isProcedureDone(desc);
3261 builder.setDone(done);
3262 return builder.build();
3263 } catch (IOException e) {
3264 throw new ServiceException(e);
3265 }
3266 }
3267
3268 @Override
3269 public ModifyNamespaceResponse modifyNamespace(RpcController controller,
3270 ModifyNamespaceRequest request) throws ServiceException {
3271 try {
3272 modifyNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
3273 return ModifyNamespaceResponse.getDefaultInstance();
3274 } catch (IOException e) {
3275 throw new ServiceException(e);
3276 }
3277 }
3278
3279 @Override
3280 public CreateNamespaceResponse createNamespace(RpcController controller,
3281 CreateNamespaceRequest request) throws ServiceException {
3282 try {
3283 createNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
3284 return CreateNamespaceResponse.getDefaultInstance();
3285 } catch (IOException e) {
3286 throw new ServiceException(e);
3287 }
3288 }
3289
3290 @Override
3291 public DeleteNamespaceResponse deleteNamespace(RpcController controller,
3292 DeleteNamespaceRequest request) throws ServiceException {
3293 try {
3294 deleteNamespace(request.getNamespaceName());
3295 return DeleteNamespaceResponse.getDefaultInstance();
3296 } catch (IOException e) {
3297 throw new ServiceException(e);
3298 }
3299 }
3300
3301 @Override
3302 public GetNamespaceDescriptorResponse getNamespaceDescriptor(
3303 RpcController controller, GetNamespaceDescriptorRequest request)
3304 throws ServiceException {
3305 try {
3306 return GetNamespaceDescriptorResponse.newBuilder()
3307 .setNamespaceDescriptor(
3308 ProtobufUtil.toProtoNamespaceDescriptor(getNamespaceDescriptor(request.getNamespaceName())))
3309 .build();
3310 } catch (IOException e) {
3311 throw new ServiceException(e);
3312 }
3313 }
3314
3315 @Override
3316 public ListNamespaceDescriptorsResponse listNamespaceDescriptors(
3317 RpcController controller, ListNamespaceDescriptorsRequest request)
3318 throws ServiceException {
3319 try {
3320 ListNamespaceDescriptorsResponse.Builder response =
3321 ListNamespaceDescriptorsResponse.newBuilder();
3322 for(NamespaceDescriptor ns: listNamespaceDescriptors()) {
3323 response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
3324 }
3325 return response.build();
3326 } catch (IOException e) {
3327 throw new ServiceException(e);
3328 }
3329 }
3330
3331 @Override
3332 public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(
3333 RpcController controller, ListTableDescriptorsByNamespaceRequest request)
3334 throws ServiceException {
3335 try {
3336 ListTableDescriptorsByNamespaceResponse.Builder b =
3337 ListTableDescriptorsByNamespaceResponse.newBuilder();
3338 for(HTableDescriptor htd: listTableDescriptorsByNamespace(request.getNamespaceName())) {
3339 b.addTableSchema(htd.convert());
3340 }
3341 return b.build();
3342 } catch (IOException e) {
3343 throw new ServiceException(e);
3344 }
3345 }
3346
3347 @Override
3348 public ListTableNamesByNamespaceResponse listTableNamesByNamespace(
3349 RpcController controller, ListTableNamesByNamespaceRequest request)
3350 throws ServiceException {
3351 try {
3352 ListTableNamesByNamespaceResponse.Builder b =
3353 ListTableNamesByNamespaceResponse.newBuilder();
3354 for (TableName tableName: listTableNamesByNamespace(request.getNamespaceName())) {
3355 b.addTableName(ProtobufUtil.toProtoTableName(tableName));
3356 }
3357 return b.build();
3358 } catch (IOException e) {
3359 throw new ServiceException(e);
3360 }
3361 }
3362
3363 private boolean isHealthCheckerConfigured() {
3364 String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
3365 return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
3366 }
3367
3368 @Override
3369 public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
3370 TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
3371 if (cpHost != null) {
3372 if (cpHost.preCreateNamespace(descriptor)) {
3373 return;
3374 }
3375 }
3376 LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
3377 tableNamespaceManager.create(descriptor);
3378 if (cpHost != null) {
3379 cpHost.postCreateNamespace(descriptor);
3380 }
3381 }
3382
3383 @Override
3384 public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
3385 TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
3386 if (cpHost != null) {
3387 if (cpHost.preModifyNamespace(descriptor)) {
3388 return;
3389 }
3390 }
3391 LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
3392 tableNamespaceManager.update(descriptor);
3393 if (cpHost != null) {
3394 cpHost.postModifyNamespace(descriptor);
3395 }
3396 }
3397
3398 @Override
3399 public void deleteNamespace(String name) throws IOException {
3400 if (cpHost != null) {
3401 if (cpHost.preDeleteNamespace(name)) {
3402 return;
3403 }
3404 }
3405 LOG.info(getClientIdAuditPrefix() + " delete " + name);
3406 tableNamespaceManager.remove(name);
3407 if (cpHost != null) {
3408 cpHost.postDeleteNamespace(name);
3409 }
3410 }
3411
3412 @Override
3413 public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
3414 boolean ready = tableNamespaceManager != null &&
3415 tableNamespaceManager.isTableAvailableAndInitialized();
3416 if (!ready) {
3417 throw new IOException("Table Namespace Manager not ready yet, try again later");
3418 }
3419 NamespaceDescriptor nsd = tableNamespaceManager.get(name);
3420 if (nsd == null) {
3421 throw new NamespaceNotFoundException(name);
3422 }
3423 return nsd;
3424 }
3425
3426 @Override
3427 public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
3428 return Lists.newArrayList(tableNamespaceManager.list());
3429 }
3430
3431 @Override
3432 public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
3433 getNamespaceDescriptor(name);
3434 return Lists.newArrayList(tableDescriptors.getByNamespace(name).values());
3435 }
3436
3437 @Override
3438 public List<TableName> listTableNamesByNamespace(String name) throws IOException {
3439 List<TableName> tableNames = Lists.newArrayList();
3440 getNamespaceDescriptor(name);
3441 for (HTableDescriptor descriptor: tableDescriptors.getByNamespace(name).values()) {
3442 tableNames.add(descriptor.getTableName());
3443 }
3444 return tableNames;
3445 }
3446
3447 @Override
3448 public ReportRegionStateTransitionResponse reportRegionStateTransition(RpcController controller,
3449 ReportRegionStateTransitionRequest req) throws ServiceException {
3450 try {
3451 RegionStateTransition rt = req.getTransition(0);
3452 TableName tableName = ProtobufUtil.toTableName(
3453 rt.getRegionInfo(0).getTableName());
3454 RegionStates regionStates = assignmentManager.getRegionStates();
3455 if (!(TableName.META_TABLE_NAME.equals(tableName)
3456 && regionStates.getRegionState(HRegionInfo.FIRST_META_REGIONINFO) != null)
3457 && !assignmentManager.isFailoverCleanupDone()) {
3458
3459
3460 throw new PleaseHoldException("Master is rebuilding user regions");
3461 }
3462 ServerName sn = ProtobufUtil.toServerName(req.getServer());
3463 String error = assignmentManager.onRegionTransition(sn, rt);
3464 ReportRegionStateTransitionResponse.Builder rrtr =
3465 ReportRegionStateTransitionResponse.newBuilder();
3466 if (error != null) {
3467 rrtr.setErrorMessage(error);
3468 }
3469 return rrtr.build();
3470 } catch (IOException ioe) {
3471 throw new ServiceException(ioe);
3472 }
3473 }
3474
3475 @Override
3476 public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
3477 checkInitialized();
3478 if (cpHost != null) {
3479 cpHost.preTruncateTable(tableName);
3480 }
3481 LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
3482 TruncateTableHandler handler = new TruncateTableHandler(tableName, this, this, preserveSplits);
3483 handler.prepare();
3484 handler.process();
3485 if (cpHost != null) {
3486 cpHost.postTruncateTable(tableName);
3487 }
3488 }
3489
3490
3491 @Override
3492 public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
3493 throws ServiceException {
3494 try {
3495 truncateTable(ProtobufUtil.toTableName(request.getTableName()), request.getPreserveSplits());
3496 } catch (IOException e) {
3497 throw new ServiceException(e);
3498 }
3499 return TruncateTableResponse.newBuilder().build();
3500 }
3501
3502 }