From 0daf483587351810c24f458e7ecc2c982e37d854 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 2 Nov 2016 13:43:39 +0100 Subject: [PATCH] Change ClusterState and PendingClusterTasksResponse's toString() to their prettyPrint format (#21245) The current XContent output is much harder to read than the prettyPrint format. This commit folds prettyPrint into toString and removes it. --- .../tasks/PendingClusterTasksResponse.java | 17 +-------- .../elasticsearch/cluster/ClusterState.java | 34 ++++++++--------- .../action/shard/ShardStateAction.java | 2 +- .../cluster/block/ClusterBlocks.java | 3 +- .../cluster/node/DiscoveryNodes.java | 9 ----- .../cluster/routing/RoutingNodes.java | 3 +- .../cluster/routing/RoutingService.java | 4 +- .../cluster/routing/RoutingTable.java | 3 +- .../allocation/IndexMetaDataUpdater.java | 2 +- .../cluster/service/ClusterService.java | 16 ++++---- .../discovery/zen/NodeJoinController.java | 2 +- .../BroadcastReplicationTests.java | 15 ++++---- .../ReplicationOperationTests.java | 7 ++-- .../TransportReplicationActionTests.java | 22 +++++------ .../elasticsearch/cluster/NoMasterNodeIT.java | 2 +- .../health/ClusterStateHealthTests.java | 4 +- .../cluster/node/DiscoveryNodesTests.java | 4 +- .../cluster/routing/PrimaryAllocationIT.java | 4 +- .../cluster/routing/UnassignedInfoTests.java | 2 +- .../allocation/AddIncrementallyTests.java | 6 +-- .../NodeVersionAllocationDeciderTests.java | 6 +-- .../RandomAllocationDeciderTests.java | 2 +- .../allocation/StartedShardsRoutingTests.java | 7 ++-- .../ClusterSerializationTests.java | 6 +-- .../ClusterStateToStringTests.java | 5 ++- .../DiscoveryWithServiceDisruptionsIT.java | 12 +++--- .../discovery/ZenUnicastDiscoveryIT.java | 2 +- .../zen/NodeJoinControllerTests.java | 4 +- .../zen/PublishClusterStateActionTests.java | 6 +-- .../index/store/CorruptedFileIT.java | 11 ++++-- ...ClusterStateServiceRandomUpdatesTests.java | 6 +-- .../recovery/FullRollingRestartIT.java | 6 ++- .../cluster/ESAllocationTestCase.java | 2 +- .../elasticsearch/test/ESIntegTestCase.java | 38 +++++++++---------- .../test/ESSingleNodeTestCase.java | 3 +- 35 files changed, 127 insertions(+), 150 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java index ae4a59fa83f..bb1afe5e19e 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import java.io.IOException; import java.util.ArrayList; @@ -59,7 +58,8 @@ public class PendingClusterTasksResponse extends ActionResponse implements Itera return pendingTasks.iterator(); } - public String prettyPrint() { + @Override + public String toString() { StringBuilder sb = new StringBuilder(); sb.append("tasks: (").append(pendingTasks.size()).append("):\n"); for (PendingClusterTask pendingClusterTask : this) { @@ -68,19 +68,6 @@ public class PendingClusterTasksResponse extends ActionResponse implements Itera return sb.toString(); } - @Override - public String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return builder.string(); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; - } - } - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startArray(Fields.TASKS); diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java index 3bbd56d9f49..82fb6476264 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -274,15 +274,16 @@ public class ClusterState implements ToXContent, Diffable { return routingNodes; } - public String prettyPrint() { + @Override + public String toString() { StringBuilder sb = new StringBuilder(); sb.append("cluster uuid: ").append(metaData.clusterUUID()).append("\n"); sb.append("version: ").append(version).append("\n"); sb.append("state uuid: ").append(stateUUID).append("\n"); sb.append("from_diff: ").append(wasReadFromDiff).append("\n"); sb.append("meta data version: ").append(metaData.version()).append("\n"); + final String TAB = " "; for (IndexMetaData indexMetaData : metaData) { - final String TAB = " "; sb.append(TAB).append(indexMetaData.getIndex()); sb.append(": v[").append(indexMetaData.getVersion()).append("]\n"); for (int shard = 0; shard < indexMetaData.getNumberOfShards(); shard++) { @@ -291,24 +292,19 @@ public class ClusterState implements ToXContent, Diffable { sb.append("isa_ids ").append(indexMetaData.inSyncAllocationIds(shard)).append("\n"); } } - sb.append(blocks().prettyPrint()); - sb.append(nodes().prettyPrint()); - sb.append(routingTable().prettyPrint()); - sb.append(getRoutingNodes().prettyPrint()); - return sb.toString(); - } - - @Override - public String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return builder.string(); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; + sb.append(blocks()); + sb.append(nodes()); + sb.append(routingTable()); + sb.append(getRoutingNodes()); + if (customs.isEmpty() == false) { + sb.append("customs:\n"); + for (ObjectObjectCursor cursor : customs) { + final String type = cursor.key; + final Custom custom = cursor.value; + sb.append(TAB).append(type).append(": ").append(custom); + } } + return sb.toString(); } /** diff --git a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index ce6473ecb42..ee56d7a61a1 100644 --- a/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -164,7 +164,7 @@ public class ShardStateAction extends AbstractComponent { @Override public void onNewClusterState(ClusterState state) { if (logger.isTraceEnabled()) { - logger.trace("new cluster state [{}] after waiting for master election to fail shard entry [{}]", state.prettyPrint(), shardEntry); + logger.trace("new cluster state [{}] after waiting for master election to fail shard entry [{}]", state, shardEntry); } sendShardAction(actionName, observer, shardEntry, listener); } diff --git a/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java b/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java index e6f04c8702c..12e6ee0f7ec 100644 --- a/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java +++ b/core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java @@ -199,7 +199,8 @@ public class ClusterBlocks extends AbstractDiffable { return new ClusterBlockException(unmodifiableSet(blocks.collect(toSet()))); } - public String prettyPrint() { + @Override + public String toString() { if (global.isEmpty() && indices().isEmpty()) { return ""; } diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index f3a8932f53c..895195d35b3 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -26,7 +26,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -398,14 +397,6 @@ public class DiscoveryNodes extends AbstractDiffable implements @Override public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("{"); - sb.append(Strings.collectionToDelimitedString(this, ",")); - sb.append("}"); - return sb.toString(); - } - - public String prettyPrint() { StringBuilder sb = new StringBuilder(); sb.append("nodes: \n"); for (DiscoveryNode node : this) { diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index bd5113029c4..ddb7969f60a 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -391,7 +391,8 @@ public class RoutingNodes implements Iterable { return shards; } - public String prettyPrint() { + @Override + public String toString() { StringBuilder sb = new StringBuilder("routing_nodes:\n"); for (RoutingNode routingNode : this) { sb.append(routingNode.prettyPrint()); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java index 9dd2cc72da8..1c3d629a72f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java @@ -109,7 +109,7 @@ public class RoutingService extends AbstractLifecycleComponent { rerouting.set(false); ClusterState state = clusterService.state(); if (logger.isTraceEnabled()) { - logger.error((Supplier) () -> new ParameterizedMessage("unexpected failure during [{}], current state:\n{}", source, state.prettyPrint()), e); + logger.error((Supplier) () -> new ParameterizedMessage("unexpected failure during [{}], current state:\n{}", source, state), e); } else { logger.error((Supplier) () -> new ParameterizedMessage("unexpected failure during [{}], current state version [{}]", source, state.version()), e); } @@ -118,7 +118,7 @@ public class RoutingService extends AbstractLifecycleComponent { } catch (Exception e) { rerouting.set(false); ClusterState state = clusterService.state(); - logger.warn((Supplier) () -> new ParameterizedMessage("failed to reroute routing table, current state:\n{}", state.prettyPrint()), e); + logger.warn((Supplier) () -> new ParameterizedMessage("failed to reroute routing table, current state:\n{}", state), e); } } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 2d960ce0450..051fd12a12b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -613,7 +613,8 @@ public class RoutingTable implements Iterable, Diffable entry : indicesRouting) { sb.append(entry.value.prettyPrint()).append('\n'); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java index 52ace3744cd..fa30a102bf6 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java @@ -233,7 +233,7 @@ public class IndexMetaDataUpdater extends RoutingChangesObserver.AbstractRouting Set oldInSyncAllocations = oldIndexMetaData.inSyncAllocationIds(shardNumber); Set idsToRemove = shardEntry.getValue().stream().map(e -> e.getAllocationId()).collect(Collectors.toSet()); assert idsToRemove.stream().allMatch(id -> oldRoutingTable.getByAllocationId(shardEntry.getKey(), id) == null) : - "removing stale ids: " + idsToRemove + ", some of which have still a routing entry: " + oldRoutingTable.prettyPrint(); + "removing stale ids: " + idsToRemove + ", some of which have still a routing entry: " + oldRoutingTable; Set remainingInSyncAllocations = Sets.difference(oldInSyncAllocations, idsToRemove); assert remainingInSyncAllocations.isEmpty() == false : "Set of in-sync ids cannot become empty for shard " + shardEntry.getKey() + " (before: " + oldInSyncAllocations + ", ids to remove: " + idsToRemove + ")"; diff --git a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index fe2adf1f4c6..4f637e05648 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -562,9 +562,9 @@ public class ClusterService extends AbstractLifecycleComponent { executionTime, previousClusterState.version(), tasksSummary, - previousClusterState.nodes().prettyPrint(), - previousClusterState.routingTable().prettyPrint(), - previousClusterState.getRoutingNodes().prettyPrint()), + previousClusterState.nodes(), + previousClusterState.routingTable(), + previousClusterState.getRoutingNodes()), e); } warnAboutSlowTaskIfNeeded(executionTime, tasksSummary); @@ -656,7 +656,7 @@ public class ClusterService extends AbstractLifecycleComponent { newClusterState.status(ClusterState.ClusterStateStatus.BEING_APPLIED); if (logger.isTraceEnabled()) { - logger.trace("cluster state updated, source [{}]\n{}", tasksSummary, newClusterState.prettyPrint()); + logger.trace("cluster state updated, source [{}]\n{}", tasksSummary, newClusterState); } else if (logger.isDebugEnabled()) { logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), tasksSummary); } @@ -759,7 +759,7 @@ public class ClusterService extends AbstractLifecycleComponent { TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS))); final long version = newClusterState.version(); final String stateUUID = newClusterState.stateUUID(); - final String prettyPrint = newClusterState.prettyPrint(); + final String fullState = newClusterState.toString(); logger.warn( (Supplier) () -> new ParameterizedMessage( "failed to apply updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}", @@ -767,7 +767,7 @@ public class ClusterService extends AbstractLifecycleComponent { version, stateUUID, tasksSummary, - prettyPrint), + fullState), e); // TODO: do we want to call updateTask.onFailure here? } @@ -826,9 +826,7 @@ public class ClusterService extends AbstractLifecycleComponent { (Supplier) () -> new ParameterizedMessage( "exception thrown by listener while notifying of cluster state processed from [{}], old cluster state:\n" + "{}\nnew cluster state:\n{}", - source, - oldState.prettyPrint(), - newState.prettyPrint()), + source, oldState, newState), e); } } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java index 3c8deee7c5f..6d77e2f48fe 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java @@ -464,7 +464,7 @@ public class NodeJoinController extends AbstractComponent { } private ClusterState.Builder becomeMasterAndTrimConflictingNodes(ClusterState currentState, List joiningNodes) { - assert currentState.nodes().getMasterNodeId() == null : currentState.prettyPrint(); + assert currentState.nodes().getMasterNodeId() == null : currentState; DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(currentState.nodes()); nodesBuilder.masterNodeId(currentState.nodes().getLocalNodeId()); ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks()) diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index cfbeb79ef7d..260f70e19ed 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushResponse; import org.elasticsearch.action.admin.indices.flush.TransportFlushAction; -import org.elasticsearch.action.support.ActionFilter; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.broadcast.BroadcastRequest; import org.elasticsearch.action.support.broadcast.BroadcastResponse; @@ -101,7 +100,7 @@ public class BroadcastReplicationTests extends ESTestCase { transportService.start(); transportService.acceptIncomingRequests(); broadcastReplicationAction = new TestBroadcastReplicationAction(Settings.EMPTY, threadPool, clusterService, transportService, - new ActionFilters(new HashSet()), new IndexNameExpressionResolver(Settings.EMPTY), null); + new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY), null); } @After @@ -120,7 +119,7 @@ public class BroadcastReplicationTests extends ESTestCase { final String index = "test"; setState(clusterService, state(index, randomBoolean(), randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { if (randomBoolean()) { @@ -139,11 +138,11 @@ public class BroadcastReplicationTests extends ESTestCase { final String index = "test"; setState(clusterService, state(index, randomBoolean(), ShardRoutingState.STARTED)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { ReplicationResponse replicationResponse = new ReplicationResponse(); - replicationResponse.setShardInfo(new ReplicationResponse.ShardInfo(1, 1, new ReplicationResponse.ShardInfo.Failure[0])); + replicationResponse.setShardInfo(new ReplicationResponse.ShardInfo(1, 1)); shardRequests.v2().onResponse(replicationResponse); } logger.info("total shards: {}, ", response.get().getTotalShards()); @@ -154,7 +153,7 @@ public class BroadcastReplicationTests extends ESTestCase { final String index = "test"; int numShards = 1 + randomInt(3); setState(clusterService, stateWithAssignedPrimariesAndOneReplica(index, numShards)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Future response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index))); int succeeded = 0; int failed = 0; @@ -184,7 +183,7 @@ public class BroadcastReplicationTests extends ESTestCase { public void testNoShards() throws InterruptedException, ExecutionException, IOException { setState(clusterService, stateWithNoShard()); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); BroadcastResponse response = executeAndAssertImmediateResponse(broadcastReplicationAction, new DummyBroadcastRequest()); assertBroadcastResponse(0, 0, 0, response, null); } @@ -194,7 +193,7 @@ public class BroadcastReplicationTests extends ESTestCase { final ShardId shardId = new ShardId(index, "_na_", 0); ClusterState clusterState = state(index, randomBoolean(), randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); List shards = broadcastReplicationAction.shards(new DummyBroadcastRequest().indices(shardId.getIndexName()), clusterState); assertThat(shards.size(), equalTo(1)); assertThat(shards.get(0), equalTo(shardId)); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index b5edc1b53c5..a49c2ae978e 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -256,7 +256,7 @@ public class ReplicationOperationTests extends ESTestCase { final ClusterState initialState, final ClusterState changedState) throws Exception { AtomicReference state = new AtomicReference<>(initialState); - logger.debug("--> using initial state:\n{}", state.get().prettyPrint()); + logger.debug("--> using initial state:\n{}", state.get()); final long primaryTerm = initialState.getMetaData().index(shardId.getIndexName()).primaryTerm(shardId.id()); final ShardRouting primaryShard = state.get().routingTable().shardRoutingTable(shardId).primaryShard(); final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) { @@ -264,7 +264,7 @@ public class ReplicationOperationTests extends ESTestCase { public Result perform(Request request) throws Exception { Result result = super.perform(request); state.set(changedState); - logger.debug("--> state after primary operation:\n{}", state.get().prettyPrint()); + logger.debug("--> state after primary operation:\n{}", state.get()); return result; } }; @@ -303,8 +303,7 @@ public class ReplicationOperationTests extends ESTestCase { logger.debug("using active shard count of [{}], assigned shards [{}], total shards [{}]." + " expecting op to [{}]. using state: \n{}", request.waitForActiveShards(), 1 + assignedReplicas, 1 + assignedReplicas + unassignedReplicas, - passesActiveShardCheck ? "succeed" : "retry", - state.prettyPrint()); + passesActiveShardCheck ? "succeed" : "retry", state); final long primaryTerm = state.metaData().index(index).primaryTerm(shardId.id()); final IndexShardRoutingTable shardRoutingTable = state.routingTable().index(index).shard(shardId.id()); PlainActionFuture listener = new PlainActionFuture<>(); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 1caac899005..709c4b830ea 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -77,7 +77,7 @@ import org.junit.Before; import org.junit.BeforeClass; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -227,7 +227,7 @@ public class TransportReplicationActionTests extends ESTestCase { randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED)); ReplicationTask task = maybeTask(); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Request request = new Request(shardId).timeout("1ms"); PlainActionFuture listener = new PlainActionFuture<>(); @@ -246,7 +246,7 @@ public class TransportReplicationActionTests extends ESTestCase { assertTrue(request.isRetrySet.get()); setState(clusterService, state(index, true, ShardRoutingState.STARTED)); - logger.debug("--> primary assigned state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> primary assigned state:\n{}", clusterService.state()); final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id()); final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId(); @@ -275,7 +275,7 @@ public class TransportReplicationActionTests extends ESTestCase { String relocationTargetNode = state.getRoutingTable().shardRoutingTable(shardId).primaryShard().relocatingNodeId(); state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(relocationTargetNode)).build(); setState(clusterService, state); - logger.debug("--> relocation ongoing state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> relocation ongoing state:\n{}", clusterService.state()); Request request = new Request(shardId).timeout("1ms").routedBasedOnClusterVersion(clusterService.state().version() + 1); PlainActionFuture listener = new PlainActionFuture<>(); @@ -295,10 +295,10 @@ public class TransportReplicationActionTests extends ESTestCase { ShardRouting relocationTarget = clusterService.state().getRoutingTable().shardRoutingTable(shardId) .shardsWithState(ShardRoutingState.INITIALIZING).get(0); AllocationService allocationService = ESAllocationTestCase.createAllocationService(); - ClusterState updatedState = allocationService.applyStartedShards(state, Arrays.asList(relocationTarget)); + ClusterState updatedState = allocationService.applyStartedShards(state, Collections.singletonList(relocationTarget)); setState(clusterService, updatedState); - logger.debug("--> relocation complete state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> relocation complete state:\n{}", clusterService.state()); IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id()); final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId(); @@ -315,7 +315,7 @@ public class TransportReplicationActionTests extends ESTestCase { // no replicas in oder to skip the replication part setState(clusterService, state(index, true, randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED)); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Request request = new Request(new ShardId("unknown_index", "_na_", 0)).timeout("1ms"); PlainActionFuture listener = new PlainActionFuture<>(); ReplicationTask task = maybeTask(); @@ -339,7 +339,7 @@ public class TransportReplicationActionTests extends ESTestCase { final ShardId shardId = new ShardId(index, "_na_", 0); // no replicas in order to skip the replication part setState(clusterService, stateWithActivePrimary(index, true, randomInt(3))); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); Request request = new Request(shardId); boolean timeout = randomBoolean(); if (timeout) { @@ -396,7 +396,7 @@ public class TransportReplicationActionTests extends ESTestCase { ReplicationTask task = maybeTask(); setState(clusterService, stateWithActivePrimary(index, randomBoolean(), 3)); - logger.debug("using state: \n{}", clusterService.state().prettyPrint()); + logger.debug("using state: \n{}", clusterService.state()); final IndexShardRoutingTable shardRoutingTable = clusterService.state().routingTable().index(index).shard(shardId.id()); final String primaryNodeId = shardRoutingTable.primaryShard().currentNodeId(); @@ -545,7 +545,7 @@ public class TransportReplicationActionTests extends ESTestCase { final String index = "test"; final ShardId shardId = new ShardId(index, "_na_", 0); ClusterState state = stateWithActivePrimary(index, true, 1 + randomInt(3), randomInt(2)); - logger.info("using state: {}", state.prettyPrint()); + logger.info("using state: {}", state); setState(clusterService, state); // check that at unknown node fails @@ -651,7 +651,7 @@ public class TransportReplicationActionTests extends ESTestCase { // no replica, we only want to test on primary final ClusterState state = state(index, true, ShardRoutingState.STARTED); setState(clusterService, state); - logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + logger.debug("--> using initial state:\n{}", clusterService.state()); final ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard(); Request request = new Request(shardId); PlainActionFuture listener = new PlainActionFuture<>(); diff --git a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java index 86b64febb04..fac8d5f7b63 100644 --- a/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java @@ -219,7 +219,7 @@ public class NoMasterNodeIT extends ESIntegTestCase { ensureSearchable("test1", "test2"); ClusterStateResponse clusterState = client().admin().cluster().prepareState().get(); - logger.info("Cluster state:\n{}", clusterState.getState().prettyPrint()); + logger.info("Cluster state:\n{}", clusterState.getState()); internalCluster().stopRandomDataNode(); assertTrue(awaitBusy(() -> { diff --git a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java index 1dd1abf4ce1..eb5c88d7e83 100644 --- a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -277,9 +277,9 @@ public class ClusterStateHealthTests extends ESTestCase { // if the inactive primaries are due solely to recovery (not failed allocation or previously being allocated) // then cluster health is YELLOW, otherwise RED if (primaryInactiveDueToRecovery(indexName, clusterState)) { - assertThat("clusterState is:\n" + clusterState.prettyPrint(), health.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + assertThat("clusterState is:\n" + clusterState, health.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); } else { - assertThat("clusterState is:\n" + clusterState.prettyPrint(), health.getStatus(), equalTo(ClusterHealthStatus.RED)); + assertThat("clusterState is:\n" + clusterState, health.getStatus(), equalTo(ClusterHealthStatus.RED)); } } } diff --git a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java index a16520faeb1..342919fb881 100644 --- a/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java @@ -135,8 +135,8 @@ public class DiscoveryNodesTests extends ESTestCase { final DiscoveryNodes discoNodesA = builderA.build(); final DiscoveryNodes discoNodesB = builderB.build(); - logger.info("nodes A: {}", discoNodesA.prettyPrint()); - logger.info("nodes B: {}", discoNodesB.prettyPrint()); + logger.info("nodes A: {}", discoNodesA); + logger.info("nodes B: {}", discoNodesB); DiscoveryNodes.Delta delta = discoNodesB.delta(discoNodesA); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 6243f138380..0d284a1e47e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -152,7 +152,9 @@ public class PrimaryAllocationIT extends ESIntegTestCase { client().admin().cluster().prepareReroute().add(new AllocateStalePrimaryAllocationCommand("test", 0, dataNodeWithNoShardCopy, true)).get(); logger.info("--> wait until shard is failed and becomes unassigned again"); - assertBusy(() -> assertTrue(client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").allPrimaryShardsUnassigned())); + assertBusy(() -> + assertTrue(client().admin().cluster().prepareState().get().getState().toString(), + client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").allPrimaryShardsUnassigned())); assertThat(client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").getShards().get(0).primaryShard().unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.ALLOCATION_FAILED)); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 60f1688ad3d..4fffcebc79b 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -306,7 +306,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build(); // make sure both replicas are marked as delayed (i.e. not reallocated) clusterState = allocation.deassociateDeadNodes(clusterState, true, "reroute"); - assertThat(clusterState.prettyPrint(), UnassignedInfo.getNumberOfDelayedUnassigned(clusterState), equalTo(2)); + assertThat(clusterState.toString(), UnassignedInfo.getNumberOfDelayedUnassigned(clusterState), equalTo(2)); } public void testFindNextDelayedAllocation() { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java index ed7a944963d..e658ff03a18 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java @@ -89,7 +89,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase { clusterState = addNodes(clusterState, service, 1, nodeOffset++); assertNumIndexShardsPerNode(clusterState, Matchers.lessThanOrEqualTo(2)); assertThat(clusterState.getRoutingNodes().unassigned().size(), equalTo(0)); - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); } public void testMinimalRelocations() { @@ -150,7 +150,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase { newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); assertThat(newState, equalTo(clusterState)); assertNumIndexShardsPerNode(clusterState, equalTo(2)); - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); } public void testMinimalRelocationsNoLimit() { @@ -212,7 +212,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase { newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); assertThat(newState, equalTo(clusterState)); assertNumIndexShardsPerNode(clusterState, equalTo(2)); - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index a56af9f2b39..f48c9acb356 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -366,7 +366,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { } private ClusterState stabilize(ClusterState clusterState, AllocationService service) { - logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes()); clusterState = service.deassociateDeadNodes(clusterState, true, "reroute"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); @@ -375,7 +375,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { logger.info("complete rebalancing"); boolean changed; do { - logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes()); ClusterState newState = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); changed = newState.equals(clusterState) == false; clusterState = newState; @@ -386,7 +386,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase { } private void assertRecoveryNodeVersions(RoutingNodes routingNodes) { - logger.trace("RoutingNodes: {}", routingNodes.prettyPrint()); + logger.trace("RoutingNodes: {}", routingNodes); List mutableShardRoutings = routingNodes.shardsWithState(ShardRoutingState.RELOCATING); for (ShardRouting r : mutableShardRoutings) { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java index 6722e048030..23992b91541 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java @@ -159,7 +159,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase { } while (clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size() != 0 || clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size() != 0 && iterations < 200); - logger.info("Done Balancing after [{}] iterations. State:\n{}", iterations, clusterState.prettyPrint()); + logger.info("Done Balancing after [{}] iterations. State:\n{}", iterations, clusterState); // we stop after 200 iterations if it didn't stabelize by then something is likely to be wrong assertThat("max num iteration exceeded", iterations, Matchers.lessThan(200)); assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(0)); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java index 454e8410484..74d3dda8e36 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java @@ -70,15 +70,14 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase { logger.info("--> test starting of shard"); ClusterState newState = allocation.applyStartedShards(state, Arrays.asList(initShard)); - assertThat("failed to start " + initShard + "\ncurrent routing table:" + newState.routingTable().prettyPrint(), - newState, not(equalTo(state))); - assertTrue(initShard + "isn't started \ncurrent routing table:" + newState.routingTable().prettyPrint(), + assertThat("failed to start " + initShard + "\ncurrent routing table:" + newState.routingTable(), newState, not(equalTo(state))); + assertTrue(initShard + "isn't started \ncurrent routing table:" + newState.routingTable(), newState.routingTable().index("test").shard(initShard.id()).allShardsStarted()); state = newState; logger.info("--> testing starting of relocating shards"); newState = allocation.applyStartedShards(state, Arrays.asList(relocatingShard.getTargetRelocatingShard())); - assertThat("failed to start " + relocatingShard + "\ncurrent routing table:" + newState.routingTable().prettyPrint(), + assertThat("failed to start " + relocatingShard + "\ncurrent routing table:" + newState.routingTable(), newState, not(equalTo(state))); ShardRouting shardRouting = newState.routingTable().index("test").shard(relocatingShard.id()).getShards().get(0); assertThat(shardRouting.state(), equalTo(ShardRoutingState.STARTED)); diff --git a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index 3f1e8f032ca..00d9a8ff096 100644 --- a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -22,6 +22,7 @@ package org.elasticsearch.cluster.serialization; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -30,7 +31,6 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.cluster.ESAllocationTestCase; import static org.hamcrest.Matchers.equalTo; @@ -56,7 +56,7 @@ public class ClusterSerializationTests extends ESAllocationTestCase { assertThat(serializedClusterState.getClusterName().value(), equalTo(clusterState.getClusterName().value())); - assertThat(serializedClusterState.routingTable().prettyPrint(), equalTo(clusterState.routingTable().prettyPrint())); + assertThat(serializedClusterState.routingTable().toString(), equalTo(clusterState.routingTable().toString())); } public void testRoutingTableSerialization() throws Exception { @@ -81,7 +81,7 @@ public class ClusterSerializationTests extends ESAllocationTestCase { StreamInput inStream = outStream.bytes().streamInput(); RoutingTable target = RoutingTable.Builder.readFrom(inStream); - assertThat(target.prettyPrint(), equalTo(source.prettyPrint())); + assertThat(target.toString(), equalTo(source.toString())); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java index 69d18933e6c..9ce3d1fcee8 100644 --- a/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java @@ -21,6 +21,7 @@ package org.elasticsearch.cluster.serialization; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -28,8 +29,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.cluster.ESAllocationTestCase; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; @@ -55,7 +56,7 @@ public class ClusterStateToStringTests extends ESAllocationTestCase { AllocationService strategy = createAllocationService(); clusterState = ClusterState.builder(clusterState).routingTable(strategy.reroute(clusterState, "reroute").routingTable()).build(); - String clusterStateString = clusterState.toString(); + String clusterStateString = Strings.toString(clusterState, true); assertNotNull(clusterStateString); assertThat(clusterStateString, containsString("test_idx")); diff --git a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java index f0c8258864a..0208664033c 100644 --- a/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java @@ -381,7 +381,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { } if (!success) { fail("node [" + node + "] has no master or has blocks, despite of being on the right side of the partition. State dump:\n" - + nodeState.prettyPrint()); + + nodeState); } } @@ -468,13 +468,13 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize()); assertEquals("different masters ", state.nodes().getMasterNodeId(), nodeState.nodes().getMasterNodeId()); assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version()); - if (!state.routingTable().prettyPrint().equals(nodeState.routingTable().prettyPrint())) { + if (!state.routingTable().toString().equals(nodeState.routingTable().toString())) { fail("different routing"); } } catch (AssertionError t) { fail("failed comparing cluster state: " + t.getMessage() + "\n" + - "--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state.prettyPrint() + - "\n--- cluster state [" + node + "]: ---\n" + nodeState.prettyPrint()); + "--- cluster state of node [" + nodes.get(0) + "]: ---\n" + state + + "\n--- cluster state [" + node + "]: ---\n" + nodeState); } } @@ -1267,7 +1267,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { final ClusterState state = client().admin().cluster().prepareState().get().getState(); if (state.metaData().hasIndex("test") == false) { - fail("index 'test' was lost. current cluster state: " + state.prettyPrint()); + fail("index 'test' was lost. current cluster state: " + state); } } @@ -1368,7 +1368,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase { assertBusy(() -> { for (String node : nodes) { ClusterState state = getNodeClusterState(node); - String failMsgSuffix = "cluster_state:\n" + state.prettyPrint(); + String failMsgSuffix = "cluster_state:\n" + state; assertThat("wrong node count on [" + node + "]. " + failMsgSuffix, state.nodes().getSize(), equalTo(nodes.size())); String otherMasterNodeName = state.nodes().getMasterNode() != null ? state.nodes().getMasterNode().getName() : null; assertThat("wrong master on node [" + node + "]. " + failMsgSuffix, otherMasterNodeName, equalTo(masterNode)); diff --git a/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java b/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java index 14561f255cf..3af2e32eefa 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java @@ -66,7 +66,7 @@ public class ZenUnicastDiscoveryIT extends ESIntegTestCase { internalCluster().startNodesAsync(currentNumNodes - unicastHostOrdinals.length).get(); if (client().admin().cluster().prepareHealth().setWaitForNodes("" + currentNumNodes).get().isTimedOut()) { - logger.info("cluster forming timed out, cluster state:\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint()); + logger.info("cluster forming timed out, cluster state:\n{}", client().admin().cluster().prepareState().get().getState()); fail("timed out waiting for cluster to form with [" + currentNumNodes + "] nodes"); } } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index eb580716622..95fcb88a7ea 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -676,10 +676,10 @@ public class NodeJoinControllerTests extends ESTestCase { protected void assertNodesInCurrentState(List expectedNodes) { final ClusterState state = clusterService.state(); - logger.info("assert for [{}] in:\n{}", expectedNodes, state.prettyPrint()); + logger.info("assert for [{}] in:\n{}", expectedNodes, state); DiscoveryNodes discoveryNodes = state.nodes(); for (DiscoveryNode node : expectedNodes) { - assertThat("missing " + node + "\n" + discoveryNodes.prettyPrint(), discoveryNodes.get(node.getId()), equalTo(node)); + assertThat("missing " + node + "\n" + discoveryNodes, discoveryNodes.get(node.getId()), equalTo(node)); } assertThat(discoveryNodes.getSize(), equalTo(expectedNodes.size())); } diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java index eb8153c8354..e1d2a226a02 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java @@ -41,8 +41,6 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; -import org.elasticsearch.discovery.zen.PublishClusterStateAction; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.node.Node; import org.elasticsearch.test.ESTestCase; @@ -852,8 +850,8 @@ public class PublishClusterStateActionTests extends ESTestCase { void assertSameState(ClusterState actual, ClusterState expected) { assertThat(actual, notNullValue()); - final String reason = "\n--> actual ClusterState: " + actual.prettyPrint() + "\n" + - "--> expected ClusterState:" + expected.prettyPrint(); + final String reason = "\n--> actual ClusterState: " + actual + "\n" + + "--> expected ClusterState:" + expected; assertThat("unequal UUIDs" + reason, actual.stateUUID(), equalTo(expected.stateUUID())); assertThat("unequal versions" + reason, actual.version(), equalTo(expected.version())); } diff --git a/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java b/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java index a996c9f4bd8..d1be0d77613 100644 --- a/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java +++ b/core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java @@ -60,8 +60,8 @@ import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; +import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; @@ -178,7 +178,8 @@ public class CorruptedFileIT extends ESIntegTestCase { .timeout("5m") // sometimes due to cluster rebalacing and random settings default timeout is just not enough. .waitForNoRelocatingShards(true)).actionGet(); if (health.isTimedOut()) { - logger.info("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for green state", health.isTimedOut(), equalTo(false)); } assertThat(health.getStatus(), equalTo(ClusterHealthStatus.GREEN)); @@ -284,7 +285,8 @@ public class CorruptedFileIT extends ESIntegTestCase { .health(Requests.clusterHealthRequest("test")).get(); if (response.getStatus() != ClusterHealthStatus.RED) { logger.info("Cluster turned red in busy loop: {}", didClusterTurnRed); - logger.info("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); } assertThat(response.getStatus(), is(ClusterHealthStatus.RED)); ClusterState state = client().admin().cluster().prepareState().get().getState(); @@ -445,7 +447,8 @@ public class CorruptedFileIT extends ESIntegTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest("test").waitForGreenStatus()).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureGreen timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for green state", actionGet.isTimedOut(), equalTo(false)); } // we are green so primaries got not corrupted. diff --git a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index 67a82d93c54..074b4a5d6bb 100644 --- a/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -91,7 +91,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice } catch (AssertionError error) { ClusterState finalState = state; logger.error((org.apache.logging.log4j.util.Supplier) () -> - new ParameterizedMessage("failed to random change state. last good state: \n{}", finalState.prettyPrint()), error); + new ParameterizedMessage("failed to random change state. last good state: \n{}", finalState), error); throw error; } } @@ -107,7 +107,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice } catch (AssertionError error) { logger.error((org.apache.logging.log4j.util.Supplier) () -> new ParameterizedMessage( "failed to apply change on [{}].\n *** Previous state ***\n{}\n *** New state ***\n{}", - node, event.previousState().prettyPrint(), event.state().prettyPrint()), error); + node, event.previousState(), event.state()), error); throw error; } @@ -117,7 +117,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice } // TODO: check if we can go to green by starting all shards and finishing all iterations - logger.info("Final cluster state: {}", state.prettyPrint()); + logger.info("Final cluster state: {}", state); } /** diff --git a/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java b/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java index a180d6feb8e..4a61bebd4db 100644 --- a/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java @@ -149,7 +149,8 @@ public class FullRollingRestartIT extends ESIntegTestCase { ClusterState state = client().admin().cluster().prepareState().get().getState(); RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); for (RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) { - assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode() + "\n" + state.prettyPrint(), recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); + assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode() + "\n" + state, + recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); } internalCluster().restartRandomDataNode(); ensureGreen(); @@ -157,7 +158,8 @@ public class FullRollingRestartIT extends ESIntegTestCase { recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); for (RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) { - assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode()+ "-- \nbefore: \n" + state.prettyPrint() + "\nafter: \n" + afterState.prettyPrint(), recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); + assertTrue("relocated from: " + recoveryState.getSourceNode() + " to: " + recoveryState.getTargetNode()+ "-- \nbefore: \n" + state, + recoveryState.getRecoverySource().getType() != RecoverySource.Type.PEER || recoveryState.getPrimary() == false); } } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index 739371d76df..47051d9072d 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -146,7 +146,7 @@ public abstract class ESAllocationTestCase extends ESTestCase { ClusterState lastClusterState; do { lastClusterState = clusterState; - logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint()); + logger.debug("ClusterState: {}", clusterState.getRoutingNodes()); clusterState = service.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)); } while (lastClusterState.equals(clusterState) == false); return clusterState; diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 1e98754a798..77c65197366 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -164,7 +164,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BooleanSupplier; import java.util.function.Function; -import java.util.stream.Collectors; import static org.elasticsearch.client.Requests.syncedFlushRequest; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; @@ -536,9 +535,7 @@ public abstract class ESIntegTestCase extends ESTestCase { for (Discovery discovery : internalCluster().getInstances(Discovery.class)) { if (discovery instanceof ZenDiscovery) { final ZenDiscovery zenDiscovery = (ZenDiscovery) discovery; - assertBusy(() -> assertThat(zenDiscovery.localNode().getName() + " still having pending states: \n" + - Arrays.stream(zenDiscovery.pendingClusterStates()).map(ClusterState::prettyPrint) - .collect(Collectors.joining("\n")), + assertBusy(() -> assertThat(zenDiscovery.localNode().getName() + " still having pending states", zenDiscovery.pendingClusterStates(), emptyArray())); } } @@ -758,17 +755,14 @@ public abstract class ESIntegTestCase extends ESTestCase { */ public void waitNoPendingTasksOnAll() throws Exception { assertNoTimeout(client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get()); - assertBusy(new Runnable() { - @Override - public void run() { - for (Client client : clients()) { - ClusterHealthResponse clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); - assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); - PendingClusterTasksResponse pendingTasks = client.admin().cluster().preparePendingClusterTasks().setLocal(true).get(); - assertThat("client " + client + " still has pending tasks " + pendingTasks.prettyPrint(), pendingTasks, Matchers.emptyIterable()); - clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); - assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); - } + assertBusy(() -> { + for (Client client : clients()) { + ClusterHealthResponse clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); + assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); + PendingClusterTasksResponse pendingTasks = client.admin().cluster().preparePendingClusterTasks().setLocal(true).get(); + assertThat("client " + client + " still has pending tasks " + pendingTasks, pendingTasks, Matchers.emptyIterable()); + clusterHealth = client.admin().cluster().prepareHealth().setLocal(true).get(); + assertThat("client " + client + " still has in flight fetch", clusterHealth.getNumberOfInFlightFetch(), equalTo(0)); } }); assertNoTimeout(client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get()); @@ -872,7 +866,8 @@ public abstract class ESIntegTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest(indices).timeout(timeout).waitForGreenStatus().waitForEvents(Priority.LANGUID).waitForNoRelocatingShards(true)).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureGreen timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); fail("timed out waiting for green state"); } assertThat(actionGet.getStatus(), equalTo(ClusterHealthStatus.GREEN)); @@ -899,7 +894,8 @@ public abstract class ESIntegTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(request).actionGet(); if (actionGet.isTimedOut()) { - logger.info("waitForRelocation timed out (status={}), cluster state:\n{}\n{}", status, client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("waitForRelocation timed out (status={}), cluster state:\n{}\n{}", status, + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for relocation", actionGet.isTimedOut(), equalTo(false)); } if (status != null) { @@ -996,7 +992,8 @@ public abstract class ESIntegTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest(indices).waitForNoRelocatingShards(true).waitForYellowStatus().waitForEvents(Priority.LANGUID)).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureYellow timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureYellow timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for yellow", actionGet.isTimedOut(), equalTo(false)); } logger.debug("indices {} are yellow", indices.length == 0 ? "[_all]" : indices); @@ -1007,7 +1004,8 @@ public abstract class ESIntegTestCase extends ESTestCase { * Prints the current cluster state as debug logging. */ public void logClusterState() { - logger.debug("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.debug("cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), client().admin().cluster().preparePendingClusterTasks().get()); } /** @@ -1108,7 +1106,7 @@ public abstract class ESIntegTestCase extends ESTestCase { if (clusterHealthResponse.isTimedOut()) { ClusterStateResponse stateResponse = client(viaNode).admin().cluster().prepareState().get(); fail("failed to reach a stable cluster of [" + nodeCount + "] nodes. Tried via [" + viaNode + "]. last cluster state:\n" - + stateResponse.getState().prettyPrint()); + + stateResponse.getState()); } assertThat(clusterHealthResponse.isTimedOut(), is(false)); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index f096e662f4a..9648eb5798e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -318,7 +318,8 @@ public abstract class ESSingleNodeTestCase extends ESTestCase { ClusterHealthResponse actionGet = client().admin().cluster() .health(Requests.clusterHealthRequest(indices).timeout(timeout).waitForGreenStatus().waitForEvents(Priority.LANGUID).waitForNoRelocatingShards(true)).actionGet(); if (actionGet.isTimedOut()) { - logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint()); + logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState(), + client().admin().cluster().preparePendingClusterTasks().get()); assertThat("timed out waiting for green state", actionGet.isTimedOut(), equalTo(false)); } assertThat(actionGet.getStatus(), equalTo(ClusterHealthStatus.GREEN));