186
186
import org .apache .hadoop .hbase .mob .MobFileCompactionChore ;
187
187
import org .apache .hadoop .hbase .monitoring .MemoryBoundedLogMessageBuffer ;
188
188
import org .apache .hadoop .hbase .monitoring .MonitoredTask ;
189
+ import org .apache .hadoop .hbase .monitoring .TaskGroup ;
189
190
import org .apache .hadoop .hbase .monitoring .TaskMonitor ;
190
191
import org .apache .hadoop .hbase .namequeues .NamedQueueRecorder ;
191
192
import org .apache .hadoop .hbase .procedure .MasterProcedureManagerHost ;
@@ -465,6 +466,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
465
466
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move" ;
466
467
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true ;
467
468
469
+ private TaskGroup startupTaskGroup ;
470
+
468
471
/**
469
472
* Initializes the HMaster. The steps are as follows:
470
473
* <p>
@@ -473,9 +476,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
473
476
* <li>Start the ActiveMasterManager.
474
477
* </ol>
475
478
* <p>
476
- * Remaining steps of initialization occur in
477
- * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
478
- * one.
479
+ * Remaining steps of initialization occur in {@link #finishActiveMasterInitialization(TaskGroup)}
480
+ * after the master becomes the active one.
479
481
*/
480
482
public HMaster (final Configuration conf ) throws IOException {
481
483
super (conf , "Master" );
@@ -908,12 +910,12 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx
908
910
* Notice that now we will not schedule a special procedure to make meta online(unless the first
909
911
* time where meta has not been created yet), we will rely on SCP to bring meta online.
910
912
*/
911
- private void finishActiveMasterInitialization (MonitoredTask status ) throws IOException ,
913
+ private void finishActiveMasterInitialization (TaskGroup startupTaskGro ) throws IOException ,
912
914
InterruptedException , KeeperException , ReplicationException , DeserializationException {
913
915
/*
914
916
* We are active master now... go initialize components we need to run.
915
917
*/
916
- status . setStatus ("Initializing Master file system" );
918
+ startupTaskGroup . addTask ("Initializing Master file system" );
917
919
918
920
this .masterActiveTime = EnvironmentEdgeManager .currentTime ();
919
921
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -926,15 +928,15 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
926
928
927
929
// warm-up HTDs cache on master initialization
928
930
if (preLoadTableDescriptors ) {
929
- status . setStatus ("Pre-loading table descriptors" );
931
+ startupTaskGroup . addTask ("Pre-loading table descriptors" );
930
932
this .tableDescriptors .getAll ();
931
933
}
932
934
933
935
// Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
934
936
// only after it has checked in with the Master. At least a few tests ask Master for clusterId
935
937
// before it has called its run method and before RegionServer has done the reportForDuty.
936
938
ClusterId clusterId = fileSystemManager .getClusterId ();
937
- status . setStatus ("Publishing Cluster ID " + clusterId + " in ZooKeeper" );
939
+ startupTaskGroup . addTask ("Publishing Cluster ID " + clusterId + " in ZooKeeper" );
938
940
ZKClusterId .setClusterId (this .zooKeeper , fileSystemManager .getClusterId ());
939
941
this .clusterId = clusterId .toString ();
940
942
@@ -953,7 +955,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
953
955
}
954
956
}
955
957
956
- status . setStatus ("Initialize ServerManager and schedule SCP for crash servers" );
958
+ startupTaskGroup . addTask ("Initialize ServerManager and schedule SCP for crash servers" );
957
959
// The below two managers must be created before loading procedures, as they will be used during
958
960
// loading.
959
961
// initialize master local region
@@ -1000,9 +1002,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1000
1002
// This manager must be accessed AFTER hbase:meta is confirmed on line..
1001
1003
this .tableStateManager = new TableStateManager (this );
1002
1004
1003
- status . setStatus ("Initializing ZK system trackers" );
1005
+ startupTaskGroup . addTask ("Initializing ZK system trackers" );
1004
1006
initializeZKBasedSystemTrackers ();
1005
- status . setStatus ("Loading last flushed sequence id of regions" );
1007
+ startupTaskGroup . addTask ("Loading last flushed sequence id of regions" );
1006
1008
try {
1007
1009
this .serverManager .loadLastFlushedSequenceIds ();
1008
1010
} catch (IOException e ) {
@@ -1018,7 +1020,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1018
1020
zombieDetector .start ();
1019
1021
1020
1022
if (!maintenanceMode ) {
1021
- status . setStatus ("Initializing master coprocessors" );
1023
+ startupTaskGroup . addTask ("Initializing master coprocessors" );
1022
1024
setQuotasObserver (conf );
1023
1025
initializeCoprocessorHost (conf );
1024
1026
} else {
@@ -1029,7 +1031,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1029
1031
}
1030
1032
1031
1033
// Checking if meta needs initializing.
1032
- status . setStatus ("Initializing meta table if this is a new deploy" );
1034
+ startupTaskGroup . addTask ("Initializing meta table if this is a new deploy" );
1033
1035
InitMetaProcedure initMetaProc = null ;
1034
1036
// Print out state of hbase:meta on startup; helps debugging.
1035
1037
if (!this .assignmentManager .getRegionStates ().hasTableRegionStates (TableName .META_TABLE_NAME )) {
@@ -1049,7 +1051,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1049
1051
this .balancer .updateClusterMetrics (getClusterMetricsWithoutCoprocessor ());
1050
1052
1051
1053
// start up all service threads.
1052
- status . setStatus ("Initializing master service threads" );
1054
+ startupTaskGroup . addTask ("Initializing master service threads" );
1053
1055
startServiceThreads ();
1054
1056
// wait meta to be initialized after we start procedure executor
1055
1057
if (initMetaProc != null ) {
@@ -1062,16 +1064,16 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1062
1064
// With this as part of master initialization, it precludes our being able to start a single
1063
1065
// server that is both Master and RegionServer. Needs more thought. TODO.
1064
1066
String statusStr = "Wait for region servers to report in" ;
1065
- status . setStatus (statusStr );
1066
- LOG .info (Objects .toString (status ));
1067
- waitForRegionServers (status );
1067
+ MonitoredTask waitRegionServer = startupTaskGroup . addTask (statusStr );
1068
+ LOG .info (Objects .toString (waitRegionServer ));
1069
+ waitForRegionServers (waitRegionServer );
1068
1070
1069
1071
// Check if master is shutting down because issue initializing regionservers or balancer.
1070
1072
if (isStopped ()) {
1071
1073
return ;
1072
1074
}
1073
1075
1074
- status . setStatus ("Starting assignment manager" );
1076
+ startupTaskGroup . addTask ("Starting assignment manager" );
1075
1077
// FIRST HBASE:META READ!!!!
1076
1078
// The below cannot make progress w/o hbase:meta being online.
1077
1079
// This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
@@ -1136,7 +1138,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1136
1138
this .balancer .updateClusterMetrics (getClusterMetricsWithoutCoprocessor ());
1137
1139
1138
1140
// Start balancer and meta catalog janitor after meta and regions have been assigned.
1139
- status . setStatus ("Starting balancer and catalog janitor" );
1141
+ startupTaskGroup . addTask ("Starting balancer and catalog janitor" );
1140
1142
this .clusterStatusChore = new ClusterStatusChore (this , balancer );
1141
1143
getChoreService ().scheduleChore (clusterStatusChore );
1142
1144
this .balancerChore = new BalancerChore (this );
@@ -1156,7 +1158,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1156
1158
if (!waitForNamespaceOnline ()) {
1157
1159
return ;
1158
1160
}
1159
- status . setStatus ("Starting cluster schema service" );
1161
+ startupTaskGroup . addTask ("Starting cluster schema service" );
1160
1162
try {
1161
1163
initClusterSchemaService ();
1162
1164
} catch (IllegalStateException e ) {
@@ -1179,7 +1181,6 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1179
1181
}
1180
1182
}
1181
1183
1182
- status .markComplete ("Initialization successful" );
1183
1184
LOG .info (String .format ("Master has completed initialization %.3fsec" ,
1184
1185
(EnvironmentEdgeManager .currentTime () - masterActiveTime ) / 1000.0f ));
1185
1186
this .masterFinishedInitializationTime = EnvironmentEdgeManager .currentTime ();
@@ -1198,6 +1199,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1198
1199
}
1199
1200
// Set master as 'initialized'.
1200
1201
setInitialized (true );
1202
+ startupTaskGroup .markComplete ("Initialization successful" );
1203
+ MonitoredTask status =
1204
+ TaskMonitor .get ().createStatus ("Progress after master initialized" , false , true );
1201
1205
1202
1206
if (tableFamilyDesc == null && replBarrierFamilyDesc == null ) {
1203
1207
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1286,6 +1290,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
1286
1290
1287
1291
this .rollingUpgradeChore = new RollingUpgradeChore (this );
1288
1292
getChoreService ().scheduleChore (rollingUpgradeChore );
1293
+ status .markComplete ("Progress after master initialized complete" );
1289
1294
}
1290
1295
1291
1296
private void createMissingCFsInMetaDuringUpgrade (TableDescriptor metaDescriptor )
@@ -2399,14 +2404,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
2399
2404
Threads .sleep (timeout );
2400
2405
}
2401
2406
}
2402
- MonitoredTask status = TaskMonitor .get ().createStatus ("Master startup" );
2403
- status .setDescription ("Master startup" );
2407
+
2408
+ // Here for the master startup process, we use TaskGroup to monitor the whole progress.
2409
+ // The UI is similar to how Hadoop designed the startup page for the NameNode.
2410
+ // See HBASE-21521 for more details.
2411
+ // We do not cleanup the startupTaskGroup, let the startup progress information
2412
+ // be permanent in the MEM.
2413
+ startupTaskGroup = TaskGroup .createTaskGroup (false , "Master startup" );
2404
2414
try {
2405
- if (activeMasterManager .blockUntilBecomingActiveMaster (timeout , status )) {
2406
- finishActiveMasterInitialization (status );
2415
+ if (activeMasterManager .blockUntilBecomingActiveMaster (timeout , startupTaskGroup )) {
2416
+ finishActiveMasterInitialization (startupTaskGroup );
2407
2417
}
2408
2418
} catch (Throwable t ) {
2409
- status . setStatus ("Failed to become active: " + t .getMessage ());
2419
+ startupTaskGroup . abort ("Failed to become active master due to: " + t .getMessage ());
2410
2420
LOG .error (HBaseMarkers .FATAL , "Failed to become active master" , t );
2411
2421
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
2412
2422
if (
@@ -2420,8 +2430,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
2420
2430
} else {
2421
2431
abort ("Unhandled exception. Starting shutdown." , t );
2422
2432
}
2423
- } finally {
2424
- status .cleanup ();
2425
2433
}
2426
2434
}
2427
2435
@@ -3097,6 +3105,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
3097
3105
return rsFatals ;
3098
3106
}
3099
3107
3108
+ public TaskGroup getStartupProgress () {
3109
+ return startupTaskGroup ;
3110
+ }
3111
+
3100
3112
/**
3101
3113
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
3102
3114
*/
0 commit comments