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