diff --git a/server/src/main/java/org/elasticsearch/action/support/local/TransportLocalClusterStateAction.java b/server/src/main/java/org/elasticsearch/action/support/local/TransportLocalClusterStateAction.java index 66f94050c9826..c49d2daffca71 100644 --- a/server/src/main/java/org/elasticsearch/action/support/local/TransportLocalClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/local/TransportLocalClusterStateAction.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionRunnable; @@ -23,6 +22,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.discovery.MasterNotDiscoveredException; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.Task; @@ -72,6 +72,8 @@ protected final void doExecute(Task task, Request request, ActionListener format("timed out while waiting for cluster to unblock in [%s] (timeout [%s])", actionName, timeout), exception ); - listener.onFailure(new ElasticsearchTimeoutException("timed out while waiting for cluster to unblock", exception)); + listener.onFailure(new MasterNotDiscoveredException(exception)); } - }, clusterState -> isTaskCancelled(task) || checkBlock(request, clusterState) == null); + }, + clusterState -> isTaskCancelled(task) + || (clusterState.nodes().getMasterNode() != null && checkBlock(request, clusterState) == null) + ); } private boolean isTaskCancelled(Task task) {