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