aboutsummaryrefslogtreecommitdiffstats
path: root/container-search/src/test
diff options
context:
space:
mode:
Diffstat (limited to 'container-search/src/test')
-rw-r--r--container-search/src/test/java/com/yahoo/prelude/fastsearch/test/FastSearcherTestCase.java11
-rw-r--r--container-search/src/test/java/com/yahoo/search/dispatch/DispatcherTest.java30
-rw-r--r--container-search/src/test/java/com/yahoo/search/dispatch/InterleavedSearchInvokerTest.java8
-rw-r--r--container-search/src/test/java/com/yahoo/search/dispatch/LoadBalancerTest.java18
-rw-r--r--container-search/src/test/java/com/yahoo/search/dispatch/MockInvoker.java6
-rw-r--r--container-search/src/test/java/com/yahoo/search/dispatch/rpc/RpcSearchInvokerTest.java6
-rw-r--r--container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/MockSearchCluster.java2
-rw-r--r--container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/SearchClusterTest.java59
8 files changed, 83 insertions, 57 deletions
diff --git a/container-search/src/test/java/com/yahoo/prelude/fastsearch/test/FastSearcherTestCase.java b/container-search/src/test/java/com/yahoo/prelude/fastsearch/test/FastSearcherTestCase.java
index 7a63eb07641..fb483a8eb7b 100644
--- a/container-search/src/test/java/com/yahoo/prelude/fastsearch/test/FastSearcherTestCase.java
+++ b/container-search/src/test/java/com/yahoo/prelude/fastsearch/test/FastSearcherTestCase.java
@@ -30,7 +30,6 @@ import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
-import java.util.Map;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -83,7 +82,7 @@ public class FastSearcherTestCase {
@Test
void testSinglePassGroupingIsForcedWithSingleNodeGroups() {
FastSearcher fastSearcher = new FastSearcher("container.0",
- MockDispatcher.create(List.of(new Node(0, "host0", 0))),
+ MockDispatcher.create(List.of(new Node("test", 0, "host0", 0))),
new SummaryParameters(null),
new ClusterParams("testhittype"),
documentdbInfoConfig("test"),
@@ -106,7 +105,7 @@ public class FastSearcherTestCase {
@Test
void testRankProfileValidation() {
FastSearcher fastSearcher = new FastSearcher("container.0",
- MockDispatcher.create(List.of(new Node(0, "host0", 0))),
+ MockDispatcher.create(List.of(new Node("test", 0, "host0", 0))),
new SummaryParameters(null),
new ClusterParams("testhittype"),
documentdbInfoConfig("test"),
@@ -125,7 +124,7 @@ public class FastSearcherTestCase {
.setHasSummaryFeatures(false)
.build());
FastSearcher backend = new FastSearcher("container.0",
- MockDispatcher.create(Collections.singletonList(new Node(0, "host0", 0))),
+ MockDispatcher.create(Collections.singletonList(new Node("test", 0, "host0", 0))),
new SummaryParameters(null),
new ClusterParams("testhittype"),
documentDb,
@@ -142,7 +141,7 @@ public class FastSearcherTestCase {
@Test
void testSinglePassGroupingIsNotForcedWithSingleNodeGroups() {
- MockDispatcher dispatcher = MockDispatcher.create(List.of(new Node(0, "host0", 0), new Node(2, "host1", 0)));
+ MockDispatcher dispatcher = MockDispatcher.create(List.of(new Node("test", 0, "host0", 0), new Node("test", 2, "host1", 0)));
FastSearcher fastSearcher = new FastSearcher("container.0",
dispatcher,
@@ -184,7 +183,7 @@ public class FastSearcherTestCase {
searchClusterB.name(clusterName);
b.searchcluster(searchClusterB);
VipStatus vipStatus = new VipStatus(b.build());
- List<Node> nodes_1 = List.of(new Node(0, "host0", 0));
+ List<Node> nodes_1 = List.of(new Node("test", 0, "host0", 0));
RpcResourcePool rpcPool_1 = new RpcResourcePool(MockDispatcher.toDispatchConfig(), MockDispatcher.toNodesConfig(nodes_1));
MockDispatcher dispatch_1 = MockDispatcher.create(nodes_1, rpcPool_1, vipStatus);
dispatch_1.clusterMonitor.shutdown();
diff --git a/container-search/src/test/java/com/yahoo/search/dispatch/DispatcherTest.java b/container-search/src/test/java/com/yahoo/search/dispatch/DispatcherTest.java
index 1278afe3759..d0f1f46d6ea 100644
--- a/container-search/src/test/java/com/yahoo/search/dispatch/DispatcherTest.java
+++ b/container-search/src/test/java/com/yahoo/search/dispatch/DispatcherTest.java
@@ -21,6 +21,7 @@ import com.yahoo.search.dispatch.searchcluster.SearchGroups;
import com.yahoo.search.searchchain.Execution;
import com.yahoo.vespa.config.search.DispatchConfig;
import com.yahoo.vespa.config.search.DispatchNodesConfig;
+import com.yahoo.yolean.UncheckedInterruptedException;
import org.junit.jupiter.api.Test;
import java.io.IOException;
@@ -36,6 +37,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.Phaser;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -77,7 +79,7 @@ public class DispatcherTest {
SearchCluster cl = new MockSearchCluster("1", 0, 0) {
@Override
public Optional<Node> localCorpusDispatchTarget() {
- return Optional.of(new Node(1, "test", 1));
+ return Optional.of(new Node("test", 1, "test", 1));
}
};
MockInvokerFactory invokerFactory = new MockInvokerFactory(cl.groupList(), dispatchConfig, (n, a) -> true);
@@ -181,7 +183,11 @@ public class DispatcherTest {
pingPhasers.put(1, new Phaser(2));
pingPhasers.put(2, new Phaser(2));
+ AtomicBoolean doPing = new AtomicBoolean();
+
PingFactory pingFactory = (node, monitor, pongHandler) -> () -> {
+ try { while ( ! doPing.getAndSet(false)) { monitor.wait(1); } } // Need to avoid hogging monitor lock while waiting for phaser.
+ catch (InterruptedException e) { throw new UncheckedInterruptedException(e, true); }
pingPhasers.get(node.key()).arriveAndAwaitAdvance();
pongHandler.handle(new Pong(2, 2));
pingPhasers.get(node.key()).arriveAndAwaitAdvance();
@@ -255,8 +261,8 @@ public class DispatcherTest {
Dispatcher dispatcher = new Dispatcher(dispatchConfig, rpcPool, cluster, invokerFactories);
ExecutorService executor = Executors.newFixedThreadPool(1);
- // Set two groups with a single node each. The first cluster-monitor has nothing to do, and is shut down immediately.
- // There are also no invokers, so the whole reconfiguration completes once the new cluster monitor has seen all nodes.
+ // Set two groups with a single node each.
+ // There are no invokers, so the whole reconfiguration completes once the cluster monitor has seen all the new nodes.
Future<?> reconfiguration = executor.submit(() -> {
dispatcher.updateWithNewConfig(new DispatchNodesConfig.Builder()
.node(new DispatchNodesConfig.Node.Builder().key(0).group(0).port(123).host("host0"))
@@ -265,8 +271,10 @@ public class DispatcherTest {
});
// Let pings return, to allow the search cluster to reconfigure.
+ doPing.set(true);
pingPhasers.get(0).arriveAndAwaitAdvance();
pingPhasers.get(0).arriveAndAwaitAdvance();
+ doPing.set(true);
pingPhasers.get(1).arriveAndAwaitAdvance();
pingPhasers.get(1).arriveAndAwaitAdvance();
// We need to wait for the cluster to have at least one group, lest dispatch will fail below.
@@ -287,9 +295,10 @@ public class DispatcherTest {
search1.search(new Query(), null);
// Wait for the current cluster monitor to be mid-ping-round.
+ doPing.set(true);
pingPhasers.get(0).arriveAndAwaitAdvance();
- // Then reconfigure the dispatcher with new nodes, replacing node0 with node2.
+ // Reconfigure the dispatcher with new nodes, removing node0 and adding node2.
reconfiguration = executor.submit(() -> {
dispatcher.updateWithNewConfig(new DispatchNodesConfig.Builder()
.node(new DispatchNodesConfig.Node.Builder().key(2).group(0).port(123).host("host2"))
@@ -297,16 +306,23 @@ public class DispatcherTest {
.build());
});
// Reconfiguration starts, but groups are only updated once the search cluster has knowledge about all of them.
+ pingPhasers.get(0).arriveAndAwaitAdvance(); // Ping for node to remove completes.
+ doPing.set(true);
+ pingPhasers.get(1).arriveAndAwaitAdvance(); // Ping for node to keep completes.
pingPhasers.get(1).arriveAndAwaitAdvance();
+ // New round of pings starts, with nodes 1 and 2.
+ doPing.set(true);
pingPhasers.get(1).arriveAndAwaitAdvance();
- pingPhasers.get(2).arriveAndAwaitAdvance();
+ pingPhasers.get(1).arriveAndAwaitAdvance();
+
// Cluster has not yet updated its group reference.
assertEquals(1, cluster.group(0).workingNodes()); // Node0 is still working.
assertSame(node0, cluster.group(0).nodes().get(0));
+
+ doPing.set(true);
+ pingPhasers.get(2).arriveAndAwaitAdvance();
pingPhasers.get(2).arriveAndAwaitAdvance();
- // Old cluster monitor is waiting for that ping to complete before it can shut down, and let reconfiguration complete.
- pingPhasers.get(0).arriveAndAwaitAdvance();
reconfiguration.get();
Node node2 = cluster.group(0).nodes().get(0);
assertNotSame(node0, node2);
diff --git a/container-search/src/test/java/com/yahoo/search/dispatch/InterleavedSearchInvokerTest.java b/container-search/src/test/java/com/yahoo/search/dispatch/InterleavedSearchInvokerTest.java
index 688cdffe22d..500201df26f 100644
--- a/container-search/src/test/java/com/yahoo/search/dispatch/InterleavedSearchInvokerTest.java
+++ b/container-search/src/test/java/com/yahoo/search/dispatch/InterleavedSearchInvokerTest.java
@@ -238,8 +238,8 @@ public class InterleavedSearchInvokerTest {
@Test
void requireThatTopKProbabilityOverrideIsDisabledOnContentSkew() throws IOException {
- Node node0 = new Node(0, "host0", 0);
- Node node1 = new Node(1, "host1", 0);
+ Node node0 = new Node("test", 0, "host0", 0);
+ Node node1 = new Node("test", 1, "host1", 0);
Group group = new Group(0, List.of(node0, node1));
node0.setActiveDocuments(1000000);
@@ -250,8 +250,8 @@ public class InterleavedSearchInvokerTest {
@Test
void requireThatTopKProbabilityOverrideIsDisabledOnLittleContent() throws IOException {
- Node node0 = new Node(0, "host0", 0);
- Node node1 = new Node(1, "host1", 0);
+ Node node0 = new Node("test", 0, "host0", 0);
+ Node node1 = new Node("test", 1, "host1", 0);
Group group = new Group(0, List.of(node0, node1));
node0.setActiveDocuments(10);
diff --git a/container-search/src/test/java/com/yahoo/search/dispatch/LoadBalancerTest.java b/container-search/src/test/java/com/yahoo/search/dispatch/LoadBalancerTest.java
index 4956698cc2f..b57d97ebb84 100644
--- a/container-search/src/test/java/com/yahoo/search/dispatch/LoadBalancerTest.java
+++ b/container-search/src/test/java/com/yahoo/search/dispatch/LoadBalancerTest.java
@@ -28,7 +28,7 @@ public class LoadBalancerTest {
private static final double delta = 0.0000001;
@Test
void requireThatLoadBalancerServesSingleNodeSetups() {
- Node n1 = new Node(0, "test-node1", 0);
+ Node n1 = new Node("test", 0, "test-node1", 0);
LoadBalancer lb = new LoadBalancer(List.of(new Group(0, List.of(n1))), LoadBalancer.Policy.ROUNDROBIN);
Optional<Group> grp = lb.takeGroup(null);
@@ -40,8 +40,8 @@ public class LoadBalancerTest {
@Test
void requireThatLoadBalancerServesMultiGroupSetups() {
- Node n1 = new Node(0, "test-node1", 0);
- Node n2 = new Node(1, "test-node2", 1);
+ Node n1 = new Node("test", 0, "test-node1", 0);
+ Node n2 = new Node("test", 1, "test-node2", 1);
LoadBalancer lb = new LoadBalancer(List.of(new Group(0, List.of(n1)), new Group(1,List.of(n2))), LoadBalancer.Policy.ROUNDROBIN);
Optional<Group> grp = lb.takeGroup(null);
@@ -53,10 +53,10 @@ public class LoadBalancerTest {
@Test
void requireThatLoadBalancerServesClusteredGroups() {
- Node n1 = new Node(0, "test-node1", 0);
- Node n2 = new Node(1, "test-node2", 0);
- Node n3 = new Node(0, "test-node3", 1);
- Node n4 = new Node(1, "test-node4", 1);
+ Node n1 = new Node("test", 0, "test-node1", 0);
+ Node n2 = new Node("test", 1, "test-node2", 0);
+ Node n3 = new Node("test", 0, "test-node3", 1);
+ Node n4 = new Node("test", 1, "test-node4", 1);
LoadBalancer lb = new LoadBalancer(List.of(new Group(0, List.of(n1,n2)), new Group(1,List.of(n3,n4))), LoadBalancer.Policy.ROUNDROBIN);
Optional<Group> grp = lb.takeGroup(null);
@@ -65,8 +65,8 @@ public class LoadBalancerTest {
@Test
void requireThatLoadBalancerReturnsDifferentGroups() {
- Node n1 = new Node(0, "test-node1", 0);
- Node n2 = new Node(1, "test-node2", 1);
+ Node n1 = new Node("test", 0, "test-node1", 0);
+ Node n2 = new Node("test", 1, "test-node2", 1);
LoadBalancer lb = new LoadBalancer(List.of(new Group(0, List.of(n1)), new Group(1,List.of(n2))), LoadBalancer.Policy.ROUNDROBIN);
// get first group
diff --git a/container-search/src/test/java/com/yahoo/search/dispatch/MockInvoker.java b/container-search/src/test/java/com/yahoo/search/dispatch/MockInvoker.java
index aca84386af7..b47015c08c6 100644
--- a/container-search/src/test/java/com/yahoo/search/dispatch/MockInvoker.java
+++ b/container-search/src/test/java/com/yahoo/search/dispatch/MockInvoker.java
@@ -3,17 +3,13 @@ package com.yahoo.search.dispatch;
import com.yahoo.prelude.fastsearch.FastHit;
import com.yahoo.search.Query;
-import com.yahoo.search.Result;
-import com.yahoo.search.dispatch.searchcluster.Group;
import com.yahoo.search.dispatch.searchcluster.Node;
import com.yahoo.search.result.Coverage;
import com.yahoo.search.result.Hit;
import com.yahoo.search.searchchain.Execution;
-import java.io.IOException;
import java.util.List;
import java.util.Optional;
-import java.util.OptionalInt;
class MockInvoker extends SearchInvoker {
@@ -23,7 +19,7 @@ class MockInvoker extends SearchInvoker {
int hitsRequested;
protected MockInvoker(int key, Coverage coverage) {
- super(Optional.of(new Node(key, "?", 0)));
+ super(Optional.of(new Node("test", key, "?", 0)));
this.coverage = coverage;
}
diff --git a/container-search/src/test/java/com/yahoo/search/dispatch/rpc/RpcSearchInvokerTest.java b/container-search/src/test/java/com/yahoo/search/dispatch/rpc/RpcSearchInvokerTest.java
index 7c1e7372507..b877bac5d74 100644
--- a/container-search/src/test/java/com/yahoo/search/dispatch/rpc/RpcSearchInvokerTest.java
+++ b/container-search/src/test/java/com/yahoo/search/dispatch/rpc/RpcSearchInvokerTest.java
@@ -33,7 +33,7 @@ public class RpcSearchInvokerTest {
var lengthHolder = new AtomicInteger();
var mockClient = parameterCollectorClient(compressionTypeHolder, payloadHolder, lengthHolder);
var mockPool = new RpcResourcePool(ImmutableMap.of(7, mockClient.createConnection("foo", 123)));
- var invoker = new RpcSearchInvoker(mockSearcher(), compressor, new Node(7, "seven", 1), mockPool, 1000);
+ var invoker = new RpcSearchInvoker(mockSearcher(), compressor, new Node("test", 7, "seven", 1), mockPool, 1000);
Query q = new Query("search/?query=test&hits=10&offset=3");
RpcSearchInvoker.RpcContext context = (RpcSearchInvoker.RpcContext) invoker.sendSearchRequest(q, null);
@@ -47,7 +47,7 @@ public class RpcSearchInvokerTest {
assertEquals(3, request.getOffset());
assertTrue(request.getQueryTreeBlob().size() > 0);
- var invoker2 = new RpcSearchInvoker(mockSearcher(), compressor, new Node(8, "eight", 1), mockPool, 1000);
+ var invoker2 = new RpcSearchInvoker(mockSearcher(), compressor, new Node("test", 8, "eight", 1), mockPool, 1000);
RpcSearchInvoker.RpcContext context2 = (RpcSearchInvoker.RpcContext) invoker2.sendSearchRequest(q, context);
assertSame(context, context2);
assertEquals(lengthHolder.get(), context.compressedPayload.uncompressedSize());
@@ -62,7 +62,7 @@ public class RpcSearchInvokerTest {
var lengthHolder = new AtomicInteger();
var mockClient = parameterCollectorClient(compressionTypeHolder, payloadHolder, lengthHolder);
var mockPool = new RpcResourcePool(ImmutableMap.of(7, mockClient.createConnection("foo", 123)));
- var invoker = new RpcSearchInvoker(mockSearcher(), compressor, new Node(7, "seven", 1), mockPool, maxHits);
+ var invoker = new RpcSearchInvoker(mockSearcher(), compressor, new Node("test", 7, "seven", 1), mockPool, maxHits);
Query q = new Query("search/?query=test&hits=10&offset=3");
invoker.sendSearchRequest(q, null);
diff --git a/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/MockSearchCluster.java b/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/MockSearchCluster.java
index cd0791a3881..6900cc5dd52 100644
--- a/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/MockSearchCluster.java
+++ b/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/MockSearchCluster.java
@@ -79,7 +79,7 @@ public class MockSearchCluster extends SearchCluster {
for (int group = 0; group < numGroups; group++) {
List<Node> groupNodes = new ArrayList<>();
for (int i = 0; i < nodesPerGroup; i++) {
- Node node = new Node(distributionKey, "host" + distributionKey, group);
+ Node node = new Node("test", distributionKey, "host" + distributionKey, group);
node.setWorking(true);
groupNodes.add(node);
distributionKey++;
diff --git a/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/SearchClusterTest.java b/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/SearchClusterTest.java
index bfe1aed1084..c3ddeac5365 100644
--- a/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/SearchClusterTest.java
+++ b/container-search/src/test/java/com/yahoo/search/dispatch/searchcluster/SearchClusterTest.java
@@ -6,12 +6,14 @@ import com.yahoo.container.handler.ClustersStatus;
import com.yahoo.container.handler.VipStatus;
import com.yahoo.net.HostName;
import com.yahoo.prelude.Pong;
+import com.yahoo.search.cluster.BaseNodeMonitor;
import com.yahoo.search.cluster.ClusterMonitor;
import com.yahoo.search.result.ErrorMessage;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -19,8 +21,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Function;
-import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static java.util.function.Function.identity;
@@ -58,7 +58,7 @@ public class SearchClusterTest {
for (String name : nodeNames) {
int key = nodes.size() % nodesPerGroup;
int group = nodes.size() / nodesPerGroup;
- nodes.add(new Node(key, name, group));
+ nodes.add(new Node("test", key, name, group));
numDocsPerNode.add(new AtomicInteger(1));
pingCounts.add(new AtomicInteger(0));
}
@@ -326,7 +326,7 @@ public class SearchClusterTest {
@Test
void requireThatPingSequenceIsUpHeld() {
- Node node = new Node(1, "n", 1);
+ Node node = new Node("test", 1, "n", 1);
assertEquals(1, node.createPingSequenceId());
assertEquals(2, node.createPingSequenceId());
assertEquals(0, node.getLastReceivedPongId());
@@ -348,7 +348,7 @@ public class SearchClusterTest {
@Test
void requireThatSingleNodeGroupIsInBalance() {
- Group group = new Group(0, List.of(new Node(1, "n", 1)));
+ Group group = new Group(0, List.of(new Node("test", 1, "n", 1)));
group.nodes().forEach(node -> node.setWorking(true));
assertTrue(group.isBalanced());
group.aggregateNodeValues();
@@ -360,7 +360,7 @@ public class SearchClusterTest {
@Test
void requireThatMultiNodeGroupDetectsBalance() {
- Group group = new Group(0, List.of(new Node(1, "n1", 1), new Node(2, "n2", 1)));
+ Group group = new Group(0, List.of(new Node("test", 1, "n1", 1), new Node("test", 2, "n2", 1)));
assertTrue(group.isBalanced());
group.nodes().forEach(node -> node.setWorking(true));
assertTrue(group.isBalanced());
@@ -386,33 +386,48 @@ public class SearchClusterTest {
@Test
void requireThatPreciselyTheRetainedNodesAreKeptWhenNodesAreUpdated() {
try (State state = new State("query", 2, IntStream.range(0, 6).mapToObj(i -> "node-" + i).toList())) {
- List<Node> referenceNodes = List.of(new Node(0, "node-0", 0),
- new Node(1, "node-1", 0),
- new Node(0, "node-2", 1),
- new Node(1, "node-3", 1),
- new Node(0, "node-4", 2),
- new Node(1, "node-5", 2));
+ state.clusterMonitor.start();
+ List<Node> referenceNodes = List.of(new Node("test", 0, "node-0", 0),
+ new Node("test", 1, "node-1", 0),
+ new Node("test", 0, "node-2", 1),
+ new Node("test", 1, "node-3", 1),
+ new Node("test", 0, "node-4", 2),
+ new Node("test", 1, "node-5", 2));
SearchGroups oldGroups = state.searchCluster.groupList();
assertEquals(Set.copyOf(referenceNodes), oldGroups.nodes());
-
- List<Node> updatedNodes = List.of(new Node(0, "node-1", 0), // Swap node-0 and node-1
- new Node(1, "node-0", 0), // Swap node-1 and node-0
- new Node(0, "node-4", 1), // Swap node-2 and node-4
- new Node(1, "node-3", 1),
- new Node(0, "node-2", 2), // Swap node-4 and node-2
- new Node(1, "node-6", 2)); // Replace node-6
- state.searchCluster.updateNodes(updatedNodes, 100.0);
+ List<BaseNodeMonitor<Node>> oldMonitors = state.clusterMonitor.nodeMonitors();
+
+ List<Node> updatedNodes = List.of(new Node("test", 0, "node-1", 0), // Swap node-0 and node-1
+ new Node("test", 1, "node-0", 0), // Swap node-1 and node-0
+ new Node("test", 0, "node-4", 1), // Swap node-2 and node-4
+ new Node("test", 1, "node-3", 1),
+ new Node("test", 0, "node-2", 2), // Swap node-4 and node-2
+ new Node("test", 1, "node-6", 2)); // Replace node-6
+ state.searchCluster.updateNodes(updatedNodes, state.clusterMonitor, 100.0);
SearchGroups newGroups = state.searchCluster.groupList();
assertEquals(Set.copyOf(updatedNodes), newGroups.nodes());
- Map<Node, Node> oldNodesByIdentity = newGroups.nodes().stream().collect(toMap(identity(), identity()));
+ Map<Node, Node> oldNodesByIdentity = oldGroups.nodes().stream().collect(toMap(identity(), identity()));
Map<Node, Node> newNodesByIdentity = newGroups.nodes().stream().collect(toMap(identity(), identity()));
assertSame(updatedNodes.get(0), newNodesByIdentity.get(updatedNodes.get(0)));
assertSame(updatedNodes.get(1), newNodesByIdentity.get(updatedNodes.get(1)));
assertSame(updatedNodes.get(2), newNodesByIdentity.get(updatedNodes.get(2)));
- assertSame(oldNodesByIdentity.get(referenceNodes.get(3)), newNodesByIdentity.get(updatedNodes.get(3)));
+ assertSame(oldNodesByIdentity.get(updatedNodes.get(3)), newNodesByIdentity.get(updatedNodes.get(3)));
assertSame(updatedNodes.get(4), newNodesByIdentity.get(updatedNodes.get(4)));
assertSame(updatedNodes.get(5), newNodesByIdentity.get(updatedNodes.get(5)));
+
+ // Also verify search-path index within group follows node order, as given by config.
+ int[] pathIndexWithinGroup = new int[3];
+ for (Node node : updatedNodes)
+ assertEquals(pathIndexWithinGroup[node.group()]++, newNodesByIdentity.get(node).pathIndex(),
+ "search path index within group should follow updated node order for: " + node);
+
+ // Precisely the one retained node keeps its monitor through reconfiguration.
+ Set<BaseNodeMonitor<Node>> retainedMonitors = new HashSet<>(state.clusterMonitor.nodeMonitors());
+ assertEquals(6, retainedMonitors.size());
+ retainedMonitors.retainAll(oldMonitors);
+ assertEquals(1, retainedMonitors.size());
+ assertSame(oldNodesByIdentity.get(updatedNodes.get(3)), retainedMonitors.iterator().next().getNode());
}
}