Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import org.elasticsearch.cluster.LocalMasterServiceTask;
import org.elasticsearch.cluster.NotMasterException;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.ProcessClusterEventTimeoutException;
Expand Down Expand Up @@ -243,9 +242,8 @@ public void onFailure(Exception e) {

static boolean isExpectedFailure(Exception e) {
return e instanceof NotMasterException
|| e instanceof FailedToCommitClusterStateException
&& e.getCause() instanceof EsRejectedExecutionException esre
&& esre.isExecutorShutdown();
&& e.getCause() instanceof EsRejectedExecutionException esre
&& esre.isExecutorShutdown();
}

});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1607,13 +1607,7 @@ public void submitTask(String source, T task, @Nullable TimeValue timeout) {
} catch (Exception e) {
assert e instanceof EsRejectedExecutionException esre && esre.isExecutorShutdown() : e;
task.onFailure(
new FailedToCommitClusterStateException(
"could not schedule timeout handler for [%s][%s] on queue [%s]",
e,
source,
task,
name
)
new NotMasterException("could not schedule timeout handler for [%s][%s] on queue [%s]", e, source, task, name)
);
return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2249,7 +2249,7 @@ public void testTimeoutRejectionBehaviourAtSubmission() {
).submitTask(source, new ClusterStateTaskListener() {
@Override
public void onFailure(Exception e) {
if (e instanceof FailedToCommitClusterStateException
if (e instanceof NotMasterException
&& e.getMessage().startsWith("could not schedule timeout handler")
&& e.getCause() instanceof EsRejectedExecutionException esre
&& esre.isExecutorShutdown()
Expand Down