diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterStateRestCancellationIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterStateRestCancellationIT.java index a0a4642d25ee7..d13b7eda30d24 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterStateRestCancellationIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterStateRestCancellationIT.java @@ -24,10 +24,11 @@ import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.tasks.TaskInfo; -import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.ToXContent; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.concurrent.CancellationException; import java.util.function.UnaryOperator; @@ -116,7 +117,7 @@ public void writeTo(StreamOutput out) { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) { + public Iterator toXContentChunked(ToXContent.Params params) { throw new AssertionError("task should have been cancelled before serializing this custom"); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleClusterStateIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleClusterStateIT.java index 0be8f79a3e1c4..e596db1e88098 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleClusterStateIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/SimpleClusterStateIT.java @@ -42,6 +42,7 @@ import org.elasticsearch.tracing.Tracer; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentFactory; import org.junit.Before; @@ -50,6 +51,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; @@ -401,8 +403,8 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - return builder; + public Iterator toXContentChunked(ToXContent.Params params) { + return Collections.emptyIterator(); } static NamedDiff readDiffFrom(StreamInput in) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java index 3ece7e345ffd6..0fc23ae86a522 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -43,7 +43,9 @@ import org.elasticsearch.common.io.stream.VersionedNamedWriteable; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.core.Nullable; +import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContentFragment; import org.elasticsearch.xcontent.XContent; import org.elasticsearch.xcontent.XContentBuilder; @@ -51,6 +53,7 @@ import java.io.IOException; import java.util.EnumSet; import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -105,7 +108,7 @@ public class ClusterState implements ToXContentFragment, Diffable public static final ClusterState EMPTY_STATE = builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build(); - public interface Custom extends NamedDiffable, ToXContentFragment { + public interface Custom extends NamedDiffable, ChunkedToXContent { /** * Returns true iff this {@link Custom} is private to the cluster and should never be send to a client. @@ -121,7 +124,7 @@ default boolean isPrivate() { * the more faithful it is the more useful it is for diagnostics. */ @Override - XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException; + Iterator toXContentChunked(Params params); } private static final NamedDiffableValueSerializer CUSTOM_VALUE_SERIALIZER = new NamedDiffableValueSerializer<>(Custom.class); @@ -619,7 +622,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (metrics.contains(Metric.CUSTOMS)) { for (Map.Entry cursor : customs.entrySet()) { builder.startObject(cursor.getKey()); - cursor.getValue().toXContent(builder, params); + ChunkedToXContent.wrapAsXContentObject(cursor.getValue()).toXContent(builder, params); builder.endObject(); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java b/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java index 948e055e06ee2..47db3993f95fc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/RepositoryCleanupInProgress.java @@ -9,13 +9,15 @@ import org.elasticsearch.Version; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.repositories.RepositoryOperation; -import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.ToXContent; import java.io.IOException; +import java.util.Iterator; import java.util.List; public final class RepositoryCleanupInProgress extends AbstractNamedDiffable implements ClusterState.Custom { @@ -62,17 +64,17 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startArray(TYPE); - for (Entry entry : entries) { - builder.startObject(); - { + public Iterator toXContentChunked(ToXContent.Params ignored) { + return Iterators.concat( + Iterators.single((builder, params) -> builder.startArray(TYPE)), + entries.stream().map(entry -> (builder, params) -> { + builder.startObject(); builder.field("repository", entry.repository); - } - builder.endObject(); - } - builder.endArray(); - return builder; + builder.endObject(); + return builder; + }).iterator(), + Iterators.single((builder, params) -> builder.endArray()) + ); } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java b/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java index 3b19fee0210bd..ab7a4a9589b92 100644 --- a/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java @@ -11,6 +11,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; import org.elasticsearch.cluster.ClusterState.Custom; +import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -18,7 +19,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.xcontent.ToXContent; -import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; import java.util.Collections; @@ -398,49 +398,41 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startArray("snapshots"); - for (Entry entry : entries.values()) { - toXContent(entry, builder); - } - builder.endArray(); - return builder; - } - - /** - * Serializes single restore operation - * - * @param entry restore operation metadata - * @param builder XContent builder - */ - public static void toXContent(Entry entry, XContentBuilder builder) throws IOException { - builder.startObject(); - builder.field("snapshot", entry.snapshot().getSnapshotId().getName()); - builder.field("repository", entry.snapshot().getRepository()); - builder.field("state", entry.state()); - builder.startArray("indices"); - { - for (String index : entry.indices()) { - builder.value(index); - } - } - builder.endArray(); - builder.startArray("shards"); - { - for (Map.Entry shardEntry : entry.shards.entrySet()) { - ShardId shardId = shardEntry.getKey(); - ShardRestoreStatus status = shardEntry.getValue(); + public Iterator toXContentChunked(ToXContent.Params ignored) { + return Iterators.concat( + Iterators.single((builder, params) -> builder.startArray("snapshots")), + entries.values().stream().map(entry -> (builder, params) -> { builder.startObject(); + builder.field("snapshot", entry.snapshot().getSnapshotId().getName()); + builder.field("repository", entry.snapshot().getRepository()); + builder.field("state", entry.state()); + builder.startArray("indices"); { - builder.field("index", shardId.getIndex()); - builder.field("shard", shardId.getId()); - builder.field("state", status.state()); + for (String index : entry.indices()) { + builder.value(index); + } + } + builder.endArray(); + builder.startArray("shards"); + { + for (Map.Entry shardEntry : entry.shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardRestoreStatus status = shardEntry.getValue(); + builder.startObject(); + { + builder.field("index", shardId.getIndex()); + builder.field("shard", shardId.getId()); + builder.field("state", status.state()); + } + builder.endObject(); + } } - builder.endObject(); - } - } - builder.endArray(); - builder.endObject(); + builder.endArray(); + builder.endObject(); + return builder; + }).iterator(), + Iterators.single((builder, params) -> builder.endArray()) + ); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java index ba9531fbe2975..26fb5d20eacea 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java @@ -11,19 +11,21 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState.Custom; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.collect.Iterators; 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.util.CollectionUtils; import org.elasticsearch.repositories.RepositoryOperation; import org.elasticsearch.snapshots.SnapshotId; -import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.ToXContent; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Objects; import java.util.Set; @@ -160,25 +162,27 @@ public Version getMinimalSupportedVersion() { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startArray(TYPE); - for (Entry entry : entries) { - builder.startObject(); - { - builder.field("repository", entry.repository()); - builder.startArray("snapshots"); - for (SnapshotId snapshot : entry.snapshots) { - builder.value(snapshot.getName()); + public Iterator toXContentChunked(ToXContent.Params ignored) { + return Iterators.concat( + Iterators.single((builder, params) -> builder.startArray(TYPE)), + entries.stream().map(entry -> (builder, params) -> { + builder.startObject(); + { + builder.field("repository", entry.repository()); + builder.startArray("snapshots"); + for (SnapshotId snapshot : entry.snapshots) { + builder.value(snapshot.getName()); + } + builder.endArray(); + builder.timeField("start_time_millis", "start_time", entry.startTime); + builder.field("repository_state_id", entry.repositoryStateId); + builder.field("state", entry.state); } - builder.endArray(); - builder.timeField("start_time_millis", "start_time", entry.startTime); - builder.field("repository_state_id", entry.repositoryStateId); - builder.field("state", entry.state); - } - builder.endObject(); - } - builder.endArray(); - return builder; + builder.endObject(); + return builder; + }).iterator(), + Iterators.single((builder, params) -> builder.endArray()) + ); } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java index 279e7ba4a6b91..1256032b050b5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java @@ -12,6 +12,7 @@ import org.elasticsearch.cluster.ClusterState.Custom; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -212,14 +213,12 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startArray("snapshots"); - final Iterator iterator = asStream().iterator(); - while (iterator.hasNext()) { - iterator.next().toXContent(builder, params); - } - builder.endArray(); - return builder; + public Iterator toXContentChunked(ToXContent.Params ignored) { + return Iterators.concat( + Iterators.single((builder, params) -> builder.startArray("snapshots")), + asStream().iterator(), + Iterators.single((builder, params) -> builder.endArray()) + ); } @Override diff --git a/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadata.java b/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadata.java index 4c64e6f404a59..9c69792914247 100644 --- a/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadata.java +++ b/server/src/main/java/org/elasticsearch/health/metadata/HealthMetadata.java @@ -12,16 +12,19 @@ import org.elasticsearch.cluster.AbstractNamedDiffable; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.NamedDiff; +import org.elasticsearch.common.collect.Iterators; 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.ByteSizeValue; import org.elasticsearch.common.unit.RelativeByteSizeValue; import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContentFragment; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; +import java.util.Iterator; import java.util.Objects; /** @@ -63,22 +66,19 @@ public static NamedDiff readDiffFrom(StreamInput in) throws } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(DISK_METADATA.getPreferredName()); - diskMetadata.toXContent(builder, params); - builder.endObject(); - return builder; + public Iterator toXContentChunked(ToXContent.Params ignored) { + return Iterators.single((builder, params) -> { + builder.startObject(DISK_METADATA.getPreferredName()); + diskMetadata.toXContent(builder, params); + builder.endObject(); + return builder; + }); } public static HealthMetadata getFromClusterState(ClusterState clusterState) { return clusterState.custom(HealthMetadata.TYPE); } - @Override - public boolean isFragment() { - return true; - } - public Disk getDiskMetadata() { return diskMetadata; } diff --git a/server/src/test/java/org/elasticsearch/cluster/RepositoryCleanupInProgressTests.java b/server/src/test/java/org/elasticsearch/cluster/RepositoryCleanupInProgressTests.java new file mode 100644 index 0000000000000..d001a7c5a2629 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/RepositoryCleanupInProgressTests.java @@ -0,0 +1,38 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.ToXContent; + +import java.io.IOException; + +import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS; +import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; + +public class RepositoryCleanupInProgressTests extends ESTestCase { + public void testChunking() throws IOException { + final var instance = new RepositoryCleanupInProgress( + randomList(10, () -> new RepositoryCleanupInProgress.Entry(randomAlphaOfLength(10), randomNonNegativeLong())) + ); + + int chunkCount = 0; + try (var builder = jsonBuilder()) { + builder.startObject(); + final var iterator = instance.toXContentChunked(EMPTY_PARAMS); + while (iterator.hasNext()) { + iterator.next().toXContent(builder, ToXContent.EMPTY_PARAMS); + chunkCount += 1; + } + builder.endObject(); + } // closing the builder verifies that the XContent is well-formed + + assertEquals(instance.entries().size() + 2, chunkCount); + } +} diff --git a/server/src/test/java/org/elasticsearch/cluster/RestoreInProgressTests.java b/server/src/test/java/org/elasticsearch/cluster/RestoreInProgressTests.java new file mode 100644 index 0000000000000..a9383fe08af1a --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/RestoreInProgressTests.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.ToXContent; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS; +import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; + +public class RestoreInProgressTests extends ESTestCase { + public void testChunking() throws IOException { + final var ripBuilder = new RestoreInProgress.Builder(); + final var entryCount = between(0, 5); + for (int i = 0; i < entryCount; i++) { + ripBuilder.add( + new RestoreInProgress.Entry( + "uuid-" + i, + new Snapshot(randomAlphaOfLength(10), new SnapshotId(randomAlphaOfLength(10), randomAlphaOfLength(10))), + randomFrom(RestoreInProgress.State.values()), + randomBoolean(), + List.of(), + Map.of() + ) + ); + } + + final var instance = ripBuilder.build(); + + int chunkCount = 0; + try (var builder = jsonBuilder()) { + builder.startObject(); + final var iterator = instance.toXContentChunked(EMPTY_PARAMS); + while (iterator.hasNext()) { + iterator.next().toXContent(builder, ToXContent.EMPTY_PARAMS); + chunkCount += 1; + } + builder.endObject(); + } // closing the builder verifies that the XContent is well-formed + + assertEquals(entryCount + 2, chunkCount); + } +} diff --git a/server/src/test/java/org/elasticsearch/cluster/SnapshotDeletionsInProgressTests.java b/server/src/test/java/org/elasticsearch/cluster/SnapshotDeletionsInProgressTests.java index b9946a2bd6038..8c18ee15afa78 100644 --- a/server/src/test/java/org/elasticsearch/cluster/SnapshotDeletionsInProgressTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/SnapshotDeletionsInProgressTests.java @@ -9,6 +9,7 @@ package org.elasticsearch.cluster; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.ToXContent; @@ -18,6 +19,7 @@ import java.util.Collections; import java.util.List; +import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; @@ -38,7 +40,7 @@ public void testXContent() throws IOException { try (XContentBuilder builder = jsonBuilder()) { builder.humanReadable(true); builder.startObject(); - sdip.toXContent(builder, ToXContent.EMPTY_PARAMS); + ChunkedToXContent.wrapAsXContentObject(sdip).toXContent(builder, ToXContent.EMPTY_PARAMS); builder.endObject(); String json = Strings.toString(builder); assertThat(json, equalTo(XContentHelper.stripWhitespace(""" @@ -56,4 +58,32 @@ public void testXContent() throws IOException { }"""))); } } + + public void testChunking() throws IOException { + final var instance = SnapshotDeletionsInProgress.of( + randomList( + 10, + () -> new SnapshotDeletionsInProgress.Entry( + Collections.emptyList(), + randomAlphaOfLength(10), + randomNonNegativeLong(), + randomNonNegativeLong(), + randomFrom(SnapshotDeletionsInProgress.State.values()) + ) + ) + ); + + int chunkCount = 0; + try (var builder = jsonBuilder()) { + builder.startObject(); + final var iterator = instance.toXContentChunked(EMPTY_PARAMS); + while (iterator.hasNext()) { + iterator.next().toXContent(builder, ToXContent.EMPTY_PARAMS); + chunkCount += 1; + } + builder.endObject(); + } // closing the builder verifies that the XContent is well-formed + + assertEquals(instance.getEntries().size() + 2, chunkCount); + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index 5bf0b302991b5..e24be3b79d5ab 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -44,11 +44,13 @@ import org.elasticsearch.test.MockLogAppender; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.ToXContent; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -1179,10 +1181,8 @@ class DelayedCustom extends AbstractNamedDiffable implement } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.endObject(); - return builder; + public Iterator toXContentChunked(ToXContent.Params params) { + return Collections.emptyIterator(); } @Override @@ -1979,8 +1979,8 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - return builder; + public Iterator toXContentChunked(ToXContent.Params params) { + return Collections.emptyIterator(); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinValidationServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinValidationServiceTests.java index 0b0ee295cea94..be076eb668caa 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinValidationServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinValidationServiceTests.java @@ -41,10 +41,11 @@ import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.ToXContent; import java.util.ArrayList; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Random; import java.util.Set; @@ -232,8 +233,8 @@ public void testJoinValidationRejectsUnreadableClusterState() { class BadCustom implements SimpleDiffable, ClusterState.Custom { @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) { - return builder; + public Iterator toXContentChunked(ToXContent.Params params) { + return Collections.emptyIterator(); } @Override diff --git a/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index 2b676bc4d3e3b..4b88627cc0260 100644 --- a/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -36,11 +37,12 @@ import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.VersionUtils; -import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.ToXContent; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -285,13 +287,12 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - { - builder.field("custom_string_object", strObject); - } - builder.endObject(); - return builder; + public Iterator toXContentChunked(ToXContent.Params ignored) { + return Iterators.concat( + Iterators.single((builder, params) -> builder.startObject()), + Iterators.single((builder, params) -> builder.field("custom_string_object", strObject)), + Iterators.single((builder, params) -> builder.endObject()) + ); } @Override @@ -329,13 +330,12 @@ public void writeTo(StreamOutput out) throws IOException { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - { - builder.field("custom_integer_object", intObject); - } - builder.endObject(); - return builder; + public Iterator toXContentChunked(ToXContent.Params ignored) { + return Iterators.concat( + Iterators.single((builder, params) -> builder.startObject()), + Iterators.single((builder, params) -> builder.field("custom_integer_object", intObject)), + Iterators.single((builder, params) -> builder.endObject()) + ); } @Override diff --git a/server/src/test/java/org/elasticsearch/health/metadata/HealthMetadataSerializationTests.java b/server/src/test/java/org/elasticsearch/health/metadata/HealthMetadataSerializationTests.java index 3661726f3d725..433a4a784efc9 100644 --- a/server/src/test/java/org/elasticsearch/health/metadata/HealthMetadataSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/health/metadata/HealthMetadataSerializationTests.java @@ -16,9 +16,14 @@ import org.elasticsearch.common.unit.RatioValue; import org.elasticsearch.common.unit.RelativeByteSizeValue; import org.elasticsearch.test.SimpleDiffableWireSerializationTestCase; +import org.elasticsearch.xcontent.ToXContent; +import java.io.IOException; import java.util.List; +import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS; +import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; + public class HealthMetadataSerializationTests extends SimpleDiffableWireSerializationTestCase { @Override @@ -102,4 +107,21 @@ static HealthMetadata.Disk mutateDiskMetadata(HealthMetadata.Disk base) { private HealthMetadata mutate(HealthMetadata base) { return new HealthMetadata(mutateDiskMetadata(base.getDiskMetadata())); } + + public void testToXContentChunking() throws IOException { + final var instance = createTestInstance(); + + int chunkCount = 0; + try (var builder = jsonBuilder()) { + builder.startObject(); + final var iterator = instance.toXContentChunked(EMPTY_PARAMS); + while (iterator.hasNext()) { + iterator.next().toXContent(builder, ToXContent.EMPTY_PARAMS); + chunkCount += 1; + } + builder.endObject(); + } // closing the builder verifies that the XContent is well-formed + + assertEquals(1, chunkCount); + } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsInProgressSerializationTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsInProgressSerializationTests.java index afc5005f8fd1c..2842c593c6a7e 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsInProgressSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsInProgressSerializationTests.java @@ -42,6 +42,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; @@ -418,115 +419,123 @@ public void testXContent() throws IOException { ) ); + String json; + long chunkCount = 0; try (XContentBuilder builder = jsonBuilder()) { builder.humanReadable(true); builder.startObject(); - sip.toXContent(builder, ToXContent.EMPTY_PARAMS); + final var iterator = sip.toXContentChunked(EMPTY_PARAMS); + while (iterator.hasNext()) { + iterator.next().toXContent(builder, ToXContent.EMPTY_PARAMS); + chunkCount += 1; + } builder.endObject(); - String json = Strings.toString(builder); - assertThat( - json, - anyOf( - equalTo(XContentHelper.stripWhitespace(""" + json = Strings.toString(builder); + } + + assertEquals(2 + sip.asStream().count(), chunkCount); + assertThat( + json, + anyOf( + equalTo(XContentHelper.stripWhitespace(""" + { + "snapshots": [ { - "snapshots": [ + "repository": "repo", + "snapshot": "name", + "uuid": "uuid", + "include_global_state": true, + "partial": true, + "state": "SUCCESS", + "indices": [ { "name": "index", "id": "uuid" } ], + "start_time": "1970-01-01T00:20:34.567Z", + "start_time_millis": 1234567, + "repository_state_id": 0, + "shards": [ { - "repository": "repo", - "snapshot": "name", - "uuid": "uuid", - "include_global_state": true, - "partial": true, + "index": { + "index_name": "index", + "index_uuid": "uuid" + }, + "shard": 0, "state": "SUCCESS", - "indices": [ { "name": "index", "id": "uuid" } ], - "start_time": "1970-01-01T00:20:34.567Z", - "start_time_millis": 1234567, - "repository_state_id": 0, - "shards": [ - { - "index": { - "index_name": "index", - "index_uuid": "uuid" - }, - "shard": 0, - "state": "SUCCESS", - "generation": "shardgen", - "node": "nodeId", - "result": { - "generation": "shardgen", - "size": "1b", - "size_in_bytes": 1, - "segments": 1 - } - }, - { - "index": { - "index_name": "index", - "index_uuid": "uuid" - }, - "shard": 1, - "state": "FAILED", - "generation": "fail-gen", - "node": "nodeId", - "reason": "failure-reason" - } - ], - "feature_states": [], - "data_streams": [] + "generation": "shardgen", + "node": "nodeId", + "result": { + "generation": "shardgen", + "size": "1b", + "size_in_bytes": 1, + "segments": 1 + } + }, + { + "index": { + "index_name": "index", + "index_uuid": "uuid" + }, + "shard": 1, + "state": "FAILED", + "generation": "fail-gen", + "node": "nodeId", + "reason": "failure-reason" } - ] - }""")), - // or the shards might be in the other order: - equalTo(XContentHelper.stripWhitespace(""" + ], + "feature_states": [], + "data_streams": [] + } + ] + }""")), + // or the shards might be in the other order: + equalTo(XContentHelper.stripWhitespace(""" + { + "snapshots": [ { - "snapshots": [ + "repository": "repo", + "snapshot": "name", + "uuid": "uuid", + "include_global_state": true, + "partial": true, + "state": "SUCCESS", + "indices": [ { "name": "index", "id": "uuid" } ], + "start_time": "1970-01-01T00:20:34.567Z", + "start_time_millis": 1234567, + "repository_state_id": 0, + "shards": [ + { + "index": { + "index_name": "index", + "index_uuid": "uuid" + }, + "shard": 1, + "state": "FAILED", + "generation": "fail-gen", + "node": "nodeId", + "reason": "failure-reason" + }, { - "repository": "repo", - "snapshot": "name", - "uuid": "uuid", - "include_global_state": true, - "partial": true, + "index": { + "index_name": "index", + "index_uuid": "uuid" + }, + "shard": 0, "state": "SUCCESS", - "indices": [ { "name": "index", "id": "uuid" } ], - "start_time": "1970-01-01T00:20:34.567Z", - "start_time_millis": 1234567, - "repository_state_id": 0, - "shards": [ - { - "index": { - "index_name": "index", - "index_uuid": "uuid" - }, - "shard": 1, - "state": "FAILED", - "generation": "fail-gen", - "node": "nodeId", - "reason": "failure-reason" - }, - { - "index": { - "index_name": "index", - "index_uuid": "uuid" - }, - "shard": 0, - "state": "SUCCESS", - "generation": "shardgen", - "node": "nodeId", - "result": { - "generation": "shardgen", - "size": "1b", - "size_in_bytes": 1, - "segments": 1 - } - } - ], - "feature_states": [], - "data_streams": [] + "generation": "shardgen", + "node": "nodeId", + "result": { + "generation": "shardgen", + "size": "1b", + "size_in_bytes": 1, + "segments": 1 + } } - ] - }""")) - ) - ); - } + ], + "feature_states": [], + "data_streams": [] + } + ] + }""")) + ) + ); } public static State randomState(Map shards) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/TokenMetadata.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/TokenMetadata.java index 58366184a4911..9d5017209edca 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/TokenMetadata.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authc/TokenMetadata.java @@ -12,10 +12,12 @@ import org.elasticsearch.cluster.NamedDiff; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.ToXContent; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; import java.util.List; public final class TokenMetadata extends AbstractNamedDiffable implements ClusterState.Custom { @@ -63,9 +65,9 @@ public String getWriteableName() { } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + public Iterator toXContentChunked(ToXContent.Params params) { // never render this to the user - return builder; + return Collections.emptyIterator(); } @Override