Skip to content
This repository has been archived by the owner on Mar 29, 2024. It is now read-only.

Test listeners for state.json and prs state update #165

Open
wants to merge 10 commits into
base: release/8.8
Choose a base branch
from
21 changes: 21 additions & 0 deletions solr/core/src/java/org/apache/solr/cloud/Overseer.java
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,8 @@ public class Overseer implements SolrCloseable {

private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

private volatile OverseerMessageListener testOverseerMessageListener = null;

enum LeaderStatus {DONT_KNOW, NO, YES}

private class ClusterStateUpdater implements Runnable, Closeable {
Expand Down Expand Up @@ -409,6 +411,9 @@ private List<ZkWriteCommand> processMessage(ClusterState clusterState,
final ZkNodeProps message, final String operation, final ZkStateWriter zkStateWriter) {
CollectionParams.CollectionAction collectionAction = CollectionParams.CollectionAction.get(operation);
if (collectionAction != null) {
if (testOverseerMessageListener != null) {
testOverseerMessageListener.onMessage(collectionAction, message);
}
switch (collectionAction) {
case CREATE:
return Collections.singletonList(new ClusterStateMutator(getSolrCloudManager()).createCollection(clusterState, message));
Expand Down Expand Up @@ -450,6 +455,9 @@ private List<ZkWriteCommand> processMessage(ClusterState clusterState,
if (overseerAction == null) {
throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
}
if (testOverseerMessageListener != null) {
testOverseerMessageListener.onMessage(overseerAction, message);
}
switch (overseerAction) {
case STATE:
return Collections.singletonList(new ReplicaMutator(getSolrCloudManager()).setState(clusterState, message));
Expand Down Expand Up @@ -1057,4 +1065,17 @@ public interface Message {

}

/**
* for testing purpose only
*/
public OverseerMessageListener getTestOverseerMessageListener() {
return testOverseerMessageListener;
}

/**
* for testing purpose only
*/
public void setTestOverseerMessageListener(OverseerMessageListener testOverseerMessageListener) {
this.testOverseerMessageListener = testOverseerMessageListener;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.solr.cloud;

import org.apache.solr.cloud.overseer.OverseerAction;import org.apache.solr.common.cloud.ZkNodeProps;import org.apache.solr.common.params.CollectionParams; /**
* register listener for overseer messages
*/
public interface OverseerMessageListener {
void onMessage(CollectionParams.CollectionAction collectionAction, ZkNodeProps message);
void onMessage(OverseerAction overseerAction, ZkNodeProps message);
}
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ void runLeaderProcess(boolean weAreReplacement, int pauseBeforeStartMs)
} else {
PerReplicaStates prs = PerReplicaStates.fetch(coll.getZNode(), zkClient, coll.getPerReplicaStates());
PerReplicaStatesOps.flipLeader(zkStateReader.getClusterState().getCollection(collection).getSlice(shardId).getReplicaNames(), id, prs)
.persist(coll.getZNode(), zkStateReader.getZkClient());
.persist(coll.getZNode(), zkStateReader.getZkClient(), zkStateReader.getTestPerReplicaStateUpdateListener());
}
}
}
Expand Down
4 changes: 2 additions & 2 deletions solr/core/src/java/org/apache/solr/cloud/ZkController.java
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@
import org.apache.solr.common.cloud.LiveNodesListener;
import org.apache.solr.common.cloud.NodesSysPropsCacher;
import org.apache.solr.common.cloud.OnReconnect;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.PerReplicaStateUpdateListener;import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.solr.common.cloud.PerReplicaStatesOps;
import org.apache.solr.common.cloud.Replica.Type;
Expand Down Expand Up @@ -1718,7 +1718,7 @@ public void publish(final CoreDescriptor cd, final Replica.State state, boolean
}
PerReplicaStates perReplicaStates = PerReplicaStates.fetch(coll.getZNode(), zkClient, coll.getPerReplicaStates());
PerReplicaStatesOps.flipState(coreNodeName, state, perReplicaStates)
.persist(coll.getZNode(), zkClient);
.persist(coll.getZNode(), zkClient, this.zkStateReader.getTestPerReplicaStateUpdateListener());
}
} finally {
MDCLoggingContext.clear();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,7 @@ public ClusterState writePendingUpdates(Map<String, ZkWriteCommand> updates, boo
DocCollection c = cmd.collection;

if (cmd.ops != null && cmd.ops.isPreOp()) {
cmd.ops.persist(path, reader.getZkClient());
cmd.ops.persist(path, reader.getZkClient(), this.reader.getTestPerReplicaStateUpdateListener());
clusterState = clusterState.copyWith(name,
cmd.collection.copyWith(PerReplicaStates.fetch(cmd.collection.getZNode(), reader.getZkClient(), null)));
}
Expand Down Expand Up @@ -280,7 +280,7 @@ public ClusterState writePendingUpdates(Map<String, ZkWriteCommand> updates, boo
}
if (cmd.ops != null && !cmd.ops.isPreOp()) {
log.info("persisting PRS states {}", Utils.toJSONString(cmd.ops.get()));
cmd.ops.persist(path, reader.getZkClient());
cmd.ops.persist(path, reader.getZkClient(), this.reader.getTestPerReplicaStateUpdateListener());
DocCollection currentCollState = clusterState.getCollection(cmd.name);
if ( currentCollState != null) {
clusterState = clusterState.copyWith(name,
Expand Down
144 changes: 124 additions & 20 deletions solr/core/src/test/org/apache/solr/core/FSPRSTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,25 +19,29 @@

import java.lang.invoke.MethodHandles;
import java.util.Arrays;
import java.lang.invoke.MethodHandles;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;

import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.SolrServerException;import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.cloud.MiniSolrCloudCluster;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.NavigableObject;
import org.apache.solr.cloud.OverseerMessageListener;import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.cloud.overseer.OverseerAction;import org.apache.solr.common.NavigableObject;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.PerReplicaStateUpdateListener;import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.solr.common.cloud.PerReplicaStatesOps;import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.cloud.ZkNodeProps;import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.LogLevel;
import org.apache.zookeeper.data.Stat;
Expand All @@ -48,6 +52,42 @@
public class FSPRSTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

public static class OverseerListener implements OverseerMessageListener {
List<ZkNodeProps> events = new LinkedList<>();

@Override
public void onMessage(CollectionParams.CollectionAction collectionAction, ZkNodeProps message) {
events.add(message);
}

@Override
public void onMessage(OverseerAction overseerAction, ZkNodeProps message) {
events.add(message);
}

@Override public String toString() {
return "OverseerListener{" +
"events=" + events + "}";
}
}

public static class PrsUpdateListener implements PerReplicaStateUpdateListener {

List<List<PerReplicaStates.Operation>> events = new LinkedList<>();

@Override
public void onUpdate(String znode, List<PerReplicaStates.Operation> operations) {
events.add(operations);
}

@Override
public String toString() {
return "PrsUpdateListener{" +
"events=" + events +
'}';
}
}

public void testShardSplitWithNodeset() throws Exception {
String COLL = "prs_shard_split_nodeset";
MiniSolrCloudCluster cluster =
Expand All @@ -56,6 +96,8 @@ public void testShardSplitWithNodeset() throws Exception {
.addConfig("conf", configset("conf2"))
.configure();

cluster.getJettySolrRunner(1).getCoreContainer().getZkController().getOverseer().setTestOverseerMessageListener(null);

try {
CollectionAdminRequest.createCollection(COLL, "conf", 2, 2)
.setMaxShardsPerNode(4)
Expand Down Expand Up @@ -113,7 +155,7 @@ public void testShardSplit() throws Exception {
.configure();
int COLL_COUNT = 2;
int SPLIT_COUNT = 1;

try {
for (int i = 0; i < COLL_COUNT; i++) {
String C = COLL + i;
Expand All @@ -125,13 +167,13 @@ public void testShardSplit() throws Exception {
for (int i = 0; i < COLL_COUNT; i++) {
String C = COLL + i;
CollectionAdminRequest.modifyCollection(C,
Collections.singletonMap("perReplicaState", "true"))
Collections.singletonMap("perReplicaState", "true"))
.process(cluster.getSolrClient());
PerReplicaStates prs = PerReplicaStates.fetch(ZkStateReader.getCollectionPath(C), cluster.getZkClient(), null);
assertTrue(prs.states.size() >= 4);
}

for (int i=1; i<3; i++) {
for (int i = 1; i < 3; i++) {
JettySolrRunner jetty = cluster.stopJettySolrRunner(i);
cluster.startJettySolrRunner(jetty);
}
Expand All @@ -154,7 +196,7 @@ public void testShardSplit() throws Exception {
for (int i = 0; i < COLL_COUNT; i++) {
String C = COLL + i;
CollectionAdminRequest.modifyCollection(C,
Collections.singletonMap("perReplicaState", "false"))
Collections.singletonMap("perReplicaState", "false"))
.process(cluster.getSolrClient());
}

Expand All @@ -172,12 +214,12 @@ public void testShardSplit() throws Exception {
for (int i = 0; i < COLL_COUNT; i++) {
String C = COLL + i;
PerReplicaStates prs = PerReplicaStates.fetch(ZkStateReader.getCollectionPath(C), cluster.getZkClient(), null);
assertEquals(0,prs.states.size());
assertEquals(0, prs.states.size());
}
for (int i = 0; i < COLL_COUNT; i++) {
String C = COLL + i;
CollectionAdminRequest.modifyCollection(C,
Collections.singletonMap("perReplicaState", "true"))
Collections.singletonMap("perReplicaState", "true"))
.process(cluster.getSolrClient());
}

Expand All @@ -200,6 +242,69 @@ public void testShardSplit() throws Exception {

}

public void testPRSCollectionCreation() throws Exception {
String COLL = "prs_test_coll";
MiniSolrCloudCluster cluster =
configureCluster(1)
.withJettyConfig(jetty -> jetty.enableV2(true))
.addConfig("conf", configset("conf2"))
.configure();

System.setProperty("overseer.node", "true");
JettySolrRunner overseerNode = cluster.startJettySolrRunner();
System.clearProperty("overseer.node");

JettySolrRunner leader = getOverseerleader(cluster);
Thread.sleep(8000);
assertEquals(overseerNode, leader);

CoreContainer dataCore = cluster.getJettySolrRunner(0).getCoreContainer();
OverseerListener dataNodeOverseerListener = new OverseerListener();
dataCore.getZkController().getOverseer().setTestOverseerMessageListener(dataNodeOverseerListener);
PrsUpdateListener dataNodePrsListener = new PrsUpdateListener();
dataCore.getZkController().zkStateReader.setTestPerReplicaStateUpdateListener(dataNodePrsListener);

CoreContainer overseerCore = leader.getCoreContainer();
OverseerListener overseerNodeOverseerListener = new OverseerListener();
overseerCore.getZkController().getOverseer().setTestOverseerMessageListener(overseerNodeOverseerListener);
PrsUpdateListener overseerNodePrsListener = new PrsUpdateListener();
overseerCore.getZkController().zkStateReader.setTestPerReplicaStateUpdateListener(overseerNodePrsListener);

CollectionAdminRequest.createCollection(COLL, "conf", 1, 1)
.setPerReplicaState(Boolean.TRUE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLL, 1, 1);

//For overseer node, it is still currently updating PRS state once but this should be fixed in the future
assertTrue("overseer node should not update prs state " + overseerNodePrsListener, overseerNodePrsListener.events.size() == 1);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add a comment here to indicate that it is still being updated by overseer node for now? otherwise reader might get confused with the error message and the actual check condition (which does allow 1 update)

assertTrue("data node should have 3 prs state update " + dataNodePrsListener , dataNodePrsListener.events.size() == 3);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Very nice that we can compare the data node update vs os node update now! 💪🏼

This has major advantage over my other test testZkNodeVersions which checks the znode version and child version but does not know who updates it.

We can consider removing testZkNodeVersions in the future. However, there's one thing that testZkNodeVersions catches while this test case does not - there are direct updates on state.json performed by overseer node, which NOT caught by dataNodeOverseerListener as it does not go through the overseer q 😞 (so it's actually 2 updates to state.json from overseer instead of 1). This is the update : https://github.com/fullstorydev/lucene-solr/blob/release/8.8/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java#L323 . in order to catch it we might need to instrument even lower level (zkclient) as discussed earlier

assertTrue("data node should not update state.json " + dataNodeOverseerListener, dataNodeOverseerListener.events.size() == 0);
//There are actually 2 updates to state.json, but one is done directly in
//https://github.com/fullstorydev/lucene-solr/blob/ca8146475f5a19f657fe178070c97d4d08534f6f/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java#L323
assertTrue("overseer node should update state.json file 1 time " + overseerNodeOverseerListener, overseerNodeOverseerListener.events.size() == 1);
overseerNode.stop();
leader.stop();
}

private JettySolrRunner getOverseerleader(MiniSolrCloudCluster cluster)throws IOException, SolrServerException {
NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
JettySolrRunner overseerJetty = null;
String overseerLeader = (String) overSeerStatus.get("leader");
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner jetty = cluster.getJettySolrRunner(i);
if (jetty.getNodeName().equals(overseerLeader)) {
overseerJetty = jetty;
break;
}
}

if (overseerJetty == null ) {
throw new RuntimeException("Overseer node not found ");
}

return overseerJetty;
}

public void testIntegration() throws Exception {
final String SHARD1 = "shard1";
final String SHARD1_0 = SHARD1 + "_0";
Expand All @@ -217,23 +322,23 @@ public void testIntegration() throws Exception {
cluster.waitForActiveCollection(COLL, 1, 2);

UpdateRequest ur = new UpdateRequest();
for(int i=0;i<10;i++) ur.add("id", ""+i);
for (int i = 0; i < 10; i++) ur.add("id", "" + i);
ur.commit(cluster.getSolrClient(), COLL);

System.setProperty(CoreContainer.SOLR_QUERY_AGGREGATOR, "true");
JettySolrRunner qaJetty = cluster.startJettySolrRunner();
System.clearProperty(CoreContainer.SOLR_QUERY_AGGREGATOR);
assertTrue(qaJetty.getCoreContainer().isQueryAggregator());

try(HttpSolrClient client = (HttpSolrClient) qaJetty.newClient()){
try (HttpSolrClient client = (HttpSolrClient) qaJetty.newClient()) {
NavigableObject result = (NavigableObject) Utils.executeGET(client.getHttpClient(),
qaJetty.getBaseUrl()+"/"+ COLL+"/select?q=*:*&wt=javabin", Utils.JAVABINCONSUMER
);
qaJetty.getBaseUrl() + "/" + COLL + "/select?q=*:*&wt=javabin", Utils.JAVABINCONSUMER
);
Collection<?> l = (Collection<?>) result._get("response", null);
assertEquals(10, l.size());
}
CollectionAdminRequest.modifyCollection(COLL,
Collections.singletonMap("perReplicaState", "true"))
Collections.singletonMap("perReplicaState", "true"))
.process(cluster.getSolrClient());
String collectionPath = ZkStateReader.getCollectionPath(COLL);
PerReplicaStates prs = PerReplicaStates.fetch(collectionPath, cluster.getZkClient(), null);
Expand All @@ -244,7 +349,7 @@ public void testIntegration() throws Exception {
Replica leader = collection.getReplica((s, replica) -> replica.isLeader());
Replica r = collection.getReplica((s, replica) -> !replica.isLeader());
CollectionAdminRequest.deleteReplica(COLL, SHARD1,
r.getName())
r.getName())
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLL, 1, 1);
prs = PerReplicaStates.fetch(collectionPath, cluster.getZkClient(), null);
Expand Down Expand Up @@ -369,7 +474,6 @@ public void testZkNodeVersions() throws Exception {
//ensure restart does not update the state.json
assertEquals(11, stat.getVersion());
}

} finally {
cluster.shutdown();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.solr.common.cloud;
import java.util.List;

public interface PerReplicaStateUpdateListener {
void onUpdate(String znode, List<PerReplicaStates.Operation> operations);
}
Loading