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