Skip to content

Commit

Permalink
Add pluggable XContentBuilder writers and human readable writers (#29120
Browse files Browse the repository at this point in the history
)

* 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
  • Loading branch information
dakrone committed Mar 20, 2018
1 parent 53d3bc4 commit d644db7
Show file tree
Hide file tree
Showing 32 changed files with 202 additions and 90 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<JvmVersion> v : versions) {
builder.startObject();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<Accountable> children = tree.getChildResources();
if (children.isEmpty() == false) {
builder.startArray(Fields.CHILDREN);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -514,7 +515,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);
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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).
Expand Down Expand Up @@ -85,6 +85,7 @@ public static XContentBuilder builder(XContent xContent, Set<String> includes, S
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));
Expand All @@ -105,14 +106,43 @@ public static XContentBuilder builder(XContent xContent, Set<String> includes, S
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
*/
Expand Down Expand Up @@ -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 {
Expand All @@ -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());
Expand Down
Original file line number Diff line number Diff line change
@@ -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();
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
Loading

0 comments on commit d644db7

Please sign in to comment.