From 4bd217c94f4cea806b1ab41b359908355ac2984c Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 20 Mar 2018 11:39:24 -0600 Subject: [PATCH] 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 --- .../cluster/health/ClusterHealthResponse.java | 2 +- .../cluster/node/info/NodesInfoResponse.java | 2 +- .../snapshots/status/SnapshotStats.java | 3 +- .../cluster/stats/ClusterStatsNodes.java | 2 +- .../segments/IndicesSegmentResponse.java | 2 +- .../cluster/SnapshotDeletionsInProgress.java | 3 +- .../cluster/SnapshotsInProgress.java | 3 +- .../AllocateUnassignedDecision.java | 6 +- .../common/xcontent/XContentBuilder.java | 77 ++++++++++++------- .../xcontent/XContentBuilderExtension.java | 64 +++++++++++++++ .../elasticsearch/index/flush/FlushStats.java | 2 +- .../org/elasticsearch/index/get/GetStats.java | 6 +- .../elasticsearch/index/merge/MergeStats.java | 6 +- .../index/recovery/RecoveryStats.java | 2 +- .../index/refresh/RefreshStats.java | 2 +- .../index/reindex/BulkByScrollTask.java | 4 +- .../index/search/stats/SearchStats.java | 8 +- .../index/shard/IndexingStats.java | 6 +- .../index/warmer/WarmerStats.java | 2 +- .../indices/recovery/RecoveryState.java | 14 ++-- .../org/elasticsearch/ingest/IngestStats.java | 3 +- .../elasticsearch/monitor/jvm/JvmStats.java | 4 +- .../org/elasticsearch/monitor/os/OsInfo.java | 3 +- .../org/elasticsearch/monitor/os/OsStats.java | 12 +-- .../monitor/process/ProcessInfo.java | 3 +- .../monitor/process/ProcessStats.java | 2 +- .../node/AdaptiveSelectionStats.java | 11 ++- .../admin/cluster/RestClusterStateAction.java | 3 +- .../search/profile/ProfileResult.java | 14 ++-- .../search/profile/query/CollectorResult.java | 12 ++- .../elasticsearch/snapshots/SnapshotInfo.java | 3 +- .../org/elasticsearch/tasks/TaskInfo.java | 6 +- 32 files changed, 202 insertions(+), 90 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java index a9a2c36970ee4..74779711c73a9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthResponse.java @@ -245,7 +245,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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"); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java index 952589766773f..09ac7e6aa13d7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java @@ -68,7 +68,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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"); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java index 5b2bdd7c614c6..e7957e0ac0818 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java @@ -22,6 +22,7 @@ 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 XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par 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; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java index f6ff6a1d643d5..2efaf2245ea04 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java @@ -488,7 +488,7 @@ static final class Fields { @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 v : versions) { builder.startObject(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java index b9296c0242fdb..e8e2f5376cd24 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java @@ -198,7 +198,7 @@ static void toXContent(XContentBuilder builder, Sort sort) throws IOException { 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 children = tree.getChildResources(); if (children.isEmpty() == false) { builder.startArray(Fields.CHILDREN); diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java index 981d6128419ba..234d1ef9f17fd 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java @@ -24,6 +24,7 @@ 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 XContentBuilder toXContent(XContentBuilder builder, Params params) throws { 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(); diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java index 2d3886aa52f12..74b748e19a7ee 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java @@ -27,6 +27,7 @@ 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 void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params p } } 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); { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocateUnassignedDecision.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocateUnassignedDecision.java index decdafd724c7d..fc2d81b38c493 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocateUnassignedDecision.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocateUnassignedDecision.java @@ -289,8 +289,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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; diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java b/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java index b5622a9c0d26e..b51add28bf539 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java @@ -43,8 +43,8 @@ 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 static XContentBuilder builder(XContent xContent, Set includes, S public static final DateTimeFormatter DEFAULT_DATE_PRINTER = ISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC); private static final Map, Writer> WRITERS; + private static final Map, HumanReadableTransformer> HUMAN_READABLE_TRANSFORMERS; static { Map, Writer> writers = new HashMap<>(); writers.put(Boolean.class, (b, v) -> b.value((Boolean) v)); @@ -105,14 +106,43 @@ public static XContentBuilder builder(XContent xContent, Set includes, S writers.put(String.class, (b, v) -> b.value((String) v)); writers.put(String[].class, (b, v) -> b.values((String[]) v)); + + Map, 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, Writer> addlWriters = service.getXContentWriters(); + Map, 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 @@ private XContentBuilder value(Iterable values, boolean ensureNoSelfReferences } //////////////////////////////////////////////////////////////////////////// - // 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 XContentBuilder percentageField(String rawFieldName, String readableField 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()); diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java b/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java new file mode 100644 index 0000000000000..610be4585eb9c --- /dev/null +++ b/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 greatly 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: + * + *
+     * {@code
+     *     Map, XContentBuilder.Writer> addlWriters = new HashMap<>();
+     *     addlWriters.put(BytesRef.class, (builder, value) -> b.value(((BytesRef) value).utf8String()));
+     *     return addlWriters;
+     * }
+     * 
+ * + * @return a map of class name to writer + */ + Map, 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 always behind the {@code toString()} method, so + * this transformer returns the raw value to be used. + * + * An example implementation: + * + *
+     * {@code
+     *     Map, XContentBuilder.HumanReadableTransformer> transformers = new HashMap<>();
+     *     transformers.put(ByteSizeValue.class, (value) -> ((ByteSizeValue) value).bytes());
+     * }
+     * 
+ * @return a map of class name to transformer used to retrieve raw value + */ + Map, XContentBuilder.HumanReadableTransformer> getXContentHumanReadableTransformers(); +} diff --git a/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java b/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java index d15a62b0a734f..4b931e47372b7 100644 --- a/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java +++ b/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java @@ -85,7 +85,7 @@ public TimeValue getTotalTime() { 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; } diff --git a/server/src/main/java/org/elasticsearch/index/get/GetStats.java b/server/src/main/java/org/elasticsearch/index/get/GetStats.java index e89f22ea85135..bff1299a348bd 100644 --- a/server/src/main/java/org/elasticsearch/index/get/GetStats.java +++ b/server/src/main/java/org/elasticsearch/index/get/GetStats.java @@ -110,11 +110,11 @@ public long current() { 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; diff --git a/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java b/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java index 94e5b31a1cd28..20329cac98ba0 100644 --- a/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java +++ b/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java @@ -189,11 +189,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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; diff --git a/server/src/main/java/org/elasticsearch/index/recovery/RecoveryStats.java b/server/src/main/java/org/elasticsearch/index/recovery/RecoveryStats.java index 4e3d71cce6299..b86f9c55f7b20 100644 --- a/server/src/main/java/org/elasticsearch/index/recovery/RecoveryStats.java +++ b/server/src/main/java/org/elasticsearch/index/recovery/RecoveryStats.java @@ -103,7 +103,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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; } diff --git a/server/src/main/java/org/elasticsearch/index/refresh/RefreshStats.java b/server/src/main/java/org/elasticsearch/index/refresh/RefreshStats.java index 1235aad885f35..9a4830368bbd6 100644 --- a/server/src/main/java/org/elasticsearch/index/refresh/RefreshStats.java +++ b/server/src/main/java/org/elasticsearch/index/refresh/RefreshStats.java @@ -96,7 +96,7 @@ public int getListeners() { 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; diff --git a/server/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java b/server/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java index 276484b055253..b6a4cc2fd74e2 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java @@ -387,12 +387,12 @@ public XContentBuilder innerXContent(XContentBuilder builder, Params params) 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) { diff --git a/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java b/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java index de0a659d5f1a6..519cd9ff9ae71 100644 --- a/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java +++ b/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java @@ -219,19 +219,19 @@ public void writeTo(StreamOutput out) throws IOException { @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; diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java b/server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java index 26eb8b469f52b..fa658c3600ea4 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexingStats.java @@ -170,18 +170,18 @@ public void writeTo(StreamOutput out) throws IOException { @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; } } diff --git a/server/src/main/java/org/elasticsearch/index/warmer/WarmerStats.java b/server/src/main/java/org/elasticsearch/index/warmer/WarmerStats.java index 8149b091a3148..63b0fe37a9b28 100644 --- a/server/src/main/java/org/elasticsearch/index/warmer/WarmerStats.java +++ b/server/src/main/java/org/elasticsearch/index/warmer/WarmerStats.java @@ -92,7 +92,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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; } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index a89fdcacb2bc3..3eb45318d7a19 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -266,7 +266,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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 void writeTo(StreamOutput out) throws IOException { @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 synchronized XContentBuilder toXContent(XContentBuilder builder, Params p 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 synchronized XContentBuilder toXContent(XContentBuilder builder, Params p 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; } diff --git a/server/src/main/java/org/elasticsearch/ingest/IngestStats.java b/server/src/main/java/org/elasticsearch/ingest/IngestStats.java index fd0d7e826c070..c4c1520fd19d4 100644 --- a/server/src/main/java/org/elasticsearch/ingest/IngestStats.java +++ b/server/src/main/java/org/elasticsearch/ingest/IngestStats.java @@ -22,6 +22,7 @@ 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 long getIngestFailedCount() { @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; diff --git a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java index 712b594b47d52..b548afadd3d88 100644 --- a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java +++ b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java @@ -190,7 +190,7 @@ public Classes getClasses() { 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 XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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(); diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java b/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java index 7046b35839098..0c81356167440 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java @@ -22,6 +22,7 @@ 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 @@ static final class Fields { @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); } diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java b/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java index 60502679c2131..637f4cf1cbe00 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java @@ -221,9 +221,9 @@ public ByteSizeValue getTotal() { @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 short getFreePercent() { @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(); diff --git a/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java b/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java index 5d74f576181d5..6bac78d2f140c 100644 --- a/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/process/ProcessInfo.java @@ -22,6 +22,7 @@ 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 @@ static final class Fields { @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(); diff --git a/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java b/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java index f85e013bc426b..1d051aac7b0c8 100644 --- a/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java +++ b/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java @@ -108,7 +108,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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) { diff --git a/server/src/main/java/org/elasticsearch/node/AdaptiveSelectionStats.java b/server/src/main/java/org/elasticsearch/node/AdaptiveSelectionStats.java index 3deb161cc8e97..819b3365ce066 100644 --- a/server/src/main/java/org/elasticsearch/node/AdaptiveSelectionStats.java +++ b/server/src/main/java/org/elasticsearch/node/AdaptiveSelectionStats.java @@ -22,6 +22,7 @@ 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 XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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(); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java index 5e3743637178e..6e55ef3671ba0 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java @@ -95,7 +95,8 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC 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); diff --git a/server/src/main/java/org/elasticsearch/search/profile/ProfileResult.java b/server/src/main/java/org/elasticsearch/search/profile/ProfileResult.java index 13dfa712ab448..d0965bca1a9b2 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/ProfileResult.java +++ b/server/src/main/java/org/elasticsearch/search/profile/ProfileResult.java @@ -23,6 +23,7 @@ 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 List getProfiledChildren() { @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()); diff --git a/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java b/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java index d553e1a8a7359..19d382dd8f380 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java +++ b/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java @@ -23,6 +23,7 @@ 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 List getProfiledChildren() { @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()); diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index b75b840e8b255..073007f4225df 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -28,6 +28,7 @@ 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 XContentBuilder toXContent(final XContentBuilder builder, final Params pa 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); diff --git a/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java b/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java index 19e9baedd753b..bc40df2b8f0c4 100644 --- a/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java +++ b/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java @@ -26,6 +26,7 @@ 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 XContentBuilder toXContent(XContentBuilder builder, Params params) throws 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());