1
0
Эх сурвалжийг харах

Add pluggable XContentBuilder writers and human readable writers (#29120)

* Add pluggable XContentBuilder writers and human readable writers

This adds the ability to use SPI to plug in writers for XContentBuilder. By
implementing the XContentBuilderProvider class we can allow Elasticsearch to
plug in different ways to encode types to JSON.

Important caveat for this, we should always try to have the class implement
`ToXContentFragment` first, however, in the case of classes from our
dependencies (think Joda classes or Lucene classes) we need a way to specify
writers for these classes.

This also makes the human-readable field writers generic and pluggable, so that
we no longer need to tie XContentBuilder to things like `TimeValue` and
`ByteSizeValue`. Contained as part of this moves all the TimeValue human
readable fields to the new `humanReadableField` method. A future commit will
move the `ByteSizeValue` calls over to this method.

Relates to #28504
Lee Hinman 7 жил өмнө
parent
commit
4bd217c94f
32 өөрчлөгдсөн 202 нэмэгдсэн , 90 устгасан
  1. 1 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java
  2. 1 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java
  3. 2 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java
  4. 1 1
      server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java
  5. 1 1
      server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java
  6. 2 1
      server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java
  7. 2 1
      server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java
  8. 4 2
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocateUnassignedDecision.java
  9. 48 29
      server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java
  10. 64 0
      server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java
  11. 1 1
      server/src/main/java/org/elasticsearch/index/flush/FlushStats.java
  12. 3 3
      server/src/main/java/org/elasticsearch/index/get/GetStats.java
  13. 3 3
      server/src/main/java/org/elasticsearch/index/merge/MergeStats.java
  14. 1 1
      server/src/main/java/org/elasticsearch/index/recovery/RecoveryStats.java
  15. 1 1
      server/src/main/java/org/elasticsearch/index/refresh/RefreshStats.java
  16. 2 2
      server/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java
  17. 4 4
      server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java
  18. 3 3
      server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java
  19. 1 1
      server/src/main/java/org/elasticsearch/index/warmer/WarmerStats.java
  20. 7 7
      server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java
  21. 2 1
      server/src/main/java/org/elasticsearch/ingest/IngestStats.java
  22. 2 2
      server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java
  23. 2 1
      server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java
  24. 6 6
      server/src/main/java/org/elasticsearch/monitor/os/OsStats.java
  25. 2 1
      server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java
  26. 1 1
      server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java
  27. 9 2
      server/src/main/java/org/elasticsearch/node/AdaptiveSelectionStats.java
  28. 2 1
      server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java
  29. 9 5
      server/src/main/java/org/elasticsearch/search/profile/ProfileResult.java
  30. 8 4
      server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java
  31. 2 1
      server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java
  32. 5 1
      server/src/main/java/org/elasticsearch/tasks/TaskInfo.java

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java

@@ -245,7 +245,7 @@ public class ClusterHealthResponse extends ActionResponse implements StatusToXCo
         builder.field(DELAYED_UNASSIGNED_SHARDS, getDelayedUnassignedShards());
         builder.field(NUMBER_OF_PENDING_TASKS, getNumberOfPendingTasks());
         builder.field(NUMBER_OF_IN_FLIGHT_FETCH, getNumberOfInFlightFetch());
-        builder.timeValueField(TASK_MAX_WAIT_TIME_IN_QUEUE_IN_MILLIS, TASK_MAX_WAIT_TIME_IN_QUEUE, getTaskMaxWaitingTime());
+        builder.humanReadableField(TASK_MAX_WAIT_TIME_IN_QUEUE_IN_MILLIS, TASK_MAX_WAIT_TIME_IN_QUEUE, getTaskMaxWaitingTime());
         builder.percentageField(ACTIVE_SHARDS_PERCENT_AS_NUMBER, ACTIVE_SHARDS_PERCENT, getActiveShardsPercent());
 
         String level = params.param("level", "cluster");

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java

@@ -68,7 +68,7 @@ public class NodesInfoResponse extends BaseNodesResponse<NodeInfo> implements To
             builder.field("version", nodeInfo.getVersion());
             builder.field("build_hash", nodeInfo.getBuild().shortHash());
             if (nodeInfo.getTotalIndexingBuffer() != null) {
-                builder.byteSizeField("total_indexing_buffer", "total_indexing_buffer_in_bytes", nodeInfo.getTotalIndexingBuffer());
+                builder.humanReadableField("total_indexing_buffer", "total_indexing_buffer_in_bytes", nodeInfo.getTotalIndexingBuffer());
             }
 
             builder.startArray("roles");

+ 2 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java

@@ -22,6 +22,7 @@ package org.elasticsearch.action.admin.cluster.snapshots.status;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Streamable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -143,7 +144,7 @@ public class SnapshotStats implements Streamable, ToXContentFragment {
         builder.byteSizeField(Fields.TOTAL_SIZE_IN_BYTES, Fields.TOTAL_SIZE, getTotalSize());
         builder.byteSizeField(Fields.PROCESSED_SIZE_IN_BYTES, Fields.PROCESSED_SIZE, getProcessedSize());
         builder.field(Fields.START_TIME_IN_MILLIS, getStartTime());
-        builder.timeValueField(Fields.TIME_IN_MILLIS, Fields.TIME, getTime());
+        builder.humanReadableField(Fields.TIME_IN_MILLIS, Fields.TIME, new TimeValue(getTime()));
         builder.endObject();
         return builder;
     }

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java

@@ -488,7 +488,7 @@ public class ClusterStatsNodes implements ToXContentFragment {
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params)
                 throws IOException {
-            builder.timeValueField(Fields.MAX_UPTIME_IN_MILLIS, Fields.MAX_UPTIME, maxUptime);
+            builder.humanReadableField(Fields.MAX_UPTIME_IN_MILLIS, Fields.MAX_UPTIME, new TimeValue(maxUptime));
             builder.startArray(Fields.VERSIONS);
             for (ObjectIntCursor<JvmVersion> v : versions) {
                 builder.startObject();

+ 1 - 1
server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java

@@ -198,7 +198,7 @@ public class IndicesSegmentResponse extends BroadcastResponse implements ToXCont
     static void toXContent(XContentBuilder builder, Accountable tree) throws IOException {
         builder.startObject();
         builder.field(Fields.DESCRIPTION, tree.toString());
-        builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, new ByteSizeValue(tree.ramBytesUsed()));
+        builder.humanReadableField(Fields.SIZE_IN_BYTES, Fields.SIZE, new ByteSizeValue(tree.ramBytesUsed()));
         Collection<Accountable> children = tree.getChildResources();
         if (children.isEmpty() == false) {
             builder.startArray(Fields.CHILDREN);

+ 2 - 1
server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java

@@ -24,6 +24,7 @@ import org.elasticsearch.cluster.ClusterState.Custom;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.snapshots.Snapshot;
 
@@ -145,7 +146,7 @@ public class SnapshotDeletionsInProgress extends AbstractNamedDiffable<Custom> i
             {
                 builder.field("repository", entry.snapshot.getRepository());
                 builder.field("snapshot", entry.snapshot.getSnapshotId().getName());
-                builder.timeValueField("start_time_millis", "start_time", entry.startTime);
+                builder.humanReadableField("start_time_millis", "start_time", new TimeValue(entry.startTime));
                 builder.field("repository_state_id", entry.repositoryStateId);
             }
             builder.endObject();

+ 2 - 1
server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java

@@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterState.Custom;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
 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.index.shard.ShardId;
@@ -512,7 +513,7 @@ public class SnapshotsInProgress extends AbstractNamedDiffable<Custom> implement
             }
         }
         builder.endArray();
-        builder.timeValueField(START_TIME_MILLIS, START_TIME, entry.startTime());
+        builder.humanReadableField(START_TIME_MILLIS, START_TIME, new TimeValue(entry.startTime()));
         builder.field(REPOSITORY_STATE_ID, entry.getRepositoryStateId());
         builder.startArray(SHARDS);
         {

+ 4 - 2
server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocateUnassignedDecision.java

@@ -289,8 +289,10 @@ public class AllocateUnassignedDecision extends AbstractAllocationDecision {
             builder.field("allocation_id", allocationId);
         }
         if (allocationStatus == AllocationStatus.DELAYED_ALLOCATION) {
-            builder.timeValueField("configured_delay_in_millis", "configured_delay", TimeValue.timeValueMillis(configuredDelayInMillis));
-            builder.timeValueField("remaining_delay_in_millis", "remaining_delay", TimeValue.timeValueMillis(remainingDelayInMillis));
+            builder.humanReadableField("configured_delay_in_millis", "configured_delay",
+                TimeValue.timeValueMillis(configuredDelayInMillis));
+            builder.humanReadableField("remaining_delay_in_millis", "remaining_delay",
+                TimeValue.timeValueMillis(remainingDelayInMillis));
         }
         nodeDecisionsToXContent(nodeDecisions, builder, params);
         return builder;

+ 48 - 29
server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java

@@ -43,8 +43,8 @@ import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
+import java.util.ServiceLoader;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 /**
  * A utility to build XContent (ie json).
@@ -85,6 +85,7 @@ public final class XContentBuilder implements Releasable, Flushable {
     public static final DateTimeFormatter DEFAULT_DATE_PRINTER = ISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC);
 
     private static final Map<Class<?>, Writer> WRITERS;
+    private static final Map<Class<?>, HumanReadableTransformer> HUMAN_READABLE_TRANSFORMERS;
     static {
         Map<Class<?>, Writer> writers = new HashMap<>();
         writers.put(Boolean.class, (b, v) -> b.value((Boolean) v));
@@ -105,14 +106,43 @@ public final class XContentBuilder implements Releasable, Flushable {
         writers.put(String.class, (b, v) -> b.value((String) v));
         writers.put(String[].class, (b, v) -> b.values((String[]) v));
 
+
+        Map<Class<?>, HumanReadableTransformer> humanReadableTransformer = new HashMap<>();
+        // These will be moved to a different class at a later time to decouple them from XContentBuilder
+        humanReadableTransformer.put(TimeValue.class, v -> ((TimeValue) v).millis());
+        humanReadableTransformer.put(ByteSizeValue.class, v -> ((ByteSizeValue) v).getBytes());
+
+        // Load pluggable extensions
+        for (XContentBuilderExtension service : ServiceLoader.load(XContentBuilderExtension.class)) {
+            Map<Class<?>, Writer> addlWriters = service.getXContentWriters();
+            Map<Class<?>, HumanReadableTransformer> addlTransformers = service.getXContentHumanReadableTransformers();
+
+            addlWriters.forEach((key, value) -> Objects.requireNonNull(value,
+                "invalid null xcontent writer for class " + key));
+            addlTransformers.forEach((key, value) -> Objects.requireNonNull(value,
+                "invalid null xcontent transformer for human readable class " + key));
+
+            writers.putAll(addlWriters);
+            humanReadableTransformer.putAll(addlTransformers);
+        }
+
         WRITERS = Collections.unmodifiableMap(writers);
+        HUMAN_READABLE_TRANSFORMERS = Collections.unmodifiableMap(humanReadableTransformer);
     }
 
     @FunctionalInterface
-    private interface Writer {
+    public interface Writer {
         void write(XContentBuilder builder, Object value) throws IOException;
     }
 
+    /**
+     * Interface for transforming complex objects into their "raw" equivalents for human-readable fields
+     */
+    @FunctionalInterface
+    public interface HumanReadableTransformer {
+        Object rawValue(Object value) throws IOException;
+    }
+
     /**
      * XContentGenerator used to build the XContent object
      */
@@ -856,33 +886,30 @@ public final class XContentBuilder implements Releasable, Flushable {
     }
 
     ////////////////////////////////////////////////////////////////////////////
-    // Misc.
+    // Human readable fields
+    //
+    // These are fields that have a "raw" value and a "human readable" value,
+    // such as time values or byte sizes. The human readable variant is only
+    // used if the humanReadable flag has been set
     //////////////////////////////////
 
-    public XContentBuilder timeValueField(String rawFieldName, String readableFieldName, TimeValue timeValue) throws IOException {
+    public XContentBuilder humanReadableField(String rawFieldName, String readableFieldName, Object value) throws IOException {
         if (humanReadable) {
-            field(readableFieldName, timeValue.toString());
+            field(readableFieldName, Objects.toString(value));
         }
-        field(rawFieldName, timeValue.millis());
-        return this;
-    }
-
-    public XContentBuilder timeValueField(String rawFieldName, String readableFieldName, long rawTime) throws IOException {
-        if (humanReadable) {
-            field(readableFieldName, new TimeValue(rawTime).toString());
+        HumanReadableTransformer transformer = HUMAN_READABLE_TRANSFORMERS.get(value.getClass());
+        if (transformer != null) {
+            Object rawValue = transformer.rawValue(value);
+            field(rawFieldName, rawValue);
+        } else {
+            throw new IllegalArgumentException("no raw transformer found for class " + value.getClass());
         }
-        field(rawFieldName, rawTime);
         return this;
     }
 
-    public XContentBuilder timeValueField(String rawFieldName, String readableFieldName, long rawTime, TimeUnit timeUnit) throws
-            IOException {
-        if (humanReadable) {
-            field(readableFieldName, new TimeValue(rawTime, timeUnit).toString());
-        }
-        field(rawFieldName, rawTime);
-        return this;
-    }
+    ////////////////////////////////////////////////////////////////////////////
+    // Misc.
+    //////////////////////////////////
 
 
     public XContentBuilder percentageField(String rawFieldName, String readableFieldName, double percentage) throws IOException {
@@ -893,14 +920,6 @@ public final class XContentBuilder implements Releasable, Flushable {
         return this;
     }
 
-    public XContentBuilder byteSizeField(String rawFieldName, String readableFieldName, ByteSizeValue byteSizeValue) throws IOException {
-        if (humanReadable) {
-            field(readableFieldName, byteSizeValue.toString());
-        }
-        field(rawFieldName, byteSizeValue.getBytes());
-        return this;
-    }
-
     public XContentBuilder byteSizeField(String rawFieldName, String readableFieldName, long rawSize) throws IOException {
         if (humanReadable) {
             field(readableFieldName, new ByteSizeValue(rawSize).toString());

+ 64 - 0
server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java

@@ -0,0 +1,64 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.common.xcontent;
+
+import java.util.Map;
+
+/**
+ * This interface provides a way for non-JDK classes to plug in a way to serialize to xcontent.
+ *
+ * It is <b>greatly</b> preferred that you implement {@link ToXContentFragment}
+ * in the class for encoding, however, in some situations you may not own the
+ * class, in which case you can add an implementation here for encoding it.
+ */
+public interface XContentBuilderExtension {
+
+    /**
+     * Used for plugging in a generic writer for a class, for example, an example implementation:
+     *
+     * <pre>
+     * {@code
+     *     Map<Class<?>, XContentBuilder.Writer> addlWriters = new HashMap<>();
+     *     addlWriters.put(BytesRef.class, (builder, value) -> b.value(((BytesRef) value).utf8String()));
+     *     return addlWriters;
+     * }
+     * </pre>
+     *
+     * @return a map of class name to writer
+     */
+    Map<Class<?>, XContentBuilder.Writer> getXContentWriters();
+
+    /**
+     * Used for plugging in a human readable version of a class's encoding. It is assumed that
+     * the human readable equivalent is <b>always</b> behind the {@code toString()} method, so
+     * this transformer returns the raw value to be used.
+     *
+     * An example implementation:
+     *
+     * <pre>
+     * {@code
+     *     Map<Class<?>, XContentBuilder.HumanReadableTransformer> transformers = new HashMap<>();
+     *     transformers.put(ByteSizeValue.class, (value) -> ((ByteSizeValue) value).bytes());
+     * }
+     * </pre>
+     * @return a map of class name to transformer used to retrieve raw value
+     */
+    Map<Class<?>, XContentBuilder.HumanReadableTransformer> getXContentHumanReadableTransformers();
+}

+ 1 - 1
server/src/main/java/org/elasticsearch/index/flush/FlushStats.java

@@ -85,7 +85,7 @@ public class FlushStats implements Streamable, ToXContentFragment {
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject(Fields.FLUSH);
         builder.field(Fields.TOTAL, total);
-        builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, totalTimeInMillis);
+        builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, getTotalTime());
         builder.endObject();
         return builder;
     }

+ 3 - 3
server/src/main/java/org/elasticsearch/index/get/GetStats.java

@@ -110,11 +110,11 @@ public class GetStats implements Streamable, ToXContentFragment {
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject(Fields.GET);
         builder.field(Fields.TOTAL, getCount());
-        builder.timeValueField(Fields.TIME_IN_MILLIS, Fields.TIME, getTimeInMillis());
+        builder.humanReadableField(Fields.TIME_IN_MILLIS, Fields.TIME, getTime());
         builder.field(Fields.EXISTS_TOTAL, existsCount);
-        builder.timeValueField(Fields.EXISTS_TIME_IN_MILLIS, Fields.EXISTS_TIME, existsTimeInMillis);
+        builder.humanReadableField(Fields.EXISTS_TIME_IN_MILLIS, Fields.EXISTS_TIME, getExistsTime());
         builder.field(Fields.MISSING_TOTAL, missingCount);
-        builder.timeValueField(Fields.MISSING_TIME_IN_MILLIS, Fields.MISSING_TIME, missingTimeInMillis);
+        builder.humanReadableField(Fields.MISSING_TIME_IN_MILLIS, Fields.MISSING_TIME, getMissingTime());
         builder.field(Fields.CURRENT, current);
         builder.endObject();
         return builder;

+ 3 - 3
server/src/main/java/org/elasticsearch/index/merge/MergeStats.java

@@ -189,11 +189,11 @@ public class MergeStats implements Streamable, ToXContentFragment {
         builder.field(Fields.CURRENT_DOCS, currentNumDocs);
         builder.byteSizeField(Fields.CURRENT_SIZE_IN_BYTES, Fields.CURRENT_SIZE, currentSizeInBytes);
         builder.field(Fields.TOTAL, total);
-        builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, totalTimeInMillis);
+        builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, getTotalTime());
         builder.field(Fields.TOTAL_DOCS, totalNumDocs);
         builder.byteSizeField(Fields.TOTAL_SIZE_IN_BYTES, Fields.TOTAL_SIZE, totalSizeInBytes);
-        builder.timeValueField(Fields.TOTAL_STOPPED_TIME_IN_MILLIS, Fields.TOTAL_STOPPED_TIME, totalStoppedTimeInMillis);
-        builder.timeValueField(Fields.TOTAL_THROTTLED_TIME_IN_MILLIS, Fields.TOTAL_THROTTLED_TIME, totalThrottledTimeInMillis);
+        builder.humanReadableField(Fields.TOTAL_STOPPED_TIME_IN_MILLIS, Fields.TOTAL_STOPPED_TIME, getTotalStoppedTime());
+        builder.humanReadableField(Fields.TOTAL_THROTTLED_TIME_IN_MILLIS, Fields.TOTAL_THROTTLED_TIME, getTotalThrottledTime());
         builder.byteSizeField(Fields.TOTAL_THROTTLE_BYTES_PER_SEC_IN_BYTES, Fields.TOTAL_THROTTLE_BYTES_PER_SEC, totalBytesPerSecAutoThrottle);
         builder.endObject();
         return builder;

+ 1 - 1
server/src/main/java/org/elasticsearch/index/recovery/RecoveryStats.java

@@ -103,7 +103,7 @@ public class RecoveryStats implements ToXContentFragment, Streamable {
         builder.startObject(Fields.RECOVERY);
         builder.field(Fields.CURRENT_AS_SOURCE, currentAsSource());
         builder.field(Fields.CURRENT_AS_TARGET, currentAsTarget());
-        builder.timeValueField(Fields.THROTTLE_TIME_IN_MILLIS, Fields.THROTTLE_TIME, throttleTime());
+        builder.humanReadableField(Fields.THROTTLE_TIME_IN_MILLIS, Fields.THROTTLE_TIME, throttleTime());
         builder.endObject();
         return builder;
     }

+ 1 - 1
server/src/main/java/org/elasticsearch/index/refresh/RefreshStats.java

@@ -96,7 +96,7 @@ public class RefreshStats implements Streamable, ToXContentFragment {
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject("refresh");
         builder.field("total", total);
-        builder.timeValueField("total_time_in_millis", "total_time", totalTimeInMillis);
+        builder.humanReadableField("total_time_in_millis", "total_time", getTotalTime());
         builder.field("listeners", listeners);
         builder.endObject();
         return builder;

+ 2 - 2
server/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java

@@ -387,12 +387,12 @@ public class BulkByScrollTask extends CancellableTask {
                 builder.field("search", searchRetries);
             }
             builder.endObject();
-            builder.timeValueField("throttled_millis", "throttled", throttled);
+            builder.humanReadableField("throttled_millis", "throttled", throttled);
             builder.field("requests_per_second", requestsPerSecond == Float.POSITIVE_INFINITY ? -1 : requestsPerSecond);
             if (reasonCancelled != null) {
                 builder.field("canceled", reasonCancelled);
             }
-            builder.timeValueField("throttled_until_millis", "throttled_until", throttledUntil);
+            builder.humanReadableField("throttled_until_millis", "throttled_until", throttledUntil);
             if (false == sliceStatuses.isEmpty()) {
                 builder.startArray("slices");
                 for (StatusOrException slice : sliceStatuses) {

+ 4 - 4
server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java

@@ -219,19 +219,19 @@ public class SearchStats implements Streamable, ToXContentFragment {
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
             builder.field(Fields.QUERY_TOTAL, queryCount);
-            builder.timeValueField(Fields.QUERY_TIME_IN_MILLIS, Fields.QUERY_TIME, queryTimeInMillis);
+            builder.humanReadableField(Fields.QUERY_TIME_IN_MILLIS, Fields.QUERY_TIME, getQueryTime());
             builder.field(Fields.QUERY_CURRENT, queryCurrent);
 
             builder.field(Fields.FETCH_TOTAL, fetchCount);
-            builder.timeValueField(Fields.FETCH_TIME_IN_MILLIS, Fields.FETCH_TIME, fetchTimeInMillis);
+            builder.humanReadableField(Fields.FETCH_TIME_IN_MILLIS, Fields.FETCH_TIME, getFetchTime());
             builder.field(Fields.FETCH_CURRENT, fetchCurrent);
 
             builder.field(Fields.SCROLL_TOTAL, scrollCount);
-            builder.timeValueField(Fields.SCROLL_TIME_IN_MILLIS, Fields.SCROLL_TIME, scrollTimeInMillis);
+            builder.humanReadableField(Fields.SCROLL_TIME_IN_MILLIS, Fields.SCROLL_TIME, getScrollTime());
             builder.field(Fields.SCROLL_CURRENT, scrollCurrent);
 
             builder.field(Fields.SUGGEST_TOTAL, suggestCount);
-            builder.timeValueField(Fields.SUGGEST_TIME_IN_MILLIS, Fields.SUGGEST_TIME, suggestTimeInMillis);
+            builder.humanReadableField(Fields.SUGGEST_TIME_IN_MILLIS, Fields.SUGGEST_TIME, getSuggestTime());
             builder.field(Fields.SUGGEST_CURRENT, suggestCurrent);
 
             return builder;

+ 3 - 3
server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java

@@ -170,18 +170,18 @@ public class IndexingStats implements Streamable, ToXContentFragment {
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
             builder.field(Fields.INDEX_TOTAL, indexCount);
-            builder.timeValueField(Fields.INDEX_TIME_IN_MILLIS, Fields.INDEX_TIME, indexTimeInMillis);
+            builder.humanReadableField(Fields.INDEX_TIME_IN_MILLIS, Fields.INDEX_TIME, getIndexTime());
             builder.field(Fields.INDEX_CURRENT, indexCurrent);
             builder.field(Fields.INDEX_FAILED, indexFailedCount);
 
             builder.field(Fields.DELETE_TOTAL, deleteCount);
-            builder.timeValueField(Fields.DELETE_TIME_IN_MILLIS, Fields.DELETE_TIME, deleteTimeInMillis);
+            builder.humanReadableField(Fields.DELETE_TIME_IN_MILLIS, Fields.DELETE_TIME, getDeleteTime());
             builder.field(Fields.DELETE_CURRENT, deleteCurrent);
 
             builder.field(Fields.NOOP_UPDATE_TOTAL, noopUpdateCount);
 
             builder.field(Fields.IS_THROTTLED, isThrottled);
-            builder.timeValueField(Fields.THROTTLED_TIME_IN_MILLIS, Fields.THROTTLED_TIME, throttleTimeInMillis);
+            builder.humanReadableField(Fields.THROTTLED_TIME_IN_MILLIS, Fields.THROTTLED_TIME, getThrottleTime());
             return builder;
         }
     }

+ 1 - 1
server/src/main/java/org/elasticsearch/index/warmer/WarmerStats.java

@@ -92,7 +92,7 @@ public class WarmerStats implements Streamable, ToXContentFragment {
         builder.startObject(Fields.WARMER);
         builder.field(Fields.CURRENT, current);
         builder.field(Fields.TOTAL, total);
-        builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, totalTimeInMillis);
+        builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, totalTime());
         builder.endObject();
         return builder;
     }

+ 7 - 7
server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java

@@ -266,7 +266,7 @@ public class RecoveryState implements ToXContentFragment, Streamable {
         if (timer.stopTime > 0) {
             builder.dateField(Fields.STOP_TIME_IN_MILLIS, Fields.STOP_TIME, timer.stopTime);
         }
-        builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, timer.time());
+        builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, new TimeValue(timer.time()));
 
         if (recoverySource.getType() == RecoverySource.Type.PEER) {
             builder.startObject(Fields.SOURCE);
@@ -444,8 +444,8 @@ public class RecoveryState implements ToXContentFragment, Streamable {
 
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
-            builder.timeValueField(Fields.CHECK_INDEX_TIME_IN_MILLIS, Fields.CHECK_INDEX_TIME, checkIndexTime);
-            builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, time());
+            builder.humanReadableField(Fields.CHECK_INDEX_TIME_IN_MILLIS, Fields.CHECK_INDEX_TIME, new TimeValue(checkIndexTime));
+            builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, new TimeValue(time()));
             return builder;
         }
     }
@@ -549,7 +549,7 @@ public class RecoveryState implements ToXContentFragment, Streamable {
             builder.field(Fields.TOTAL, total);
             builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", recoveredPercent()));
             builder.field(Fields.TOTAL_ON_START, totalOnStart);
-            builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, time());
+            builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, new TimeValue(time()));
             return builder;
         }
     }
@@ -924,9 +924,9 @@ public class RecoveryState implements ToXContentFragment, Streamable {
                 builder.endArray();
             }
             builder.endObject();
-            builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, time());
-            builder.timeValueField(Fields.SOURCE_THROTTLE_TIME_IN_MILLIS, Fields.SOURCE_THROTTLE_TIME, sourceThrottling());
-            builder.timeValueField(Fields.TARGET_THROTTLE_TIME_IN_MILLIS, Fields.TARGET_THROTTLE_TIME, targetThrottling());
+            builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, new TimeValue(time()));
+            builder.humanReadableField(Fields.SOURCE_THROTTLE_TIME_IN_MILLIS, Fields.SOURCE_THROTTLE_TIME, sourceThrottling());
+            builder.humanReadableField(Fields.TARGET_THROTTLE_TIME_IN_MILLIS, Fields.TARGET_THROTTLE_TIME, targetThrottling());
             return builder;
         }
 

+ 2 - 1
server/src/main/java/org/elasticsearch/ingest/IngestStats.java

@@ -22,6 +22,7 @@ package org.elasticsearch.ingest;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 
@@ -157,7 +158,7 @@ public class IngestStats implements Writeable, ToXContentFragment {
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
             builder.field("count", ingestCount);
-            builder.timeValueField("time_in_millis", "time", ingestTimeInMillis, TimeUnit.MILLISECONDS);
+            builder.humanReadableField("time_in_millis", "time", new TimeValue(ingestTimeInMillis, TimeUnit.MILLISECONDS));
             builder.field("current", ingestCurrent);
             builder.field("failed", ingestFailedCount);
             return builder;

+ 2 - 2
server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java

@@ -190,7 +190,7 @@ public class JvmStats implements Writeable, ToXContentFragment {
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject(Fields.JVM);
         builder.field(Fields.TIMESTAMP, timestamp);
-        builder.timeValueField(Fields.UPTIME_IN_MILLIS, Fields.UPTIME, uptime);
+        builder.humanReadableField(Fields.UPTIME_IN_MILLIS, Fields.UPTIME, new TimeValue(uptime));
 
         builder.startObject(Fields.MEM);
 
@@ -229,7 +229,7 @@ public class JvmStats implements Writeable, ToXContentFragment {
         for (GarbageCollector collector : gc) {
             builder.startObject(collector.getName());
             builder.field(Fields.COLLECTION_COUNT, collector.getCollectionCount());
-            builder.timeValueField(Fields.COLLECTION_TIME_IN_MILLIS, Fields.COLLECTION_TIME, collector.collectionTime);
+            builder.humanReadableField(Fields.COLLECTION_TIME_IN_MILLIS, Fields.COLLECTION_TIME, new TimeValue(collector.collectionTime));
             builder.endObject();
         }
         builder.endObject();

+ 2 - 1
server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java

@@ -22,6 +22,7 @@ package org.elasticsearch.monitor.os;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent.Params;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -103,7 +104,7 @@ public class OsInfo implements Writeable, ToXContentFragment {
     @Override
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject(Fields.OS);
-        builder.timeValueField(Fields.REFRESH_INTERVAL_IN_MILLIS, Fields.REFRESH_INTERVAL, refreshInterval);
+        builder.humanReadableField(Fields.REFRESH_INTERVAL_IN_MILLIS, Fields.REFRESH_INTERVAL, new TimeValue(refreshInterval));
         if (name != null) {
             builder.field(Fields.NAME, name);
         }

+ 6 - 6
server/src/main/java/org/elasticsearch/monitor/os/OsStats.java

@@ -221,9 +221,9 @@ public class OsStats implements Writeable, ToXContentFragment {
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
             builder.startObject(Fields.SWAP);
-            builder.byteSizeField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, getTotal());
-            builder.byteSizeField(Fields.FREE_IN_BYTES, Fields.FREE, getFree());
-            builder.byteSizeField(Fields.USED_IN_BYTES, Fields.USED, getUsed());
+            builder.humanReadableField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, getTotal());
+            builder.humanReadableField(Fields.FREE_IN_BYTES, Fields.FREE, getFree());
+            builder.humanReadableField(Fields.USED_IN_BYTES, Fields.USED, getUsed());
             builder.endObject();
             return builder;
         }
@@ -273,9 +273,9 @@ public class OsStats implements Writeable, ToXContentFragment {
         @Override
         public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
             builder.startObject(Fields.MEM);
-            builder.byteSizeField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, getTotal());
-            builder.byteSizeField(Fields.FREE_IN_BYTES, Fields.FREE, getFree());
-            builder.byteSizeField(Fields.USED_IN_BYTES, Fields.USED, getUsed());
+            builder.humanReadableField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, getTotal());
+            builder.humanReadableField(Fields.FREE_IN_BYTES, Fields.FREE, getFree());
+            builder.humanReadableField(Fields.USED_IN_BYTES, Fields.USED, getUsed());
             builder.field(Fields.FREE_PERCENT, getFreePercent());
             builder.field(Fields.USED_PERCENT, getUsedPercent());
             builder.endObject();

+ 2 - 1
server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java

@@ -22,6 +22,7 @@ package org.elasticsearch.monitor.process;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent.Params;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -83,7 +84,7 @@ public class ProcessInfo implements Writeable, ToXContentFragment {
     @Override
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
         builder.startObject(Fields.PROCESS);
-        builder.timeValueField(Fields.REFRESH_INTERVAL_IN_MILLIS, Fields.REFRESH_INTERVAL, refreshInterval);
+        builder.humanReadableField(Fields.REFRESH_INTERVAL_IN_MILLIS, Fields.REFRESH_INTERVAL, new TimeValue(refreshInterval));
         builder.field(Fields.ID, id);
         builder.field(Fields.MLOCKALL, mlockall);
         builder.endObject();

+ 1 - 1
server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java

@@ -108,7 +108,7 @@ public class ProcessStats implements Writeable, ToXContentFragment {
         if (cpu != null) {
             builder.startObject(Fields.CPU);
             builder.field(Fields.PERCENT, cpu.percent);
-            builder.timeValueField(Fields.TOTAL_IN_MILLIS, Fields.TOTAL, cpu.total);
+            builder.humanReadableField(Fields.TOTAL_IN_MILLIS, Fields.TOTAL, new TimeValue(cpu.total));
             builder.endObject();
         }
         if (mem != null) {

+ 9 - 2
server/src/main/java/org/elasticsearch/node/AdaptiveSelectionStats.java

@@ -22,6 +22,7 @@ package org.elasticsearch.node;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.common.xcontent.ToXContent.Params;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
@@ -73,8 +74,14 @@ public class AdaptiveSelectionStats implements Writeable, ToXContentFragment {
                 long outgoingSearches = clientOutgoingConnections.getOrDefault(nodeId, 0L);
                 builder.field("outgoing_searches", outgoingSearches);
                 builder.field("avg_queue_size", stats.queueSize);
-                builder.timeValueField("avg_service_time_ns", "avg_service_time", (long) stats.serviceTime, TimeUnit.NANOSECONDS);
-                builder.timeValueField("avg_response_time_ns", "avg_response_time", (long) stats.responseTime, TimeUnit.NANOSECONDS);
+                if (builder.humanReadable()) {
+                    builder.field("avg_service_time", new TimeValue((long) stats.serviceTime, TimeUnit.NANOSECONDS).toString());
+                }
+                builder.field("avg_service_time_ns", (long) stats.serviceTime);
+                if (builder.humanReadable()) {
+                    builder.field("avg_response_time", new TimeValue((long) stats.responseTime, TimeUnit.NANOSECONDS).toString());
+                }
+                builder.field("avg_response_time_ns", (long) stats.responseTime);
                 builder.field("rank", String.format(Locale.ROOT, "%.1f", stats.rank(outgoingSearches)));
             }
             builder.endObject();

+ 2 - 1
server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java

@@ -95,7 +95,8 @@ public class RestClusterStateAction extends BaseRestHandler {
             public RestResponse buildResponse(ClusterStateResponse response, XContentBuilder builder) throws Exception {
                 builder.startObject();
                 builder.field(Fields.CLUSTER_NAME, response.getClusterName().value());
-                builder.byteSizeField(Fields.CLUSTER_STATE_SIZE_IN_BYTES, Fields.CLUSTER_STATE_SIZE, response.getTotalCompressedSize());
+                builder.humanReadableField(Fields.CLUSTER_STATE_SIZE_IN_BYTES, Fields.CLUSTER_STATE_SIZE,
+                        response.getTotalCompressedSize());
                 response.getState().toXContent(builder, request);
                 builder.endObject();
                 return new BytesRestResponse(RestStatus.OK, builder);

+ 9 - 5
server/src/main/java/org/elasticsearch/search/profile/ProfileResult.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContentObject;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -147,11 +148,14 @@ public final class ProfileResult implements Writeable, ToXContentObject {
 
     @Override
     public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
-        builder = builder.startObject()
-                .field(TYPE.getPreferredName(), type)
-                .field(DESCRIPTION.getPreferredName(), description)
-                .timeValueField(NODE_TIME_RAW.getPreferredName(), NODE_TIME.getPreferredName(), getTime(), TimeUnit.NANOSECONDS)
-                .field(BREAKDOWN.getPreferredName(), timings);
+        builder.startObject();
+        builder.field(TYPE.getPreferredName(), type);
+        builder.field(DESCRIPTION.getPreferredName(), description);
+        if (builder.humanReadable()) {
+            builder.field(NODE_TIME.getPreferredName(), new TimeValue(getTime(), TimeUnit.NANOSECONDS).toString());
+        }
+        builder.field(NODE_TIME_RAW.getPreferredName(), getTime());
+        builder.field(BREAKDOWN.getPreferredName(), timings);
 
         if (!children.isEmpty()) {
             builder = builder.startArray(CHILDREN.getPreferredName());

+ 8 - 4
server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java

@@ -23,6 +23,7 @@ import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.ToXContentObject;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -142,10 +143,13 @@ public class CollectorResult implements ToXContentObject, Writeable {
 
     @Override
     public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException {
-        builder = builder.startObject()
-                .field(NAME.getPreferredName(), getName())
-                .field(REASON.getPreferredName(), getReason())
-                .timeValueField(TIME_NANOS.getPreferredName(), TIME.getPreferredName(), getTime(), TimeUnit.NANOSECONDS);
+        builder = builder.startObject();
+        builder.field(NAME.getPreferredName(), getName());
+        builder.field(REASON.getPreferredName(), getReason());
+        if (builder.humanReadable()) {
+            builder.field(TIME.getPreferredName(), new TimeValue(getTime(), TimeUnit.NANOSECONDS).toString());
+        }
+        builder.field(TIME_NANOS.getPreferredName(), getTime());
 
         if (!children.isEmpty()) {
             builder = builder.startArray(CHILDREN.getPreferredName());

+ 2 - 1
server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java

@@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.joda.FormatDateTimeFormatter;
 import org.elasticsearch.common.joda.Joda;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -392,7 +393,7 @@ public final class SnapshotInfo implements Comparable<SnapshotInfo>, ToXContent,
         if (verbose || endTime != 0) {
             builder.field(END_TIME, DATE_TIME_FORMATTER.printer().print(endTime));
             builder.field(END_TIME_IN_MILLIS, endTime);
-            builder.timeValueField(DURATION_IN_MILLIS, DURATION, endTime - startTime);
+            builder.humanReadableField(DURATION_IN_MILLIS, DURATION, new TimeValue(endTime - startTime));
         }
         if (verbose || !shardFailures.isEmpty()) {
             builder.startArray(FAILURES);

+ 5 - 1
server/src/main/java/org/elasticsearch/tasks/TaskInfo.java

@@ -26,6 +26,7 @@ import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ConstructingObjectParser;
 import org.elasticsearch.common.xcontent.ToXContent.Params;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
@@ -196,7 +197,10 @@ public final class TaskInfo implements Writeable, ToXContentFragment {
             builder.field("description", description);
         }
         builder.dateField("start_time_in_millis", "start_time", startTime);
-        builder.timeValueField("running_time_in_nanos", "running_time", runningTimeNanos, TimeUnit.NANOSECONDS);
+        if (builder.humanReadable()) {
+            builder.field("running_time", new TimeValue(runningTimeNanos, TimeUnit.NANOSECONDS).toString());
+        }
+        builder.field("running_time_in_nanos", runningTimeNanos);
         builder.field("cancellable", cancellable);
         if (parentTaskId.isSet()) {
             builder.field("parent_task_id", parentTaskId.toString());