|  | @@ -35,7 +35,9 @@ import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.node.DiscoveryNodes;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.Nullable;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.Priority;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.StopWatch;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.component.AbstractLifecycleComponent;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.lease.Releasable;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.ClusterSettings;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.Settings;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.unit.TimeValue;
 | 
	
	
		
			
				|  | @@ -47,6 +49,7 @@ import org.elasticsearch.common.util.iterable.Iterables;
 | 
	
		
			
				|  |  |  import org.elasticsearch.threadpool.Scheduler;
 | 
	
		
			
				|  |  |  import org.elasticsearch.threadpool.ThreadPool;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +import java.util.Arrays;
 | 
	
		
			
				|  |  |  import java.util.Collection;
 | 
	
		
			
				|  |  |  import java.util.Collections;
 | 
	
		
			
				|  |  |  import java.util.Iterator;
 | 
	
	
		
			
				|  | @@ -61,6 +64,7 @@ import java.util.concurrent.atomic.AtomicReference;
 | 
	
		
			
				|  |  |  import java.util.function.Consumer;
 | 
	
		
			
				|  |  |  import java.util.function.Function;
 | 
	
		
			
				|  |  |  import java.util.function.Supplier;
 | 
	
		
			
				|  |  | +import java.util.stream.Collectors;
 | 
	
		
			
				|  |  |  import java.util.stream.Stream;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import static org.elasticsearch.cluster.service.ClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING;
 | 
	
	
		
			
				|  | @@ -389,15 +393,18 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
 | 
	
		
			
				|  |  |          final ClusterState previousClusterState = state.get();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          long startTimeMS = currentTimeInMillis();
 | 
	
		
			
				|  |  | +        final StopWatch stopWatch = new StopWatch();
 | 
	
		
			
				|  |  |          final ClusterState newClusterState;
 | 
	
		
			
				|  |  |          try {
 | 
	
		
			
				|  |  | -            newClusterState = task.apply(previousClusterState);
 | 
	
		
			
				|  |  | +            try (Releasable ignored = stopWatch.timing("running task [" + task.source + ']')) {
 | 
	
		
			
				|  |  | +                newClusterState = task.apply(previousClusterState);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  |          } catch (Exception e) {
 | 
	
		
			
				|  |  |              TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, currentTimeInMillis() - startTimeMS));
 | 
	
		
			
				|  |  |              logger.trace(() -> new ParameterizedMessage(
 | 
	
		
			
				|  |  |                  "failed to execute cluster state applier in [{}], state:\nversion [{}], source [{}]\n{}",
 | 
	
		
			
				|  |  |                  executionTime, previousClusterState.version(), task.source, previousClusterState), e);
 | 
	
		
			
				|  |  | -            warnAboutSlowTaskIfNeeded(executionTime, task.source);
 | 
	
		
			
				|  |  | +            warnAboutSlowTaskIfNeeded(executionTime, task.source, stopWatch);
 | 
	
		
			
				|  |  |              task.listener.onFailure(task.source, e);
 | 
	
		
			
				|  |  |              return;
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -405,7 +412,7 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
 | 
	
		
			
				|  |  |          if (previousClusterState == newClusterState) {
 | 
	
		
			
				|  |  |              TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, currentTimeInMillis() - startTimeMS));
 | 
	
		
			
				|  |  |              logger.debug("processing [{}]: took [{}] no change in cluster state", task.source, executionTime);
 | 
	
		
			
				|  |  | -            warnAboutSlowTaskIfNeeded(executionTime, task.source);
 | 
	
		
			
				|  |  | +            warnAboutSlowTaskIfNeeded(executionTime, task.source, stopWatch);
 | 
	
		
			
				|  |  |              task.listener.onSuccess(task.source);
 | 
	
		
			
				|  |  |          } else {
 | 
	
		
			
				|  |  |              if (logger.isTraceEnabled()) {
 | 
	
	
		
			
				|  | @@ -415,12 +422,12 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
 | 
	
		
			
				|  |  |                  logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), task.source);
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |              try {
 | 
	
		
			
				|  |  | -                applyChanges(task, previousClusterState, newClusterState);
 | 
	
		
			
				|  |  | +                applyChanges(task, previousClusterState, newClusterState, stopWatch);
 | 
	
		
			
				|  |  |                  TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, currentTimeInMillis() - startTimeMS));
 | 
	
		
			
				|  |  |                  logger.debug("processing [{}]: took [{}] done applying updated cluster state (version: {}, uuid: {})", task.source,
 | 
	
		
			
				|  |  |                      executionTime, newClusterState.version(),
 | 
	
		
			
				|  |  |                      newClusterState.stateUUID());
 | 
	
		
			
				|  |  | -                warnAboutSlowTaskIfNeeded(executionTime, task.source);
 | 
	
		
			
				|  |  | +                warnAboutSlowTaskIfNeeded(executionTime, task.source, stopWatch);
 | 
	
		
			
				|  |  |                  task.listener.onSuccess(task.source);
 | 
	
		
			
				|  |  |              } catch (Exception e) {
 | 
	
		
			
				|  |  |                  TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, currentTimeInMillis() - startTimeMS));
 | 
	
	
		
			
				|  | @@ -438,7 +445,7 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private void applyChanges(UpdateTask task, ClusterState previousClusterState, ClusterState newClusterState) {
 | 
	
		
			
				|  |  | +    private void applyChanges(UpdateTask task, ClusterState previousClusterState, ClusterState newClusterState, StopWatch stopWatch) {
 | 
	
		
			
				|  |  |          ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(task.source, newClusterState, previousClusterState);
 | 
	
		
			
				|  |  |          // new cluster state, notify all listeners
 | 
	
		
			
				|  |  |          final DiscoveryNodes.Delta nodesDelta = clusterChangedEvent.nodesDelta();
 | 
	
	
		
			
				|  | @@ -451,24 +458,28 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          logger.trace("connecting to nodes of cluster state with version {}", newClusterState.version());
 | 
	
		
			
				|  |  | -        connectToNodesAndWait(newClusterState);
 | 
	
		
			
				|  |  | +        try (Releasable ignored = stopWatch.timing("connecting to new nodes")) {
 | 
	
		
			
				|  |  | +            connectToNodesAndWait(newClusterState);
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          // nothing to do until we actually recover from the gateway or any other block indicates we need to disable persistency
 | 
	
		
			
				|  |  |          if (clusterChangedEvent.state().blocks().disableStatePersistence() == false && clusterChangedEvent.metaDataChanged()) {
 | 
	
		
			
				|  |  |              logger.debug("applying settings from cluster state with version {}", newClusterState.version());
 | 
	
		
			
				|  |  |              final Settings incomingSettings = clusterChangedEvent.state().metaData().settings();
 | 
	
		
			
				|  |  | -            clusterSettings.applySettings(incomingSettings);
 | 
	
		
			
				|  |  | +            try (Releasable ignored = stopWatch.timing("applying settings")) {
 | 
	
		
			
				|  |  | +                clusterSettings.applySettings(incomingSettings);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          logger.debug("apply cluster state with version {}", newClusterState.version());
 | 
	
		
			
				|  |  | -        callClusterStateAppliers(clusterChangedEvent);
 | 
	
		
			
				|  |  | +        callClusterStateAppliers(clusterChangedEvent, stopWatch);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          nodeConnectionsService.disconnectFromNodesExcept(newClusterState.nodes());
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          logger.debug("set locally applied cluster state to version {}", newClusterState.version());
 | 
	
		
			
				|  |  |          state.set(newClusterState);
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        callClusterStateListeners(clusterChangedEvent);
 | 
	
		
			
				|  |  | +        callClusterStateListeners(clusterChangedEvent, stopWatch);
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      protected void connectToNodesAndWait(ClusterState newClusterState) {
 | 
	
	
		
			
				|  | @@ -483,18 +494,22 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private void callClusterStateAppliers(ClusterChangedEvent clusterChangedEvent) {
 | 
	
		
			
				|  |  | +    private void callClusterStateAppliers(ClusterChangedEvent clusterChangedEvent, StopWatch stopWatch) {
 | 
	
		
			
				|  |  |          clusterStateAppliers.forEach(applier -> {
 | 
	
		
			
				|  |  |              logger.trace("calling [{}] with change to version [{}]", applier, clusterChangedEvent.state().version());
 | 
	
		
			
				|  |  | -            applier.applyClusterState(clusterChangedEvent);
 | 
	
		
			
				|  |  | +            try (Releasable ignored = stopWatch.timing("running applier [" + applier + "]")) {
 | 
	
		
			
				|  |  | +                applier.applyClusterState(clusterChangedEvent);
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  |          });
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private void callClusterStateListeners(ClusterChangedEvent clusterChangedEvent) {
 | 
	
		
			
				|  |  | +    private void callClusterStateListeners(ClusterChangedEvent clusterChangedEvent, StopWatch stopWatch) {
 | 
	
		
			
				|  |  |          Stream.concat(clusterStateListeners.stream(), timeoutClusterStateListeners.stream()).forEach(listener -> {
 | 
	
		
			
				|  |  |              try {
 | 
	
		
			
				|  |  |                  logger.trace("calling [{}] with change to version [{}]", listener, clusterChangedEvent.state().version());
 | 
	
		
			
				|  |  | -                listener.clusterChanged(clusterChangedEvent);
 | 
	
		
			
				|  |  | +                try (Releasable ignored = stopWatch.timing("notifying listener [" + listener + "]")) {
 | 
	
		
			
				|  |  | +                    listener.clusterChanged(clusterChangedEvent);
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  |              } catch (Exception ex) {
 | 
	
		
			
				|  |  |                  logger.warn("failed to notify ClusterStateListener", ex);
 | 
	
		
			
				|  |  |              }
 | 
	
	
		
			
				|  | @@ -532,10 +547,11 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    protected void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source) {
 | 
	
		
			
				|  |  | +    private void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source, StopWatch stopWatch) {
 | 
	
		
			
				|  |  |          if (executionTime.getMillis() > slowTaskLoggingThreshold.getMillis()) {
 | 
	
		
			
				|  |  | -            logger.warn("cluster state applier task [{}] took [{}] which is above the warn threshold of {}", source, executionTime,
 | 
	
		
			
				|  |  | -                slowTaskLoggingThreshold);
 | 
	
		
			
				|  |  | +            logger.warn("cluster state applier task [{}] took [{}] which is above the warn threshold of [{}]: {}", source, executionTime,
 | 
	
		
			
				|  |  | +                slowTaskLoggingThreshold, Arrays.stream(stopWatch.taskInfo())
 | 
	
		
			
				|  |  | +                    .map(ti -> '[' + ti.getTaskName() + "] took [" + ti.getTime().millis() + "ms]").collect(Collectors.joining(", ")));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 |