183
183
import org .apache .hadoop .hbase .mob .MobFileCompactionChore ;
184
184
import org .apache .hadoop .hbase .monitoring .MemoryBoundedLogMessageBuffer ;
185
185
import org .apache .hadoop .hbase .monitoring .MonitoredTask ;
186
+ import org .apache .hadoop .hbase .monitoring .TaskGroup ;
186
187
import org .apache .hadoop .hbase .monitoring .TaskMonitor ;
187
188
import org .apache .hadoop .hbase .procedure .MasterProcedureManagerHost ;
188
189
import org .apache .hadoop .hbase .procedure .flush .MasterFlushTableProcedureManager ;
@@ -447,6 +448,8 @@ public class HMaster extends HRegionServer implements MasterServices {
447
448
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move" ;
448
449
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true ;
449
450
451
+ private TaskGroup startupTaskGroup ;
452
+
450
453
/**
451
454
* Initializes the HMaster. The steps are as follows:
452
455
* <p>
@@ -455,9 +458,8 @@ public class HMaster extends HRegionServer implements MasterServices {
455
458
* <li>Start the ActiveMasterManager.
456
459
* </ol>
457
460
* <p>
458
- * Remaining steps of initialization occur in
459
- * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
460
- * one.
461
+ * Remaining steps of initialization occur in {@link #finishActiveMasterInitialization()} after
462
+ * the master becomes the active one.
461
463
*/
462
464
public HMaster (final Configuration conf ) throws IOException {
463
465
super (conf );
@@ -887,12 +889,12 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx
887
889
* Notice that now we will not schedule a special procedure to make meta online(unless the first
888
890
* time where meta has not been created yet), we will rely on SCP to bring meta online.
889
891
*/
890
- private void finishActiveMasterInitialization (MonitoredTask status ) throws IOException ,
891
- InterruptedException , KeeperException , ReplicationException , DeserializationException {
892
+ private void finishActiveMasterInitialization () throws IOException , InterruptedException ,
893
+ KeeperException , ReplicationException , DeserializationException {
892
894
/*
893
895
* We are active master now... go initialize components we need to run.
894
896
*/
895
- status . setStatus ("Initializing Master file system" );
897
+ startupTaskGroup . addTask ("Initializing Master file system" );
896
898
897
899
this .masterActiveTime = EnvironmentEdgeManager .currentTime ();
898
900
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -905,15 +907,15 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
905
907
906
908
// warm-up HTDs cache on master initialization
907
909
if (preLoadTableDescriptors ) {
908
- status . setStatus ("Pre-loading table descriptors" );
910
+ startupTaskGroup . addTask ("Pre-loading table descriptors" );
909
911
this .tableDescriptors .getAll ();
910
912
}
911
913
912
914
// Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
913
915
// only after it has checked in with the Master. At least a few tests ask Master for clusterId
914
916
// before it has called its run method and before RegionServer has done the reportForDuty.
915
917
ClusterId clusterId = fileSystemManager .getClusterId ();
916
- status . setStatus ("Publishing Cluster ID " + clusterId + " in ZooKeeper" );
918
+ startupTaskGroup . addTask ("Publishing Cluster ID " + clusterId + " in ZooKeeper" );
917
919
ZKClusterId .setClusterId (this .zooKeeper , fileSystemManager .getClusterId ());
918
920
this .clusterId = clusterId .toString ();
919
921
@@ -932,7 +934,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
932
934
}
933
935
}
934
936
935
- status . setStatus ("Initialize ServerManager and schedule SCP for crash servers" );
937
+ startupTaskGroup . addTask ("Initialize ServerManager and schedule SCP for crash servers" );
936
938
// The below two managers must be created before loading procedures, as they will be used during
937
939
// loading.
938
940
// initialize master local region
@@ -982,8 +984,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
982
984
? new MirroringTableStateManager (this )
983
985
: new TableStateManager (this );
984
986
985
- status . setStatus ("Initializing ZK system trackers" );
987
+ startupTaskGroup . addTask ("Initializing ZK system trackers" );
986
988
initializeZKBasedSystemTrackers ();
989
+
987
990
// Set ourselves as active Master now our claim has succeeded up in zk.
988
991
this .activeMaster = true ;
989
992
@@ -995,19 +998,19 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
995
998
996
999
// This is for backwards compatibility
997
1000
// See HBASE-11393
998
- status . setStatus ("Update TableCFs node in ZNode" );
1001
+ startupTaskGroup . addTask ("Update TableCFs node in ZNode" );
999
1002
ReplicationPeerConfigUpgrader tableCFsUpdater =
1000
1003
new ReplicationPeerConfigUpgrader (zooKeeper , conf );
1001
1004
tableCFsUpdater .copyTableCFs ();
1002
1005
1003
1006
if (!maintenanceMode ) {
1004
- status . setStatus ("Initializing master coprocessors" );
1007
+ startupTaskGroup . addTask ("Initializing master coprocessors" );
1005
1008
setQuotasObserver (conf );
1006
1009
initializeCoprocessorHost (conf );
1007
1010
}
1008
1011
1009
1012
// Checking if meta needs initializing.
1010
- status . setStatus ("Initializing meta table if this is a new deploy" );
1013
+ startupTaskGroup . addTask ("Initializing meta table if this is a new deploy" );
1011
1014
InitMetaProcedure initMetaProc = null ;
1012
1015
// Print out state of hbase:meta on startup; helps debugging.
1013
1016
if (!this .assignmentManager .getRegionStates ().hasTableRegionStates (TableName .META_TABLE_NAME )) {
@@ -1030,7 +1033,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1030
1033
this .balancer .updateClusterMetrics (getClusterMetricsWithoutCoprocessor ());
1031
1034
1032
1035
// start up all service threads.
1033
- status . setStatus ("Initializing master service threads" );
1036
+ startupTaskGroup . addTask ("Initializing master service threads" );
1034
1037
startServiceThreads ();
1035
1038
// wait meta to be initialized after we start procedure executor
1036
1039
if (initMetaProc != null ) {
@@ -1046,16 +1049,16 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1046
1049
// With this as part of master initialization, it precludes our being able to start a single
1047
1050
// server that is both Master and RegionServer. Needs more thought. TODO.
1048
1051
String statusStr = "Wait for region servers to report in" ;
1049
- status . setStatus (statusStr );
1050
- LOG .info (Objects .toString (status ));
1051
- waitForRegionServers (status );
1052
+ MonitoredTask waitRegionServer = startupTaskGroup . addTask (statusStr );
1053
+ LOG .info (Objects .toString (waitRegionServer ));
1054
+ waitForRegionServers (waitRegionServer );
1052
1055
1053
1056
// Check if master is shutting down because issue initializing regionservers or balancer.
1054
1057
if (isStopped ()) {
1055
1058
return ;
1056
1059
}
1057
1060
1058
- status . setStatus ("Starting assignment manager" );
1061
+ startupTaskGroup . addTask ("Starting assignment manager" );
1059
1062
// FIRST HBASE:META READ!!!!
1060
1063
// The below cannot make progress w/o hbase:meta being online.
1061
1064
// This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
@@ -1132,7 +1135,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1132
1135
this .balancer .updateClusterMetrics (getClusterMetricsWithoutCoprocessor ());
1133
1136
1134
1137
// Start balancer and meta catalog janitor after meta and regions have been assigned.
1135
- status . setStatus ("Starting balancer and catalog janitor" );
1138
+ startupTaskGroup . addTask ("Starting balancer and catalog janitor" );
1136
1139
this .clusterStatusChore = new ClusterStatusChore (this , balancer );
1137
1140
getChoreService ().scheduleChore (clusterStatusChore );
1138
1141
this .balancerChore = new BalancerChore (this );
@@ -1154,7 +1157,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1154
1157
if (!waitForNamespaceOnline ()) {
1155
1158
return ;
1156
1159
}
1157
- status . setStatus ("Starting cluster schema service" );
1160
+ startupTaskGroup . addTask ("Starting cluster schema service" );
1158
1161
try {
1159
1162
initClusterSchemaService ();
1160
1163
} catch (IllegalStateException e ) {
@@ -1177,7 +1180,6 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1177
1180
}
1178
1181
}
1179
1182
1180
- status .markComplete ("Initialization successful" );
1181
1183
LOG .info (String .format ("Master has completed initialization %.3fsec" ,
1182
1184
(EnvironmentEdgeManager .currentTime () - masterActiveTime ) / 1000.0f ));
1183
1185
this .masterFinishedInitializationTime = EnvironmentEdgeManager .currentTime ();
@@ -1196,6 +1198,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1196
1198
}
1197
1199
// Set master as 'initialized'.
1198
1200
setInitialized (true );
1201
+ startupTaskGroup .markComplete ("Initialization successful" );
1202
+ MonitoredTask status =
1203
+ TaskMonitor .get ().createStatus ("Progress after master initialized" , false , true );
1199
1204
1200
1205
if (tableFamilyDesc == null && replBarrierFamilyDesc == null ) {
1201
1206
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1284,6 +1289,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1284
1289
1285
1290
this .rollingUpgradeChore = new RollingUpgradeChore (this );
1286
1291
getChoreService ().scheduleChore (rollingUpgradeChore );
1292
+ status .markComplete ("Progress after master initialized complete" );
1287
1293
}
1288
1294
1289
1295
private void createMissingCFsInMetaDuringUpgrade (TableDescriptor metaDescriptor )
@@ -2385,14 +2391,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
2385
2391
Threads .sleep (timeout );
2386
2392
}
2387
2393
}
2388
- MonitoredTask status = TaskMonitor .get ().createStatus ("Master startup" );
2389
- status .setDescription ("Master startup" );
2394
+
2395
+ // Here for the master startup process, we use TaskGroup to monitor the whole progress.
2396
+ // The UI is similar to how Hadoop designed the startup page for the NameNode.
2397
+ // See HBASE-21521 for more details.
2398
+ // We do not cleanup the startupTaskGroup, let the startup progress information
2399
+ // be permanent in the MEM.
2400
+ startupTaskGroup = TaskMonitor .createTaskGroup (true , "Master startup" );
2390
2401
try {
2391
- if (activeMasterManager .blockUntilBecomingActiveMaster (timeout , status )) {
2392
- finishActiveMasterInitialization (status );
2402
+ if (activeMasterManager .blockUntilBecomingActiveMaster (timeout , startupTaskGroup )) {
2403
+ finishActiveMasterInitialization ();
2393
2404
}
2394
2405
} catch (Throwable t ) {
2395
- status . setStatus ("Failed to become active: " + t .getMessage ());
2406
+ startupTaskGroup . abort ("Failed to become active master due to: " + t .getMessage ());
2396
2407
LOG .error (HBaseMarkers .FATAL , "Failed to become active master" , t );
2397
2408
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
2398
2409
if (
@@ -2406,8 +2417,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
2406
2417
} else {
2407
2418
abort ("Unhandled exception. Starting shutdown." , t );
2408
2419
}
2409
- } finally {
2410
- status .cleanup ();
2411
2420
}
2412
2421
}
2413
2422
@@ -3097,6 +3106,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
3097
3106
return rsFatals ;
3098
3107
}
3099
3108
3109
+ public TaskGroup getStartupProgress () {
3110
+ return startupTaskGroup ;
3111
+ }
3112
+
3100
3113
/**
3101
3114
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
3102
3115
*/
0 commit comments