SOLR-15052: Per-replica states for reducing overseer bottlenecks (trunk) (#2177)

This commit is contained in:
Noble Paul 2021-01-19 02:59:41 +11:00 committed by GitHub
parent 4b508aef24
commit 8505d4d416
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
59 changed files with 1572 additions and 164 deletions

View File

@ -209,6 +209,8 @@ New Features
* SOLR-14560: Add interleaving support in Learning To Rank. (Alessandro Benedetti, Christine Poerschke)
* SOLR-15052: Reducing overseer bottlenecks using per-replica states (noble, Ishan Chattopadhyaya)
Improvements
---------------------
* SOLR-14942: Reduce leader election time on node shutdown by removing election nodes before closing cores.
@ -385,6 +387,8 @@ New Features
* SOLR-14615: CPU Utilization Based Circuit Breaker (Atri Sharma)
* SOLR-15052: Reducing overseer bottlenecks using per-replica states (noble, Ishan Chattopadhyaya)
Improvements
---------------------

View File

@ -19,18 +19,13 @@ package org.apache.solr.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.List;
import java.util.ArrayList;
import java.util.List;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.common.cloud.ZkMaintenanceUtils;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.*;
import org.apache.solr.common.util.RetryUtil;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.CreateMode;
@ -179,7 +174,14 @@ class ShardLeaderElectionContextBase extends ElectionContext {
ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
assert zkController != null;
assert zkController.getOverseer() != null;
DocCollection coll = zkStateReader.getCollection(this.collection);
if (coll == null || ZkController.sendToOverseer(coll, id)) {
zkController.getOverseer().offerStateUpdate(Utils.toJSON(m));
} 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());
}
}
}

View File

@ -64,31 +64,8 @@ import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.BeforeReconnect;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ConnectionManager;
import org.apache.solr.common.cloud.DefaultConnectionStrategy;
import org.apache.solr.common.cloud.DefaultZkACLProvider;
import org.apache.solr.common.cloud.DefaultZkCredentialsProvider;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocCollectionWatcher;
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.*;
import org.apache.solr.common.cloud.Replica.Type;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.UrlScheme;
import org.apache.solr.common.cloud.ZkACLProvider;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkCredentialsProvider;
import org.apache.solr.common.cloud.ZkMaintenanceUtils;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.SolrParams;
@ -1606,12 +1583,40 @@ public class ZkController implements Closeable {
if (updateLastState) {
cd.getCloudDescriptor().setLastPublished(state);
}
DocCollection coll = zkStateReader.getCollection(collection);
if (forcePublish || sendToOverseer(coll, coreNodeName)) {
overseerJobQueue.offer(Utils.toJSON(m));
} else {
if (log.isDebugEnabled()) {
log.debug("bypassed overseer for message : {}", Utils.toJSONString(m));
}
PerReplicaStates perReplicaStates = PerReplicaStates.fetch(coll.getZNode(), zkClient, coll.getPerReplicaStates());
PerReplicaStatesOps.flipState(coreNodeName, state, perReplicaStates)
.persist(coll.getZNode(), zkClient);
}
} finally {
MDCLoggingContext.clear();
}
}
/**
* Whether a message needs to be sent to overseer or not
*/
static boolean sendToOverseer(DocCollection coll, String replicaName) {
if (coll == null) return true;
if (!coll.isPerReplicaState()) return true;
Replica r = coll.getReplica(replicaName);
if (r == null) return true;
Slice shard = coll.getSlice(r.shard);
if (shard == null) return true;//very unlikely
if (shard.getState() == Slice.State.RECOVERY) return true;
if (shard.getParent() != null) return true;
for (Slice slice : coll.getSlices()) {
if (Objects.equals(shard.getName(), slice.getParent())) return true;
}
return false;
}
public ZkShardTerms getShardTerms(String collection, String shardId) {
return getCollectionTerms(collection).getShard(shardId);
}

View File

@ -36,10 +36,10 @@ import com.google.common.collect.ImmutableMap;
import org.apache.commons.lang3.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.BadVersionException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
@ -75,6 +75,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.TimeSource;
@ -84,7 +85,6 @@ import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardRequest;
import org.apache.solr.handler.component.ShardResponse;
import org.apache.solr.logging.MDCLoggingContext;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.CreateMode;
@ -142,6 +142,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
ZkStateReader.REPLICATION_FACTOR, "1",
ZkStateReader.NRT_REPLICAS, "1",
ZkStateReader.TLOG_REPLICAS, "0",
DocCollection.PER_REPLICA_STATE, null,
ZkStateReader.PULL_REPLICAS, "0"));
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

View File

@ -25,13 +25,7 @@ import java.util.Map;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.*;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -46,10 +40,12 @@ public class CollectionMutator {
protected final SolrCloudManager cloudManager;
protected final DistribStateManager stateManager;
protected final SolrZkClient zkClient;
public CollectionMutator(SolrCloudManager cloudManager) {
this.cloudManager = cloudManager;
this.stateManager = cloudManager.getDistribStateManager();
this.zkClient = SliceMutator.getZkClient(cloudManager);
}
public ZkWriteCommand createShard(final ClusterState clusterState, ZkNodeProps message) {
@ -107,7 +103,21 @@ public class CollectionMutator {
DocCollection coll = clusterState.getCollection(message.getStr(COLLECTION_PROP));
Map<String, Object> m = coll.shallowCopy();
boolean hasAnyOps = false;
PerReplicaStatesOps replicaOps = null;
for (String prop : CollectionAdminRequest.MODIFIABLE_COLLECTION_PROPERTIES) {
if (prop.equals(DocCollection.PER_REPLICA_STATE)) {
String val = message.getStr(DocCollection.PER_REPLICA_STATE);
if (val == null) continue;
boolean enable = Boolean.parseBoolean(val);
if (enable == coll.isPerReplicaState()) {
//already enabled
log.error("trying to set perReplicaState to {} from {}", val, coll.isPerReplicaState());
continue;
}
replicaOps = PerReplicaStatesOps.modifyCollection(coll, enable, PerReplicaStates.fetch(coll.getZNode(), zkClient, null));
}
if (message.containsKey(prop)) {
hasAnyOps = true;
if (message.get(prop) == null) {
@ -136,8 +146,12 @@ public class CollectionMutator {
return ZkStateWriter.NO_OP;
}
return new ZkWriteCommand(coll.getName(),
new DocCollection(coll.getName(), coll.getSlicesMap(), m, coll.getRouter(), coll.getZNodeVersion()));
DocCollection collection = new DocCollection(coll.getName(), coll.getSlicesMap(), m, coll.getRouter(), coll.getZNodeVersion());
if (replicaOps == null){
return new ZkWriteCommand(coll.getName(), collection);
} else {
return new ZkWriteCommand(coll.getName(), collection, replicaOps, true);
}
}
public static DocCollection updateSlice(String collectionName, DocCollection collection, Slice slice) {

View File

@ -26,6 +26,7 @@ import java.util.Map.Entry;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
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.ZkNodeProps;
@ -45,6 +46,8 @@ public class NodeMutator {
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
List<String> downedReplicas = new ArrayList<>();
String collection = entry.getKey();
DocCollection docCollection = entry.getValue();
@ -68,6 +71,7 @@ public class NodeMutator {
Replica.State.DOWN, replica.type, props);
newReplicas.put(replica.getName(), newReplica);
needToUpdateCollection = true;
downedReplicas.add(replica.getName());
}
}
@ -76,9 +80,14 @@ public class NodeMutator {
}
if (needToUpdateCollection) {
if (docCollection.isPerReplicaState()) {
zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy),
PerReplicaStatesOps.downReplicas(downedReplicas, docCollection.getPerReplicaStates()), false));
} else {
zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy)));
}
}
}
return zkWriteCommands;
}

View File

@ -39,8 +39,11 @@ import org.apache.solr.cloud.api.collections.SplitShardCmd;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.PerReplicaStatesOps;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
@ -50,6 +53,7 @@ import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.overseer.CollectionMutator.checkCollectionKeyExistence;
import static org.apache.solr.cloud.overseer.CollectionMutator.checkKeyExistence;
import static org.apache.solr.cloud.overseer.SliceMutator.getZkClient;
import static org.apache.solr.common.params.CommonParams.NAME;
public class ReplicaMutator {
@ -57,10 +61,12 @@ public class ReplicaMutator {
protected final SolrCloudManager cloudManager;
protected final DistribStateManager stateManager;
protected SolrZkClient zkClient;
public ReplicaMutator(SolrCloudManager cloudManager) {
this.cloudManager = cloudManager;
this.stateManager = cloudManager.getDistribStateManager();
this.zkClient = getZkClient(cloudManager);
}
protected Replica setProperty(Replica replica, String key, String value) {
@ -260,6 +266,7 @@ public class ReplicaMutator {
log.info("Failed to update state because the replica does not exist, {}", message);
return ZkStateWriter.NO_OP;
}
boolean persistCollectionState = collection != null && collection.isPerReplicaState();
if (coreNodeName == null) {
coreNodeName = ClusterStateMutator.getAssignedCoreNodeName(collection,
@ -271,6 +278,7 @@ public class ReplicaMutator {
log.info("Failed to update state because the replica does not exist, {}", message);
return ZkStateWriter.NO_OP;
}
persistCollectionState = true;
// if coreNodeName is null, auto assign one
coreNodeName = Assign.assignCoreNodeName(stateManager, collection);
}
@ -285,6 +293,7 @@ public class ReplicaMutator {
if (sliceName != null) {
log.debug("shard={} is already registered", sliceName);
}
persistCollectionState = true;
}
if (sliceName == null) {
//request new shardId
@ -295,6 +304,7 @@ public class ReplicaMutator {
}
sliceName = Assign.assignShard(collection, numShards);
log.info("Assigning new node to shard shard={}", sliceName);
persistCollectionState = true;
}
Slice slice = collection != null ? collection.getSlice(sliceName) : null;
@ -366,8 +376,13 @@ public class ReplicaMutator {
DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection, slice);
log.debug("Collection is now: {}", newCollection);
if (collection != null && collection.isPerReplicaState()) {
PerReplicaStates prs = PerReplicaStates.fetch(collection.getZNode(), zkClient, collection.getPerReplicaStates());
return new ZkWriteCommand(collectionName, newCollection, PerReplicaStatesOps.flipState(replica.getName(), replica.getState(), prs), persistCollectionState);
} else{
return new ZkWriteCommand(collectionName, newCollection);
}
}
private DocCollection checkAndCompleteShardSplit(ClusterState prevState, DocCollection collection, String coreNodeName, String sliceName, Replica replica) {
Slice slice = collection.getSlice(sliceName);

View File

@ -25,14 +25,18 @@ import java.util.Set;
import com.google.common.collect.ImmutableSet;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.Assign;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.PerReplicaStatesOps;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.solr.common.cloud.RoutingRule;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -51,10 +55,21 @@ public class SliceMutator {
protected final SolrCloudManager cloudManager;
protected final DistribStateManager stateManager;
protected final SolrZkClient zkClient;
public SliceMutator(SolrCloudManager cloudManager) {
this.cloudManager = cloudManager;
this.stateManager = cloudManager.getDistribStateManager();
this.zkClient = getZkClient(cloudManager);
}
static SolrZkClient getZkClient(SolrCloudManager cloudManager) {
if (cloudManager instanceof SolrClientCloudManager) {
SolrClientCloudManager manager = (SolrClientCloudManager) cloudManager;
return manager.getZkClient();
} else {
return null;
}
}
public ZkWriteCommand addReplica(ClusterState clusterState, ZkNodeProps message) {
@ -80,9 +95,17 @@ public class SliceMutator {
ZkStateReader.STATE_PROP, message.getStr(ZkStateReader.STATE_PROP),
ZkStateReader.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP),
ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)), coll, slice);
if (collection.isPerReplicaState()) {
PerReplicaStates prs = PerReplicaStates.fetch(collection.getZNode(), zkClient, collection.getPerReplicaStates());
return new ZkWriteCommand(coll, updateReplica(collection, sl, replica.getName(), replica),
PerReplicaStatesOps.addReplica(replica.getName(), replica.getState(), replica.isLeader(), prs), true);
} else {
return new ZkWriteCommand(coll, updateReplica(collection, sl, replica.getName(), replica));
}
}
public ZkWriteCommand removeReplica(ClusterState clusterState, ZkNodeProps message) {
final String cnn = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
final String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
@ -106,8 +129,13 @@ public class SliceMutator {
newSlices.put(slice.getName(), slice);
}
if (coll.isPerReplicaState()) {
return new ZkWriteCommand(collection, coll.copyWithSlices(newSlices), PerReplicaStatesOps.deleteReplica(cnn, coll.getPerReplicaStates()) , true);
} else {
return new ZkWriteCommand(collection, coll.copyWithSlices(newSlices));
}
}
public ZkWriteCommand setShardLeader(ClusterState clusterState, ZkNodeProps message) {
String leaderUrl = ZkCoreNodeProps.getCoreUrl(message);
@ -124,6 +152,7 @@ public class SliceMutator {
Slice slice = slices.get(sliceName);
Replica oldLeader = slice.getLeader();
Replica newLeader = null;
final Map<String, Replica> newReplicas = new LinkedHashMap<>();
for (Replica replica : slice.getReplicas()) {
// TODO: this should only be calculated once and cached somewhere?
@ -132,7 +161,7 @@ public class SliceMutator {
if (replica == oldLeader && !coreURL.equals(leaderUrl)) {
replica = new ReplicaMutator(cloudManager).unsetLeader(replica);
} else if (coreURL.equals(leaderUrl)) {
replica = new ReplicaMutator(cloudManager).setLeader(replica);
newLeader = replica = new ReplicaMutator(cloudManager).setLeader(replica);
}
newReplicas.put(replica.getName(), replica);
@ -141,8 +170,17 @@ public class SliceMutator {
Map<String, Object> newSliceProps = slice.shallowCopy();
newSliceProps.put(Slice.REPLICAS, newReplicas);
slice = new Slice(slice.getName(), newReplicas, slice.getProperties(), collectionName);
if (coll.isPerReplicaState()) {
PerReplicaStates prs = PerReplicaStates.fetch(coll.getZNode(), zkClient, coll.getPerReplicaStates());
return new ZkWriteCommand(collectionName, CollectionMutator.updateSlice(collectionName, coll, slice),
PerReplicaStatesOps.flipLeader(
slice.getReplicaNames(),
newLeader == null ? null : newLeader.getName(),
prs), false);
} else {
return new ZkWriteCommand(collectionName, CollectionMutator.updateSlice(collectionName, coll, slice));
}
}
public ZkWriteCommand updateShardState(ClusterState clusterState, ZkNodeProps message) {
String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);

View File

@ -27,6 +27,7 @@ import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.CreateMode;
@ -63,7 +64,7 @@ public class ZkStateWriter {
protected final ZkStateReader reader;
protected final Stats stats;
protected Map<String, DocCollection> updates = new HashMap<>();
protected Map<String, ZkWriteCommand> updates = new HashMap<>();
private int numUpdates = 0;
protected ClusterState clusterState = null;
protected long lastUpdatedTime = 0;
@ -111,15 +112,47 @@ public class ZkStateWriter {
if (cmds.isEmpty()) return prevState;
if (isNoOps(cmds)) return prevState;
boolean forceFlush = false;
if (cmds.size() == 1) {
//most messages result in only one command. let's deal with it right away
ZkWriteCommand cmd = cmds.get(0);
if (cmd.collection != null && cmd.collection.isPerReplicaState()) {
//we do not wish to batch any updates for collections with per-replica state because
// these changes go to individual ZK nodes and there is zero advantage to batching
//now check if there are any updates for the same collection already present
if (updates.containsKey(cmd.name)) {
//this should not happen
// but let's get those updates out anyway
writeUpdate(updates.remove(cmd.name));
}
//now let's write the current message
try {
return writeUpdate(cmd);
} finally {
if (callback !=null) callback.onWrite();
}
}
} else {
//there are more than one commands created as a result of this message
for (ZkWriteCommand cmd : cmds) {
if (cmd.collection != null && cmd.collection.isPerReplicaState()) {
// we don't try to optimize for this case. let's flush out all after this
forceFlush = true;
break;
}
}
}
for (ZkWriteCommand cmd : cmds) {
if (cmd == NO_OP) continue;
prevState = prevState.copyWith(cmd.name, cmd.collection);
updates.put(cmd.name, cmd.collection);
updates.put(cmd.name, cmd);
numUpdates++;
}
clusterState = prevState;
if (maybeFlushAfter()) {
if (forceFlush || maybeFlushAfter()) {
ClusterState state = writePendingUpdates();
if (callback != null) {
callback.onWrite();
@ -149,7 +182,15 @@ public class ZkStateWriter {
public boolean hasPendingUpdates() {
return numUpdates != 0;
}
public ClusterState writeUpdate(ZkWriteCommand command) throws IllegalStateException, KeeperException, InterruptedException {
Map<String, ZkWriteCommand> commands = new HashMap<>();
commands.put(command.name, command);
return writePendingUpdates(commands);
}
public ClusterState writePendingUpdates() throws KeeperException, InterruptedException {
return writePendingUpdates(updates);
}
/**
* Writes all pending updates to ZooKeeper and returns the modified cluster state
*
@ -158,20 +199,30 @@ public class ZkStateWriter {
* @throws KeeperException if any ZooKeeper operation results in an error
* @throws InterruptedException if the current thread is interrupted
*/
public ClusterState writePendingUpdates() throws IllegalStateException, KeeperException, InterruptedException {
public ClusterState writePendingUpdates(Map<String, ZkWriteCommand> updates) throws IllegalStateException, KeeperException, InterruptedException {
if (invalidState) {
throw new IllegalStateException("ZkStateWriter has seen a tragic error, this instance can no longer be used");
}
if (!hasPendingUpdates()) return clusterState;
if ((updates == this.updates)
&& !hasPendingUpdates()) {
return clusterState;
}
Timer.Context timerContext = stats.time("update_state");
boolean success = false;
try {
if (!updates.isEmpty()) {
for (Map.Entry<String, DocCollection> entry : updates.entrySet()) {
for (Map.Entry<String, ZkWriteCommand> entry : updates.entrySet()) {
String name = entry.getKey();
String path = ZkStateReader.getCollectionPath(name);
DocCollection c = entry.getValue();
ZkWriteCommand cmd = entry.getValue();
DocCollection c = cmd.collection;
if(cmd.ops != null && cmd.ops.isPreOp()) {
cmd.ops.persist(path, reader.getZkClient());
clusterState = clusterState.copyWith(name,
cmd.collection.copyWith(PerReplicaStates.fetch(cmd.collection.getZNode(), reader.getZkClient(), null)));
}
if (!cmd.persistCollState) continue;
if (c == null) {
// let's clean up the state.json of this collection only, the rest should be clean by delete collection cmd
log.debug("going to delete state.json {}", path);
@ -192,6 +243,14 @@ public class ZkStateWriter {
clusterState = clusterState.copyWith(name, newCollection);
}
}
if(cmd.ops != null && !cmd.ops.isPreOp()) {
cmd.ops.persist(path, reader.getZkClient());
DocCollection currentCollState = clusterState.getCollection(cmd.name);
if ( currentCollState != null) {
clusterState = clusterState.copyWith(name,
currentCollState.copyWith(PerReplicaStates.fetch(currentCollState.getZNode(), reader.getZkClient(), null)));
}
}
}
updates.clear();

View File

@ -17,16 +17,34 @@
package org.apache.solr.cloud.overseer;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.PerReplicaStatesOps;
public class ZkWriteCommand {
public final String name;
public final DocCollection collection;
public final boolean noop;
public final boolean noop;
// persist the collection state. If this is false, it means the collection state is not modified
public final boolean persistCollState;
public final PerReplicaStatesOps ops;
public ZkWriteCommand(String name, DocCollection collection, PerReplicaStatesOps replicaOps, boolean persistCollState) {
boolean isPerReplicaState = collection.isPerReplicaState();
this.name = name;
this.collection = collection;
this.noop = false;
this.ops = isPerReplicaState ? replicaOps : null;
this.persistCollState = isPerReplicaState ? persistCollState : true;
}
public ZkWriteCommand(String name, DocCollection collection) {
this.name = name;
this.collection = collection;
this.noop = false;
persistCollState = true;
this.ops = collection != null && collection.isPerReplicaState() ?
PerReplicaStatesOps.touchChildren():
null;
}
/**
@ -36,6 +54,8 @@ public class ZkWriteCommand {
this.noop = true;
this.name = null;
this.collection = null;
this.ops = null;
persistCollState = true;
}
public static ZkWriteCommand noop() {

View File

@ -19,17 +19,7 @@ package org.apache.solr.handler.admin;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.*;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
@ -118,6 +108,7 @@ import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHan
import static org.apache.solr.cloud.api.collections.RoutedAlias.CREATE_COLLECTION_PREFIX;
import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;
import static org.apache.solr.common.cloud.DocCollection.PER_REPLICA_STATE;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
@ -462,6 +453,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
TLOG_REPLICAS,
NRT_REPLICAS,
WAIT_FOR_FINAL_STATE,
PER_REPLICA_STATE,
ALIAS);
if (props.get(REPLICATION_FACTOR) != null && props.get(NRT_REPLICAS) != null) {

View File

@ -16,13 +16,6 @@
*/
package org.apache.solr.cloud;
import static java.util.Arrays.asList;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_DEF;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.NUM_SHARDS_PROP;
import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
import static org.apache.solr.common.params.CollectionAdminParams.DEFAULTS;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.file.Path;
@ -77,6 +70,13 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.Arrays.asList;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_DEF;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.NUM_SHARDS_PROP;
import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
import static org.apache.solr.common.params.CollectionAdminParams.DEFAULTS;
@LuceneTestCase.Slow
public class CollectionsAPISolrJTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@ -104,6 +104,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
public void testCreateWithDefaultConfigSet() throws Exception {
String collectionName = "solrj_default_configset";
CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, 2, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collectionName, 2, 4);
@ -236,6 +237,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
assertEquals("2", String.valueOf(clusterProperty));
CollectionAdminResponse response = CollectionAdminRequest
.createCollection(COLL_NAME, "conf", null, null, null, null)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
assertEquals(0, response.getStatus());
assertTrue(response.isSuccess());
@ -406,7 +408,9 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
public void testCreateAndDeleteAlias() throws IOException, SolrServerException {
final String collection = "aliasedCollection";
CollectionAdminRequest.createCollection(collection, "conf", 1, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection, "conf", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
CollectionAdminResponse response
= CollectionAdminRequest.createAlias("solrj_alias", collection).process(cluster.getSolrClient());
@ -421,6 +425,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
final String collectionName = "solrj_test_splitshard";
CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collectionName, 2, 2);
@ -475,6 +480,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
cluster.getJettySolrRunners().forEach(j -> j.getCoreContainer().getAllowPaths().add(tmpDir));
CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.withProperty(CoreAdminParams.DATA_DIR, dataDir.toString())
.withProperty(CoreAdminParams.ULOG_DIR, ulogDir.toString())
.process(cluster.getSolrClient());
@ -501,6 +507,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
final String collectionName = "solrj_replicatests";
CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collectionName, 1, 2);
@ -569,6 +576,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
final String propName = "testProperty";
CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collectionName, 2, 4);
@ -600,6 +608,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
public void testColStatus() throws Exception {
final String collectionName = "collectionStatusTest";
CollectionAdminRequest.createCollection(collectionName, "conf2", 2, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collectionName, 2, 4);

View File

@ -18,7 +18,9 @@ package org.apache.solr.cloud;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@ -29,12 +31,16 @@ import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.apache.solr.cloud.SolrCloudTestCase.configureCluster;
public class ZkSolrClientTest extends SolrTestCaseJ4 {
@BeforeClass
@ -376,6 +382,31 @@ public class ZkSolrClientTest extends SolrTestCaseJ4 {
}
}
public void testZkBehavior() throws Exception {
MiniSolrCloudCluster cluster =
configureCluster(4)
.withJettyConfig(jetty -> jetty.enableV2(true))
.configure();
try {
SolrZkClient zkClient = cluster.getZkClient();
zkClient.create("/test-node", null, CreateMode.PERSISTENT, true);
Stat stat = zkClient.exists("/test-node", null, true);
int cversion = stat.getCversion();
List<Op> ops = Arrays.asList(
Op.create("/test-node/abc", null, zkClient.getZkACLProvider().getACLsToAdd("/test-node/abc"), CreateMode.PERSISTENT),
Op.delete("/test-node/abc", -1));
zkClient.multi(ops, true);
stat = zkClient.exists("/test-node", null, true);
assertTrue(stat.getCversion() >= cversion + 2);
} finally {
cluster.shutdown();
}
}
@Override
public void tearDown() throws Exception {
super.tearDown();

View File

@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.SolrPing;
import org.apache.solr.client.solrj.response.SolrPingResponse;
import org.apache.solr.cloud.MiniSolrCloudCluster;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.request.SolrQueryRequest;
@ -188,6 +189,7 @@ public class PingRequestHandlerTest extends SolrTestCaseJ4 {
String configName = "solrCloudCollectionConfig";
miniCluster.uploadConfigSet(SolrTestCaseJ4.TEST_PATH().resolve("collection1").resolve("conf"), configName);
CollectionAdminRequest.createCollection(collectionName, configName, NUM_SHARDS, REPLICATION_FACTOR)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(miniCluster.getSolrClient());
// Send distributed and non-distributed ping query

View File

@ -63,7 +63,9 @@ public class TestSQLHandler extends SolrCloudTestCase {
collection = COLLECTIONORALIAS;
}
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection, "conf", 2, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collection, 2, 2);
if (useAlias) {
CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());

View File

@ -24,9 +24,9 @@ import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -34,16 +34,16 @@ import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase.Nightly;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.TestUtil;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.GenericSolrRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
@ -51,9 +51,8 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.util.TimeOut;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeOut;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -96,6 +95,7 @@ public class TestStressThreadBackup extends SolrCloudTestCase {
.configure();
assertEquals(0, (CollectionAdminRequest.createCollection(DEFAULT_TEST_COLLECTION_NAME, "conf1", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient()).getStatus()));
adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
initCoreNameAndSolrCoreClient();

View File

@ -80,6 +80,7 @@ public class HealthCheckHandlerTest extends SolrCloudTestCase {
try (HttpSolrClient httpSolrClient = getHttpSolrClient(cluster.getJettySolrRunner(0).getBaseUrl().toString())) {
CollectionAdminResponse collectionAdminResponse = CollectionAdminRequest.createCollection("test", "_default", 1, 1)
.withProperty("solr.directoryFactory", "solr.StandardDirectoryFactory")
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(httpSolrClient);
assertEquals(0, collectionAdminResponse.getStatus());
SolrResponse response = req.process(httpSolrClient);

View File

@ -78,7 +78,8 @@ public class MetricsHistoryHandlerTest extends SolrCloudTestCase {
// create .system collection
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL,
"conf", 1, 1);
"conf", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE);
create.process(solrClient);
CloudUtil.waitForState(cloudManager, "failed to create " + CollectionAdminParams.SYSTEM_COLL,
CollectionAdminParams.SYSTEM_COLL, CloudUtil.clusterShape(1, 1));

View File

@ -30,6 +30,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.MiniSolrCloudCluster;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.params.ModifiableSolrParams;
@ -140,6 +141,7 @@ public class SearchHandlerTest extends SolrTestCaseJ4
miniCluster.uploadConfigSet(SolrTestCaseJ4.TEST_PATH().resolve("collection1/conf"), configName);
CollectionAdminRequest.createCollection(collectionName, configName, 2, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(miniCluster.getSolrClient());
QueryRequest req = new QueryRequest();
@ -182,6 +184,7 @@ public class SearchHandlerTest extends SolrTestCaseJ4
miniCluster.uploadConfigSet(SolrTestCaseJ4.TEST_PATH().resolve("collection1/conf"), configName);
CollectionAdminRequest.createCollection(collectionName, configName, 2, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(miniCluster.getSolrClient());
ModifiableSolrParams params = new ModifiableSolrParams();
@ -229,6 +232,7 @@ public class SearchHandlerTest extends SolrTestCaseJ4
miniCluster.uploadConfigSet(SolrTestCaseJ4.TEST_PATH().resolve("collection1/conf"), configName);
CollectionAdminRequest.createCollection(collectionName, configName, 2, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(miniCluster.getSolrClient());
ModifiableSolrParams params = new ModifiableSolrParams();

View File

@ -78,10 +78,12 @@ public class TestSubQueryTransformerDistrib extends SolrCloudTestCase {
int replicas = 2 ;
CollectionAdminRequest.createCollection(people, configName, shards, replicas)
.withProperty("config", "solrconfig-doctransformers.xml")
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.withProperty("schema", "schema-docValuesJoin.xml")
.process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(depts, configName, shards, replicas)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.withProperty("config", "solrconfig-doctransformers.xml")
.withProperty("schema",
differentUniqueId ? "schema-minimal-with-another-uniqkey.xml":

View File

@ -51,6 +51,8 @@ public class TestManagedSchemaAPI extends SolrCloudTestCase {
public void test() throws Exception {
String collection = "testschemaapi";
CollectionAdminRequest.createCollection(collection, "conf1", 1, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
testModifyField(collection);
testReloadAndAddSimple(collection);

View File

@ -109,6 +109,7 @@ public class TestCloudJSONFacetJoinDomain extends SolrCloudTestCase {
collectionProperties.put("config", "solrconfig-tlog.xml");
collectionProperties.put("schema", "schema_latest.xml");
CollectionAdminRequest.createCollection(COLLECTION_NAME, configName, numShards, repFactor)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.setProperties(collectionProperties)
.process(cluster.getSolrClient());

View File

@ -138,6 +138,7 @@ public class TestCloudJSONFacetSKG extends SolrCloudTestCase {
collectionProperties.put("config", "solrconfig-tlog.xml");
collectionProperties.put("schema", "schema_latest.xml");
CollectionAdminRequest.createCollection(COLLECTION_NAME, configName, numShards, repFactor)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.setProperties(collectionProperties)
.process(cluster.getSolrClient());

View File

@ -132,6 +132,7 @@ public class TestCloudJSONFacetSKGEquiv extends SolrCloudTestCase {
collectionProperties.put("config", "solrconfig-tlog.xml");
collectionProperties.put("schema", "schema_latest.xml");
CollectionAdminRequest.createCollection(COLLECTION_NAME, configName, numShards, repFactor)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.setProperties(collectionProperties)
.process(cluster.getSolrClient());

View File

@ -54,9 +54,11 @@ public class CrossCollectionJoinQueryTest extends SolrCloudTestCase {
CollectionAdminRequest.createCollection("products", "ccjoin", NUM_SHARDS, NUM_REPLICAS)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("parts", "ccjoin", NUM_SHARDS, NUM_REPLICAS)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
}

View File

@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.MiniSolrCloudCluster;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.CompositeIdRouter;
import org.apache.solr.common.cloud.ImplicitDocRouter;
@ -200,7 +201,7 @@ public class TestDistribIDF extends SolrTestCaseJ4 {
response = create.process(solrCluster.getSolrClient());
solrCluster.waitForActiveCollection(name, 3, 3);
} else {
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(name,config,2,1);
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(name,config,2,1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE);
response = create.process(solrCluster.getSolrClient());
solrCluster.waitForActiveCollection(name, 2, 2);
}

View File

@ -88,7 +88,7 @@ public class TemplateUpdateProcessorTest extends SolrCloudTestCase {
params.add("commit", "true");
UpdateRequest add = new UpdateRequest().add(solrDoc);
add.setParams(params);
NamedList<Object> result = cluster.getSolrClient().request(CollectionAdminRequest.createCollection("c", "conf1", 1, 1));
NamedList<Object> result = cluster.getSolrClient().request(CollectionAdminRequest.createCollection("c", "conf1", 1, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE));
Utils.toJSONString(result.asMap(4));
AbstractFullDistribZkTestBase.waitForCollection(cluster.getSolrClient().getZkStateReader(), "c",1);
cluster.getSolrClient().request(add, "c");

View File

@ -56,6 +56,7 @@ public class TestDistributedTracing extends SolrCloudTestCase {
waitForSampleRateUpdated(1.0);
CollectionAdminRequest
.createCollection(COLLECTION, "config", 2, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 4);
}

View File

@ -87,6 +87,9 @@ If this parameter is specified, the router will look at the value of the field i
+
Please note that <<realtime-get.adoc#realtime-get,RealTime Get>> or retrieval by document ID would also require the parameter `\_route_` (or `shard.keys`) to avoid a distributed search.
`perReplicaState`::
If `true` the states of individual replicas will be maintained as individual child of the `state.json`. default is `false`
`property._name_=_value_`::
Set core property _name_ to _value_. See the section <<defining-core-properties.adoc#defining-core-properties,Defining core.properties>> for details on supported properties and values.

View File

@ -25,6 +25,7 @@ import java.util.List;
import java.util.NoSuchElementException;
import org.apache.solr.common.SolrCloseable;
import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
@ -100,6 +101,12 @@ public interface DistribStateManager extends SolrCloseable {
return tree;
}
default PerReplicaStates getReplicaStates(String path) throws KeeperException, InterruptedException {
throw new UnsupportedOperationException("Not implemented");
}
/**
* Remove data recursively.
* @param root root path

View File

@ -37,8 +37,8 @@ import org.apache.http.util.EntityUtils;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.common.cloud.SolrZkClient;
@ -187,6 +187,9 @@ public class SolrClientCloudManager implements SolrCloudManager {
return EMPTY;
}
}
public SolrZkClient getZkClient() {
return zkClient;
}
@Override
public DistributedQueueFactory getDistributedQueueFactory() {

View File

@ -27,6 +27,7 @@ import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.NotEmptyException;
import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.cloud.PerReplicaStates;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@ -181,4 +182,9 @@ public class ZkDistribStateManager implements DistribStateManager {
public SolrZkClient getZkClient() {
return zkClient;
}
@Override
public PerReplicaStates getReplicaStates(String path) throws KeeperException, InterruptedException {
return PerReplicaStates.fetch(path, zkClient, null);
}
}

View File

@ -54,6 +54,7 @@ import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import static org.apache.solr.common.cloud.DocCollection.PER_REPLICA_STATE;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.READ_ONLY;
@ -80,6 +81,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
public static final java.util.List<String> MODIFIABLE_COLLECTION_PROPERTIES = Arrays.asList(
REPLICATION_FACTOR,
COLL_CONF,
PER_REPLICA_STATE,
READ_ONLY);
protected final CollectionAction action;
@ -433,6 +435,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
protected Integer nrtReplicas;
protected Integer pullReplicas;
protected Integer tlogReplicas;
protected Boolean perReplicaState;
protected Properties properties;
protected String alias;
@ -473,6 +476,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
public Create setReplicationFactor(Integer repl) { this.nrtReplicas = repl; return this; }
public Create setRule(String... s){ this.rule = s; return this; }
public Create setSnitch(String... s){ this.snitch = s; return this; }
public Create setPerReplicaState(Boolean b) {this.perReplicaState = b; return this; }
public Create setAlias(String alias) {
this.alias = alias;
@ -489,6 +493,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
public Integer getNumNrtReplicas() { return nrtReplicas; }
public Integer getNumTlogReplicas() {return tlogReplicas;}
public Integer getNumPullReplicas() {return pullReplicas;}
public Boolean getPerReplicaState() {return perReplicaState;}
/**
* Provide the name of the shards to be created, separated by commas
@ -559,6 +564,9 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
if (tlogReplicas != null) {
params.set(ZkStateReader.TLOG_REPLICAS, tlogReplicas);
}
if(Boolean.TRUE.equals(perReplicaState)) {
params.set(PER_REPLICA_STATE, perReplicaState);
}
params.setNonNull(ALIAS, alias);
return params;
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.solr.common.cloud;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@ -23,15 +24,19 @@ import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import java.util.function.Supplier;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.noggit.JSONWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Immutable state of the cloud. Normally you can get the state by using
@ -39,6 +44,8 @@ import org.noggit.JSONWriter;
* @lucene.experimental
*/
public class ClusterState implements JSONWriter.Writable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final Map<String, CollectionRef> collectionStates, immutableCollectionStates;
private Set<String> liveNodes;
@ -241,6 +248,14 @@ public class ClusterState implements JSONWriter.Writable {
Map<String,Object> props;
Map<String,Slice> slices;
if (Boolean.parseBoolean(String.valueOf(objs.get(DocCollection.PER_REPLICA_STATE)))) {
if(log.isDebugEnabled()) {
log.debug("a collection {} has per-replica state", name);
}
//this collection has replica states stored outside
ReplicaStatesProvider rsp = REPLICASTATES_PROVIDER.get();
if (rsp instanceof StatesProvider) ((StatesProvider) rsp).isPerReplicaState = true;
}
@SuppressWarnings({"unchecked"})
Map<String, Object> sliceObjs = (Map<String, Object>) objs.get(DocCollection.SHARDS);
if (sliceObjs == null) {
@ -383,5 +398,63 @@ public class ClusterState implements JSONWriter.Writable {
public int size() {
return collectionStates.size();
}
interface ReplicaStatesProvider {
Optional<ReplicaStatesProvider> get();
PerReplicaStates getStates();
}
private static final ReplicaStatesProvider EMPTYSTATEPROVIDER = new ReplicaStatesProvider() {
@Override
public Optional<ReplicaStatesProvider> get() {
return Optional.empty();
}
@Override
public PerReplicaStates getStates() {
throw new RuntimeException("Invalid operation");
}
};
private static ThreadLocal<ReplicaStatesProvider> REPLICASTATES_PROVIDER = new ThreadLocal<>();
public static ReplicaStatesProvider getReplicaStatesProvider() {
return (REPLICASTATES_PROVIDER.get() == null)? EMPTYSTATEPROVIDER: REPLICASTATES_PROVIDER.get() ;
}
public static void initReplicaStateProvider(Supplier<PerReplicaStates> replicaStatesSupplier) {
REPLICASTATES_PROVIDER.set(new StatesProvider(replicaStatesSupplier));
}
public static void clearReplicaStateProvider(){
REPLICASTATES_PROVIDER.remove();
}
private static class StatesProvider implements ReplicaStatesProvider {
private final Supplier<PerReplicaStates> replicaStatesSupplier;
private PerReplicaStates perReplicaStates;
private boolean isPerReplicaState = false;
public StatesProvider(Supplier<PerReplicaStates> replicaStatesSupplier) {
this.replicaStatesSupplier = replicaStatesSupplier;
}
@Override
public Optional<ReplicaStatesProvider> get() {
return isPerReplicaState ? Optional.of(this) : Optional.empty();
}
@Override
public PerReplicaStates getStates() {
if (perReplicaStates == null) perReplicaStates = replicaStatesSupplier.get();
return perReplicaStates;
}
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.solr.common.cloud;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
@ -30,6 +31,8 @@ import java.util.function.BiConsumer;
import java.util.function.BiPredicate;
import org.noggit.JSONWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
@ -42,9 +45,12 @@ import static org.apache.solr.common.util.Utils.toJSONString;
* Models a Collection in zookeeper (but that Java name is obviously taken, hence "DocCollection")
*/
public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String DOC_ROUTER = "router";
public static final String SHARDS = "shards";
public static final String PER_REPLICA_STATE = "perReplicaState";
private final int znodeVersion;
@ -55,12 +61,17 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
private final Map<String, List<Replica>> nodeNameReplicas;
private final Map<String, List<Replica>> nodeNameLeaderReplicas;
private final DocRouter router;
private final String znode;
private final Integer replicationFactor;
private final Integer numNrtReplicas;
private final Integer numTlogReplicas;
private final Integer numPullReplicas;
private final Boolean readOnly;
private final Boolean perReplicaState;
private final Map<String, Replica> replicaMap = new HashMap<>();
private volatile PerReplicaStates perReplicaStates;
public DocCollection(String name, Map<String, Slice> slices, Map<String, Object> props, DocRouter router) {
this(name, slices, props, router, Integer.MAX_VALUE);
@ -86,6 +97,8 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
this.numNrtReplicas = (Integer) verifyProp(props, NRT_REPLICAS, 0);
this.numTlogReplicas = (Integer) verifyProp(props, TLOG_REPLICAS, 0);
this.numPullReplicas = (Integer) verifyProp(props, PULL_REPLICAS, 0);
this.perReplicaState = (Boolean) verifyProp(props, PER_REPLICA_STATE, Boolean.FALSE);
ClusterState.getReplicaStatesProvider().get().ifPresent(it -> perReplicaStates = it.getStates());
Boolean readOnly = (Boolean) verifyProp(props, READ_ONLY);
this.readOnly = readOnly == null ? Boolean.FALSE : readOnly;
@ -98,13 +111,44 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
}
for (Replica replica : slice.getValue()) {
addNodeNameReplica(replica);
if(perReplicaState) {
replicaMap.put(replica.getName(), replica);
}
}
}
this.activeSlicesArr = activeSlices.values().toArray(new Slice[activeSlices.size()]);
this.router = router;
this.znode = ZkStateReader.getCollectionPath(name);
assert name != null && slices != null;
}
/**Update our state with a state of a {@link Replica}
* Used to create a new Collection State when only a replica is updated
*/
public DocCollection copyWith( PerReplicaStates newPerReplicaStates) {
if (log.isDebugEnabled()) {
log.debug("collection :{} going to be updated : per-replica state :{} -> {}",
name,
getChildNodesVersion(), newPerReplicaStates.cversion);
}
if(getChildNodesVersion() == newPerReplicaStates.cversion) return this;
Set<String> modifiedReplicas = PerReplicaStates.findModifiedReplicas(newPerReplicaStates, this.perReplicaStates);
if(modifiedReplicas.isEmpty()) return this; //nothing is modified
Map<String, Slice> modifiedShards = new HashMap<>(getSlicesMap());
for (String s : modifiedReplicas) {
Replica replica = getReplica(s);
if(replica != null) {
Replica newReplica = replica.copyWith(newPerReplicaStates.get(s));
Slice shard = modifiedShards.get(replica.shard);
modifiedShards.put(replica.shard, shard.copyWith(newReplica));
}
}
DocCollection result = new DocCollection(getName(), modifiedShards, propMap, router, znodeVersion);
result.perReplicaStates = newPerReplicaStates;
return result;
}
private void addNodeNameReplica(Replica replica) {
List<Replica> replicas = nodeNameReplicas.get(replica.getNodeName());
if (replicas == null) {
@ -136,6 +180,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
case PULL_REPLICAS:
case TLOG_REPLICAS:
return Integer.parseInt(o.toString());
case PER_REPLICA_STATE:
case READ_ONLY:
return Boolean.parseBoolean(o.toString());
case "snitch":
@ -149,10 +194,11 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
* @param slices the new set of Slices
* @return the resulting DocCollection
*/
public DocCollection copyWithSlices(Map<String, Slice> slices){
return new DocCollection(getName(), slices, propMap, router, znodeVersion);
public DocCollection copyWithSlices(Map<String, Slice> slices) {
DocCollection result = new DocCollection(getName(), slices, propMap, router, znodeVersion);
result.perReplicaStates = perReplicaStates;
return result;
}
/**
* Return collection name.
*/
@ -224,6 +270,16 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
public int getZNodeVersion(){
return znodeVersion;
}
public int getChildNodesVersion() {
return perReplicaStates == null ? -1 : perReplicaStates.cversion;
}
public boolean isModified(int dataVersion, int childVersion) {
if (dataVersion > znodeVersion) return true;
if (childVersion > getChildNodesVersion()) return true;
return false;
}
/**
* @return replication factor for this collection or null if no
@ -233,6 +289,10 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
return replicationFactor;
}
public String getZNode(){
return znode;
}
public DocRouter getRouter() {
return router;
}
@ -243,7 +303,9 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
@Override
public String toString() {
return "DocCollection("+name+"/" + znodeVersion + ")=" + toJSONString(this);
return "DocCollection("+name+"/" + znode + "/" + znodeVersion+" "
+ (perReplicaStates == null ? "": perReplicaStates.toString())+")="
+ toJSONString(this);
}
@Override
@ -255,6 +317,9 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
}
public Replica getReplica(String coreNodeName) {
if (perReplicaState) {
return replicaMap.get(coreNodeName);
}
for (Slice slice : slices.values()) {
Replica replica = slice.getReplica(coreNodeName);
if (replica != null) return replica;
@ -375,6 +440,14 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
public Integer getNumPullReplicas() {
return numPullReplicas;
}
public boolean isPerReplicaState() {
return Boolean.TRUE.equals(perReplicaState);
}
public PerReplicaStates getPerReplicaStates() {
return perReplicaStates;
}
public int getExpectedReplicaCount(Replica.Type type, int def) {
Integer result = null;

View File

@ -0,0 +1,312 @@
/*
* 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.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.BiConsumer;
import org.apache.solr.cluster.api.SimpleMap;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.annotation.JsonProperty;
import org.apache.solr.common.util.ReflectMapWriter;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.WrappedSimpleMap;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.params.CommonParams.VERSION;
/**
* This represents the individual replica states in a collection
* This is an immutable object. When states are modified, a new instance is constructed
*/
public class PerReplicaStates implements ReflectMapWriter {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final char SEPARATOR = ':';
//no:of times to retry in case of a CAS failure
public static final int MAX_RETRIES = 5;
//znode path where thisis loaded from
@JsonProperty
public final String path;
// the child version of that znode
@JsonProperty
public final int cversion;
//states of individual replicas
@JsonProperty
public final SimpleMap<State> states;
/**
* Construct with data read from ZK
* @param path path from where this is loaded
* @param cversion the current child version of the znode
* @param states the per-replica states (the list of all child nodes)
*/
public PerReplicaStates(String path, int cversion, List<String> states) {
this.path = path;
this.cversion = cversion;
Map<String, State> tmp = new LinkedHashMap<>();
for (String state : states) {
State rs = State.parse(state);
if (rs == null) continue;
State existing = tmp.get(rs.replica);
if (existing == null) {
tmp.put(rs.replica, rs);
} else {
tmp.put(rs.replica, rs.insert(existing));
}
}
this.states = new WrappedSimpleMap<>(tmp);
}
/**Get the changed replicas
*/
public static Set<String> findModifiedReplicas(PerReplicaStates old, PerReplicaStates fresh) {
Set<String> result = new HashSet<>();
if (fresh == null) {
old.states.forEachKey(result::add);
return result;
}
old.states.forEachEntry((s, state) -> {
// the state is modified or missing
if (!Objects.equals(fresh.get(s) , state)) result.add(s);
});
fresh.states.forEachEntry((s, state) -> { if (old.get(s) == null ) result.add(s);
});
return result;
}
/**
* Fetch the latest {@link PerReplicaStates} . It fetches data after checking the {@link Stat#getCversion()} of state.json.
* If this is not modified, the same object is returned
*/
public static PerReplicaStates fetch(String path, SolrZkClient zkClient, PerReplicaStates current) {
try {
if (current != null) {
Stat stat = zkClient.exists(current.path, null, true);
if (stat == null) return new PerReplicaStates(path, -1, Collections.emptyList());
if (current.cversion == stat.getCversion()) return current;// not modifiedZkStateReaderTest
}
Stat stat = new Stat();
List<String> children = zkClient.getChildren(path, null, stat, true);
return new PerReplicaStates(path, stat.getCversion(), Collections.unmodifiableList(children));
} catch (KeeperException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error fetching per-replica states", e);
} catch (InterruptedException e) {
SolrZkClient.checkInterrupted(e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Thread interrupted when loading per-replica states from " + path, e);
}
}
public static String getReplicaName(String s) {
int idx = s.indexOf(SEPARATOR);
if (idx > 0) {
return s.substring(0, idx);
}
return null;
}
public State get(String replica) {
return states.get(replica);
}
public static class Operation {
public final Type typ;
public final State state;
public Operation(Type typ, State replicaState) {
this.typ = typ;
this.state = replicaState;
}
public enum Type {
//add a new node
ADD,
//delete an existing node
DELETE
}
@Override
public String toString() {
return typ.toString() + " : " + state;
}
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("{").append(path).append("/[").append(cversion).append("]: [");
appendStates(sb);
return sb.append("]}").toString();
}
private StringBuilder appendStates(StringBuilder sb) {
states.forEachEntry(new BiConsumer<String, State>() {
int count = 0;
@Override
public void accept(String s, State state) {
if (count++ > 0) sb.append(", ");
sb.append(state.asString);
for (State d : state.getDuplicates()) sb.append(d.asString);
}
});
return sb;
}
/**
* The state of a replica as stored as a node under /collections/collection-name/state.json/replica-state
*/
public static class State implements MapWriter {
public final String replica;
public final Replica.State state;
public final Boolean isLeader;
public final int version;
public final String asString;
/**
* if there are multiple entries for the same replica, e.g: core_node_1:12:A core_node_1:13:D
* <p>
* the entry with '13' is the latest and the one with '12' is considered a duplicate
* <p>
* These are unlikely, but possible
*/
final State duplicate;
private State(String serialized, List<String> pieces) {
this.asString = serialized;
replica = pieces.get(0);
version = Integer.parseInt(pieces.get(1));
String encodedStatus = pieces.get(2);
this.state = Replica.getState(encodedStatus);
isLeader = pieces.size() > 3 && "L".equals(pieces.get(3));
duplicate = null;
}
public static State parse(String serialized) {
List<String> pieces = StrUtils.splitSmart(serialized, ':');
if (pieces.size() < 3) return null;
return new State(serialized, pieces);
}
public State(String replica, Replica.State state, Boolean isLeader, int version) {
this(replica, state, isLeader, version, null);
}
public State(String replica, Replica.State state, Boolean isLeader, int version, State duplicate) {
this.replica = replica;
this.state = state == null ? Replica.State.ACTIVE : state;
this.isLeader = isLeader == null ? Boolean.FALSE : isLeader;
this.version = version;
asString = serialize();
this.duplicate = duplicate;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put(NAME, replica);
ew.put(VERSION, version);
ew.put(ZkStateReader.STATE_PROP, state.toString());
if (isLeader) ew.put(Slice.LEADER, isLeader);
ew.putIfNotNull("duplicate", duplicate);
}
private State insert(State duplicate) {
assert this.replica.equals(duplicate.replica);
if (this.version >= duplicate.version) {
if (this.duplicate != null) {
duplicate = new State(duplicate.replica, duplicate.state, duplicate.isLeader, duplicate.version, this.duplicate);
}
return new State(this.replica, this.state, this.isLeader, this.version, duplicate);
} else {
return duplicate.insert(this);
}
}
/**
* fetch duplicates entries for this replica
*/
List<State> getDuplicates() {
if (duplicate == null) return Collections.emptyList();
List<State> result = new ArrayList<>();
State current = duplicate;
while (current != null) {
result.add(current);
current = current.duplicate;
}
return result;
}
private String serialize() {
StringBuilder sb = new StringBuilder(replica)
.append(":")
.append(version)
.append(":")
.append(state.shortName);
if (isLeader) sb.append(":").append("L");
return sb.toString();
}
@Override
public String toString() {
return asString;
}
@Override
public boolean equals(Object o) {
if (o instanceof State) {
State that = (State) o;
return Objects.equals(this.asString, that.asString);
}
return false;
}
@Override
public int hashCode() {
return asString.hashCode();
}
}
}

View File

@ -0,0 +1,305 @@
/*
* 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.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.Collections.singletonList;
/**
* This is a helper class that encapsulates various operations performed on the per-replica states
* Do not directly manipulate the per replica states as it can become difficult to debug them
*/
public class PerReplicaStatesOps {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private PerReplicaStates rs;
List<PerReplicaStates.Operation> ops;
private boolean preOp = true;
final Function<PerReplicaStates, List<PerReplicaStates.Operation>> fun;
PerReplicaStatesOps(Function<PerReplicaStates, List<PerReplicaStates.Operation>> fun) {
this.fun = fun;
}
/**
* Persist a set of operations to Zookeeper
*/
private void persist(List<PerReplicaStates.Operation> operations, String znode, SolrZkClient zkClient) throws KeeperException, InterruptedException {
if (operations == null || operations.isEmpty()) return;
if (log.isDebugEnabled()) {
log.debug("Per-replica state being persisted for : '{}', ops: {}", znode, operations);
}
List<Op> ops = new ArrayList<>(operations.size());
for (PerReplicaStates.Operation op : operations) {
//the state of the replica is being updated
String path = znode + "/" + op.state.asString;
ops.add(op.typ == PerReplicaStates.Operation.Type.ADD ?
Op.create(path, null, zkClient.getZkACLProvider().getACLsToAdd(path), CreateMode.PERSISTENT) :
Op.delete(path, -1));
}
try {
zkClient.multi(ops, true);
} catch (KeeperException e) {
if(log.isErrorEnabled()) {
log.error("multi op exception " , e);
}
throw e;
}
}
/**There is a possibility that a replica may have some leftover entries . delete them too
*/
private static List<PerReplicaStates.Operation> addDeleteStaleNodes(List<PerReplicaStates.Operation> ops, PerReplicaStates.State rs) {
while (rs != null) {
ops.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.DELETE, rs));
rs = rs.duplicate;
}
return ops;
}
/**
* This is a persist operation with retry if a write fails due to stale state
*/
public void persist(String znode, SolrZkClient zkClient) throws KeeperException, InterruptedException {
List<PerReplicaStates.Operation> operations = ops;
for (int i = 0; i < PerReplicaStates.MAX_RETRIES; i++) {
try {
persist(operations, znode, zkClient);
return;
} catch (KeeperException.NodeExistsException | KeeperException.NoNodeException e) {
//state is stale
if(log.isInfoEnabled()) {
log.info("stale state for {} , attempt: {}. retrying...", znode, i);
}
operations = refresh(PerReplicaStates.fetch(znode, zkClient, null));
}
}
}
public PerReplicaStates getPerReplicaStates() {
return rs;
}
/**
* state of a replica is changed
*
* @param newState the new state
*/
public static PerReplicaStatesOps flipState(String replica, Replica.State newState, PerReplicaStates rs) {
return new PerReplicaStatesOps(prs -> {
List<PerReplicaStates.Operation> operations = new ArrayList<>(2);
PerReplicaStates.State existing = rs.get(replica);
if (existing == null) {
operations.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(replica, newState, Boolean.FALSE, 0)));
} else {
operations.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(replica, newState, existing.isLeader, existing.version + 1)));
addDeleteStaleNodes(operations, existing);
}
if (log.isDebugEnabled()) {
log.debug("flipState on {}, {} -> {}, ops :{}", rs.path, replica, newState, operations);
}
return operations;
}).init(rs);
}
/**
* Switch a collection from/to perReplicaState=true
*/
public static PerReplicaStatesOps modifyCollection(DocCollection coll, boolean enable, PerReplicaStates rs) {
return new PerReplicaStatesOps(prs -> enable ? enable(coll) : disable(prs)).init(rs);
}
private static List<PerReplicaStates.Operation> enable(DocCollection coll) {
List<PerReplicaStates.Operation> result = new ArrayList<>();
coll.forEachReplica((s, r) -> result.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(r.getName(), r.getState(), r.isLeader(), 0))));
return result;
}
private static List<PerReplicaStates.Operation> disable(PerReplicaStates prs) {
List<PerReplicaStates.Operation> result = new ArrayList<>();
prs.states.forEachEntry((s, state) -> result.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.DELETE, state)));
return result;
}
/**
* Flip the leader replica to a new one
*
* @param allReplicas allReplicas of the shard
* @param next next leader
*/
public static PerReplicaStatesOps flipLeader(Set<String> allReplicas, String next, PerReplicaStates rs) {
return new PerReplicaStatesOps(prs -> {
List<PerReplicaStates.Operation> ops = new ArrayList<>();
if (next != null) {
PerReplicaStates.State st = rs.get(next);
if (st != null) {
if (!st.isLeader) {
ops.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(st.replica, Replica.State.ACTIVE, Boolean.TRUE, st.version + 1)));
ops.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.DELETE, st));
}
//else do not do anything , that node is the leader
} else {
//there is no entry for the new leader.
//create one
ops.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(next, Replica.State.ACTIVE, Boolean.TRUE, 0)));
}
}
//now go through all other replicas and unset previous leader
for (String r : allReplicas) {
PerReplicaStates.State st = rs.get(r);
if (st == null) continue;//unlikely
if (!Objects.equals(r, next)) {
if (st.isLeader) {
//some other replica is the leader now. unset
ops.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(st.replica, st.state, Boolean.FALSE, st.version + 1)));
ops.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.DELETE, st));
}
}
}
if (log.isDebugEnabled()) {
log.debug("flipLeader on:{}, {} -> {}, ops: {}", rs.path, allReplicas, next, ops);
}
return ops;
}).init(rs);
}
/**
* Delete a replica entry from per-replica states
*
* @param replica name of the replica to be deleted
*/
public static PerReplicaStatesOps deleteReplica(String replica, PerReplicaStates rs) {
return new PerReplicaStatesOps(prs -> {
List<PerReplicaStates.Operation> result;
if (rs == null) {
result = Collections.emptyList();
} else {
PerReplicaStates.State state = rs.get(replica);
result = addDeleteStaleNodes(new ArrayList<>(), state);
}
return result;
}).init(rs);
}
public static PerReplicaStatesOps addReplica(String replica, Replica.State state, boolean isLeader, PerReplicaStates rs) {
return new PerReplicaStatesOps(perReplicaStates -> singletonList(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD,
new PerReplicaStates.State(replica, state, isLeader, 0)))).init(rs);
}
/**
* mark a bunch of replicas as DOWN
*/
public static PerReplicaStatesOps downReplicas(List<String> replicas, PerReplicaStates rs) {
return new PerReplicaStatesOps(prs -> {
List<PerReplicaStates.Operation> operations = new ArrayList<>();
for (String replica : replicas) {
PerReplicaStates.State r = rs.get(replica);
if (r != null) {
if (r.state == Replica.State.DOWN && !r.isLeader) continue;
operations.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(replica, Replica.State.DOWN, Boolean.FALSE, r.version + 1)));
addDeleteStaleNodes(operations, r);
} else {
operations.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(replica, Replica.State.DOWN, Boolean.FALSE, 0)));
}
}
if (log.isDebugEnabled()) {
log.debug("for coll: {} down replicas {}, ops {}", rs, replicas, operations);
}
return operations;
}).init(rs);
}
/**
* Just creates and deletes a dummy entry so that the {@link Stat#getCversion()} of states.json
* is updated
*/
public static PerReplicaStatesOps touchChildren() {
PerReplicaStatesOps result = new PerReplicaStatesOps(prs -> {
List<PerReplicaStates.Operation> operations = new ArrayList<>(2);
PerReplicaStates.State st = new PerReplicaStates.State(".dummy." + System.nanoTime(), Replica.State.DOWN, Boolean.FALSE, 0);
operations.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, st));
operations.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.DELETE, st));
if (log.isDebugEnabled()) {
log.debug("touchChildren {}", operations);
}
return operations;
});
result.preOp = false;
result.ops = result.refresh(null);
return result;
}
PerReplicaStatesOps init(PerReplicaStates rs) {
if (rs == null) return null;
get(rs);
return this;
}
public List<PerReplicaStates.Operation> get() {
return ops;
}
public List<PerReplicaStates.Operation> get(PerReplicaStates rs) {
ops = refresh(rs);
if (ops == null) ops = Collections.emptyList();
this.rs = rs;
return ops;
}
/**
* To be executed before collection state.json is persisted
*/
public boolean isPreOp() {
return preOp;
}
/**
* This method should compute the set of ZK operations for a given action
* for instance, a state change may result in 2 operations on per-replica states (1 CREATE and 1 DELETE)
* if a multi operation fails because the state got modified from behind,
* refresh the operation and try again
*
* @param prs The latest state
*/
List<PerReplicaStates.Operation> refresh(PerReplicaStates prs) {
if (fun != null) return fun.apply(prs);
return null;
}
@Override
public String toString() {
return ops.toString();
}
}

View File

@ -17,6 +17,7 @@
package org.apache.solr.common.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
@ -29,13 +30,15 @@ import java.util.function.BiPredicate;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.util.Utils;
import org.noggit.JSONWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.ConditionalMapWriter.NON_NULL_VAL;
import static org.apache.solr.common.ConditionalMapWriter.dedupeKeyPredicate;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
public class Replica extends ZkNodeProps implements MapWriter {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
/**
* The replica's state. In general, if the node the replica is hosted on is
* not under {@code /live_nodes} in ZK, the replica's state should be
@ -53,7 +56,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
* {@link ClusterState#liveNodesContain(String)}).
* </p>
*/
ACTIVE,
ACTIVE("A"),
/**
* The first state before {@link State#RECOVERING}. A node in this state
@ -64,13 +67,13 @@ public class Replica extends ZkNodeProps implements MapWriter {
* should not be relied on.
* </p>
*/
DOWN,
DOWN("D"),
/**
* The node is recovering from the leader. This might involve peer-sync,
* full replication or finding out things are already in sync.
*/
RECOVERING,
RECOVERING("R"),
/**
* Recovery attempts have not worked, something is not right.
@ -80,7 +83,15 @@ public class Replica extends ZkNodeProps implements MapWriter {
* cluster and it's state should be discarded.
* </p>
*/
RECOVERY_FAILED;
RECOVERY_FAILED("F");
/**short name for a state. Used to encode this in the state node see {@link PerReplicaStates.State}
*/
public final String shortName;
State(String c) {
this.shortName = c;
}
@Override
public String toString() {
@ -124,6 +135,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
public final String core;
public final Type type;
public final String shard, collection;
private PerReplicaStates.State replicaState;
// mutable
private State state;
@ -137,6 +149,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
this.node = (String) propMap.get(ZkStateReader.NODE_NAME_PROP);
this.core = (String) propMap.get(ZkStateReader.CORE_NAME_PROP);
this.type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
readPrs();
// default to ACTIVE
this.state = State.getState(String.valueOf(propMap.getOrDefault(ZkStateReader.STATE_PROP, State.ACTIVE.toString())));
validate();
@ -158,6 +171,7 @@ public class Replica extends ZkNodeProps implements MapWriter {
if (props != null) {
this.propMap.putAll(props);
}
readPrs();
validate();
propMap.put(BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(this.node));
}
@ -177,13 +191,26 @@ public class Replica extends ZkNodeProps implements MapWriter {
this.shard = String.valueOf(details.get("shard"));
this.core = String.valueOf(details.get("core"));
this.node = String.valueOf(details.get("node_name"));
type = Replica.Type.valueOf(String.valueOf(details.getOrDefault(ZkStateReader.REPLICA_TYPE, "NRT")));
state = State.getState(String.valueOf(details.getOrDefault(ZkStateReader.STATE_PROP, "active")));
this.propMap.putAll(details);
readPrs();
type = Replica.Type.valueOf(String.valueOf(propMap.getOrDefault(ZkStateReader.REPLICA_TYPE, "NRT")));
if(state == null) state = State.getState(String.valueOf(propMap.getOrDefault(ZkStateReader.STATE_PROP, "active")));
validate();
propMap.put(BASE_URL_PROP, UrlScheme.INSTANCE.getBaseUrlForNodeName(this.node));
}
private void readPrs() {
ClusterState.getReplicaStatesProvider().get().ifPresent(it -> {
log.debug("A replica {} state fetched from per-replica state", name);
replicaState = it.getStates().get(name);
if(replicaState!= null) {
propMap.put(ZkStateReader.STATE_PROP, replicaState.state.toString().toLowerCase(Locale.ROOT));
if (replicaState.isLeader) propMap.put(Slice.LEADER, "true");
}
}) ;
}
private final void validate() {
Objects.requireNonNull(this.name, "'name' must not be null");
Objects.requireNonNull(this.core, "'core' must not be null");
@ -295,6 +322,24 @@ public class Replica extends ZkNodeProps implements MapWriter {
final String propertyValue = getStr(propertyKey);
return propertyValue;
}
public Replica copyWith(PerReplicaStates.State state) {
log.debug("A replica is updated with new state : {}", state);
Map<String, Object> props = new LinkedHashMap<>(propMap);
if (state == null) {
props.put(ZkStateReader.STATE_PROP, State.DOWN.toString());
props.remove(Slice.LEADER);
} else {
props.put(ZkStateReader.STATE_PROP, state.state.toString());
if (state.isLeader) props.put(Slice.LEADER, "true");
}
Replica r = new Replica(name, props, collection, shard);
r.replicaState = state;
return r;
}
public PerReplicaStates.State getReplicaState() {
return replicaState;
}
public Object clone() {
return new Replica(name, node, collection, shard, core, state, type, propMap);
@ -305,6 +350,17 @@ public class Replica extends ZkNodeProps implements MapWriter {
ew.put(name, _allPropsWriter());
}
private static final Map<String, State> STATES = new HashMap<>();
static {
STATES.put(Replica.State.ACTIVE.shortName, Replica.State.ACTIVE);
STATES.put(Replica.State.DOWN.shortName, Replica.State.DOWN);
STATES.put(Replica.State.RECOVERING.shortName, Replica.State.RECOVERING);
STATES.put(Replica.State.RECOVERY_FAILED.shortName, Replica.State.RECOVERY_FAILED);
}
public static State getState(String shortName) {
return STATES.get(shortName);
}
private MapWriter _allPropsWriter() {
BiPredicate<CharSequence, Object> p = dedupeKeyPredicate(new HashSet<>())
@ -330,7 +386,6 @@ public class Replica extends ZkNodeProps implements MapWriter {
.put(ZkStateReader.STATE_PROP, state.toString(), p);
};
}
@Override
public void write(JSONWriter jsonWriter) {
Map<String, Object> map = new LinkedHashMap<>();

View File

@ -16,6 +16,7 @@
*/
package org.apache.solr.common.cloud;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
@ -25,11 +26,14 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import org.apache.solr.common.cloud.Replica.Type;
import org.noggit.JSONWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.util.Utils.toJSONString;
@ -37,6 +41,8 @@ import static org.apache.solr.common.util.Utils.toJSONString;
* A Slice contains immutable information about a logical shard (all replicas that share the same shard id).
*/
public class Slice extends ZkNodeProps implements Iterable<Replica> {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public final String collection;
/** Loads multiple slices into a Map from a generic Map that probably came from deserialized JSON. */
@ -61,6 +67,16 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
return replicas.values().iterator();
}
/**Make a copy with a modified replica
*/
public Slice copyWith(Replica modified) {
if(log.isDebugEnabled()) {
log.debug("modified replica : {}", modified);
}
Map<String, Replica> replicasCopy = new LinkedHashMap<>(replicas);
replicasCopy.put(modified.getName(), modified);
return new Slice(name, replicasCopy, propMap, collection);
}
/** The slice's state. */
public enum State {
@ -210,7 +226,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
private Replica findLeader() {
for (Replica replica : replicas.values()) {
if (replica.getStr(LEADER) != null) {
if (replica.isLeader()) {
assert replica.getType() == Type.TLOG || replica.getType() == Type.NRT: "Pull replica should not become leader!";
return replica;
}
@ -235,6 +251,10 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
return replicas.values();
}
public Set<String> getReplicaNames() {
return Collections.unmodifiableSet(replicas.keySet());
}
/**
* Gets all replicas that match a predicate
*/

View File

@ -330,6 +330,18 @@ public class SolrZkClient implements Closeable {
}
}
/**
* Returns children of the node at the path
*/
public List<String> getChildren(final String path, final Watcher watcher,Stat stat, boolean retryOnConnLoss)
throws KeeperException, InterruptedException {
if (retryOnConnLoss) {
return zkCmdExecutor.retryOperation(() -> keeper.getChildren(path, wrapWatcher(watcher) , stat));
} else {
return keeper.getChildren(path, wrapWatcher(watcher), stat);
}
}
/**
* Returns node's data
*/

View File

@ -655,7 +655,7 @@ public class ZkStateReader implements SolrCloseable {
private class LazyCollectionRef extends ClusterState.CollectionRef {
private final String collName;
private long lastUpdateTime;
private volatile long lastUpdateTime;
private DocCollection cachedDocCollection;
public LazyCollectionRef(String collName) {
@ -670,12 +670,12 @@ public class ZkStateReader implements SolrCloseable {
if (!allowCached || lastUpdateTime < 0 || System.nanoTime() - lastUpdateTime > LAZY_CACHE_TIME) {
boolean shouldFetch = true;
if (cachedDocCollection != null) {
Stat exists = null;
Stat freshStats = null;
try {
exists = zkClient.exists(getCollectionPath(collName), null, true);
freshStats = zkClient.exists(getCollectionPath(collName), null, true);
} catch (Exception e) {
}
if (exists != null && exists.getVersion() == cachedDocCollection.getZNodeVersion()) {
if (freshStats != null && !cachedDocCollection.isModified(freshStats.getVersion(), freshStats.getCversion())) {
shouldFetch = false;
}
}
@ -853,14 +853,16 @@ public class ZkStateReader implements SolrCloseable {
* Get shard leader properties, with retry if none exist.
*/
public Replica getLeaderRetry(String collection, String shard, int timeout) throws InterruptedException {
AtomicReference<DocCollection> coll = new AtomicReference<>();
AtomicReference<Replica> leader = new AtomicReference<>();
try {
waitForState(collection, timeout, TimeUnit.MILLISECONDS, (n, c) -> {
if (c == null)
return false;
coll.set(c);
Replica l = getLeader(n, c, shard);
if (l != null) {
log.debug("leader found for {}/{} to be {}", collection, shard, l);
leader.set(l);
return true;
}
@ -1195,9 +1197,11 @@ public class ZkStateReader implements SolrCloseable {
*/
class StateWatcher implements Watcher {
private final String coll;
private final String collectionPath;
StateWatcher(String coll) {
this.coll = coll;
collectionPath = getCollectionPath(coll);
}
@Override
@ -1219,17 +1223,28 @@ public class ZkStateReader implements SolrCloseable {
event, coll, liveNodes.size());
}
refreshAndWatch();
refreshAndWatch(event.getType());
}
public void refreshAndWatch() {
refreshAndWatch(null);
}
/**
* Refresh collection state from ZK and leave a watch for future changes.
* As a side effect, updates {@link #clusterState} and {@link #watchedCollectionStates}
* with the results of the refresh.
*/
public void refreshAndWatch() {
public void refreshAndWatch(EventType eventType) {
try {
if (eventType == null || eventType == EventType.NodeChildrenChanged) {
refreshAndWatchChildren();
if (eventType == EventType.NodeChildrenChanged) {
//only per-replica states modified. return
return;
}
}
DocCollection newState = fetchCollectionState(coll, this);
updateWatchedCollection(coll, newState);
synchronized (getUpdateLock()) {
@ -1246,6 +1261,29 @@ public class ZkStateReader implements SolrCloseable {
log.error("Unwatched collection: [{}]", coll, e);
}
}
private void refreshAndWatchChildren() throws KeeperException, InterruptedException {
Stat stat = new Stat();
List<String> replicaStates = null;
try {
replicaStates = zkClient.getChildren(collectionPath, this, stat, true);
PerReplicaStates newStates = new PerReplicaStates(collectionPath, stat.getCversion(), replicaStates);
DocCollection oldState = watchedCollectionStates.get(coll);
final DocCollection newState = oldState != null ?
oldState.copyWith(newStates) :
fetchCollectionState(coll, null);
updateWatchedCollection(coll, newState);
synchronized (getUpdateLock()) {
constructState(Collections.singleton(coll));
}
if (log.isDebugEnabled()) {
log.debug("updated per-replica states changed for: {}, ver: {} , new vals: {}", coll, stat.getCversion(), replicaStates);
}
} catch (NoNodeException e) {
log.info("{} is deleted, stop watching children", collectionPath);
}
}
}
/**
@ -1422,6 +1460,16 @@ public class ZkStateReader implements SolrCloseable {
private DocCollection fetchCollectionState(String coll, Watcher watcher) throws KeeperException, InterruptedException {
String collectionPath = getCollectionPath(coll);
while (true) {
ClusterState.initReplicaStateProvider(() -> {
try {
PerReplicaStates replicaStates = PerReplicaStates.fetch(collectionPath, zkClient, null);
log.info("per-replica-state ver: {} fetched for initializing {} ", replicaStates.cversion, collectionPath);
return replicaStates;
} catch (Exception e) {
//TODO
throw new RuntimeException(e);
}
});
try {
Stat stat = new Stat();
byte[] data = zkClient.getData(collectionPath, watcher, stat, true);
@ -1439,6 +1487,8 @@ public class ZkStateReader implements SolrCloseable {
}
}
return null;
} finally {
ClusterState.clearReplicaStateProvider();
}
}
}
@ -1566,11 +1616,28 @@ public class ZkStateReader implements SolrCloseable {
}
DocCollection state = clusterState.getCollectionOrNull(collection);
state = updatePerReplicaState(state);
if (stateWatcher.onStateChanged(state) == true) {
removeDocCollectionWatcher(collection, stateWatcher);
}
}
private DocCollection updatePerReplicaState(DocCollection c) {
if (c == null || !c.isPerReplicaState()) return c;
PerReplicaStates current = c.getPerReplicaStates();
PerReplicaStates newPrs = PerReplicaStates.fetch(c.getZNode(), zkClient, current);
if (newPrs != current) {
if(log.isDebugEnabled()) {
log.debug("update for a fresh per-replica-state {}", c.getName());
}
DocCollection modifiedColl = c.copyWith(newPrs);
updateWatchedCollection(c.getName(), modifiedColl);
return modifiedColl;
} else {
return c;
}
}
/**
* Block until a CollectionStatePredicate returns true, or the wait times out
*
@ -1804,7 +1871,9 @@ public class ZkStateReader implements SolrCloseable {
break;
}
} else {
if (oldState.getZNodeVersion() >= newState.getZNodeVersion()) {
int oldCVersion = oldState.getPerReplicaStates() == null ? -1 : oldState.getPerReplicaStates().cversion;
int newCVersion = newState.getPerReplicaStates() == null ? -1 : newState.getPerReplicaStates().cversion;
if (oldState.getZNodeVersion() >= newState.getZNodeVersion() && oldCVersion >= newCVersion) {
// no change to state, but we might have been triggered by the addition of a
// state watcher, so run notifications
updated = true;
@ -2199,6 +2268,7 @@ public class ZkStateReader implements SolrCloseable {
}
public DocCollection getCollection(String collection) {
return clusterState.getCollectionOrNull(collection);
return clusterState == null ? null : clusterState.getCollectionOrNull(collection);
}
}

View File

@ -90,6 +90,11 @@
"type": "boolean",
"description": "If true then request will complete only when all affected replicas become active.",
"default": false
},
"perReplicaState": {
"type": "boolean",
"description": "Use Per replica states",
"default": false
}
},
"required": [

View File

@ -25,14 +25,11 @@ import java.util.List;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.util.ExternalPaths;
import org.junit.After;
import org.junit.BeforeClass;
@ -70,7 +67,9 @@ public class IndexingNestedDocuments extends SolrCloudTestCase {
*/
public void testIndexingAnonKids() throws Exception {
final String collection = "test_anon";
CollectionAdminRequest.createCollection(collection, ANON_KIDS_CONFIG, 1, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection, ANON_KIDS_CONFIG, 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.getSolrClient().setDefaultCollection(collection);
//

View File

@ -57,7 +57,9 @@ public class JsonRequestApiHeatmapFacetingTest extends SolrCloudTestCase {
final List<String> solrUrls = new ArrayList<>();
solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
indexSpatialData();
}

View File

@ -64,7 +64,8 @@ public class JsonRequestApiTest extends SolrCloudTestCase {
final List<String> solrUrls = new ArrayList<>();
solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
up.setParam("collection", COLLECTION_NAME);

View File

@ -69,6 +69,7 @@ public class UsingSolrJRefGuideExamplesTest extends SolrCloudTestCase {
.configure();
CollectionAdminResponse response = CollectionAdminRequest.createCollection("techproducts", "conf", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("techproducts", 1, 1);
}

View File

@ -47,6 +47,7 @@ import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.client.solrj.response.SolrPingResponse;
@ -60,8 +61,10 @@ import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.PerReplicaStates;
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.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
@ -71,6 +74,7 @@ import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.handler.admin.CollectionsHandler;
import org.apache.solr.handler.admin.ConfigSetsHandler;
import org.apache.solr.handler.admin.CoreAdminHandler;
import org.apache.solr.util.LogLevel;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -80,11 +84,14 @@ import org.junit.rules.ExpectedException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
/**
* This test would be faster if we simulated the zk state instead.
*/
@Slow
@LogLevel("org.apache.solr.cloud.Overseer=INFO;org.apache.solr.common.cloud=INFO;org.apache.solr.cloud.api.collections=INFO;org.apache.solr.cloud.overseer=INFO")
public class CloudSolrClientTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@ -140,7 +147,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
@Test
public void testParallelUpdateQTime() throws Exception {
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 2);
UpdateRequest req = new UpdateRequest();
for (int i=0; i<10; i++) {
@ -157,6 +166,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
public void testOverwriteOption() throws Exception {
CollectionAdminRequest.createCollection("overwrite", "conf", 1, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), TIMEOUT);
cluster.waitForActiveCollection("overwrite", 1, 1);
@ -180,10 +190,14 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
@Test
public void testAliasHandling() throws Exception {
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 2);
CollectionAdminRequest.createCollection(COLLECTION2, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION2, "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION2, 2, 2);
CloudSolrClient client = getRandomClient();
@ -228,7 +242,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
@Test
public void testRouting() throws Exception {
CollectionAdminRequest.createCollection("routing_collection", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("routing_collection", "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("routing_collection", 2, 2);
AbstractUpdateRequest request = new UpdateRequest()
@ -412,6 +428,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
// all its cores on the same node.
// Hence the below configuration for our collection
CollectionAdminRequest.createCollection(collectionName, "conf", liveNodes, liveNodes)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), TIMEOUT);
cluster.waitForActiveCollection(collectionName, liveNodes, liveNodes * liveNodes);
// Add some new documents
@ -482,7 +499,8 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
int liveNodes = cluster.getJettySolrRunners().size();
// For testing replica.type, we want to have all replica types available for the collection
CollectionAdminRequest.createCollection(collectionName, "conf", 1, liveNodes/3, liveNodes/3, liveNodes/3)
CollectionAdminRequest.createCollection(collectionName, "conf", 1, liveNodes / 3, liveNodes / 3, liveNodes / 3)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), TIMEOUT);
cluster.waitForActiveCollection(collectionName, 1, liveNodes);
@ -632,8 +650,10 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
try (CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress())) {
String async1 = CollectionAdminRequest.createCollection("multicollection1", "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.processAsync(client);
String async2 = CollectionAdminRequest.createCollection("multicollection2", "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.processAsync(client);
CollectionAdminRequest.waitForAsyncRequest(async1, client, TIMEOUT);
@ -776,7 +796,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
@Test
public void testVersionsAreReturned() throws Exception {
CollectionAdminRequest.createCollection("versions_collection", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("versions_collection", "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("versions_collection", 2, 2);
// assert that "adds" are returned
@ -825,7 +847,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
@Test
public void testInitializationWithSolrUrls() throws Exception {
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 2);
CloudSolrClient client = httpBasedCloudSolrClient;
SolrInputDocument doc = new SolrInputDocument("id", "1", "title_s", "my doc");
@ -1021,7 +1045,9 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
@Test
public void testPing() throws Exception {
final String testCollection = "ping_test";
CollectionAdminRequest.createCollection(testCollection, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(testCollection, "conf", 2, 1)
.setPerReplicaState(USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(testCollection, 2, 2);
final SolrClient clientUnderTest = getRandomClient();
@ -1030,4 +1056,34 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
assertEquals("This should be OK", 0, response.getStatus());
}
public void testPerReplicaStateCollection() throws Exception {
CollectionAdminRequest.createCollection("versions_collection", "conf", 2, 1)
.process(cluster.getSolrClient());
String testCollection = "perReplicaState_test";
int liveNodes = cluster.getJettySolrRunners().size();
CollectionAdminRequest.createCollection(testCollection, "conf", 2, 2)
.setPerReplicaState(Boolean.TRUE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(testCollection, 2, 4);
final SolrClient clientUnderTest = getRandomClient();
final SolrPingResponse response = clientUnderTest.ping(testCollection);
assertEquals("This should be OK", 0, response.getStatus());
DocCollection c = cluster.getSolrClient().getZkStateReader().getCollection(testCollection);
c.forEachReplica((s, replica) -> assertNotNull(replica.getReplicaState()));
PerReplicaStates prs = PerReplicaStates.fetch(ZkStateReader.getCollectionPath(testCollection), cluster.getZkClient(), null);
assertEquals(4, prs.states.size());
testCollection = "perReplicaState_testv2";
new V2Request.Builder("/collections")
.withMethod(POST)
.withPayload("{create: {name: perReplicaState_testv2, config : conf, numShards : 2, nrtReplicas : 2, perReplicaState : true}}")
.build()
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(testCollection, 2, 4);
c = cluster.getSolrClient().getZkStateReader().getCollection(testCollection);
c.forEachReplica((s, replica) -> assertNotNull(replica.getReplicaState()));
prs = PerReplicaStates.fetch(ZkStateReader.getCollectionPath(testCollection), cluster.getZkClient(), null);
assertEquals(4, prs.states.size());
}
}

View File

@ -41,15 +41,14 @@ import org.apache.solr.common.util.Utils;
import org.apache.solr.security.BasicAuthPlugin;
import org.apache.solr.security.RuleBasedAuthorizationPlugin;
import org.apache.solr.util.TimeOut;
import static org.apache.solr.security.Sha256AuthenticationProvider.getSaltedHashedValue;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.security.Sha256AuthenticationProvider.getSaltedHashedValue;
/**
* tests various streaming expressions (via the SolrJ {@link SolrStream} API) against a SolrCloud cluster
* using both Authenticationand Role based Authorization
@ -126,6 +125,7 @@ public class CloudAuthStreamTest extends SolrCloudTestCase {
for (String collection : Arrays.asList(COLLECTION_X, COLLECTION_Y)) {
CollectionAdminRequest.createCollection(collection, "_default", 2, 2)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.setBasicAuthCredentials(ADMIN_USER, ADMIN_USER)
.process(cluster.getSolrClient());
}

View File

@ -74,7 +74,9 @@ public class JDBCStreamTest extends SolrCloudTestCase {
} else {
collection = COLLECTIONORALIAS;
}
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection, "conf", 2, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
if (useAlias) {

View File

@ -65,7 +65,10 @@ public class MathExpressionTest extends SolrCloudTestCase {
collection = COLLECTIONORALIAS;
}
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection, "conf", 2, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
if (useAlias) {

View File

@ -68,7 +68,9 @@ public class SelectWithEvaluatorsTest extends SolrCloudTestCase {
} else {
collection = COLLECTIONORALIAS;
}
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection, "conf", 2, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
false, true, TIMEOUT);
if (useAlias) {

View File

@ -96,7 +96,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
collection = COLLECTIONORALIAS;
}
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collection, 2, 2);
@ -2679,7 +2680,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testUpdateStream() throws Exception {
CollectionAdminRequest.createCollection("destinationCollection", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("destinationCollection", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("destinationCollection", 2, 2);
new UpdateRequest()
@ -2773,7 +2775,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testParallelUpdateStream() throws Exception {
CollectionAdminRequest.createCollection("parallelDestinationCollection", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("parallelDestinationCollection", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("parallelDestinationCollection", 2, 2);
new UpdateRequest()
@ -2871,7 +2874,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testParallelDaemonUpdateStream() throws Exception {
CollectionAdminRequest.createCollection("parallelDestinationCollection1", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("parallelDestinationCollection1", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("parallelDestinationCollection1", 2, 2);
new UpdateRequest()
@ -3045,7 +3049,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
public void testParallelTerminatingDaemonUpdateStream() throws Exception {
Assume.assumeTrue(!useAlias);
CollectionAdminRequest.createCollection("parallelDestinationCollection1", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("parallelDestinationCollection1", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("parallelDestinationCollection1", 2, 2);
new UpdateRequest()
@ -3231,7 +3236,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testCommitStream() throws Exception {
CollectionAdminRequest.createCollection("destinationCollection", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("destinationCollection", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("destinationCollection", 2, 2);
new UpdateRequest()
@ -3324,7 +3330,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testParallelCommitStream() throws Exception {
CollectionAdminRequest.createCollection("parallelDestinationCollection", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("parallelDestinationCollection", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("parallelDestinationCollection", 2, 2);
new UpdateRequest()
@ -3422,7 +3429,8 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testParallelDaemonCommitStream() throws Exception {
CollectionAdminRequest.createCollection("parallelDestinationCollection1", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("parallelDestinationCollection1", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("parallelDestinationCollection1", 2, 2);
new UpdateRequest()
@ -3639,11 +3647,14 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
public void testClassifyStream() throws Exception {
Assume.assumeTrue(!useAlias);
CollectionAdminRequest.createCollection("modelCollection", "ml", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("modelCollection", "ml", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("modelCollection", 2, 2);
CollectionAdminRequest.createCollection("uknownCollection", "ml", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("uknownCollection", "ml", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("uknownCollection", 2, 2);
CollectionAdminRequest.createCollection("checkpointCollection", "ml", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("checkpointCollection", "ml", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection("checkpointCollection", 2, 2);
UpdateRequest updateRequest = new UpdateRequest();
@ -3867,11 +3878,14 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testExecutorStream() throws Exception {
CollectionAdminRequest.createCollection("workQueue", "conf", 2, 1).processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("workQueue", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
cluster.waitForActiveCollection("workQueue", 2, 2);
CollectionAdminRequest.createCollection("mainCorpus", "conf", 2, 1).processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("mainCorpus", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
cluster.waitForActiveCollection("mainCorpus", 2, 2);
CollectionAdminRequest.createCollection("destination", "conf", 2, 1).processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("destination", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
cluster.waitForActiveCollection("destination", 2, 2);
UpdateRequest workRequest = new UpdateRequest();
@ -3933,11 +3947,14 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
@Test
public void testParallelExecutorStream() throws Exception {
CollectionAdminRequest.createCollection("workQueue1", "conf", 2, 1).processAndWait(cluster.getSolrClient(),DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("workQueue1", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(),DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("mainCorpus1", "conf", 2, 1).processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("mainCorpus1", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("destination1", "conf", 2, 1).processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
CollectionAdminRequest.createCollection("destination1", "conf", 2, 1).setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
cluster.waitForActiveCollection("workQueue1", 2, 2);
cluster.waitForActiveCollection("mainCorpus1", 2, 2);

View File

@ -24,10 +24,10 @@ import java.util.List;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
import org.apache.solr.client.solrj.response.json.BucketJsonFacet;
import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.client.solrj.response.json.BucketJsonFacet;
import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.util.ExternalPaths;
@ -56,7 +56,9 @@ public class DirectJsonQueryRequestFacetingIntegrationTest extends SolrCloudTest
final List<String> solrUrls = new ArrayList<>();
solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
up.setParam("collection", COLLECTION_NAME);

View File

@ -29,10 +29,10 @@ import java.util.Map;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
import org.apache.solr.client.solrj.response.json.BucketJsonFacet;
import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.client.solrj.response.json.BucketJsonFacet;
import org.apache.solr.client.solrj.response.json.NestableJsonFacet;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrDocumentList;
@ -63,7 +63,9 @@ public class JsonQueryRequestFacetingIntegrationTest extends SolrCloudTestCase {
final List<String> solrUrls = new ArrayList<>();
solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
ContentStreamUpdateRequest up = new ContentStreamUpdateRequest("/update");
up.setParam("collection", COLLECTION_NAME);

View File

@ -47,7 +47,9 @@ public class JsonQueryRequestHeatmapFacetingTest extends SolrCloudTestCase {
final List<String> solrUrls = new ArrayList<>();
solrUrls.add(cluster.getJettySolrRunner(0).getBaseUrl().toString());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION_NAME, CONFIG_NAME, 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
indexSpatialData();
}

View File

@ -97,6 +97,7 @@ public class TestCloudCollectionsListeners extends SolrCloudTestCase {
assertFalse("CloudCollectionsListener not triggered after registration", newResults.get(2).contains("testcollection1"));
CollectionAdminRequest.createCollection("testcollection1", "config", 4, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
client.waitForState("testcollection1", MAX_WAIT_TIMEOUT, TimeUnit.SECONDS,
(n, c) -> DocCollection.isFullyActive(n, c, 4, 1));
@ -110,6 +111,7 @@ public class TestCloudCollectionsListeners extends SolrCloudTestCase {
client.getZkStateReader().removeCloudCollectionsListener(watcher1);
CollectionAdminRequest.createCollection("testcollection2", "config", 4, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
cluster.waitForActiveCollection("testcollection2", 4, 4);
@ -136,10 +138,12 @@ public class TestCloudCollectionsListeners extends SolrCloudTestCase {
CloudSolrClient client = cluster.getSolrClient();
CollectionAdminRequest.createCollection("testcollection1", "config", 4, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
cluster.waitForActiveCollection("testcollection1", 4, 4);
CollectionAdminRequest.createCollection("testcollection2", "config", 4, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
cluster.waitForActiveCollection("testcollection2", 4, 4);

View File

@ -33,6 +33,7 @@ import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.util.ExecutorUtil;
import org.junit.After;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -121,6 +122,7 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
// note: one node in our cluster is unsed by collection
CollectionAdminRequest.createCollection("testcollection", "config", CLUSTER_SIZE, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
client.waitForState("testcollection", MAX_WAIT_TIMEOUT, TimeUnit.SECONDS,
@ -169,6 +171,7 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
CloudSolrClient client = cluster.getSolrClient();
CollectionAdminRequest.createCollection("currentstate", "config", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
final CountDownLatch latch = new CountDownLatch(1);
@ -199,6 +202,7 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
CloudSolrClient client = cluster.getSolrClient();
CollectionAdminRequest.createCollection("waitforstate", "config", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
client.waitForState("waitforstate", MAX_WAIT_TIMEOUT, TimeUnit.SECONDS,
@ -217,6 +221,7 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
}
@Test
@Ignore
public void testCanWaitForNonexistantCollection() throws Exception {
Future<Boolean> future = waitInBackground("delayed", MAX_WAIT_TIMEOUT, TimeUnit.SECONDS,
@ -247,6 +252,7 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
CloudSolrClient client = cluster.getSolrClient();
CollectionAdminRequest.createCollection("falsepredicate", "config", 4, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.processAndWait(client, MAX_WAIT_TIMEOUT);
client.waitForState("falsepredicate", MAX_WAIT_TIMEOUT, TimeUnit.SECONDS,
@ -301,6 +307,7 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
@Test
public void testDeletionsTriggerWatches() throws Exception {
CollectionAdminRequest.createCollection("tobedeleted", "config", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(cluster.getSolrClient());
Future<Boolean> future = waitInBackground("tobedeleted", MAX_WAIT_TIMEOUT, TimeUnit.SECONDS,
@ -315,7 +322,9 @@ public class TestCollectionStateWatchers extends SolrCloudTestCase {
public void testLiveNodeChangesTriggerWatches() throws Exception {
final CloudSolrClient client = cluster.getSolrClient();
CollectionAdminRequest.createCollection("test_collection", "config", 1, 1).process(client);
CollectionAdminRequest.createCollection("test_collection", "config", 1, 1)
.setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
.process(client);
Future<Boolean> future = waitInBackground("test_collection", MAX_WAIT_TIMEOUT, TimeUnit.SECONDS,
(l, c) -> (l.size() == 1 + CLUSTER_SIZE));

View File

@ -0,0 +1,133 @@
/*
* 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.Set;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.Replica.State;
import org.apache.zookeeper.CreateMode;
import org.junit.After;
import org.junit.Before;
public class TestPerReplicaStates extends SolrCloudTestCase {
@Before
public void prepareCluster() throws Exception {
configureCluster(4)
.configure();
}
@After
public void tearDownCluster() throws Exception {
shutdownCluster();
}
public void testBasic() {
PerReplicaStates.State rs = new PerReplicaStates.State("R1", State.ACTIVE, Boolean.FALSE, 1);
assertEquals("R1:1:A", rs.asString);
rs = new PerReplicaStates.State("R1", State.DOWN, Boolean.TRUE, 1);
assertEquals("R1:1:D:L", rs.asString);
rs = PerReplicaStates.State.parse (rs.asString);
assertEquals(State.DOWN, rs.state);
}
public void testEntries() {
PerReplicaStates entries = new PerReplicaStates("state.json", 0, ImmutableList.of("R1:2:A", "R1:1:A:L", "R1:0:D", "R2:0:D", "R3:0:A"));
assertEquals(2, entries.get("R1").version);
entries = new PerReplicaStates("state.json", 0, ImmutableList.of("R1:1:A:L", "R1:2:A", "R2:0:D", "R3:0:A", "R1:0:D"));
assertEquals(2, entries.get("R1").version);
assertEquals(2, entries.get("R1").getDuplicates().size());
Set<String> modified = PerReplicaStates.findModifiedReplicas(entries, new PerReplicaStates("state.json", 0, ImmutableList.of("R1:1:A:L", "R1:2:A", "R2:0:D", "R3:1:A", "R1:0:D")));
assertEquals(1, modified.size());
assertTrue(modified.contains("R3"));
modified = PerReplicaStates.findModifiedReplicas( entries,
new PerReplicaStates("state.json", 0, ImmutableList.of("R1:1:A:L", "R1:2:A", "R2:0:D", "R3:1:A", "R1:0:D", "R4:0:A")));
assertEquals(2, modified.size());
assertTrue(modified.contains("R3"));
assertTrue(modified.contains("R4"));
modified = PerReplicaStates.findModifiedReplicas( entries,
new PerReplicaStates("state.json", 0, ImmutableList.of("R1:1:A:L", "R1:2:A", "R3:1:A", "R1:0:D", "R4:0:A")));
assertEquals(3, modified.size());
assertTrue(modified.contains("R3"));
assertTrue(modified.contains("R4"));
assertTrue(modified.contains("R2"));
}
public void testReplicaStateOperations() throws Exception {
String root = "/testReplicaStateOperations";
cluster.getZkClient().create(root, null, CreateMode.PERSISTENT, true);
ImmutableList<String> states = ImmutableList.of("R1:2:A", "R1:1:A:L", "R1:0:D", "R3:0:A", "R4:13:A");
for (String state : states) {
cluster.getZkClient().create(root + "/" + state, null, CreateMode.PERSISTENT, true);
}
ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
PerReplicaStates rs = PerReplicaStates.fetch (root, zkStateReader.getZkClient(),null);
assertEquals(3, rs.states.size());
assertTrue(rs.cversion >= 5);
PerReplicaStatesOps ops = PerReplicaStatesOps.addReplica("R5",State.ACTIVE, false, rs);
assertEquals(1, ops.get().size());
assertEquals(PerReplicaStates.Operation.Type.ADD , ops.ops.get(0).typ );
ops.persist(root,cluster.getZkClient());
rs = PerReplicaStates.fetch (root, zkStateReader.getZkClient(),null);
assertEquals(4, rs.states.size());
assertTrue(rs.cversion >= 6);
assertEquals(6, cluster.getZkClient().getChildren(root, null,true).size());
ops = PerReplicaStatesOps.flipState("R1", State.DOWN , rs);
assertEquals(4, ops.ops.size());
assertEquals(PerReplicaStates.Operation.Type.ADD, ops.ops.get(0).typ);
assertEquals(PerReplicaStates.Operation.Type.DELETE, ops.ops.get(1).typ);
assertEquals(PerReplicaStates.Operation.Type.DELETE, ops.ops.get(2).typ);
assertEquals(PerReplicaStates.Operation.Type.DELETE, ops.ops.get(3).typ);
ops.persist(root, cluster.getZkClient());
rs = PerReplicaStates.fetch (root, zkStateReader.getZkClient(),null);
assertEquals(4, rs.states.size());
assertEquals(3, rs.states.get("R1").version);
ops = PerReplicaStatesOps.deleteReplica("R5" , rs);
assertEquals(1, ops.ops.size());
ops.persist(root,cluster.getZkClient());
rs = PerReplicaStates.fetch (root, zkStateReader.getZkClient(),null);
assertEquals(3, rs.states.size());
ops = PerReplicaStatesOps.flipLeader(ImmutableSet.of("R4","R3","R1"), "R4",rs);
assertEquals(2, ops.ops.size());
assertEquals(PerReplicaStates.Operation.Type.ADD, ops.ops.get(0).typ);
assertEquals(PerReplicaStates.Operation.Type.DELETE, ops.ops.get(1).typ);
ops.persist(root,cluster.getZkClient());
rs = PerReplicaStates.fetch (root, zkStateReader.getZkClient(),null);
ops = PerReplicaStatesOps.flipLeader(ImmutableSet.of("R4","R3","R1"),"R3",rs);
assertEquals(4, ops.ops.size());
ops.persist(root,cluster.getZkClient());
rs =PerReplicaStates.fetch (root, zkStateReader.getZkClient(),null);
assertTrue(rs.get("R3").isLeader);
}
}

View File

@ -81,6 +81,7 @@ import org.slf4j.LoggerFactory;
public class SolrCloudTestCase extends SolrTestCaseJ4 {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final Boolean USE_PER_REPLICA_STATE = Boolean.parseBoolean(System.getProperty("use.per-replica", "false"));
public static final int DEFAULT_TIMEOUT = 45; // this is an important timeout for test stability - can't be too short