Browse Source

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.
Boaz Leskes 9 years ago
parent
commit
0daf483587
35 changed files with 127 additions and 150 deletions
  1. 2 15
      core/src/main/java/org/elasticsearch/action/admin/cluster/tasks/PendingClusterTasksResponse.java
  2. 15 19
      core/src/main/java/org/elasticsearch/cluster/ClusterState.java
  3. 1 1
      core/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java
  4. 2 1
      core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java
  5. 0 9
      core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java
  6. 2 1
      core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java
  7. 2 2
      core/src/main/java/org/elasticsearch/cluster/routing/RoutingService.java
  8. 2 1
      core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java
  9. 1 1
      core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java
  10. 7 9
      core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java
  11. 1 1
      core/src/main/java/org/elasticsearch/discovery/zen/NodeJoinController.java
  12. 7 8
      core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java
  13. 3 4
      core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java
  14. 11 11
      core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java
  15. 1 1
      core/src/test/java/org/elasticsearch/cluster/NoMasterNodeIT.java
  16. 2 2
      core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java
  17. 2 2
      core/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodesTests.java
  18. 3 1
      core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java
  19. 1 1
      core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java
  20. 3 3
      core/src/test/java/org/elasticsearch/cluster/routing/allocation/AddIncrementallyTests.java
  21. 3 3
      core/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java
  22. 1 1
      core/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java
  23. 3 4
      core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java
  24. 3 3
      core/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java
  25. 3 2
      core/src/test/java/org/elasticsearch/cluster/serialization/ClusterStateToStringTests.java
  26. 6 6
      core/src/test/java/org/elasticsearch/discovery/DiscoveryWithServiceDisruptionsIT.java
  27. 1 1
      core/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryIT.java
  28. 2 2
      core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java
  29. 2 4
      core/src/test/java/org/elasticsearch/discovery/zen/PublishClusterStateActionTests.java
  30. 7 4
      core/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java
  31. 3 3
      core/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java
  32. 4 2
      core/src/test/java/org/elasticsearch/recovery/FullRollingRestartIT.java
  33. 1 1
      test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java
  34. 18 20
      test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java
  35. 2 1
      test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java

+ 2 - 15
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);

+ 15 - 19
core/src/main/java/org/elasticsearch/cluster/ClusterState.java

@@ -274,15 +274,16 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
         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<ClusterState> {
                 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<String, Custom> cursor : customs) {
+                final String type = cursor.key;
+                final Custom custom = cursor.value;
+                sb.append(TAB).append(type).append(": ").append(custom);
+            }
         }
+        return sb.toString();
     }
 
     /**

+ 1 - 1
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);
             }

+ 2 - 1
core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java

@@ -199,7 +199,8 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
         return new ClusterBlockException(unmodifiableSet(blocks.collect(toSet())));
     }
 
-    public String prettyPrint() {
+    @Override
+    public String toString() {
         if (global.isEmpty() && indices().isEmpty()) {
             return "";
         }

+ 0 - 9
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<DiscoveryNodes> 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) {

+ 2 - 1
core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java

@@ -391,7 +391,8 @@ public class RoutingNodes implements Iterable<RoutingNode> {
         return shards;
     }
 
-    public String prettyPrint() {
+    @Override
+    public String toString() {
         StringBuilder sb = new StringBuilder("routing_nodes:\n");
         for (RoutingNode routingNode : this) {
             sb.append(routingNode.prettyPrint());

+ 2 - 2
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);
         }
     }
 }

+ 2 - 1
core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java

@@ -613,7 +613,8 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
         }
     }
 
-    public String prettyPrint() {
+    @Override
+    public String toString() {
         StringBuilder sb = new StringBuilder("routing_table (version ").append(version).append("):\n");
         for (ObjectObjectCursor<String, IndexRoutingTable> entry : indicesRouting) {
             sb.append(entry.value.prettyPrint()).append('\n');

+ 1 - 1
core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java

@@ -233,7 +233,7 @@ public class IndexMetaDataUpdater extends RoutingChangesObserver.AbstractRouting
                 Set<String> oldInSyncAllocations = oldIndexMetaData.inSyncAllocationIds(shardNumber);
                 Set<String> 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<String> 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 + ")";

+ 7 - 9
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);
             }
         }

+ 1 - 1
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<DiscoveryNode> 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())

+ 7 - 8
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<ActionFilter>()), 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<BroadcastResponse> response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index)));
         for (Tuple<ShardId, ActionListener<ReplicationResponse>> 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<BroadcastResponse> response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index)));
         for (Tuple<ShardId, ActionListener<ReplicationResponse>> 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<BroadcastResponse> 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<ShardId> shards = broadcastReplicationAction.shards(new DummyBroadcastRequest().indices(shardId.getIndexName()), clusterState);
         assertThat(shards.size(), equalTo(1));
         assertThat(shards.get(0), equalTo(shardId));

+ 3 - 4
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<ClusterState> 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<TestPrimary.Result> listener = new PlainActionFuture<>();

+ 11 - 11
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<Response> 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<Response> 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<Response> 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<Response> listener = new PlainActionFuture<>();

+ 1 - 1
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(() -> {

+ 2 - 2
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));
             }
         }
     }

+ 2 - 2
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);
 

+ 3 - 1
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));
     }
 

+ 1 - 1
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() {

+ 3 - 3
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());
     }
 
 

+ 3 - 3
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<ShardRouting> mutableShardRoutings = routingNodes.shardsWithState(ShardRoutingState.RELOCATING);
         for (ShardRouting r : mutableShardRoutings) {

+ 1 - 1
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));

+ 3 - 4
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));

+ 3 - 3
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()));
     }
 
 }

+ 3 - 2
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"));

+ 6 - 6
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));

+ 1 - 1
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");
         }
     }

+ 2 - 2
core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java

@@ -676,10 +676,10 @@ public class NodeJoinControllerTests extends ESTestCase {
 
     protected void assertNodesInCurrentState(List<DiscoveryNode> 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()));
     }

+ 2 - 4
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()));
     }

+ 7 - 4
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.

+ 3 - 3
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);
     }
 
     /**

+ 4 - 2
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);
         }
     }
 }

+ 1 - 1
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;

+ 18 - 20
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));
     }

+ 2 - 1
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));