[TEST] Enable transport tracer for RemoteClusterServiceTests#testCollectNodes #25301

This commit is contained in:
Simon Willnauer 2017-07-05 11:22:37 +02:00
parent 8e861b3896
commit ca351b60b7
1 changed files with 14 additions and 5 deletions

View File

@ -21,11 +21,13 @@ package org.elasticsearch.transport;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
@ -271,19 +273,23 @@ public class RemoteClusterServiceTests extends ESTestCase {
}
@TestLogging("org.elasticsearch.test.transport.MockTransportService.tracer:TRACE")
public void testCollectNodes() throws InterruptedException, IOException {
final Settings settings = Settings.EMPTY;
final Settings settings = Settings.builder()
.put("transport.tracer.include", ClusterStateAction.NAME)
.build();
final List<DiscoveryNode> knownNodes_c1 = new CopyOnWriteArrayList<>();
final List<DiscoveryNode> knownNodes_c2 = new CopyOnWriteArrayList<>();
try (MockTransportService c1N1 =
startTransport("cluster_1_node_1", knownNodes_c1, Version.CURRENT);
startTransport("cluster_1_node_1", knownNodes_c1, Version.CURRENT, settings);
MockTransportService c1N2 =
startTransport("cluster_1_node_2", knownNodes_c1, Version.CURRENT);
startTransport("cluster_1_node_2", knownNodes_c1, Version.CURRENT, settings);
MockTransportService c2N1 =
startTransport("cluster_2_node_1", knownNodes_c2, Version.CURRENT);
startTransport("cluster_2_node_1", knownNodes_c2, Version.CURRENT, settings);
MockTransportService c2N2 =
startTransport("cluster_2_node_2", knownNodes_c2, Version.CURRENT)) {
startTransport("cluster_2_node_2", knownNodes_c2, Version.CURRENT, settings)) {
final DiscoveryNode c1N1Node = c1N1.getLocalDiscoNode();
final DiscoveryNode c1N2Node = c1N2.getLocalDiscoNode();
final DiscoveryNode c2N1Node = c2N1.getLocalDiscoNode();
@ -385,8 +391,11 @@ public class RemoteClusterServiceTests extends ESTestCase {
assertEquals("no such remote cluster: [no such cluster]", ex.get().getMessage());
}
{
logger.info("closing all source nodes");
// close all targets and check for the transport level failure path
IOUtils.close(c1N1, c1N2, c2N1, c2N2);
logger.info("all source nodes are closed");
CountDownLatch failLatch = new CountDownLatch(1);
AtomicReference<Exception> ex = new AtomicReference<>();
service.collectNodes(new HashSet<>(Arrays.asList("cluster_1", "cluster_2")),