From 88f8d58c8bec33494139a10c921afd55d3895d7c Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Mon, 22 Jun 2015 14:59:35 +0200 Subject: [PATCH] Cluster Health: Add max wait time for pending task and active shard percentage In order to get a quick overview using by simply checking the cluster state and its corresponding cat API, the following two attributes have been added to the cluster health response: * task max waiting time, the time value of the first task of the queue and how long it has been waiting * active shards percent: The percentage of the number of shards that are in initializing state This makes the cluster health API handy to check, when a fully restarted cluster is back up and running. Closes #10805 --- .../cluster/health/ClusterHealthResponse.java | 52 ++++++++++++++++++- .../health/TransportClusterHealthAction.java | 20 ++++--- .../elasticsearch/cluster/ClusterService.java | 6 +++ .../service/InternalClusterService.java | 39 +++++--------- .../util/concurrent/EsThreadPoolExecutor.java | 9 ++-- .../PrioritizedEsThreadPoolExecutor.java | 22 +++++++- .../util/concurrent/PrioritizedRunnable.java | 11 ++++ .../common/xcontent/XContentBuilder.java | 9 ++++ .../rest/action/cat/RestHealthAction.java | 4 ++ .../cluster/ClusterHealthResponsesTests.java | 11 ++-- .../test/cluster/NoopClusterService.java | 6 ++- .../test/cluster/TestClusterService.java | 5 ++ .../test/cat.health/10_basic.yaml | 4 ++ 13 files changed, 155 insertions(+), 43 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java index a87df801a6310..258d60722019a 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java @@ -27,8 +27,10 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.RoutingTableValidation; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; @@ -60,6 +62,8 @@ public class ClusterHealthResponse extends ActionResponse implements Iterable validationFailures; @@ -70,15 +74,19 @@ public class ClusterHealthResponse extends ActionResponse implements Iterable shardRoutings = clusterState.getRoutingTable().allShards(); + int activeShardCount = 0; + int totalShardCount = 0; + for (ShardRouting shardRouting : shardRoutings) { + if (shardRouting.active()) activeShardCount++; + totalShardCount++; + } + this.activeShardsPercent = (((double) activeShardCount) / totalShardCount) * 100; + } } public String getClusterName() { @@ -200,6 +222,21 @@ public Map getIndices() { return indices; } + /** + * + * @return The maximum wait time of all tasks in the queue + */ + public TimeValue getTaskMaxWaitingTime() { + return taskMaxWaitingTime; + } + + /** + * The percentage of active shards, should be 100% in a green system + */ + public double getActiveShardsPercent() { + return activeShardsPercent; + } + @Override public Iterator iterator() { return indices.values().iterator(); @@ -244,6 +281,9 @@ public void readFrom(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_1_7_0)) { delayedUnassignedShards= in.readInt(); } + + activeShardsPercent = in.readDouble(); + taskMaxWaitingTime = TimeValue.readTimeValue(in); } @Override @@ -274,6 +314,8 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_1_7_0)) { out.writeInt(delayedUnassignedShards); } + out.writeDouble(activeShardsPercent); + taskMaxWaitingTime.writeTo(out); } @@ -299,6 +341,10 @@ static final class Fields { static final XContentBuilderString NUMBER_OF_PENDING_TASKS = new XContentBuilderString("number_of_pending_tasks"); static final XContentBuilderString NUMBER_OF_IN_FLIGHT_FETCH = new XContentBuilderString("number_of_in_flight_fetch"); static final XContentBuilderString DELAYED_UNASSIGNED_SHARDS = new XContentBuilderString("delayed_unassigned_shards"); + static final XContentBuilderString TASK_MAX_WAIT_TIME_IN_QUEUE = new XContentBuilderString("task_max_waiting_in_queue"); + static final XContentBuilderString TASK_MAX_WAIT_TIME_IN_QUEUE_IN_MILLIS = new XContentBuilderString("task_max_waiting_in_queue_millis"); + static final XContentBuilderString ACTIVE_SHARDS_PERCENT_AS_NUMBER = new XContentBuilderString("active_shards_percent_as_number"); + static final XContentBuilderString ACTIVE_SHARDS_PERCENT = new XContentBuilderString("active_shards_percent"); static final XContentBuilderString ACTIVE_PRIMARY_SHARDS = new XContentBuilderString("active_primary_shards"); static final XContentBuilderString ACTIVE_SHARDS = new XContentBuilderString("active_shards"); static final XContentBuilderString RELOCATING_SHARDS = new XContentBuilderString("relocating_shards"); @@ -323,6 +369,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(Fields.DELAYED_UNASSIGNED_SHARDS, getDelayedUnassignedShards()); builder.field(Fields.NUMBER_OF_PENDING_TASKS, getNumberOfPendingTasks()); builder.field(Fields.NUMBER_OF_IN_FLIGHT_FETCH, getNumberOfInFlightFetch()); + builder.timeValueField(Fields.TASK_MAX_WAIT_TIME_IN_QUEUE_IN_MILLIS, Fields.TASK_MAX_WAIT_TIME_IN_QUEUE, getTaskMaxWaitingTime()); + builder.percentageField(Fields.ACTIVE_SHARDS_PERCENT_AS_NUMBER, Fields.ACTIVE_SHARDS_PERCENT, getActiveShardsPercent()); String level = params.param("level", "cluster"); boolean outputIndices = "indices".equals(level) || "shards".equals(level); diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java index f283421e01d32..c9d3ec27f4e2e 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/health/TransportClusterHealthAction.java @@ -25,9 +25,6 @@ import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; @@ -170,12 +167,14 @@ public void onTimeout(TimeValue timeout) { } private boolean validateRequest(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor) { - ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(), gatewayAllocator.getNumberOfInFlightFetch()); + ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(), + gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMaxTaskWaitTime()); return prepareResponse(request, response, clusterState, waitFor); } private ClusterHealthResponse getResponse(final ClusterHealthRequest request, ClusterState clusterState, final int waitFor, boolean timedOut) { - ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(), gatewayAllocator.getNumberOfInFlightFetch()); + ClusterHealthResponse response = clusterHealth(request, clusterState, clusterService.numberOfPendingTasks(), + gatewayAllocator.getNumberOfInFlightFetch(), clusterService.getMaxTaskWaitTime()); boolean valid = prepareResponse(request, response, clusterState, waitFor); assert valid || timedOut; // we check for a timeout here since this method might be called from the wait_for_events @@ -259,20 +258,25 @@ private boolean prepareResponse(final ClusterHealthRequest request, final Cluste } - private ClusterHealthResponse clusterHealth(ClusterHealthRequest request, ClusterState clusterState, int numberOfPendingTasks, int numberOfInFlightFetch) { + private ClusterHealthResponse clusterHealth(ClusterHealthRequest request, ClusterState clusterState, int numberOfPendingTasks, int numberOfInFlightFetch, + TimeValue pendingTaskTimeInQueue) { if (logger.isTraceEnabled()) { logger.trace("Calculating health based on state version [{}]", clusterState.version()); } + String[] concreteIndices; try { concreteIndices = clusterState.metaData().concreteIndices(request.indicesOptions(), request.indices()); } catch (IndexMissingException e) { // one of the specified indices is not there - treat it as RED. - ClusterHealthResponse response = new ClusterHealthResponse(clusterName.value(), Strings.EMPTY_ARRAY, clusterState, numberOfPendingTasks, numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(settings, clusterState)); + ClusterHealthResponse response = new ClusterHealthResponse(clusterName.value(), Strings.EMPTY_ARRAY, clusterState, + numberOfPendingTasks, numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(settings, clusterState), + pendingTaskTimeInQueue); response.status = ClusterHealthStatus.RED; return response; } - return new ClusterHealthResponse(clusterName.value(), concreteIndices, clusterState, numberOfPendingTasks, numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(settings, clusterState)); + return new ClusterHealthResponse(clusterName.value(), concreteIndices, clusterState, numberOfPendingTasks, + numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(settings, clusterState), pendingTaskTimeInQueue); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterService.java b/core/src/main/java/org/elasticsearch/cluster/ClusterService.java index 4ee9afb3e98f2..2a3d3c12e46c0 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterService.java @@ -120,4 +120,10 @@ public interface ClusterService extends LifecycleComponent { */ int numberOfPendingTasks(); + /** + * Returns the maximum wait time for tasks in the queue + * + * @returns A zero time value if the queue is empty, otherwise the time value oldest task waiting in the queue + */ + TimeValue getMaxTaskWaitTime(); } diff --git a/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java b/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java index 621bfb9868fc8..964a94624feab 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java @@ -243,7 +243,7 @@ public void add(@Nullable final TimeValue timeout, final TimeoutClusterStateList } // call the post added notification on the same event thread try { - updateTasksExecutor.execute(new TimedPrioritizedRunnable(Priority.HIGH, "_add_listener_") { + updateTasksExecutor.execute(new PrioritizedRunnable(Priority.HIGH) { @Override public void run() { if (timeout != null) { @@ -312,10 +312,10 @@ public List pendingTasks() { final Object task = pending.task; if (task == null) { continue; - } else if (task instanceof TimedPrioritizedRunnable) { - TimedPrioritizedRunnable runnable = (TimedPrioritizedRunnable) task; + } else if (task instanceof UpdateTask) { + UpdateTask runnable = (UpdateTask) task; source = runnable.source(); - timeInQueue = runnable.timeSinceCreatedInMillis(); + timeInQueue = runnable.getAgeInMillis(); } else { assert false : "expected TimedPrioritizedRunnable got " + task.getClass(); source = "unknown [" + task.getClass() + "]"; @@ -332,37 +332,26 @@ public int numberOfPendingTasks() { return updateTasksExecutor.getNumberOfPendingTasks(); } + @Override + public TimeValue getMaxTaskWaitTime() { + return updateTasksExecutor.getMaxTaskWaitTime(); + } + + class UpdateTask extends PrioritizedRunnable { - static abstract class TimedPrioritizedRunnable extends PrioritizedRunnable { - private final long creationTimeNS; + public final ClusterStateUpdateTask updateTask; protected final String source; - protected TimedPrioritizedRunnable(Priority priority, String source) { + + UpdateTask(String source, Priority priority, ClusterStateUpdateTask updateTask) { super(priority); + this.updateTask = updateTask; this.source = source; - this.creationTimeNS = System.nanoTime(); - } - - public long timeSinceCreatedInMillis() { - // max with 0 to make sure we always return a non negative number - // even if time shifts. - return Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - creationTimeNS)); } public String source() { return source; } - } - - class UpdateTask extends TimedPrioritizedRunnable { - - public final ClusterStateUpdateTask updateTask; - - - UpdateTask(String source, Priority priority, ClusterStateUpdateTask updateTask) { - super(priority, source); - this.updateTask = updateTask; - } @Override public void run() { diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/EsThreadPoolExecutor.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/EsThreadPoolExecutor.java index 8e21065e2f4ae..9cfb6875993e9 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/EsThreadPoolExecutor.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/EsThreadPoolExecutor.java @@ -20,7 +20,10 @@ package org.elasticsearch.common.util.concurrent; -import java.util.concurrent.*; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; /** * An extension to thread pool executor, allowing (in the future) to add specific additional stats to it. @@ -67,8 +70,8 @@ protected synchronized void terminated() { } } - public static interface ShutdownListener { - public void onTerminated(); + public interface ShutdownListener { + void onTerminated(); } @Override diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java index a63d154080423..c4ed23752356f 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedEsThreadPoolExecutor.java @@ -36,6 +36,7 @@ */ public class PrioritizedEsThreadPoolExecutor extends EsThreadPoolExecutor { + private static final TimeValue NO_WAIT_TIME_VALUE = TimeValue.timeValueMillis(0); private AtomicLong insertionOrder = new AtomicLong(); private Queue current = ConcurrentCollections.newQueue(); @@ -56,6 +57,26 @@ public int getNumberOfPendingTasks() { return size; } + /** + * Returns the waiting time of the first task in the queue + */ + public TimeValue getMaxTaskWaitTime() { + if (getQueue().size() == 0) { + return NO_WAIT_TIME_VALUE; + } + + long now = System.nanoTime(); + long oldestCreationDateInNanos = now; + for (Runnable queuedRunnable : getQueue()) { + if (queuedRunnable instanceof PrioritizedRunnable) { + oldestCreationDateInNanos = Math.min(oldestCreationDateInNanos, + ((PrioritizedRunnable) queuedRunnable).getCreationDateInNanos()); + } + } + + return TimeValue.timeValueNanos(now - oldestCreationDateInNanos); + } + private void addPending(List runnables, List pending, boolean executing) { for (Runnable runnable : runnables) { if (runnable instanceof TieBreakingPrioritizedRunnable) { @@ -191,7 +212,6 @@ private void runAndClean(Runnable run) { timeoutFuture = null; } } - } private final class PrioritizedFutureTask extends FutureTask implements Comparable { diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedRunnable.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedRunnable.java index bd830ba75b9c2..09d644e664f40 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedRunnable.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/PrioritizedRunnable.java @@ -19,6 +19,7 @@ package org.elasticsearch.common.util.concurrent; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.unit.TimeValue; /** * @@ -26,6 +27,7 @@ public abstract class PrioritizedRunnable implements Runnable, Comparable { private final Priority priority; + private final long creationDate; public static PrioritizedRunnable wrap(Runnable runnable, Priority priority) { return new Wrapped(runnable, priority); @@ -33,6 +35,15 @@ public static PrioritizedRunnable wrap(Runnable runnable, Priority priority) { protected PrioritizedRunnable(Priority priority) { this.priority = priority; + creationDate = System.nanoTime(); + } + + public long getCreationDateInNanos() { + return creationDate; + } + + public long getAgeInMillis() { + return Math.max(0, (System.nanoTime() - creationDate) / 1000); } @Override diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java index fb31bd89a95fb..f239525e6bc7d 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java @@ -43,6 +43,7 @@ import java.math.RoundingMode; import java.util.Calendar; import java.util.Date; +import java.util.Locale; import java.util.Map; /** @@ -957,6 +958,14 @@ public XContentBuilder byteSizeField(XContentBuilderString rawFieldName, XConten return this; } + public XContentBuilder percentageField(XContentBuilderString rawFieldName, XContentBuilderString readableFieldName, double percentage) throws IOException { + if (humanReadable) { + field(readableFieldName, String.format(Locale.ROOT, "%1.1f%%", percentage)); + } + field(rawFieldName, percentage); + return this; + } + public XContentBuilder value(Boolean value) throws IOException { if (value == null) { return nullValue(); diff --git a/core/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java b/core/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java index 365638c9cff5a..eb35da99d2145 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java @@ -80,6 +80,8 @@ Table getTableWithHeader(final RestRequest request) { t.addCell("init", "alias:i,shards.initializing,shardsInitializing;text-align:right;desc:number of initializing nodes"); t.addCell("unassign", "alias:u,shards.unassigned,shardsUnassigned;text-align:right;desc:number of unassigned shards"); t.addCell("pending_tasks", "alias:pt,pendingTasks;text-align:right;desc:number of pending tasks"); + t.addCell("max_task_wait_time", "alias:mtwt,maxTaskWaitTime;text-align:right;desc:wait time of longest task pending"); + t.addCell("active_shards_percent", "alias:asp,activeShardsPercent;text-align:right;desc:active number of shards in percent"); t.endHeaders(); return t; @@ -103,6 +105,8 @@ private Table buildTable(final ClusterHealthResponse health, final RestRequest r t.addCell(health.getInitializingShards()); t.addCell(health.getUnassignedShards()); t.addCell(health.getNumberOfPendingTasks()); + t.addCell(health.getTaskMaxWaitingTime().millis() == 0 ? "-" : health.getTaskMaxWaitingTime()); + t.addCell(String.format(Locale.ROOT, "%1.1f%%", health.getActiveShardsPercent())); t.endRow(); return t; } diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java index 345f396034248..eab639ac77c7d 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterHealthResponsesTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.routing.*; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchTestCase; import org.hamcrest.Matchers; @@ -38,8 +39,9 @@ import java.io.IOException; +import static org.hamcrest.CoreMatchers.allOf; import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.*; public class ClusterHealthResponsesTests extends ElasticsearchTestCase { @@ -193,13 +195,16 @@ public void testClusterHealth() throws IOException { int pendingTasks = randomIntBetween(0, 200); int inFlight = randomIntBetween(0, 200); int delayedUnassigned = randomIntBetween(0, 200); - ClusterHealthResponse clusterHealth = new ClusterHealthResponse("bla", clusterState.metaData().concreteIndices(IndicesOptions.strictExpand(), (String[]) null), clusterState, pendingTasks, inFlight, delayedUnassigned); + TimeValue pendingTaskInQueueTime = TimeValue.timeValueMillis(randomIntBetween(1000, 100000)); + ClusterHealthResponse clusterHealth = new ClusterHealthResponse("bla", clusterState.metaData().concreteIndices(IndicesOptions.strictExpand(), (String[]) null), clusterState, pendingTasks, inFlight, delayedUnassigned, pendingTaskInQueueTime); logger.info("cluster status: {}, expected {}", clusterHealth.getStatus(), counter.status()); clusterHealth = maybeSerialize(clusterHealth); assertClusterHealth(clusterHealth, counter); assertThat(clusterHealth.getNumberOfPendingTasks(), Matchers.equalTo(pendingTasks)); assertThat(clusterHealth.getNumberOfInFlightFetch(), Matchers.equalTo(inFlight)); assertThat(clusterHealth.getDelayedUnassignedShards(), Matchers.equalTo(delayedUnassigned)); + assertThat(clusterHealth.getTaskMaxWaitingTime().millis(), is(pendingTaskInQueueTime.millis())); + assertThat(clusterHealth.getActiveShardsPercent(), is(allOf(greaterThanOrEqualTo(0.0), lessThanOrEqualTo(100.0)))); } ClusterHealthResponse maybeSerialize(ClusterHealthResponse clusterHealth) throws IOException { @@ -227,7 +232,7 @@ public void testValidations() throws IOException { metaData.put(indexMetaData, true); routingTable.add(indexRoutingTable); ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build(); - ClusterHealthResponse clusterHealth = new ClusterHealthResponse("bla", clusterState.metaData().concreteIndices(IndicesOptions.strictExpand(), (String[]) null), clusterState, 0, 0, 0); + ClusterHealthResponse clusterHealth = new ClusterHealthResponse("bla", clusterState.metaData().concreteIndices(IndicesOptions.strictExpand(), (String[]) null), clusterState, 0, 0, 0, TimeValue.timeValueMillis(0)); clusterHealth = maybeSerialize(clusterHealth); // currently we have no cluster level validation failures as index validation issues are reported per index. assertThat(clusterHealth.getValidationFailures(), Matchers.hasSize(0)); diff --git a/core/src/test/java/org/elasticsearch/test/cluster/NoopClusterService.java b/core/src/test/java/org/elasticsearch/test/cluster/NoopClusterService.java index 18f712e725924..834e7d540c408 100644 --- a/core/src/test/java/org/elasticsearch/test/cluster/NoopClusterService.java +++ b/core/src/test/java/org/elasticsearch/test/cluster/NoopClusterService.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.test.cluster; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.block.ClusterBlock; @@ -135,6 +134,11 @@ public int numberOfPendingTasks() { return 0; } + @Override + public TimeValue getMaxTaskWaitTime() { + return TimeValue.timeValueMillis(0); + } + @Override public Lifecycle.State lifecycleState() { return null; diff --git a/core/src/test/java/org/elasticsearch/test/cluster/TestClusterService.java b/core/src/test/java/org/elasticsearch/test/cluster/TestClusterService.java index 6a61665e355a4..007f6b45d1d8e 100644 --- a/core/src/test/java/org/elasticsearch/test/cluster/TestClusterService.java +++ b/core/src/test/java/org/elasticsearch/test/cluster/TestClusterService.java @@ -192,6 +192,11 @@ public int numberOfPendingTasks() { throw new UnsupportedOperationException(); } + @Override + public TimeValue getMaxTaskWaitTime() { + throw new UnsupportedOperationException(); + } + @Override public Lifecycle.State lifecycleState() { throw new UnsupportedOperationException(); diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.health/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.health/10_basic.yaml index 4c8607ea6e9b4..ad4fd2f5521fc 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.health/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.health/10_basic.yaml @@ -18,6 +18,8 @@ init .+ \n unassign .+ \n pending_tasks .+ \n + max_task_wait_time .+ \n + active_shards_percent .+ \n $/ @@ -44,6 +46,8 @@ \d+ \s+ # init \d+ \s+ # unassign \d+ \s+ # pending_tasks + (-|\d+[.]\d+ms|s) \s+ # max task waiting time + \d+\.\d+% \s+ # active shards percent \n )+ $/