SOLR-11285: Simulation framework for autoscaling.

This commit is contained in:
Andrzej Bialecki 2017-12-14 12:13:05 +01:00
parent d66d9549d7
commit d6d2e3b2e3
126 changed files with 9187 additions and 844 deletions

View File

@ -49,6 +49,10 @@ Apache UIMA 2.3.1
Apache ZooKeeper 3.4.10
Jetty 9.3.20.v20170531
New Features
----------------------
* SOLR-11285: Simulation framework for autoscaling. (ab)
Optimizations
----------------------

View File

@ -20,7 +20,7 @@ import java.lang.invoke.MethodHandles;
import java.util.concurrent.TimeUnit;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -35,9 +35,7 @@ public class ActionThrottle {
private final TimeSource timeSource;
public ActionThrottle(String name, long minMsBetweenActions) {
this.name = name;
this.minMsBetweenActions = minMsBetweenActions;
this.timeSource = TimeSource.NANO_TIME;
this(name, minMsBetweenActions, TimeSource.NANO_TIME);
}
public ActionThrottle(String name, long minMsBetweenActions, TimeSource timeSource) {
@ -47,16 +45,20 @@ public class ActionThrottle {
}
public ActionThrottle(String name, long minMsBetweenActions, long lastActionStartedAt) {
this(name, minMsBetweenActions, lastActionStartedAt, TimeSource.NANO_TIME);
}
public ActionThrottle(String name, long minMsBetweenActions, long lastActionStartedAt, TimeSource timeSource) {
this.name = name;
this.minMsBetweenActions = minMsBetweenActions;
this.lastActionStartedAt = lastActionStartedAt;
this.timeSource = TimeSource.NANO_TIME;
this.timeSource = timeSource;
}
public void reset() {
lastActionStartedAt = null;
}
public void markAttemptingAction() {
lastActionStartedAt = timeSource.getTime();
}

View File

@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.common.SolrCloseableLatch;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@ -48,7 +49,6 @@ import org.apache.solr.handler.component.ShardHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.Assign.getNodesForNewReplicas;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_CONF;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
@ -56,6 +56,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonAdminParams.TIMEOUT;
import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@ -69,81 +70,30 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
@Override
public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
addReplica(ocmh.zkStateReader.getClusterState(), message, results, null);
addReplica(state, message, results, null);
}
ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
throws IOException, InterruptedException {
log.debug("addReplica() : {}", Utils.toJSONString(message));
boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
final String asyncId = message.getStr(ASYNC);
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
message = assignReplicaDetails(ocmh.cloudManager, clusterState, message, sessionWrapper);
String collection = message.getStr(COLLECTION_PROP);
DocCollection coll = clusterState.getCollection(collection);
String node = message.getStr(CoreAdminParams.NODE);
String shard = message.getStr(SHARD_ID_PROP);
String coreName = message.getStr(CoreAdminParams.NAME);
String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
int timeout = message.getInt("timeout", 10 * 60); // 10 minutes
int timeout = message.getInt(TIMEOUT, 10 * 60); // 10 minutes
Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT));
boolean parallel = message.getBool("parallel", false);
boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
if (StringUtils.isBlank(coreName)) {
coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
}
final String asyncId = message.getStr(ASYNC);
DocCollection coll = clusterState.getCollection(collection);
if (coll == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
}
if (coll.getSlice(shard) == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Collection: " + collection + " shard: " + shard + " does not exist");
}
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
// Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
if (!skipCreateReplicaInClusterState) {
if (CreateShardCmd.usePolicyFramework(coll, ocmh)) {
if (node == null) {
if(coll.getPolicyName() != null) message.getProperties().put(Policy.POLICY, coll.getPolicyName());
node = Assign.identifyNodes(ocmh,
clusterState,
Collections.emptyList(),
collection,
message,
Collections.singletonList(shard),
replicaType == Replica.Type.NRT ? 0 : 1,
replicaType == Replica.Type.TLOG ? 0 : 1,
replicaType == Replica.Type.PULL ? 0 : 1
).get(0).node;
sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
}
} else {
node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
ocmh.overseer.getSolrCloudManager()).get(0).nodeName;// TODO: use replica type in this logic too
}
}
log.info("Node Identified {} for creating new replica", node);
if (!clusterState.liveNodesContain(node)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
}
if (coreName == null) {
coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, shard, replicaType);
} else if (!skipCreateReplicaInClusterState) {
//Validate that the core name is unique in that collection
for (Slice slice : coll.getSlices()) {
for (Replica replica : slice.getReplicas()) {
String replicaCoreName = replica.getStr(CORE_NAME_PROP);
if (coreName.equals(replicaCoreName)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Another replica with the same core name already exists" +
" for this collection");
}
}
}
}
ModifiableSolrParams params = new ModifiableSolrParams();
ZkStateReader zkStateReader = ocmh.zkStateReader;
@ -210,6 +160,8 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
// For tracking async calls.
Map<String,String> requestMap = new HashMap<>();
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ocmh.sendShardRequest(node, params, shardHandler, asyncId, requestMap);
final String fnode = node;
@ -253,4 +205,75 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
);
}
public static ZkNodeProps assignReplicaDetails(SolrCloudManager cloudManager, ClusterState clusterState,
ZkNodeProps message, AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
String collection = message.getStr(COLLECTION_PROP);
String node = message.getStr(CoreAdminParams.NODE);
String shard = message.getStr(SHARD_ID_PROP);
String coreName = message.getStr(CoreAdminParams.NAME);
String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT));
if (StringUtils.isBlank(coreName)) {
coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
}
DocCollection coll = clusterState.getCollection(collection);
if (coll == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " does not exist");
}
if (coll.getSlice(shard) == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Collection: " + collection + " shard: " + shard + " does not exist");
}
// Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
if (!skipCreateReplicaInClusterState) {
if (CloudUtil.usePolicyFramework(coll, cloudManager)) {
if (node == null) {
if(coll.getPolicyName() != null) message.getProperties().put(Policy.POLICY, coll.getPolicyName());
node = Assign.identifyNodes(cloudManager,
clusterState,
Collections.emptyList(),
collection,
message,
Collections.singletonList(shard),
replicaType == Replica.Type.NRT ? 0 : 1,
replicaType == Replica.Type.TLOG ? 0 : 1,
replicaType == Replica.Type.PULL ? 0 : 1
).get(0).node;
sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
}
} else {
node = Assign.getNodesForNewReplicas(clusterState, collection, shard, 1, node,
cloudManager).get(0).nodeName;// TODO: use replica type in this logic too
}
}
log.info("Node Identified {} for creating new replica", node);
if (!clusterState.liveNodesContain(node)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
}
if (coreName == null) {
coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), coll, shard, replicaType);
} else if (!skipCreateReplicaInClusterState) {
//Validate that the core name is unique in that collection
for (Slice slice : coll.getSlices()) {
for (Replica replica : slice.getReplicas()) {
String replicaCoreName = replica.getStr(CORE_NAME_PROP);
if (coreName.equals(replicaCoreName)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Another replica with the same core name already exists" +
" for this collection");
}
}
}
}
if (coreNodeName != null) {
message = message.plus(CoreAdminParams.CORE_NODE_NAME, coreNodeName);
}
message = message.plus(CoreAdminParams.NAME, coreName);
message = message.plus(CoreAdminParams.NODE, node);
return message;
}
}

View File

@ -114,7 +114,7 @@ public class Assign {
}
}
public static String assignNode(DistribStateManager stateManager, DocCollection collection) {
public static String assignCoreNodeName(DistribStateManager stateManager, DocCollection collection) {
// for backward compatibility;
int defaultValue = defaultCounterValue(collection, false);
String coreNodeName = "core_node" + incAndGetId(stateManager, collection.getName(), defaultValue);
@ -170,7 +170,7 @@ public class Assign {
return returnShardId;
}
private static String buildCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
private static String buildSolrCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
// TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type
return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
}
@ -187,20 +187,20 @@ public class Assign {
return defaultValue * 20;
}
public static String buildCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
Slice slice = collection.getSlice(shard);
int defaultValue = defaultCounterValue(collection, newCollection);
int replicaNum = incAndGetId(stateManager, collection.getName(), defaultValue);
String coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
String coreName = buildSolrCoreName(collection.getName(), shard, type, replicaNum);
while (existCoreName(coreName, slice)) {
replicaNum = incAndGetId(stateManager, collection.getName(), defaultValue);
coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
coreName = buildSolrCoreName(collection.getName(), shard, type, replicaNum);
}
return coreName;
}
public static String buildCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
return buildCoreName(stateManager, collection, shard, type, false);
public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
return buildSolrCoreName(stateManager, collection, shard, type, false);
}
private static boolean existCoreName(String coreName, Slice slice) {
@ -237,7 +237,7 @@ public class Assign {
return nodeList;
}
public static List<ReplicaPosition> identifyNodes(OverseerCollectionMessageHandler ocmh,
public static List<ReplicaPosition> identifyNodes(SolrCloudManager cloudManager,
ClusterState clusterState,
List<String> nodeList,
String collectionName,
@ -248,7 +248,7 @@ public class Assign {
int numPullReplicas) throws IOException, InterruptedException {
List<Map> rulesMap = (List) message.get("rule");
String policyName = message.getStr(POLICY);
AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
if (rulesMap == null && policyName == null && autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) {
log.debug("Identify nodes using default");
@ -283,7 +283,7 @@ public class Assign {
(List<Map>) message.get(SNITCH),
new HashMap<>(),//this is a new collection. So, there are no nodes in any shard
nodeList,
ocmh.overseer.getSolrCloudManager(),
cloudManager,
clusterState);
Map<ReplicaPosition, String> nodeMappings = replicaAssigner.getNodeMappings();
@ -294,7 +294,7 @@ public class Assign {
if (message.getStr(CREATE_NODE_SET) == null)
nodeList = Collections.emptyList();// unless explicitly specified do not pass node list to Policy
return getPositionsUsingPolicy(collectionName,
shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, ocmh.overseer.getSolrCloudManager(), nodeList);
shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, cloudManager, nodeList);
}
}
@ -397,7 +397,7 @@ public class Assign {
nodesList);
return replicaPositions;
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error closing CloudSolrClient",e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error getting replica locations", e);
} finally {
if (log.isTraceEnabled()) {
if (replicaPositions != null)

View File

@ -42,7 +42,7 @@ public class CloudConfigSetService extends ConfigSetService {
try {
// for back compat with cores that can create collections without the collections API
if (!zkController.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cd.getCollectionName(), true)) {
CreateCollectionCmd.createCollectionZkNode(zkController.getZkClient(), cd.getCollectionName(), cd.getCloudDescriptor().getParams());
CreateCollectionCmd.createCollectionZkNode(zkController.getSolrCloudManager().getDistribStateManager(), cd.getCollectionName(), cd.getCloudDescriptor().getParams());
}
} catch (KeeperException e) {
SolrException.log(log, null, e);

View File

@ -25,6 +25,8 @@ import java.util.List;
import java.util.Map;
import org.apache.commons.io.FileUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.DocCollection;
@ -130,4 +132,9 @@ public class CloudUtil {
}
static boolean usePolicyFramework(DocCollection collection, SolrCloudManager cloudManager)
throws IOException, InterruptedException {
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
return !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || collection.getPolicyName() != null;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.solr.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
@ -26,11 +27,18 @@ import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.common.SolrException;
@ -40,7 +48,6 @@ import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -51,6 +58,7 @@ import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.admin.ConfigSetsHandlerApi;
import org.apache.solr.handler.component.ShardHandler;
@ -70,7 +78,6 @@ import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
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.REPLICATION_FACTOR;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@ -81,11 +88,13 @@ import static org.apache.solr.common.util.StrUtils.formatString;
public class CreateCollectionCmd implements Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private SolrZkClient zkClient;
private final TimeSource timeSource;
private final DistribStateManager stateManager;
public CreateCollectionCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.zkClient = ocmh.zkStateReader.getZkClient();
this.stateManager = ocmh.cloudManager.getDistribStateManager();
this.timeSource = ocmh.cloudManager.getTimeSource();
}
@Override
@ -103,95 +112,20 @@ public class CreateCollectionCmd implements Cmd {
}
ocmh.validateConfigOrThrowSolrException(configName);
PolicyHelper.SessionWrapper sessionWrapper = null;
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
try {
// look at the replication factor and see if it matches reality
// if it does not, find best nodes to create more cores
int numTlogReplicas = message.getInt(TLOG_REPLICAS, 0);
int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
int numPullReplicas = message.getInt(PULL_REPLICAS, 0);
Map autoScalingJson = Utils.getJson(ocmh.zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
String policy = message.getStr(Policy.POLICY);
boolean usePolicyFramework = autoScalingJson.get(Policy.CLUSTER_POLICY) != null || policy != null;
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
final String async = message.getStr(ASYNC);
Integer numSlices = message.getInt(NUM_SLICES, null);
String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
List<String> nodeList = new ArrayList<>();
List<String> shardNames = new ArrayList<>();
if(ImplicitDocRouter.NAME.equals(router)){
ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
numSlices = shardNames.size();
} else {
if (numSlices == null ) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required param (when using CompositeId router).");
}
ClusterStateMutator.getShardNames(numSlices, shardNames);
}
int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, 1);
if (usePolicyFramework && message.getStr(MAX_SHARDS_PER_NODE) != null && maxShardsPerNode > 0) {
throw new SolrException(ErrorCode.BAD_REQUEST, "'maxShardsPerNode>0' is not supported when autoScaling policies are used");
}
if (maxShardsPerNode == -1 || usePolicyFramework) maxShardsPerNode = Integer.MAX_VALUE;
if (numNrtReplicas + numTlogReplicas <= 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
}
if (numSlices <= 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " must be > 0");
}
// we need to look at every node and see how many cores it serves
// add our new cores to existing nodes serving the least number of cores
// but (for now) require that each core goes on a distinct node.
final List<String> nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM);
List<ReplicaPosition> replicaPositions;
if (nodeList.isEmpty()) {
log.warn("It is unusual to create a collection ("+collectionName+") without cores.");
replicaPositions = new ArrayList<>();
} else {
int totalNumReplicas = numNrtReplicas + numTlogReplicas + numPullReplicas;
if (totalNumReplicas > nodeList.size()) {
log.warn("Specified number of replicas of "
+ totalNumReplicas
+ " on collection "
+ collectionName
+ " is higher than the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
+ nodeList.size()
+ "). It's unusual to run two replica of the same slice on the same Solr-instance.");
}
int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
Integer.MAX_VALUE :
maxShardsPerNode * nodeList.size();
int requestedShardsToCreate = numSlices * totalNumReplicas;
if (maxShardsAllowedToCreate < requestedShardsToCreate) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create collection " + collectionName + ". Value of "
+ MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
+ ", and the number of nodes currently live or live and part of your "+CREATE_NODE_SET+" is " + nodeList.size()
+ ". This allows a maximum of " + maxShardsAllowedToCreate
+ " to be created. Value of " + NUM_SLICES + " is " + numSlices
+ ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
+ ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
+ " and value of " + PULL_REPLICAS + " is " + numPullReplicas
+ ". This requires " + requestedShardsToCreate
+ " shards to be created (higher than the allowed number)");
}
replicaPositions = Assign.identifyNodes(ocmh
, clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
}
List<ReplicaPosition> replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, message,
nodeList, shardNames, sessionWrapper);
ZkStateReader zkStateReader = ocmh.zkStateReader;
boolean isLegacyCloud = Overseer.isLegacy(zkStateReader);
ocmh.createConfNode(configName, collectionName, isLegacyCloud);
ocmh.createConfNode(stateManager, configName, collectionName, isLegacyCloud);
Map<String,String> collectionParams = new HashMap<>();
Map<String,Object> collectionProps = message.getProperties();
@ -201,16 +135,16 @@ public class CreateCollectionCmd implements Cmd {
}
}
createCollectionZkNode(zkClient, collectionName, collectionParams);
createCollectionZkNode(stateManager, collectionName, collectionParams);
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
// wait for a while until we don't see the collection
TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS);
TimeOut waitUntil = new TimeOut(30, TimeUnit.SECONDS, timeSource);
boolean created = false;
while (! waitUntil.hasTimedOut()) {
Thread.sleep(100);
created = zkStateReader.getClusterState().hasCollection(collectionName);
waitUntil.sleep(100);
created = ocmh.cloudManager.getClusterStateProvider().getClusterState().hasCollection(collectionName);
if(created) break;
}
if (!created)
@ -225,12 +159,14 @@ public class CreateCollectionCmd implements Cmd {
Map<String, String> requestMap = new HashMap<>();
log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , nrtReplicas : {2}, tlogReplicas: {3}, pullReplicas: {4}",
collectionName, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas));
log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , message : {2}",
collectionName, shardNames, message));
Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
for (ReplicaPosition replicaPosition : replicaPositions) {
String nodeName = replicaPosition.node;
String coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), zkStateReader.getClusterState().getCollection(collectionName),
String coreName = Assign.buildSolrCoreName(ocmh.cloudManager.getDistribStateManager(),
ocmh.cloudManager.getClusterStateProvider().getClusterState().getCollection(collectionName),
replicaPosition.shard, replicaPosition.type, true);
log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
, coreName, replicaPosition.shard, collectionName, nodeName));
@ -260,7 +196,7 @@ public class CreateCollectionCmd implements Cmd {
params.set(COLL_CONF, configName);
params.set(CoreAdminParams.COLLECTION, collectionName);
params.set(CoreAdminParams.SHARD, replicaPosition.shard);
params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices);
params.set(ZkStateReader.NUM_SHARDS_PROP, shardNames.size());
params.set(CoreAdminParams.NEW_COLLECTION, "true");
params.set(CoreAdminParams.REPLICA_TYPE, replicaPosition.type.name());
@ -320,10 +256,93 @@ public class CreateCollectionCmd implements Cmd {
} catch (Exception ex) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
} finally {
if(sessionWrapper != null) sessionWrapper.release();
if (sessionWrapper.get() != null) sessionWrapper.get().release();
}
}
public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
ZkNodeProps message,
List<String> nodeList, List<String> shardNames,
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
final String collectionName = message.getStr(NAME);
// look at the replication factor and see if it matches reality
// if it does not, find best nodes to create more cores
int numTlogReplicas = message.getInt(TLOG_REPLICAS, 0);
int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, numTlogReplicas>0?0:1));
int numPullReplicas = message.getInt(PULL_REPLICAS, 0);
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
String policy = message.getStr(Policy.POLICY);
boolean usePolicyFramework = !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || policy != null;
Integer numSlices = message.getInt(NUM_SLICES, null);
String router = message.getStr("router.name", DocRouter.DEFAULT_NAME);
if(ImplicitDocRouter.NAME.equals(router)){
ClusterStateMutator.getShardNames(shardNames, message.getStr("shards", null));
numSlices = shardNames.size();
} else {
if (numSlices == null ) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required param (when using CompositeId router).");
}
if (numSlices <= 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NUM_SLICES + " must be > 0");
}
ClusterStateMutator.getShardNames(numSlices, shardNames);
}
int maxShardsPerNode = message.getInt(MAX_SHARDS_PER_NODE, 1);
if (usePolicyFramework && message.getStr(MAX_SHARDS_PER_NODE) != null && maxShardsPerNode > 0) {
throw new SolrException(ErrorCode.BAD_REQUEST, "'maxShardsPerNode>0' is not supported when autoScaling policies are used");
}
if (maxShardsPerNode == -1 || usePolicyFramework) maxShardsPerNode = Integer.MAX_VALUE;
if (numNrtReplicas + numTlogReplicas <= 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
}
// we need to look at every node and see how many cores it serves
// add our new cores to existing nodes serving the least number of cores
// but (for now) require that each core goes on a distinct node.
List<ReplicaPosition> replicaPositions;
nodeList.addAll(Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM));
if (nodeList.isEmpty()) {
log.warn("It is unusual to create a collection ("+collectionName+") without cores.");
replicaPositions = new ArrayList<>();
} else {
int totalNumReplicas = numNrtReplicas + numTlogReplicas + numPullReplicas;
if (totalNumReplicas > nodeList.size()) {
log.warn("Specified number of replicas of "
+ totalNumReplicas
+ " on collection "
+ collectionName
+ " is higher than the number of Solr instances currently live or live and part of your " + CREATE_NODE_SET + "("
+ nodeList.size()
+ "). It's unusual to run two replica of the same slice on the same Solr-instance.");
}
int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
Integer.MAX_VALUE :
maxShardsPerNode * nodeList.size();
int requestedShardsToCreate = numSlices * totalNumReplicas;
if (maxShardsAllowedToCreate < requestedShardsToCreate) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create collection " + collectionName + ". Value of "
+ MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
+ ", and the number of nodes currently live or live and part of your "+CREATE_NODE_SET+" is " + nodeList.size()
+ ". This allows a maximum of " + maxShardsAllowedToCreate
+ " to be created. Value of " + NUM_SLICES + " is " + numSlices
+ ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
+ ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
+ " and value of " + PULL_REPLICAS + " is " + numPullReplicas
+ ". This requires " + requestedShardsToCreate
+ " shards to be created (higher than the allowed number)");
}
replicaPositions = Assign.identifyNodes(cloudManager
, clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
}
return replicaPositions;
}
String getConfigName(String coll, ZkNodeProps message) throws KeeperException, InterruptedException {
String configName = message.getStr(COLL_CONF);
@ -375,12 +394,12 @@ public class CreateCollectionCmd implements Cmd {
}
}
public static void createCollectionZkNode(SolrZkClient zkClient, String collection, Map<String,String> params) {
public static void createCollectionZkNode(DistribStateManager stateManager, String collection, Map<String,String> params) {
log.debug("Check for collection zkNode:" + collection);
String collectionPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
try {
if (!zkClient.exists(collectionPath, true)) {
if (!stateManager.hasData(collectionPath)) {
log.debug("Creating collection in ZooKeeper:" + collection);
try {
@ -394,7 +413,7 @@ public class CreateCollectionCmd implements Cmd {
// if the config name wasn't passed in, use the default
if (!collectionProps.containsKey(ZkController.CONFIGNAME_PROP)) {
// users can create the collection node and conf link ahead of time, or this may return another option
getConfName(zkClient, collection, collectionPath, collectionProps);
getConfName(stateManager, collection, collectionPath, collectionProps);
}
} else if (System.getProperty("bootstrap_confdir") != null) {
@ -417,19 +436,21 @@ public class CreateCollectionCmd implements Cmd {
// the conf name should should be the collection name of this core
collectionProps.put(ZkController.CONFIGNAME_PROP, collection);
} else {
getConfName(zkClient, collection, collectionPath, collectionProps);
getConfName(stateManager, collection, collectionPath, collectionProps);
}
collectionProps.remove(ZkStateReader.NUM_SHARDS_PROP); // we don't put numShards in the collections properties
ZkNodeProps zkProps = new ZkNodeProps(collectionProps);
zkClient.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, null, true);
stateManager.makePath(collectionPath, Utils.toJSON(zkProps), CreateMode.PERSISTENT, false);
} catch (KeeperException e) {
// it's okay if the node already exists
if (e.code() != KeeperException.Code.NODEEXISTS) {
throw e;
}
} catch (AlreadyExistsException e) {
// it's okay if the node already exists
}
} else {
log.debug("Collection zkNode exists");
@ -441,6 +462,8 @@ public class CreateCollectionCmd implements Cmd {
return;
}
throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
} catch (IOException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
} catch (InterruptedException e) {
Thread.interrupted();
throw new SolrException(ErrorCode.SERVER_ERROR, "Error creating collection node in Zookeeper", e);
@ -448,8 +471,8 @@ public class CreateCollectionCmd implements Cmd {
}
private static void getConfName(SolrZkClient zkClient, String collection, String collectionPath, Map<String,Object> collectionProps) throws KeeperException,
InterruptedException {
private static void getConfName(DistribStateManager stateManager, String collection, String collectionPath, Map<String,Object> collectionProps) throws IOException,
KeeperException, InterruptedException {
// check for configName
log.debug("Looking for collection configName");
if (collectionProps.containsKey("configName")) {
@ -461,17 +484,17 @@ public class CreateCollectionCmd implements Cmd {
int retry = 1;
int retryLimt = 6;
for (; retry < retryLimt; retry++) {
if (zkClient.exists(collectionPath, true)) {
ZkNodeProps cProps = ZkNodeProps.load(zkClient.getData(collectionPath, null, null, true));
if (stateManager.hasData(collectionPath)) {
VersionedData data = stateManager.getData(collectionPath);
ZkNodeProps cProps = ZkNodeProps.load(data.getData());
if (cProps.containsKey(ZkController.CONFIGNAME_PROP)) {
break;
}
}
try {
configNames = zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE, null,
true);
} catch (NoNodeException e) {
configNames = stateManager.listData(ZkConfigManager.CONFIGS_ZKNODE);
} catch (NoSuchElementException | NoNodeException e) {
// just keep trying
}
@ -507,15 +530,4 @@ public class CreateCollectionCmd implements Cmd {
"Could not find configName for collection " + collection + " found:" + configNames);
}
}
public static boolean usePolicyFramework(ZkStateReader zkStateReader, ZkNodeProps message) {
Map autoScalingJson = Collections.emptyMap();
try {
autoScalingJson = Utils.getJson(zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
} catch (Exception e) {
return false;
}
return autoScalingJson.get(Policy.CLUSTER_POLICY) != null || message.getStr(Policy.POLICY) != null;
}
}

View File

@ -25,11 +25,12 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.common.SolrCloseableLatch;
import org.apache.solr.common.SolrException;
@ -76,59 +77,21 @@ public class CreateShardCmd implements Cmd {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'collection' and 'shard' are required parameters");
DocCollection collection = clusterState.getCollection(collectionName);
int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
int numTlogReplicas = message.getInt(TLOG_REPLICAS, collection.getInt(TLOG_REPLICAS, 0));
int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
if (numNrtReplicas + numTlogReplicas <= 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
}
Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
ZkStateReader zkStateReader = ocmh.zkStateReader;
PolicyHelper.SessionWrapper sessionWrapper = null;
boolean usePolicyFramework = usePolicyFramework(collection,ocmh);
List<ReplicaPosition> positions = null;
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
SolrCloseableLatch countDownLatch;
try {
if (usePolicyFramework) {
if (collection.getPolicyName() != null) message.getProperties().put(Policy.POLICY, collection.getPolicyName());
positions = Assign.identifyNodes(ocmh,
clusterState,
Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM),
collectionName,
message,
Collections.singletonList(sliceName),
numNrtReplicas,
numTlogReplicas,
numPullReplicas);
sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
} else {
List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas,
createNodeSetStr, ocmh.overseer.getSolrCloudManager());
int i = 0;
positions = new ArrayList<>();
for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
Replica.Type.TLOG, numTlogReplicas,
Replica.Type.PULL, numPullReplicas
).entrySet()) {
for (int j = 0; j < e.getValue(); j++) {
positions.add(new ReplicaPosition(sliceName, j + 1, e.getKey(), sortedNodeList.get(i % sortedNodeList.size()).nodeName));
i++;
}
}
}
List<ReplicaPosition> positions = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, sessionWrapper);
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
// wait for a while until we see the shard
ocmh.waitForNewShard(collectionName, sliceName);
String async = message.getStr(ASYNC);
countDownLatch = new SolrCloseableLatch(totalReplicas, ocmh);
countDownLatch = new SolrCloseableLatch(positions.size(), ocmh);
for (ReplicaPosition position : positions) {
String nodeName = position.node;
String coreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), collection, sliceName, position.type);
String coreName = Assign.buildSolrCoreName(ocmh.cloudManager.getDistribStateManager(), collection, sliceName, position.type);
log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName
+ " on " + nodeName);
@ -166,7 +129,7 @@ public class CreateShardCmd implements Cmd {
});
}
} finally {
if(sessionWrapper != null) sessionWrapper.release();
if (sessionWrapper.get() != null) sessionWrapper.get().release();
}
log.debug("Waiting for create shard action to complete");
@ -177,9 +140,52 @@ public class CreateShardCmd implements Cmd {
}
static boolean usePolicyFramework(DocCollection collection, OverseerCollectionMessageHandler ocmh)
throws IOException, InterruptedException {
AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
return !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty() || collection.getPolicyName() != null;
public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
String collectionName, ZkNodeProps message, AtomicReference< PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
String sliceName = message.getStr(SHARD_ID_PROP);
DocCollection collection = clusterState.getCollection(collectionName);
int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
int numTlogReplicas = message.getInt(TLOG_REPLICAS, collection.getInt(TLOG_REPLICAS, 0));
int totalReplicas = numNrtReplicas + numPullReplicas + numTlogReplicas;
if (numNrtReplicas + numTlogReplicas <= 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
}
Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
boolean usePolicyFramework = CloudUtil.usePolicyFramework(collection, cloudManager);
List<ReplicaPosition> positions;
if (usePolicyFramework) {
if (collection.getPolicyName() != null) message.getProperties().put(Policy.POLICY, collection.getPolicyName());
positions = Assign.identifyNodes(cloudManager,
clusterState,
Assign.getLiveOrLiveAndCreateNodeSetList(clusterState.getLiveNodes(), message, RANDOM),
collection.getName(),
message,
Collections.singletonList(sliceName),
numNrtReplicas,
numTlogReplicas,
numPullReplicas);
sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
} else {
List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collection.getName(), sliceName, totalReplicas,
createNodeSetStr, cloudManager);
int i = 0;
positions = new ArrayList<>();
for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
Replica.Type.TLOG, numTlogReplicas,
Replica.Type.PULL, numPullReplicas
).entrySet()) {
for (int j = 0; j < e.getValue(); j++) {
positions.add(new ReplicaPosition(sliceName, j + 1, e.getKey(), sortedNodeList.get(i % sortedNodeList.size()).nodeName));
i++;
}
}
}
return positions;
}
}

View File

@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.snapshots.SolrSnapshotManager;
import org.apache.solr.util.TimeOut;
@ -49,9 +50,11 @@ import static org.apache.solr.common.params.CommonParams.NAME;
public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final TimeSource timeSource;
public DeleteCollectionCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.timeSource = ocmh.cloudManager.getTimeSource();
}
@Override
@ -94,13 +97,13 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
// wait for a while until we don't see the collection
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
boolean removed = false;
while (! timeout.hasTimedOut()) {
Thread.sleep(100);
timeout.sleep(100);
removed = !zkStateReader.getClusterState().hasCollection(collection);
if (removed) {
Thread.sleep(500); // just a bit of time so it's more likely other
timeout.sleep(500); // just a bit of time so it's more likely other
// readers see on return
break;
}

View File

@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
@ -55,9 +56,11 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
public class DeleteShardCmd implements Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final TimeSource timeSource;
public DeleteShardCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.timeSource = ocmh.cloudManager.getTimeSource();
}
@Override
@ -134,14 +137,14 @@ public class DeleteShardCmd implements Cmd {
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
// wait for a while until we don't see the shard
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
boolean removed = false;
while (!timeout.hasTimedOut()) {
Thread.sleep(100);
timeout.sleep(100);
DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
removed = collection.getSlice(sliceId) == null;
if (removed) {
Thread.sleep(100); // just a bit of time so it's more likely other readers see on return
timeout.sleep(100); // just a bit of time so it's more likely other readers see on return
break;
}
}

View File

@ -38,6 +38,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardHandlerFactory;
@ -63,9 +64,11 @@ import static org.apache.solr.common.util.Utils.makeMap;
public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final TimeSource timeSource;
public MigrateCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.timeSource = ocmh.cloudManager.getTimeSource();
}
@ -179,10 +182,10 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
// wait for a while until we see the new rule
log.info("Waiting to see routing rule updated in clusterstate");
TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS);
TimeOut waitUntil = new TimeOut(60, TimeUnit.SECONDS, timeSource);
boolean added = false;
while (!waitUntil.hasTimedOut()) {
Thread.sleep(100);
waitUntil.sleep(100);
sourceCollection = zkStateReader.getClusterState().getCollection(sourceCollection.getName());
sourceSlice = sourceCollection.getSlice(sourceSlice.getName());
Map<String, RoutingRule> rules = sourceSlice.getRoutingRules();
@ -257,7 +260,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
tempSourceCollectionName, targetLeader.getNodeName());
String tempCollectionReplica2 = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
String tempCollectionReplica2 = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
props = new HashMap<>();
props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());

View File

@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.util.TimeOut;
@ -54,9 +55,11 @@ public class MoveReplicaCmd implements Cmd{
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
private final TimeSource timeSource;
public MoveReplicaCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
this.timeSource = ocmh.cloudManager.getTimeSource();
}
@Override
@ -158,11 +161,11 @@ public class MoveReplicaCmd implements Cmd{
return;
}
TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(20L, TimeUnit.SECONDS, timeSource);
while (!timeOut.hasTimedOut()) {
coll = ocmh.zkStateReader.getClusterState().getCollection(coll.getName());
if (coll.getReplica(replica.getName()) != null) {
Thread.sleep(100);
timeOut.sleep(100);
} else {
break;
}
@ -233,7 +236,7 @@ public class MoveReplicaCmd implements Cmd{
private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async,
DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
String newCoreName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
String newCoreName = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
ZkNodeProps addReplicasProps = new ZkNodeProps(
COLLECTION_PROP, coll.getName(),
SHARD_ID_PROP, slice.getName(),

View File

@ -80,7 +80,7 @@ public class Overseer implements SolrCloseable {
enum LeaderStatus {DONT_KNOW, NO, YES}
private class ClusterStateUpdater implements Runnable, Closeable {
private final ZkStateReader reader;
private final SolrZkClient zkClient;
private final String myId;
@ -88,7 +88,7 @@ public class Overseer implements SolrCloseable {
private final ZkDistributedQueue stateUpdateQueue;
//TODO remove in 9.0, we do not push message into this queue anymore
//Internal queue where overseer stores events that have not yet been published into cloudstate
//If Overseer dies while extracting the main queue a new overseer will start from this queue
//If Overseer dies while extracting the main queue a new overseer will start from this queue
private final ZkDistributedQueue workQueue;
// Internal map which holds the information about running tasks.
private final DistributedMap runningMap;
@ -120,7 +120,7 @@ public class Overseer implements SolrCloseable {
public Stats getWorkQueueStats() {
return workQueue.getZkStats();
}
@Override
public void run() {
@ -442,7 +442,7 @@ public class Overseer implements SolrCloseable {
}
static class OverseerThread extends Thread implements Closeable {
public static class OverseerThread extends Thread implements Closeable {
protected volatile boolean isClosed;
private Closeable thread;
@ -466,9 +466,9 @@ public class Overseer implements SolrCloseable {
public boolean isClosed() {
return this.isClosed;
}
}
private OverseerThread ccThread;
private OverseerThread updaterThread;
@ -478,7 +478,7 @@ public class Overseer implements SolrCloseable {
private final ZkStateReader reader;
private final ShardHandler shardHandler;
private final UpdateShardHandler updateShardHandler;
private final String adminPath;
@ -505,7 +505,7 @@ public class Overseer implements SolrCloseable {
this.stats = new Stats();
this.config = config;
}
public synchronized void start(String id) {
this.id = id;
closed = false;

View File

@ -36,6 +36,10 @@ import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
@ -65,6 +69,7 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardHandlerFactory;
@ -73,6 +78,7 @@ import org.apache.solr.handler.component.ShardResponse;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -143,8 +149,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
ShardHandlerFactory shardHandlerFactory;
String adminPath;
ZkStateReader zkStateReader;
SolrCloudManager cloudManager;
String myId;
Stats stats;
TimeSource timeSource;
// Set that tracks collections that are currently being processed by a running task.
// This is used for handling mutual exclusion of the tasks.
@ -182,6 +190,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
this.myId = myId;
this.stats = stats;
this.overseer = overseer;
this.cloudManager = overseer.getSolrCloudManager();
this.timeSource = cloudManager.getTimeSource();
this.isClosed = false;
commandMap = new ImmutableMap.Builder<CollectionAction, Cmd>()
.put(REPLACENODE, new ReplaceNodeCmd(this))
@ -229,7 +239,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
CollectionAction action = getCollectionAction(operation);
Cmd command = commandMap.get(action);
if (command != null) {
command.call(zkStateReader.getClusterState(), message, results);
command.call(cloudManager.getClusterStateProvider().getClusterState(), message, results);
} else {
throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
+ operation);
@ -423,9 +433,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS);
TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS, timeSource);
while (! timeout.hasTimedOut()) {
Thread.sleep(100);
timeout.sleep(100);
DocCollection docCollection = zkStateReader.getClusterState().getCollection(collectionName);
if (docCollection == null) { // someone already deleted the collection
return true;
@ -465,7 +475,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
boolean firstLoop = true;
// wait for a while until the state format changes
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
while (! timeout.hasTimedOut()) {
DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
if (collection == null) {
@ -483,7 +493,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, MIGRATESTATEFORMAT.toLower(), COLLECTION_PROP, collectionName);
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
}
Thread.sleep(100);
timeout.sleep(100);
}
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not migrate state format for collection: " + collectionName);
}
@ -642,16 +652,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
validateConfigOrThrowSolrException(configName);
boolean isLegacyCloud = Overseer.isLegacy(zkStateReader);
createConfNode(configName, collectionName, isLegacyCloud);
createConfNode(cloudManager.getDistribStateManager(), configName, collectionName, isLegacyCloud);
reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
}
overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(message));
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
boolean areChangesVisible = true;
while (!timeout.hasTimedOut()) {
DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
DocCollection collection = cloudManager.getClusterStateProvider().getClusterState().getCollection(collectionName);
areChangesVisible = true;
for (Map.Entry<String,Object> updateEntry : message.getProperties().entrySet()) {
String updateKey = updateEntry.getKey();
@ -663,7 +673,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
}
if (areChangesVisible) break;
Thread.sleep(100);
timeout.sleep(100);
}
if (!areChangesVisible)
@ -680,7 +690,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
Map<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreNames) throws InterruptedException {
Map<String, Replica> result = new HashMap<>();
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, timeSource);
while (true) {
DocCollection coll = zkStateReader.getClusterState().getCollection(collectionName);
for (String coreName : coreNames) {
@ -746,8 +756,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
void validateConfigOrThrowSolrException(String configName) throws KeeperException, InterruptedException {
boolean isValid = zkStateReader.getZkClient().exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true);
void validateConfigOrThrowSolrException(String configName) throws IOException, KeeperException, InterruptedException {
boolean isValid = cloudManager.getDistribStateManager().hasData(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName);
if(!isValid) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Can not find the specified config set: " + configName);
}
@ -757,16 +767,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
* This doesn't validate the config (path) itself and is just responsible for creating the confNode.
* That check should be done before the config node is created.
*/
void createConfNode(String configName, String coll, boolean isLegacyCloud) throws KeeperException, InterruptedException {
public static void createConfNode(DistribStateManager stateManager, String configName, String coll, boolean isLegacyCloud) throws IOException, AlreadyExistsException, BadVersionException, KeeperException, InterruptedException {
if (configName != null) {
String collDir = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll;
log.debug("creating collections conf node {} ", collDir);
byte[] data = Utils.toJSON(makeMap(ZkController.CONFIGNAME_PROP, configName));
if (zkStateReader.getZkClient().exists(collDir, true)) {
zkStateReader.getZkClient().setData(collDir, data, true);
if (stateManager.hasData(collDir)) {
stateManager.setData(collDir, data, -1);
} else {
zkStateReader.getZkClient().makePath(collDir, data, true);
stateManager.makePath(collDir, data, CreateMode.PERSISTENT, false);
}
} else {
if(isLegacyCloud){
@ -775,7 +785,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
throw new SolrException(ErrorCode.BAD_REQUEST,"Unable to get config name");
}
}
}
private void collectionCmd(ZkNodeProps message, ModifiableSolrParams params,

View File

@ -223,7 +223,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
try {
List<ReplicaPosition> replicaPositions = Assign.identifyNodes(
ocmh, clusterState,
ocmh.cloudManager, clusterState,
nodeList, restoreCollectionName,
message, sliceNames,
numNrtReplicas, numTlogReplicas, numPullReplicas);

View File

@ -26,9 +26,11 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.cloud.overseer.OverseerAction;
@ -79,56 +81,26 @@ public class SplitShardCmd implements Cmd {
}
public boolean split(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
String collectionName = message.getStr("collection");
String slice = message.getStr(ZkStateReader.SHARD_ID_PROP);
boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
String collectionName = message.getStr(CoreAdminParams.COLLECTION);
log.info("Split shard invoked");
ZkStateReader zkStateReader = ocmh.zkStateReader;
zkStateReader.forceUpdateCollection(collectionName);
AtomicReference<String> slice = new AtomicReference<>();
slice.set(message.getStr(ZkStateReader.SHARD_ID_PROP));
String splitKey = message.getStr("split.key");
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
DocCollection collection = clusterState.getCollection(collectionName);
DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
PolicyHelper.SessionWrapper sessionWrapper = null;
Slice parentSlice;
if (slice == null) {
if (router instanceof CompositeIdRouter) {
Collection<Slice> searchSlices = router.getSearchSlicesSingle(splitKey, new ModifiableSolrParams(), collection);
if (searchSlices.isEmpty()) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to find an active shard for split.key: " + splitKey);
}
if (searchSlices.size() > 1) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Splitting a split.key: " + splitKey + " which spans multiple shards is not supported");
}
parentSlice = searchSlices.iterator().next();
slice = parentSlice.getName();
log.info("Split by route.key: {}, parent shard is: {} ", splitKey, slice);
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Split by route key can only be used with CompositeIdRouter or subclass. Found router: "
+ router.getClass().getName());
}
} else {
parentSlice = collection.getSlice(slice);
}
if (parentSlice == null) {
// no chance of the collection being null because ClusterState#getCollection(String) would have thrown
// an exception already
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
}
Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
// find the leader for the shard
Replica parentShardLeader = null;
try {
parentShardLeader = zkStateReader.getLeaderRetry(collectionName, slice, 10000);
parentShardLeader = zkStateReader.getLeaderRetry(collectionName, slice.get(), 10000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
@ -140,80 +112,13 @@ public class SplitShardCmd implements Cmd {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNodeName() + " is not live anymore!");
}
DocRouter.Range range = parentSlice.getRange();
if (range == null) {
range = new PlainIdRouter().fullRange();
}
List<DocRouter.Range> subRanges = new ArrayList<>();
List<String> subSlices = new ArrayList<>();
List<String> subShardNames = new ArrayList<>();
List<DocRouter.Range> subRanges = null;
String rangesStr = message.getStr(CoreAdminParams.RANGES);
if (rangesStr != null) {
String[] ranges = rangesStr.split(",");
if (ranges.length == 0 || ranges.length == 1) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least two ranges specified to split a shard");
} else {
subRanges = new ArrayList<>(ranges.length);
for (int i = 0; i < ranges.length; i++) {
String r = ranges[i];
try {
subRanges.add(DocRouter.DEFAULT.fromString(r));
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception in parsing hexadecimal hash range: " + r, e);
}
if (!subRanges.get(i).isSubsetOf(range)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Specified hash range: " + r + " is not a subset of parent shard's range: " + range.toString());
}
}
List<DocRouter.Range> temp = new ArrayList<>(subRanges); // copy to preserve original order
Collections.sort(temp);
if (!range.equals(new DocRouter.Range(temp.get(0).min, temp.get(temp.size() - 1).max))) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Specified hash ranges: " + rangesStr + " do not cover the entire range of parent shard: " + range);
}
for (int i = 1; i < temp.size(); i++) {
if (temp.get(i - 1).max + 1 != temp.get(i).min) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Specified hash ranges: " + rangesStr
+ " either overlap with each other or " + "do not cover the entire range of parent shard: " + range);
}
}
}
} else if (splitKey != null) {
if (router instanceof CompositeIdRouter) {
CompositeIdRouter compositeIdRouter = (CompositeIdRouter) router;
subRanges = compositeIdRouter.partitionRangeByKey(splitKey, range);
if (subRanges.size() == 1) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey
+ " has a hash range that is exactly equal to hash range of shard: " + slice);
}
for (DocRouter.Range subRange : subRanges) {
if (subRange.min == subRange.max) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey + " must be a compositeId");
}
}
log.info("Partitioning parent shard " + slice + " range: " + parentSlice.getRange() + " yields: " + subRanges);
rangesStr = "";
for (int i = 0; i < subRanges.size(); i++) {
DocRouter.Range subRange = subRanges.get(i);
rangesStr += subRange.toString();
if (i < subRanges.size() - 1) rangesStr += ',';
}
}
} else {
// todo: fixed to two partitions?
subRanges = router.partitionRange(2, range);
}
String rangesStr = fillRanges(ocmh.cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames);
try {
List<String> subSlices = new ArrayList<>(subRanges.size());
List<String> subShardNames = new ArrayList<>(subRanges.size());
String nodeName = parentShardLeader.getNodeName();
for (int i = 0; i < subRanges.size(); i++) {
String subSlice = slice + "_" + i;
subSlices.add(subSlice);
String subShardName = Assign.buildCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), collection, subSlice, Replica.Type.NRT);
subShardNames.add(subShardName);
}
boolean oldShardsDeleted = false;
for (String subSlice : subSlices) {
@ -252,6 +157,7 @@ public class SplitShardCmd implements Cmd {
final String asyncId = message.getStr(ASYNC);
Map<String, String> requestMap = new HashMap<>();
String nodeName = parentShardLeader.getNodeName();
for (int i = 0; i < subRanges.size(); i++) {
String subSlice = subSlices.get(i);
@ -300,6 +206,8 @@ public class SplitShardCmd implements Cmd {
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), results, null);
}
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ocmh.processResponses(results, shardHandler, true, "SPLITSHARD failed to create subshard leaders", asyncId, requestMap);
for (String subShardName : subShardNames) {
@ -388,7 +296,7 @@ public class SplitShardCmd implements Cmd {
// TODO: change this to handle sharding a slice into > 2 sub-shards.
List<ReplicaPosition> replicaPositions = Assign.identifyNodes(ocmh,
List<ReplicaPosition> replicaPositions = Assign.identifyNodes(ocmh.cloudManager,
clusterState,
new ArrayList<>(clusterState.getLiveNodes()),
collectionName,
@ -401,15 +309,15 @@ public class SplitShardCmd implements Cmd {
for (ReplicaPosition replicaPosition : replicaPositions) {
String sliceName = replicaPosition.shard;
String subShardNodeName = replicaPosition.node;
String shardName = collectionName + "_" + sliceName + "_replica" + (replicaPosition.index);
String solrCoreName = collectionName + "_" + sliceName + "_replica" + (replicaPosition.index);
log.info("Creating replica shard " + shardName + " as part of slice " + sliceName + " of collection "
log.info("Creating replica shard " + solrCoreName + " as part of slice " + sliceName + " of collection "
+ collectionName + " on " + subShardNodeName);
ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
ZkStateReader.COLLECTION_PROP, collectionName,
ZkStateReader.SHARD_ID_PROP, sliceName,
ZkStateReader.CORE_NAME_PROP, shardName,
ZkStateReader.CORE_NAME_PROP, solrCoreName,
ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
ZkStateReader.NODE_NAME_PROP, subShardNodeName,
@ -421,7 +329,7 @@ public class SplitShardCmd implements Cmd {
propMap.put(COLLECTION_PROP, collectionName);
propMap.put(SHARD_ID_PROP, sliceName);
propMap.put("node", subShardNodeName);
propMap.put(CoreAdminParams.NAME, shardName);
propMap.put(CoreAdminParams.NAME, solrCoreName);
// copy over property params:
for (String key : message.keySet()) {
if (key.startsWith(COLL_PROP_PREFIX)) {
@ -478,7 +386,7 @@ public class SplitShardCmd implements Cmd {
DistributedQueue inQueue = Overseer.getStateUpdateQueue(zkStateReader.getZkClient());
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
propMap.put(slice, Slice.State.INACTIVE.toString());
propMap.put(slice.get(), Slice.State.INACTIVE.toString());
for (String subSlice : subSlices) {
propMap.put(subSlice, Slice.State.ACTIVE.toString());
}
@ -507,7 +415,7 @@ public class SplitShardCmd implements Cmd {
log.info("Successfully created all replica shards for all sub-slices " + subSlices);
ocmh.commit(results, slice, parentShardLeader);
ocmh.commit(results, slice.get(), parentShardLeader);
return true;
} catch (SolrException e) {
@ -519,4 +427,116 @@ public class SplitShardCmd implements Cmd {
if (sessionWrapper != null) sessionWrapper.release();
}
}
public static Slice getParentSlice(ClusterState clusterState, String collectionName, AtomicReference<String> slice, String splitKey) {
DocCollection collection = clusterState.getCollection(collectionName);
DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
Slice parentSlice;
if (slice.get() == null) {
if (router instanceof CompositeIdRouter) {
Collection<Slice> searchSlices = router.getSearchSlicesSingle(splitKey, new ModifiableSolrParams(), collection);
if (searchSlices.isEmpty()) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to find an active shard for split.key: " + splitKey);
}
if (searchSlices.size() > 1) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Splitting a split.key: " + splitKey + " which spans multiple shards is not supported");
}
parentSlice = searchSlices.iterator().next();
slice.set(parentSlice.getName());
log.info("Split by route.key: {}, parent shard is: {} ", splitKey, slice);
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Split by route key can only be used with CompositeIdRouter or subclass. Found router: "
+ router.getClass().getName());
}
} else {
parentSlice = collection.getSlice(slice.get());
}
if (parentSlice == null) {
// no chance of the collection being null because ClusterState#getCollection(String) would have thrown
// an exception already
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
}
return parentSlice;
}
public static String fillRanges(SolrCloudManager cloudManager, ZkNodeProps message, DocCollection collection, Slice parentSlice,
List<DocRouter.Range> subRanges, List<String> subSlices, List<String> subShardNames) {
String splitKey = message.getStr("split.key");
DocRouter.Range range = parentSlice.getRange();
if (range == null) {
range = new PlainIdRouter().fullRange();
}
DocRouter router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
String rangesStr = message.getStr(CoreAdminParams.RANGES);
if (rangesStr != null) {
String[] ranges = rangesStr.split(",");
if (ranges.length == 0 || ranges.length == 1) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least two ranges specified to split a shard");
} else {
for (int i = 0; i < ranges.length; i++) {
String r = ranges[i];
try {
subRanges.add(DocRouter.DEFAULT.fromString(r));
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception in parsing hexadecimal hash range: " + r, e);
}
if (!subRanges.get(i).isSubsetOf(range)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Specified hash range: " + r + " is not a subset of parent shard's range: " + range.toString());
}
}
List<DocRouter.Range> temp = new ArrayList<>(subRanges); // copy to preserve original order
Collections.sort(temp);
if (!range.equals(new DocRouter.Range(temp.get(0).min, temp.get(temp.size() - 1).max))) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Specified hash ranges: " + rangesStr + " do not cover the entire range of parent shard: " + range);
}
for (int i = 1; i < temp.size(); i++) {
if (temp.get(i - 1).max + 1 != temp.get(i).min) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Specified hash ranges: " + rangesStr
+ " either overlap with each other or " + "do not cover the entire range of parent shard: " + range);
}
}
}
} else if (splitKey != null) {
if (router instanceof CompositeIdRouter) {
CompositeIdRouter compositeIdRouter = (CompositeIdRouter) router;
List<DocRouter.Range> tmpSubRanges = compositeIdRouter.partitionRangeByKey(splitKey, range);
if (tmpSubRanges.size() == 1) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey
+ " has a hash range that is exactly equal to hash range of shard: " + parentSlice.getName());
}
for (DocRouter.Range subRange : tmpSubRanges) {
if (subRange.min == subRange.max) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The split.key: " + splitKey + " must be a compositeId");
}
}
subRanges.addAll(tmpSubRanges);
log.info("Partitioning parent shard " + parentSlice.getName() + " range: " + parentSlice.getRange() + " yields: " + subRanges);
rangesStr = "";
for (int i = 0; i < subRanges.size(); i++) {
DocRouter.Range subRange = subRanges.get(i);
rangesStr += subRange.toString();
if (i < subRanges.size() - 1) rangesStr += ',';
}
}
} else {
// todo: fixed to two partitions?
subRanges.addAll(router.partitionRange(2, range));
}
for (int i = 0; i < subRanges.size(); i++) {
String subSlice = parentSlice.getName() + "_" + i;
subSlices.add(subSlice);
String subShardName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), collection, subSlice, Replica.Type.NRT);
subShardNames.add(subShardName);
}
return rangesStr;
}
}

View File

@ -133,13 +133,13 @@ public class AutoScaling {
*/
public static class TriggerFactoryImpl extends TriggerFactory {
private final SolrCloudManager dataProvider;
private final SolrCloudManager cloudManager;
private final SolrResourceLoader loader;
public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager dataProvider) {
Objects.requireNonNull(dataProvider);
public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager cloudManager) {
Objects.requireNonNull(cloudManager);
Objects.requireNonNull(loader);
this.dataProvider = dataProvider;
this.cloudManager = cloudManager;
this.loader = loader;
}
@ -150,11 +150,11 @@ public class AutoScaling {
}
switch (type) {
case NODEADDED:
return new NodeAddedTrigger(name, props, loader, dataProvider);
return new NodeAddedTrigger(name, props, loader, cloudManager);
case NODELOST:
return new NodeLostTrigger(name, props, loader, dataProvider);
return new NodeLostTrigger(name, props, loader, cloudManager);
case SEARCHRATE:
return new SearchRateTrigger(name, props, loader, dataProvider);
return new SearchRateTrigger(name, props, loader, cloudManager);
default:
throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
}

View File

@ -36,26 +36,23 @@ import java.util.stream.Stream;
import org.apache.solr.api.Api;
import org.apache.solr.api.ApiBag;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
import org.apache.solr.cloud.ZkDistributedQueueFactory;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.util.CommandOperation;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.handler.RequestHandlerUtils;
import org.apache.solr.request.SolrQueryRequest;
@ -63,7 +60,7 @@ import org.apache.solr.request.SolrRequestHandler;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.security.AuthorizationContext;
import org.apache.solr.security.PermissionNameProvider;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -80,15 +77,18 @@ import static org.apache.solr.common.params.CommonParams.JSON;
public class AutoScalingHandler extends RequestHandlerBase implements PermissionNameProvider {
public static final String HANDLER_PATH = "/admin/autoscaling";
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected final CoreContainer container;
protected final SolrCloudManager cloudManager;
protected final SolrResourceLoader loader;
private final List<Map<String, String>> DEFAULT_ACTIONS = new ArrayList<>(3);
private static Set<String> singletonCommands = Stream.of("set-cluster-preferences", "set-cluster-policy")
.collect(collectingAndThen(toSet(), Collections::unmodifiableSet));
private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
private final TimeSource timeSource;
public AutoScalingHandler(CoreContainer container) {
this.container = container;
public AutoScalingHandler(SolrCloudManager cloudManager, SolrResourceLoader loader) {
this.cloudManager = cloudManager;
this.loader = loader;
this.timeSource = cloudManager.getTimeSource();
Map<String, String> map = new HashMap<>(2);
map.put(NAME, "compute_plan");
map.put(CLASS, "solr.ComputePlanAction");
@ -116,7 +116,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
}
AutoScalingConfig autoScalingConf = container.getZkController().zkStateReader.getAutoScalingConfig();
AutoScalingConfig autoScalingConf = cloudManager.getDistribStateManager().getAutoScalingConfig();
if (parts.size() == 2) {
autoScalingConf.writeMap(new MapWriter.EntryWriter() {
@ -154,21 +154,14 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
private void handleSuggestions(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) throws IOException {
try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) {
DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
rsp.getValues().add("suggestions",
PolicyHelper.getSuggestions(autoScalingConf, new SolrClientCloudManager(queueFactory, build)));
}
rsp.getValues().add("suggestions",
PolicyHelper.getSuggestions(autoScalingConf, cloudManager));
}
public void processOps(SolrQueryRequest req, SolrQueryResponse rsp, List<CommandOperation> ops)
throws KeeperException, InterruptedException, IOException {
while (true) {
AutoScalingConfig initialConfig = container.getZkController().zkStateReader.getAutoScalingConfig();
AutoScalingConfig initialConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
AutoScalingConfig currentConfig = initialConfig;
for (CommandOperation op : ops) {
switch (op.name) {
@ -216,7 +209,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
if (!currentConfig.equals(initialConfig)) {
// update in ZK
if (zkSetAutoScalingConfig(container.getZkController().getZkStateReader(), currentConfig)) {
if (setAutoScalingConfig(currentConfig)) {
break;
} else {
// someone else updated the config, get the latest one and re-apply our ops
@ -244,12 +237,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
private void handleDiagnostics(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) throws IOException {
Policy policy = autoScalingConf.getPolicy();
try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) {
DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, new SolrClientCloudManager(queueFactory, build)));
}
rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, cloudManager));
}
private AutoScalingConfig handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
@ -302,7 +290,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
return currentConfig;
}
container.getZkController().getZkStateReader().getClusterState().forEachCollection(coll -> {
cloudManager.getClusterStateProvider().getClusterState().forEachCollection(coll -> {
if (policyName.equals(coll.getPolicyName()))
op.addError(StrUtils.formatString("policy : {0} is being used by collection {1}", policyName, coll.getName()));
});
@ -470,7 +458,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
// validate that we can load the listener class
// todo allow creation from blobstore
try {
container.getResourceLoader().findClass(listenerClass, TriggerListener.class);
loader.findClass(listenerClass, TriggerListener.class);
} catch (Exception e) {
log.warn("error loading listener class ", e);
op.addError("Listener not found: " + listenerClass + ". error message:" + e.getMessage());
@ -535,7 +523,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
}
String klass = action.get(CLASS);
try {
container.getResourceLoader().findClass(klass, TriggerAction.class);
loader.findClass(klass, TriggerAction.class);
} catch (Exception e) {
log.warn("Could not load class : ", e);
op.addError("Action not found: " + klass + " " + e.getMessage());
@ -632,26 +620,22 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
}
private boolean zkSetAutoScalingConfig(ZkStateReader reader, AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
private boolean setAutoScalingConfig(AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
verifyAutoScalingConf(currentConfig);
try {
reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion(), true);
} catch (KeeperException.BadVersionException bve) {
cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion());
} catch (BadVersionException bve) {
// somebody else has changed the configuration so we must retry
return false;
}
//log.debug("-- saved version " + currentConfig.getZkVersion() + ": " + currentConfig);
return true;
}
private void verifyAutoScalingConf(AutoScalingConfig autoScalingConf) throws IOException {
try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) {
DistributedQueueFactory queueFactory = new ZkDistributedQueueFactory(container.getZkController().getZkClient());
Policy.Session session = autoScalingConf.getPolicy()
.createSession(new SolrClientCloudManager(queueFactory, build));
log.debug("Verified autoscaling configuration");
}
Policy.Session session = autoScalingConf.getPolicy()
.createSession(cloudManager);
log.debug("Verified autoscaling configuration");
}
@Override

View File

@ -33,8 +33,10 @@ import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.Pair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -57,12 +59,13 @@ public class ComputePlanAction extends TriggerActionBase {
if (autoScalingConf.isEmpty()) {
throw new Exception("Action: " + getName() + " executed but no policy is configured");
}
// Policy.Session session = cloudManager.getDistribStateManager().getAutoScalingConfig().getPolicy().createSession(cloudManager);
// return new PolicyHelper.SessionWrapper(session, null);
PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getSession(cloudManager);
Policy.Session session = sessionWrapper.get();
// Policy policy = autoScalingConf.getPolicy();
if (log.isTraceEnabled()) {
ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
log.trace("-- session: {}", session);
log.trace("-- state: {}", state);
}
try {
Suggester suggester = getSuggester(session, event, cloudManager);
while (true) {
@ -99,12 +102,10 @@ public class ComputePlanAction extends TriggerActionBase {
case NODEADDED:
suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
.hint(Suggester.Hint.TARGET_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
log.debug("NODEADDED Created suggester with targetNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
break;
case NODELOST:
suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
.hint(Suggester.Hint.SRC_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
log.debug("NODELOST Created suggester with srcNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
break;
case SEARCHRATE:
Map<String, Map<String, Double>> hotShards = (Map<String, Map<String, Double>>)event.getProperty(AutoScalingParams.SHARD);
@ -125,13 +126,10 @@ public class ComputePlanAction extends TriggerActionBase {
} else {
// collection || shard || replica -> ADDREPLICA
suggester = session.getSuggester(CollectionParams.CollectionAction.ADDREPLICA);
Set<String> collections = new HashSet<>();
// XXX improve this when AddReplicaSuggester supports coll_shard hint
hotReplicas.forEach(r -> collections.add(r.getCollection()));
hotShards.forEach((coll, shards) -> collections.add(coll));
hotCollections.forEach((coll, rate) -> collections.add(coll));
for (String coll : collections) {
suggester = suggester.hint(Suggester.Hint.COLL, coll);
Set<Pair> collectionShards = new HashSet<>();
hotShards.forEach((coll, shards) -> shards.forEach((s, r) -> collectionShards.add(new Pair(coll, s))));
for (Pair<String, String> colShard : collectionShards) {
suggester = suggester.hint(Suggester.Hint.COLL_SHARD, colShard);
}
}
break;

View File

@ -56,7 +56,7 @@ public class ExecutePlanAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
SolrCloudManager dataProvider = context.getCloudManager();
SolrCloudManager cloudManager = context.getCloudManager();
List<SolrRequest> operations = (List<SolrRequest>) context.getProperty("operations");
if (operations == null || operations.isEmpty()) {
log.info("No operations to execute for event: {}", event);
@ -64,7 +64,7 @@ public class ExecutePlanAction extends TriggerActionBase {
}
try {
for (SolrRequest operation : operations) {
log.info("Executing operation: {}", operation.getParams());
log.debug("Executing operation: {}", operation.getParams());
try {
SolrResponse response = null;
int counter = 0;
@ -73,22 +73,22 @@ public class ExecutePlanAction extends TriggerActionBase {
// waitForFinalState so that the end effects of operations are visible
req.setWaitForFinalState(true);
String asyncId = event.getSource() + '/' + event.getId() + '/' + counter;
String znode = saveAsyncId(dataProvider.getDistribStateManager(), event, asyncId);
String znode = saveAsyncId(cloudManager.getDistribStateManager(), event, asyncId);
log.debug("Saved requestId: {} in znode: {}", asyncId, znode);
// TODO: find a better way of using async calls using dataProvider API !!!
req.setAsyncId(asyncId);
SolrResponse asyncResponse = dataProvider.request(req);
SolrResponse asyncResponse = cloudManager.request(req);
if (asyncResponse.getResponse().get("error") != null) {
throw new IOException("" + asyncResponse.getResponse().get("error"));
}
asyncId = (String)asyncResponse.getResponse().get("requestid");
CollectionAdminRequest.RequestStatusResponse statusResponse = waitForTaskToFinish(dataProvider, asyncId,
CollectionAdminRequest.RequestStatusResponse statusResponse = waitForTaskToFinish(cloudManager, asyncId,
DEFAULT_TASK_TIMEOUT_SECONDS, TimeUnit.SECONDS);
if (statusResponse != null) {
RequestStatusState state = statusResponse.getRequestStatus();
if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED || state == RequestStatusState.NOT_FOUND) {
try {
dataProvider.getDistribStateManager().removeData(znode, -1);
cloudManager.getDistribStateManager().removeData(znode, -1);
} catch (Exception e) {
log.warn("Unexpected exception while trying to delete znode: " + znode, e);
}
@ -96,7 +96,7 @@ public class ExecutePlanAction extends TriggerActionBase {
response = statusResponse;
}
} else {
response = dataProvider.request(operation);
response = cloudManager.request(operation);
}
NamedList<Object> result = response.getResponse();
context.getProperties().compute("responses", (s, o) -> {
@ -106,16 +106,15 @@ public class ExecutePlanAction extends TriggerActionBase {
return responses;
});
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to talk to ZooKeeper", e);
// } catch (InterruptedException e) {
// Thread.currentThread().interrupt();
// throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected exception executing operation: " + operation.getParams(), e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected exception executing operation: " + operation.getParams(), e);
}
// counter++;
}
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@ -124,22 +123,22 @@ public class ExecutePlanAction extends TriggerActionBase {
}
static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager dataProvider, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager cloudManager, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
long timeoutSeconds = timeUnit.toSeconds(duration);
RequestStatusState state = RequestStatusState.NOT_FOUND;
CollectionAdminRequest.RequestStatusResponse statusResponse = null;
for (int i = 0; i < timeoutSeconds; i++) {
try {
statusResponse = (CollectionAdminRequest.RequestStatusResponse)dataProvider.request(CollectionAdminRequest.requestStatus(requestId));
statusResponse = (CollectionAdminRequest.RequestStatusResponse)cloudManager.request(CollectionAdminRequest.requestStatus(requestId));
state = statusResponse.getRequestStatus();
if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
log.info("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
dataProvider.request(CollectionAdminRequest.deleteAsyncId(requestId));
log.debug("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
return statusResponse;
} else if (state == RequestStatusState.NOT_FOUND) {
// the request for this id was never actually submitted! no harm done, just bail out
log.warn("Task with requestId={} was not found on overseer", requestId);
dataProvider.request(CollectionAdminRequest.deleteAsyncId(requestId));
cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
return statusResponse;
}
} catch (Exception e) {
@ -154,11 +153,12 @@ public class ExecutePlanAction extends TriggerActionBase {
throw e;
}
log.error("Unexpected Exception while querying status of requestId=" + requestId, e);
throw e;
}
if (i > 0 && i % 5 == 0) {
log.debug("Task with requestId={} still not complete after {}s. Last state={}", requestId, i * 5, state);
}
TimeUnit.SECONDS.sleep(5);
cloudManager.getTimeSource().sleep(5000);
}
log.debug("Task with requestId={} did not complete within 5 minutes. Last state={}", requestId, state);
return statusResponse;

View File

@ -66,8 +66,8 @@ public class HttpTriggerListener extends TriggerListenerBase {
private boolean followRedirects;
@Override
public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
super.init(dataProvider, config);
public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
super.init(cloudManager, config);
urlTemplate = (String)config.properties.get("url");
payloadTemplate = (String)config.properties.get("payload");
contentType = (String)config.properties.get("contentType");
@ -148,7 +148,7 @@ public class HttpTriggerListener extends TriggerListenerBase {
});
headers.put("Content-Type", type);
try {
dataProvider.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
cloudManager.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
} catch (IOException e) {
LOG.warn("Exception sending request for event " + event, e);
}

View File

@ -35,7 +35,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.TimeSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -45,8 +44,6 @@ import org.slf4j.LoggerFactory;
public class NodeAddedTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final TimeSource timeSource;
private Set<String> lastLiveNodes;
private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>();
@ -55,7 +52,6 @@ public class NodeAddedTrigger extends TriggerBase {
SolrResourceLoader loader,
SolrCloudManager cloudManager) {
super(TriggerEventType.NODEADDED, name, properties, loader, cloudManager);
this.timeSource = TimeSource.CURRENT_TIME;
lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
log.debug("Initial livenodes: {}", lastLiveNodes);
log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
@ -71,7 +67,7 @@ public class NodeAddedTrigger extends TriggerBase {
// don't add nodes that have since gone away
if (lastLiveNodes.contains(n)) {
log.debug("Adding node from marker path: {}", n);
nodeNameVsTimeAdded.put(n, timeSource.getTime());
nodeNameVsTimeAdded.put(n, cloudManager.getTimeSource().getTime());
}
removeMarker(n);
});
@ -131,7 +127,7 @@ public class NodeAddedTrigger extends TriggerBase {
log.debug("Running NodeAddedTrigger {}", name);
Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
log.debug("Found livenodes: {}", newLiveNodes);
log.debug("Found livenodes: {}", newLiveNodes.size());
// have any nodes that we were tracking been removed from the cluster?
// if so, remove them from the tracking map
@ -142,7 +138,7 @@ public class NodeAddedTrigger extends TriggerBase {
Set<String> copyOfNew = new HashSet<>(newLiveNodes);
copyOfNew.removeAll(lastLiveNodes);
copyOfNew.forEach(n -> {
long eventTime = timeSource.getTime();
long eventTime = cloudManager.getTimeSource().getTime();
log.debug("Tracking new node: {} at time {}", n, eventTime);
nodeNameVsTimeAdded.put(n, eventTime);
});
@ -154,7 +150,7 @@ public class NodeAddedTrigger extends TriggerBase {
Map.Entry<String, Long> entry = it.next();
String nodeName = entry.getKey();
Long timeAdded = entry.getValue();
long now = timeSource.getTime();
long now = cloudManager.getTimeSource().getTime();
if (TimeUnit.SECONDS.convert(now - timeAdded, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
nodeNames.add(nodeName);
times.add(timeAdded);
@ -163,7 +159,8 @@ public class NodeAddedTrigger extends TriggerBase {
AutoScaling.TriggerEventProcessor processor = processorRef.get();
if (!nodeNames.isEmpty()) {
if (processor != null) {
log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name, nodeNames, times, timeSource.getTime());
log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name,
nodeNames, times, cloudManager.getTimeSource().getTime());
if (processor.process(new NodeAddedEvent(getEventType(), getName(), times, nodeNames))) {
// remove from tracking set only if the fire was accepted
nodeNames.forEach(n -> {

View File

@ -34,7 +34,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.TimeSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -44,8 +43,6 @@ import org.slf4j.LoggerFactory;
public class NodeLostTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final TimeSource timeSource;
private Set<String> lastLiveNodes;
private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>();
@ -54,7 +51,6 @@ public class NodeLostTrigger extends TriggerBase {
SolrResourceLoader loader,
SolrCloudManager dataProvider) {
super(TriggerEventType.NODELOST, name, properties, loader, dataProvider);
this.timeSource = TimeSource.CURRENT_TIME;
lastLiveNodes = new HashSet<>(dataProvider.getClusterStateProvider().getLiveNodes());
log.debug("Initial livenodes: {}", lastLiveNodes);
}
@ -69,7 +65,7 @@ public class NodeLostTrigger extends TriggerBase {
// don't add nodes that have since came back
if (!lastLiveNodes.contains(n)) {
log.debug("Adding lost node from marker path: {}", n);
nodeNameVsTimeRemoved.put(n, timeSource.getTime());
nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTime());
}
removeMarker(n);
});
@ -122,12 +118,12 @@ public class NodeLostTrigger extends TriggerBase {
synchronized (this) {
if (isClosed) {
log.warn("NodeLostTrigger ran but was already closed");
throw new RuntimeException("Trigger has been closed");
return;
}
}
Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes);
log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes.size());
// have any nodes that we were tracking been added to the cluster?
// if so, remove them from the tracking map
@ -139,7 +135,7 @@ public class NodeLostTrigger extends TriggerBase {
copyOfLastLiveNodes.removeAll(newLiveNodes);
copyOfLastLiveNodes.forEach(n -> {
log.debug("Tracking lost node: {}", n);
nodeNameVsTimeRemoved.put(n, timeSource.getTime());
nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTime());
});
// has enough time expired to trigger events for a node?
@ -149,7 +145,7 @@ public class NodeLostTrigger extends TriggerBase {
Map.Entry<String, Long> entry = it.next();
String nodeName = entry.getKey();
Long timeRemoved = entry.getValue();
long now = timeSource.getTime();
long now = cloudManager.getTimeSource().getTime();
if (TimeUnit.SECONDS.convert(now - timeRemoved, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
nodeNames.add(nodeName);
times.add(timeRemoved);

View File

@ -57,7 +57,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final SolrCloudManager dataProvider;
private final SolrCloudManager cloudManager;
private final CloudConfig cloudConfig;
@ -80,11 +80,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
private AutoScalingConfig autoScalingConfig;
public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager dataProvider, CloudConfig cloudConfig) {
this.dataProvider = dataProvider;
public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager cloudManager, CloudConfig cloudConfig) {
this.cloudManager = cloudManager;
this.cloudConfig = cloudConfig;
scheduledTriggers = new ScheduledTriggers(loader, dataProvider);
triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, dataProvider);
scheduledTriggers = new ScheduledTriggers(loader, cloudManager);
triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
}
@Override
@ -114,11 +114,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
// we automatically add a trigger for auto add replicas if it does not exists already
while (!isClosed) {
try {
AutoScalingConfig autoScalingConfig = dataProvider.getDistribStateManager().getAutoScalingConfig();
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
AutoScalingConfig withAutoAddReplicasTrigger = withAutoAddReplicasTrigger(autoScalingConfig);
if (withAutoAddReplicasTrigger.equals(autoScalingConfig)) break;
log.debug("Adding .autoAddReplicas trigger");
dataProvider.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(withAutoAddReplicasTrigger), withAutoAddReplicasTrigger.getZkVersion());
cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(withAutoAddReplicasTrigger), withAutoAddReplicasTrigger.getZkVersion());
break;
} catch (BadVersionException bve) {
// somebody else has changed the configuration so we must retry
@ -225,7 +225,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
throw new IllegalStateException("Caught AlreadyClosedException from ScheduledTriggers, but we're not closed yet!", e);
}
}
DistribStateManager stateManager = dataProvider.getDistribStateManager();
DistribStateManager stateManager = cloudManager.getDistribStateManager();
if (cleanOldNodeLostMarkers) {
log.debug("-- clean old nodeLost markers");
try {
@ -259,7 +259,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
private void removeNodeMarker(String path, String nodeName) {
path = path + "/" + nodeName;
try {
dataProvider.getDistribStateManager().removeData(path, -1);
cloudManager.getDistribStateManager().removeData(path, -1);
log.debug(" -- deleted " + path);
} catch (NoSuchElementException e) {
// ignore
@ -297,7 +297,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
if (isClosed) {
return;
}
AutoScalingConfig currentConfig = dataProvider.getDistribStateManager().getAutoScalingConfig(watcher);
AutoScalingConfig currentConfig = cloudManager.getDistribStateManager().getAutoScalingConfig(watcher);
log.debug("Refreshing {} with znode version {}", ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, currentConfig.getZkVersion());
if (znodeVersion >= currentConfig.getZkVersion()) {
// protect against reordered watcher fires by ensuring that we only move forward

View File

@ -38,12 +38,12 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
@ -78,13 +78,14 @@ import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_SCHEDULE_D
*/
public class ScheduledTriggers implements Closeable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
public static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
public static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
public static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
public static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
static final Map<String, Object> DEFAULT_PROPERTIES = new HashMap<>();
// Note: values must be all in milliseconds!
static {
DEFAULT_PROPERTIES.put(TRIGGER_SCHEDULE_DELAY_SECONDS, DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
DEFAULT_PROPERTIES.put(TRIGGER_COOLDOWN_PERIOD_SECONDS, DEFAULT_COOLDOWN_PERIOD_SECONDS);
@ -114,7 +115,7 @@ public class ScheduledTriggers implements Closeable {
private final AtomicLong cooldownPeriod = new AtomicLong(TimeUnit.SECONDS.toNanos(DEFAULT_COOLDOWN_PERIOD_SECONDS));
private final AtomicInteger triggerDelay = new AtomicInteger(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
private final AtomicLong triggerDelay = new AtomicLong(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
private final AtomicReference<ActionThrottle> actionThrottle;
@ -136,14 +137,14 @@ public class ScheduledTriggers implements Closeable {
scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor"));
actionThrottle = new AtomicReference<>(new ActionThrottle("action", TimeUnit.SECONDS.toMillis(DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS)));
actionThrottle = new AtomicReference<>(new ActionThrottle("action", TimeUnit.SECONDS.toMillis(DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS), cloudManager.getTimeSource()));
this.cloudManager = cloudManager;
this.stateManager = cloudManager.getDistribStateManager();
this.loader = loader;
queueStats = new Stats();
listeners = new TriggerListeners();
// initialize cooldown timer
cooldownStart.set(System.nanoTime() - cooldownPeriod.get());
cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
}
/**
@ -168,7 +169,9 @@ public class ScheduledTriggers implements Closeable {
scheduledTriggers.forEach((s, scheduledTrigger) -> {
if (scheduledTrigger.scheduledFuture.cancel(false)) {
scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(
scheduledTrigger, 0, triggerDelay.get(), TimeUnit.SECONDS);
scheduledTrigger, 0,
cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
TimeUnit.MILLISECONDS);
} else {
log.debug("Failed to cancel scheduled task: {}", s);
}
@ -188,9 +191,10 @@ public class ScheduledTriggers implements Closeable {
if (oldThrottle.getLastActionStartedAt() != null) {
newThrottle = new ActionThrottle("action",
minMsBetweenActions,
oldThrottle.getLastActionStartedAt());
oldThrottle.getLastActionStartedAt(),
cloudManager.getTimeSource());
} else {
newThrottle = new ActionThrottle("action", minMsBetweenActions);
newThrottle = new ActionThrottle("action", minMsBetweenActions, cloudManager.getTimeSource());
}
this.actionThrottle.set(newThrottle);
break;
@ -200,12 +204,17 @@ public class ScheduledTriggers implements Closeable {
this.autoScalingConfig = autoScalingConfig;
// reset cooldown and actionThrottle
cooldownStart.set(System.nanoTime() - cooldownPeriod.get());
cooldownStart.set(cloudManager.getTimeSource().getTime() - cooldownPeriod.get());
actionThrottle.get().reset();
listeners.setAutoScalingConfig(autoScalingConfig);
}
@VisibleForTesting
void resetActionThrottle() {
actionThrottle.get().reset();
}
/**
* Adds a new trigger or replaces an existing one. The replaced trigger, if any, is closed
* <b>before</b> the new trigger is run. If a trigger is replaced with itself then this
@ -261,7 +270,7 @@ public class ScheduledTriggers implements Closeable {
}
boolean replaying = event.getProperty(TriggerEvent.REPLAYING) != null ? (Boolean)event.getProperty(TriggerEvent.REPLAYING) : false;
AutoScaling.Trigger source = scheduledSource.trigger;
if (source.isClosed()) {
if (scheduledSource.isClosed || source.isClosed()) {
String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the source trigger: %s has already been closed", event.toString(), source);
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
log.warn(msg);
@ -269,7 +278,7 @@ public class ScheduledTriggers implements Closeable {
return false;
}
// reject events during cooldown period
if (cooldownStart.get() + cooldownPeriod.get() > System.nanoTime()) {
if (cooldownStart.get() + cooldownPeriod.get() > cloudManager.getTimeSource().getTime()) {
log.debug("-------- Cooldown period - rejecting event: " + event);
event.getProperties().put(TriggerEvent.COOLDOWN, true);
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "In cooldown period.");
@ -288,8 +297,16 @@ public class ScheduledTriggers implements Closeable {
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.STARTED);
List<TriggerAction> actions = source.getActions();
if (actions != null) {
if (actionExecutor.isShutdown()) {
String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the executor has already been closed", event.toString(), source);
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
log.warn(msg);
// we do not want to lose this event just because the trigger was closed, perhaps a replacement will need it
return false;
}
actionExecutor.submit(() -> {
assert hasPendingActions.get();
long eventProcessingStart = cloudManager.getTimeSource().getTime();
log.debug("-- processing actions for " + event);
try {
// let the action executor thread wait instead of the trigger thread so we use the throttle here
@ -324,9 +341,11 @@ public class ScheduledTriggers implements Closeable {
} catch (Exception e) {
log.warn("Exception executing actions", e);
} finally {
cooldownStart.set(System.nanoTime());
cooldownStart.set(cloudManager.getTimeSource().getTime());
hasPendingActions.set(false);
}
log.debug("-- processing took {} ms for event id={}",
TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getTime() - eventProcessingStart), event.id);
});
} else {
if (enqueued) {
@ -347,7 +366,9 @@ public class ScheduledTriggers implements Closeable {
}
});
newTrigger.init(); // mark as ready for scheduling
scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0, triggerDelay.get(), TimeUnit.SECONDS);
scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0,
cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
TimeUnit.MILLISECONDS);
}
private void waitForPendingTasks(AutoScaling.Trigger newTrigger, List<TriggerAction> actions) throws AlreadyClosedException {

View File

@ -37,7 +37,6 @@ import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.metrics.SolrCoreMetricManager;
import org.apache.solr.util.TimeSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -47,7 +46,6 @@ import org.slf4j.LoggerFactory;
public class SearchRateTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final TimeSource timeSource;
private final String handler;
private final String collection;
private final String shard;
@ -63,7 +61,6 @@ public class SearchRateTrigger extends TriggerBase {
SolrResourceLoader loader,
SolrCloudManager cloudManager) {
super(TriggerEventType.SEARCHRATE, name, properties, loader, cloudManager);
this.timeSource = TimeSource.CURRENT_TIME;
this.state.put("lastCollectionEvent", lastCollectionEvent);
this.state.put("lastNodeEvent", lastNodeEvent);
this.state.put("lastShardEvent", lastShardEvent);
@ -168,6 +165,9 @@ public class SearchRateTrigger extends TriggerBase {
});
});
});
if (metricTags.isEmpty()) {
continue;
}
Map<String, Object> rates = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
rates.forEach((tag, rate) -> {
ReplicaInfo info = metricTags.get(tag);
@ -184,7 +184,7 @@ public class SearchRateTrigger extends TriggerBase {
});
}
long now = timeSource.getTime();
long now = cloudManager.getTimeSource().getTime();
// check for exceeded rates and filter out those with less than waitFor from previous events
Map<String, Double> hotNodes = nodeRates.entrySet().stream()
.filter(entry -> node.equals(Policy.ANY) || node.equals(entry.getKey()))
@ -274,7 +274,7 @@ public class SearchRateTrigger extends TriggerBase {
private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
log.debug("name=" + name + ", lastTime=" + lastTime + ", elapsed=" + elapsed);
log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
return false;
}

View File

@ -72,8 +72,8 @@ public class SystemLogListener extends TriggerListenerBase {
private boolean enabled = true;
@Override
public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
super.init(dataProvider, config);
public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
super.init(cloudManager, config);
collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
}
@ -119,7 +119,7 @@ public class SystemLogListener extends TriggerListenerBase {
UpdateRequest req = new UpdateRequest();
req.add(doc);
req.setParam(CollectionAdminParams.COLLECTION, collection);
dataProvider.request(req);
cloudManager.request(req);
} catch (Exception e) {
if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
// relatively benign

View File

@ -27,7 +27,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -48,7 +48,7 @@ public class TriggerEventQueue {
// TODO: collect stats
this.delegate = cloudManager.getDistributedQueueFactory().makeQueue(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName);
this.triggerName = triggerName;
this.timeSource = TimeSource.CURRENT_TIME;
this.timeSource = cloudManager.getTimeSource();
}
public boolean offerEvent(TriggerEvent event) {

View File

@ -27,11 +27,11 @@ import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
public abstract class TriggerListenerBase implements TriggerListener {
protected AutoScalingConfig.TriggerListenerConfig config;
protected SolrCloudManager dataProvider;
protected SolrCloudManager cloudManager;
@Override
public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
this.dataProvider = dataProvider;
public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
this.cloudManager = cloudManager;
this.config = config;
}

View File

@ -43,12 +43,12 @@ import static org.apache.solr.common.params.CommonParams.NAME;
public class CollectionMutator {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected final SolrCloudManager dataProvider;
protected final SolrCloudManager cloudManager;
protected final DistribStateManager stateManager;
public CollectionMutator(SolrCloudManager dataProvider) {
this.dataProvider = dataProvider;
this.stateManager = dataProvider.getDistribStateManager();
public CollectionMutator(SolrCloudManager cloudManager) {
this.cloudManager = cloudManager;
this.stateManager = cloudManager.getDistribStateManager();
}
public ZkWriteCommand createShard(final ClusterState clusterState, ZkNodeProps message) {

View File

@ -245,7 +245,7 @@ public class ReplicaMutator {
log.debug("node=" + coreNodeName + " is already registered");
} else {
// if coreNodeName is null, auto assign one
coreNodeName = Assign.assignNode(stateManager, collection);
coreNodeName = Assign.assignCoreNodeName(stateManager, collection);
}
message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP,
coreNodeName);

View File

@ -73,7 +73,7 @@ public class SliceMutator {
if (message.getStr(ZkStateReader.CORE_NODE_NAME_PROP) != null) {
coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
} else {
coreNodeName = Assign.assignNode(stateManager, collection);
coreNodeName = Assign.assignCoreNodeName(stateManager, collection);
}
Replica replica = new Replica(coreNodeName,
makeMap(

View File

@ -540,8 +540,6 @@ public class CoreContainer {
// may want to add some configuration here in the future
metricsCollectorHandler.init(null);
autoScalingHandler = createHandler(AutoScalingHandler.HANDLER_PATH, AutoScalingHandler.class.getName(), AutoScalingHandler.class);
containerHandlers.put(AUTHZ_PATH, securityConfHandler);
securityConfHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AUTHZ_PATH);
containerHandlers.put(AUTHC_PATH, securityConfHandler);
@ -690,6 +688,10 @@ public class CoreContainer {
if (isZooKeeperAware()) {
zkSys.getZkController().checkOverseerDesignate();
// initialize this handler here when SolrCloudManager is ready
autoScalingHandler = new AutoScalingHandler(getZkController().getSolrCloudManager(), loader);
containerHandlers.put(AutoScalingHandler.HANDLER_PATH, autoScalingHandler);
autoScalingHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AutoScalingHandler.HANDLER_PATH);
}
// This is a bit redundant but these are two distinct concepts for all they're accomplished at the same time.
status |= LOAD_COMPLETE | INITIAL_CORE_LOAD_COMPLETE;

View File

@ -45,6 +45,7 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.CdcrUpdateLog;
import org.apache.solr.util.TimeOut;
@ -273,7 +274,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED) {
Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
}
TimeOut timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
if (closed) {
log.warn("Cancelling waiting for bootstrap on target: {} shard: {} to complete", targetCollection, shard);
@ -285,7 +286,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
try {
log.info("CDCR bootstrap running for {} seconds, sleeping for {} ms",
BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS), BOOTSTRAP_RETRY_DELAY_MS);
Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
@ -309,7 +310,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
while (!closed && sendBootstrapCommand() != BootstrapStatus.SUBMITTED) {
Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
}
timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS); // reset the timer
timeOut = new TimeOut(BOOTSTRAP_TIMEOUT_SECONDS, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
retries++;
}
} else if (status == BootstrapStatus.NOTFOUND || status == BootstrapStatus.CANCELLED) {
@ -321,13 +322,13 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
}
retries = 1;
timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS); // reset the timer
timeOut = new TimeOut(6L * 3600L * 3600L, TimeUnit.SECONDS, TimeSource.NANO_TIME); // reset the timer
} else if (status == BootstrapStatus.UNKNOWN || status == BootstrapStatus.SUBMITTED) {
log.info("CDCR bootstrap is " + (status == BootstrapStatus.UNKNOWN ? "unknown" : "submitted"),
BOOTSTRAP_TIMEOUT_SECONDS - timeOut.timeLeft(TimeUnit.SECONDS));
// we were not able to query the status on the remote end
// so just sleep for a bit and try again
Thread.sleep(BOOTSTRAP_RETRY_DELAY_MS);
timeOut.sleep(BOOTSTRAP_RETRY_DELAY_MS);
}
}
} catch (InterruptedException e) {

View File

@ -391,7 +391,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
results.add("status", status);
}
enum CollectionOperation implements CollectionOp {
public enum CollectionOperation implements CollectionOp {
/**
* very simple currently, you can pass a template collection, and the new collection is created on
* every node the template collection is on

View File

@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.ZkSolrResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrResourceLoader;
@ -93,7 +94,7 @@ public class SchemaManager {
if (timeout < 1) {
timeout = 600;
}
TimeOut timeOut = new TimeOut(timeout, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(timeout, TimeUnit.SECONDS, TimeSource.NANO_TIME);
SolrCore core = req.getCore();
String errorMsg = "Unable to persist managed schema. ";
List errors = Collections.emptyList();

View File

@ -78,6 +78,7 @@ import org.apache.solr.common.util.JsonSchemaValidator;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.common.util.ValidatingJsonMap;
import org.apache.solr.core.CoreContainer;
@ -347,7 +348,7 @@ public class HttpSolrCall {
if (rsp.getValues().get("success") == null) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not auto-create " + SYSTEM_COLL + " collection: "+ Utils.toJSONString(rsp.getValues()));
}
TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS, TimeSource.NANO_TIME);
for (; ; ) {
if (cores.getZkController().getClusterState().getCollectionOrNull(SYSTEM_COLL) != null) {
break;
@ -355,7 +356,7 @@ public class HttpSolrCall {
if (timeOut.hasTimedOut()) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find " + SYSTEM_COLL + " collection even after 3 seconds");
}
Thread.sleep(50);
timeOut.sleep(50);
}
}

View File

@ -68,6 +68,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.Hash;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.component.RealTimeGetComponent;
@ -1142,7 +1143,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
private long waitForDependentUpdates(AddUpdateCommand cmd, long versionOnUpdate,
boolean isReplayOrPeersync, VersionBucket bucket) throws IOException {
long lastFoundVersion = 0;
TimeOut waitTimeout = new TimeOut(5, TimeUnit.SECONDS);
TimeOut waitTimeout = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
vinfo.lockForUpdate();
try {

View File

@ -19,6 +19,7 @@ package org.apache.solr.util;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.util.StringHelper;
import org.apache.solr.common.util.TimeSource;
/**
* Helper class for generating unique ID-s.

View File

@ -18,26 +18,34 @@ package org.apache.solr.util;
import java.util.concurrent.TimeUnit;
import org.apache.solr.common.util.TimeSource;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
public class TimeOut {
private final long timeoutAt, startTime;
private final TimeSource timeSource;
public TimeOut(long interval, TimeUnit unit) {
startTime = System.nanoTime();
public TimeOut(long interval, TimeUnit unit, TimeSource timeSource) {
this.timeSource = timeSource;
startTime = timeSource.getTime();
this.timeoutAt = startTime + NANOSECONDS.convert(interval, unit);
}
public boolean hasTimedOut() {
return System.nanoTime() > timeoutAt;
return timeSource.getTime() > timeoutAt;
}
public void sleep(long ms) throws InterruptedException {
timeSource.sleep(ms);
}
public long timeLeft(TimeUnit unit) {
return unit.convert(timeoutAt - System.nanoTime(), NANOSECONDS);
return unit.convert(timeoutAt - timeSource.getTime(), NANOSECONDS);
}
public long timeElapsed(TimeUnit unit) {
return unit.convert(System.nanoTime() - startTime, NANOSECONDS);
return unit.convert(timeSource.getTime() - startTime, NANOSECONDS);
}
}

View File

@ -1,57 +0,0 @@
/*
* 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.util;
import java.util.concurrent.TimeUnit;
import org.apache.solr.common.util.SuppressForbidden;
/**
* Source of timestamps.
*/
public abstract class TimeSource {
/** Implementation that uses {@link System#currentTimeMillis()}. */
public static final class CurrentTimeSource extends TimeSource {
@Override
@SuppressForbidden(reason = "Needed to provide timestamps based on currentTimeMillis.")
public long getTime() {
return TimeUnit.NANOSECONDS.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
}
}
/** Implementation that uses {@link System#nanoTime()}. */
public static final class NanoTimeSource extends TimeSource {
@Override
public long getTime() {
return System.nanoTime();
}
}
/** This instance uses {@link CurrentTimeSource} for generating timestamps. */
public static final TimeSource CURRENT_TIME = new CurrentTimeSource();
/** This instance uses {@link NanoTimeSource} for generating timestamps. */
public static final TimeSource NANO_TIME = new NanoTimeSource();
/**
* Return a timestamp, in nanosecond unit.
*/
public abstract long getTime();
}

View File

@ -20,6 +20,7 @@ import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.concurrent.TimeUnit;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -120,7 +121,7 @@ public class TransformerProvider {
lastFilename = filename;
lastTemplates = result;
cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS);
cacheExpiresTimeout = new TimeOut(cacheLifetimeSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
return result;
}

View File

@ -21,7 +21,7 @@ import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.junit.Test;
public class ActionThrottleTest extends SolrTestCaseJ4 {
@ -40,6 +40,16 @@ public class ActionThrottleTest extends SolrTestCaseJ4 {
return returnValues.get(index++);
}
@Override
public void sleep(long ms) throws InterruptedException {
throw new UnsupportedOperationException();
}
@Override
public long convertDelay(TimeUnit fromUnit, long value, TimeUnit toUnit) {
throw new UnsupportedOperationException();
}
}
// use the same time source as ActionThrottle

View File

@ -73,7 +73,7 @@ public class AddReplicaTest extends SolrCloudTestCase {
success = true;
break;
}
assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
assertFalse(rsp.toString(), rsp.getRequestStatus() == RequestStatusState.FAILED);
Thread.sleep(500);
}
assertTrue(success);

View File

@ -75,11 +75,11 @@ public class AssignTest extends SolrTestCaseJ4 {
zkClientData.get(invocation.getArgument(0)));
// TODO: fix this to be independent of ZK
ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
String nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
String nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node1", nodeName);
nodeName = Assign.assignNode(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node1", nodeName);
nodeName = Assign.assignNode(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node2", nodeName);
}
@ -145,8 +145,8 @@ public class AssignTest extends SolrTestCaseJ4 {
slices.put("shard2", new Slice("shard2", new HashMap<>(), null));
DocCollection docCollection = new DocCollection("collection1", slices, null, DocRouter.DEFAULT);
assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildSolrCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildSolrCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
} finally {
server.shutdown();
}

View File

@ -34,6 +34,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.AfterClass;
@ -264,7 +265,7 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
}
}
waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS));
waitForReplicationFromReplicas(DEFAULT_COLLECTION, zkStateReader, new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
// waitForAllWarmingSearchers();
Set<String> addFails = getAddFails(indexTreads);

View File

@ -32,6 +32,7 @@ import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.AfterClass;
@ -206,7 +207,7 @@ public class ChaosMonkeySafeLeaderWithPullReplicasTest extends AbstractFullDistr
log.info("collection state: " + printClusterStateInfo(DEFAULT_COLLECTION));
waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS));
waitForReplicationFromReplicas(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME));
// waitForAllWarmingSearchers();
checkShardConsistency(batchSize == 1, true);

View File

@ -66,6 +66,7 @@ import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoBean.Category;
@ -458,7 +459,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
.add("id", "7")
.add("id", "8")
.commit(cluster.getSolrClient(), collectionName);
TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
try {
long numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@ -514,7 +515,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
boolean allTimesAreCorrect = false;
while (! timeout.hasTimedOut()) {

View File

@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
@ -140,7 +141,7 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
@Override
public void run() {
final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS);
final TimeOut timeout = new TimeOut(timeToRunSec, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (! timeout.hasTimedOut() && failure.get() == null) {
doWork();
}

View File

@ -35,6 +35,7 @@ import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.handler.ReplicationHandler;
import org.apache.solr.update.processor.DocExpirationUpdateProcessorFactory;
import org.apache.solr.util.TimeOut;
@ -175,7 +176,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
SolrParams params)
throws SolrServerException, InterruptedException, IOException {
final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS);
final TimeOut timeout = new TimeOut(maxTimeLimitSeconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
long numFound = cluster.getSolrClient().query(COLLECTION, params).getResults().getNumFound();
while (0L < numFound && ! timeout.hasTimedOut()) {
Thread.sleep(Math.max(1, Math.min(5000, timeout.timeLeft(TimeUnit.MILLISECONDS))));

View File

@ -41,6 +41,7 @@ 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.ModifiableSolrParams;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
import org.slf4j.Logger;
@ -153,7 +154,7 @@ public class LeaderFailureAfterFreshStartTest extends AbstractFullDistribZkTestB
// shutdown the original leader
log.info("Now shutting down initial leader");
forceNodeFailures(singletonList(initialLeaderJetty));
waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info , new TimeOut(15, SECONDS));
waitForNewLeader(cloudClient, "shard1", (Replica)initialLeaderJetty.client.info , new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
waitTillNodesActive();
log.info("Updating mappings from zk");
updateMappingsFromZk(jettys, clients, true);

View File

@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.RoutingRule;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
import org.junit.BeforeClass;
@ -177,7 +178,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
@Override
public void run() {
TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
for (int id = 26*3; id < 500 && ! timeout.hasTimedOut(); id++) {
String shardKey = "" + (char) ('a' + (id % 26)); // See comment in ShardRoutingTest for hash distribution
SolrInputDocument doc = new SolrInputDocument();

View File

@ -44,7 +44,9 @@ import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ObjectCache;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.handler.component.ShardHandlerFactory;
@ -87,6 +89,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
private static ClusterState clusterStateMock;
private static SolrZkClient solrZkClientMock;
private static DistribStateManager stateManagerMock;
private static ObjectCache objectCache;
private static AutoScalingConfig autoScalingConfig = new AutoScalingConfig(Collections.emptyMap());
private final Map zkMap = new HashMap();
private final Map<String, ClusterState.CollectionRef> collectionsSet = new HashMap<>();
@ -136,6 +139,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
overseerMock = mock(Overseer.class);
zkControllerMock = mock(ZkController.class);
cloudDataProviderMock = mock(SolrCloudManager.class);
objectCache = new ObjectCache();
clusterStateProviderMock = mock(ClusterStateProvider.class);
stateManagerMock = mock(DistribStateManager.class);
}
@ -173,6 +177,9 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
reset(overseerMock);
reset(zkControllerMock);
reset(cloudDataProviderMock);
objectCache.clear();
when(cloudDataProviderMock.getObjectCache()).thenReturn(objectCache);
when(cloudDataProviderMock.getTimeSource()).thenReturn(TimeSource.NANO_TIME);
reset(clusterStateProviderMock);
reset(stateManagerMock);
@ -537,7 +544,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
}
protected void waitForEmptyQueue(long maxWait) throws Exception {
final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS);
final TimeOut timeout = new TimeOut(maxWait, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
while (queue.peek() != null) {
if (timeout.hasTimedOut())
fail("Queue not empty within " + maxWait + " ms");

View File

@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
@ -51,7 +52,7 @@ public class OverseerRolesTest extends SolrCloudTestCase {
}
private void waitForNewOverseer(int seconds, Predicate<String> state) throws Exception {
TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS);
TimeOut timeout = new TimeOut(seconds, TimeUnit.SECONDS, TimeSource.NANO_TIME);
String current = null;
while (timeout.hasTimedOut() == false) {
current = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());

View File

@ -48,6 +48,7 @@ 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.ModifiableSolrParams;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.util.TimeOut;
@ -162,7 +163,7 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
log.info("Now shutting down initial leader");
forceNodeFailures(singletonList(initialLeaderJetty));
log.info("Updating mappings from zk");
waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS));
waitForNewLeader(cloudClient, "shard1", (Replica) initialLeaderJetty.client.info, new TimeOut(15, SECONDS, TimeSource.NANO_TIME));
updateMappingsFromZk(jettys, clients, true);
assertEquals("PeerSynced node did not become leader", nodePeerSynced, shardToLeaderJetty.get("shard1"));

View File

@ -54,6 +54,7 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.util.DefaultSolrThreadFactory;
@ -389,7 +390,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
}
private void assertSliceAndReplicaCount(String collection, int numSlices, int numReplicas, int timeOutInMs) throws InterruptedException {
TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS);
TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
Collection<Slice> slices = clusterState.getCollection(collection).getActiveSlices();

View File

@ -24,6 +24,7 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.MockCoreContainer.MockCoreDescriptor;
import org.apache.solr.util.TimeOut;
@ -98,7 +99,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
// kill the replica
int children = cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size();
ChaosMonkey.stop(notLeader.jetty);
TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
if (children > cloudClient.getZkStateReader().getZkClient().getChildren("/live_nodes", null, true).size()) {
break;
@ -173,7 +174,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
// this should have published a down state so assert that cversion has incremented
assertTrue(getOverseerCversion() > cversion);
timeOut = new TimeOut(30, TimeUnit.SECONDS);
timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
Replica r = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getReplica(replica.getName());
if (r.getState() == Replica.State.DOWN) {
@ -217,7 +218,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
thread = new LeaderInitiatedRecoveryThread(zkController, coreContainer,
DEFAULT_COLLECTION, SHARD1, replicaCoreNodeProps, 1, coreContainer.getCores().iterator().next().getCoreDescriptor());
thread.publishDownState(replicaCoreNodeProps.getCoreName(), replica.getName(), replica.getNodeName(), replicaCoreNodeProps.getCoreUrl(), false);
timeOut = new TimeOut(30, TimeUnit.SECONDS);
timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
Replica.State state = zkController.getLeaderInitiatedRecoveryState(DEFAULT_COLLECTION, SHARD1, replica.getName());
if (state == Replica.State.DOWN) {

View File

@ -52,6 +52,7 @@ import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
@ -229,7 +230,7 @@ public class TestPullReplica extends SolrCloudTestCase {
assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
}
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.PULL))) {
//TODO: assert replication < REPLICATION_TIMEOUT_SECS
try (HttpSolrClient readOnlyReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@ -500,7 +501,7 @@ public class TestPullReplica extends SolrCloudTestCase {
}
private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query) throws IOException, SolrServerException, InterruptedException {
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
for (Replica r:replicas) {
try (HttpSolrClient replicaClient = getHttpSolrClient(r.getCoreUrl())) {
while (true) {
@ -521,7 +522,7 @@ public class TestPullReplica extends SolrCloudTestCase {
}
private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
LOG.info("Collection not yet deleted");
try {

View File

@ -43,6 +43,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
@ -86,7 +87,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
proxies.put(proxy.getUrl(), proxy);
jettys.put(proxy.getUrl(), jetty);
}
TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (true) {
try {
CollectionAdminRequest.ClusterProp clusterPropRequest = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false");
@ -159,7 +160,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
assertNumberOfReplicas(numShards, 0, numShards, true, true);// Replica should still be active, since it doesn't disconnect from ZooKeeper
{
long numFound = 0;
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (numFound < 20 && !t.hasTimedOut()) {
Thread.sleep(200);
numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
@ -233,7 +234,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
}
private void assertNumDocs(int numDocs, SolrClient client, int timeoutSecs) throws InterruptedException, SolrServerException, IOException {
TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS);
TimeOut t = new TimeOut(timeoutSecs, TimeUnit.SECONDS, TimeSource.NANO_TIME);
long numFound = -1;
while (!t.hasTimedOut()) {
Thread.sleep(200);
@ -299,7 +300,7 @@ public class TestPullReplicaErrorHandling extends SolrCloudTestCase {
}
private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
LOG.info("Collection not yet deleted");
try {

View File

@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
@ -118,7 +119,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
// 2> All the replicas we _think_ are leaders are in the 0th position in the leader election queue.
// 3> The node that ZooKeeper thinks is the leader is the one we think should be the leader.
void checkConsistency() throws InterruptedException, KeeperException {
TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
boolean checkAppearOnce = false;
boolean checkElectionZero = false;
boolean checkZkLeadersAgree = false;
@ -323,7 +324,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
boolean waitForAllPreferreds() throws KeeperException, InterruptedException {
boolean goAgain = true;
TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
while (! timeout.hasTimedOut()) {
goAgain = false;
Map<String, Slice> slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();

View File

@ -60,6 +60,7 @@ import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.update.SolrIndexWriter;
@ -232,7 +233,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
assertEquals(1, leaderClient.query(new SolrQuery("*:*")).getResults().getNumFound());
}
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS);
TimeOut t = new TimeOut(REPLICATION_TIMEOUT_SECS, TimeUnit.SECONDS, TimeSource.NANO_TIME);
for (Replica r:s.getReplicas(EnumSet.of(Replica.Type.TLOG))) {
//TODO: assert replication < REPLICATION_TIMEOUT_SECS
try (HttpSolrClient tlogReplicaClient = getHttpSolrClient(r.getCoreUrl())) {
@ -400,7 +401,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
docCollection = assertNumberOfReplicas(0, 1, 0, true, true);
// Wait until a new leader is elected
TimeOut t = new TimeOut(30, TimeUnit.SECONDS);
TimeOut t = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!t.hasTimedOut()) {
docCollection = getCollectionState(collectionName);
Replica leader = docCollection.getSlice("shard1").getLeader();
@ -488,7 +489,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
waitForNumDocsInAllActiveReplicas(2);
// There are a small delay between new searcher and copy over old updates operation
TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
if (assertCopyOverOldUpdates(1, timeCopyOverPerCores)) {
break;
@ -748,7 +749,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
}
private void waitForNumDocsInAllReplicas(int numDocs, Collection<Replica> replicas, String query, int timeout) throws IOException, SolrServerException, InterruptedException {
TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS);
TimeOut t = new TimeOut(timeout, TimeUnit.SECONDS, TimeSource.NANO_TIME);
for (Replica r:replicas) {
if (!r.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
continue;
@ -772,7 +773,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
}
private void waitForDeletion(String collection) throws InterruptedException, KeeperException {
TimeOut t = new TimeOut(10, TimeUnit.SECONDS);
TimeOut t = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (cluster.getSolrClient().getZkStateReader().getClusterState().hasCollection(collection)) {
try {
Thread.sleep(100);

View File

@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.util.DefaultSolrThreadFactory;
@ -73,7 +74,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
private void checkCoreNamePresenceAndSliceCount(String collectionName, String coreName,
boolean shouldBePresent, int expectedSliceCount) throws Exception {
final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
Boolean isPresent = null; // null meaning "don't know"
while (null == isPresent || shouldBePresent != isPresent.booleanValue()) {
final DocCollection docCollection = getCommonCloudSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName);

View File

@ -36,6 +36,7 @@ import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeOut;
import org.junit.BeforeClass;
@ -177,7 +178,7 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
Thread.sleep(100);
if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

View File

@ -37,6 +37,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.junit.BeforeClass;
import org.junit.Test;
@ -44,6 +45,7 @@ import org.junit.Test;
import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(3)
@ -141,7 +143,7 @@ public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
Thread.sleep(100);
if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");

View File

@ -40,6 +40,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.data.Stat;
@ -69,7 +70,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
}
private static void testAutoAddReplicas() throws Exception {
TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
ZkNodeProps loaded = ZkNodeProps.load(data);

View File

@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling;
import java.util.HashMap;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
/**
*
*/
public class CapturedEvent {
public final AutoScalingConfig.TriggerListenerConfig config;
public final TriggerEventProcessorStage stage;
public final String actionName;
public final TriggerEvent event;
public final String message;
public final Map<String, Object> context = new HashMap<>();
public final long timestamp;
public CapturedEvent(long timestamp, ActionContext context, AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName,
TriggerEvent event, String message) {
if (context != null) {
context.toMap(this.context);
}
this.config = config;
this.stage = stage;
this.actionName = actionName;
this.event = event;
this.message = message;
this.timestamp = timestamp;
}
@Override
public String toString() {
return "CapturedEvent{" +
"timestamp=" + timestamp +
", stage=" + stage +
", actionName='" + actionName + '\'' +
", event=" + event +
", context=" + context +
", config=" + config +
", message='" + message + '\'' +
'}';
}
}

View File

@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
@ -38,11 +39,13 @@ import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.LogLevel;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -139,6 +142,19 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
assertEquals(response.get("result").toString(), "success");
}
@After
public void printState() throws Exception {
log.debug("-------------_ FINAL STATE --------------");
SolrCloudManager cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
for (String node: cloudManager.getClusterStateProvider().getLiveNodes()) {
Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, ImplicitSnitch.tags);
log.debug("* Node values: " + node + "\n" + Utils.toJSONString(values));
}
log.debug("* Live nodes: " + cloudManager.getClusterStateProvider().getLiveNodes());
ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
state.forEachCollection(coll -> log.debug("* Collection " + coll.getName() + " state: " + coll));
}
@Test
public void testNodeLost() throws Exception {
// let's start a node so that we have at least two

View File

@ -41,7 +41,7 @@ import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.apache.zookeeper.data.Stat;
import org.junit.Before;
import org.junit.BeforeClass;

View File

@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;

View File

@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;

View File

@ -56,7 +56,7 @@ import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeOut;
import org.apache.solr.util.TimeSource;
import org.apache.solr.common.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZKUtil;
import org.apache.zookeeper.data.Stat;
@ -87,6 +87,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
private static AtomicBoolean triggerFired;
private static Set<TriggerEvent> events = ConcurrentHashMap.newKeySet();
private static ZkStateReader zkStateReader;
private static SolrCloudManager cloudManager;
// use the same time source as triggers use
private static final TimeSource timeSource = TimeSource.CURRENT_TIME;
@ -146,6 +147,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
// lets start a node
cluster.startJettySolrRunner();
}
cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
}
private void deleteChildrenRecursively(String path) throws Exception {
@ -322,7 +324,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
Thread.sleep(200);
}
@ -338,7 +340,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
cluster.stopJettySolrRunner(index);
// ensure that the old trigger sees the stopped node, todo find a better way to do this
Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
waitForSeconds = 0;
setTriggerCommand = "{" +
@ -386,7 +388,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(2, TimeUnit.SECONDS, cloudManager.getTimeSource());
while (actionInitCalled.getCount() == 0 && !timeOut.hasTimedOut()) {
Thread.sleep(200);
}
@ -396,7 +398,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
JettySolrRunner newNode = cluster.startJettySolrRunner();
// ensure that the old trigger sees the new node, todo find a better way to do this
Thread.sleep(500 + TimeUnit.MILLISECONDS.convert(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS));
Thread.sleep(500 + TimeUnit.SECONDS.toMillis(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS));
waitForSeconds = 0;
setTriggerCommand = "{" +
@ -954,52 +956,22 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
assertEquals(TriggerEventType.NODELOST, ev.getEventType());
}
private static class TestEvent {
final AutoScalingConfig.TriggerListenerConfig config;
final TriggerEventProcessorStage stage;
final String actionName;
final TriggerEvent event;
final String message;
final long timestamp;
TestEvent(AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName, TriggerEvent event, String message) {
this.config = config;
this.stage = stage;
this.actionName = actionName;
this.event = event;
this.message = message;
this.timestamp = timeSource.getTime();
}
@Override
public String toString() {
return "TestEvent{" +
"timestamp=" + timestamp +
", config=" + config +
", stage=" + stage +
", actionName='" + actionName + '\'' +
", event=" + event +
", message='" + message + '\'' +
'}';
}
}
static Map<String, List<TestEvent>> listenerEvents = new HashMap<>();
static Map<String, List<CapturedEvent>> listenerEvents = new HashMap<>();
static CountDownLatch listenerCreated = new CountDownLatch(1);
static boolean failDummyAction = false;
public static class TestTriggerListener extends TriggerListenerBase {
@Override
public void init(SolrCloudManager dataProvider, AutoScalingConfig.TriggerListenerConfig config) {
super.init(dataProvider, config);
public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
super.init(cloudManager, config);
listenerCreated.countDown();
}
@Override
public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
ActionContext context, Throwable error, String message) {
List<TestEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
lst.add(new TestEvent(config, stage, actionName, event, message));
List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
lst.add(new CapturedEvent(timeSource.getTime(), context, config, stage, actionName, event, message));
}
}
@ -1079,38 +1051,38 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
Thread.sleep(2000);
// check foo events
List<TestEvent> testEvents = listenerEvents.get("foo");
assertNotNull("foo events: " + testEvents, testEvents);
assertEquals("foo events: " + testEvents, 5, testEvents.size());
List<CapturedEvent> capturedEvents = listenerEvents.get("foo");
assertNotNull("foo events: " + capturedEvents, capturedEvents);
assertEquals("foo events: " + capturedEvents, 5, capturedEvents.size());
assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
assertEquals("test", testEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
assertEquals("test", capturedEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
assertEquals("test", testEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
assertEquals("test", capturedEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(3).stage);
assertEquals("test1", testEvents.get(3).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(3).stage);
assertEquals("test1", capturedEvents.get(3).actionName);
assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(4).stage);
assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(4).stage);
// check bar events
testEvents = listenerEvents.get("bar");
assertNotNull("bar events", testEvents);
assertEquals("bar events", 4, testEvents.size());
capturedEvents = listenerEvents.get("bar");
assertNotNull("bar events", capturedEvents);
assertEquals("bar events", 4, capturedEvents.size());
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
assertEquals("test", testEvents.get(0).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
assertEquals("test", capturedEvents.get(0).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
assertEquals("test", testEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
assertEquals("test", capturedEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
assertEquals("test1", testEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
assertEquals("test1", capturedEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.SUCCEEDED, testEvents.get(3).stage);
assertEquals(TriggerEventProcessorStage.SUCCEEDED, capturedEvents.get(3).stage);
// reset
triggerFired.set(false);
@ -1125,37 +1097,37 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
Thread.sleep(2000);
// check foo events
testEvents = listenerEvents.get("foo");
assertNotNull("foo events: " + testEvents, testEvents);
assertEquals("foo events: " + testEvents, 4, testEvents.size());
capturedEvents = listenerEvents.get("foo");
assertNotNull("foo events: " + capturedEvents, capturedEvents);
assertEquals("foo events: " + capturedEvents, 4, capturedEvents.size());
assertEquals(TriggerEventProcessorStage.STARTED, testEvents.get(0).stage);
assertEquals(TriggerEventProcessorStage.STARTED, capturedEvents.get(0).stage);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(1).stage);
assertEquals("test", testEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(1).stage);
assertEquals("test", capturedEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(2).stage);
assertEquals("test", testEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(2).stage);
assertEquals("test", capturedEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
assertEquals("test1", testEvents.get(3).actionName);
assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
assertEquals("test1", capturedEvents.get(3).actionName);
// check bar events
testEvents = listenerEvents.get("bar");
assertNotNull("bar events", testEvents);
assertEquals("bar events", 4, testEvents.size());
capturedEvents = listenerEvents.get("bar");
assertNotNull("bar events", capturedEvents);
assertEquals("bar events", 4, capturedEvents.size());
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(0).stage);
assertEquals("test", testEvents.get(0).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(0).stage);
assertEquals("test", capturedEvents.get(0).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, testEvents.get(1).stage);
assertEquals("test", testEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.AFTER_ACTION, capturedEvents.get(1).stage);
assertEquals("test", capturedEvents.get(1).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, testEvents.get(2).stage);
assertEquals("test1", testEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.BEFORE_ACTION, capturedEvents.get(2).stage);
assertEquals("test1", capturedEvents.get(2).actionName);
assertEquals(TriggerEventProcessorStage.FAILED, testEvents.get(3).stage);
assertEquals("test1", testEvents.get(3).actionName);
assertEquals(TriggerEventProcessorStage.FAILED, capturedEvents.get(3).stage);
assertEquals("test1", capturedEvents.get(3).actionName);
}
@Test
@ -1200,7 +1172,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
// wait for listener to capture the SUCCEEDED stage
Thread.sleep(1000);
List<TestEvent> capturedEvents = listenerEvents.get("bar");
List<CapturedEvent> capturedEvents = listenerEvents.get("bar");
// we may get a few IGNORED events if other tests caused events within cooldown period
assertTrue(capturedEvents.toString(), capturedEvents.size() > 0);
long prevTimestamp = capturedEvents.get(capturedEvents.size() - 1).timestamp;
@ -1220,11 +1192,11 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
capturedEvents = listenerEvents.get("bar");
assertTrue(capturedEvents.toString(), capturedEvents.size() > 1);
for (int i = 0; i < capturedEvents.size() - 1; i++) {
TestEvent ev = capturedEvents.get(i);
CapturedEvent ev = capturedEvents.get(i);
assertEquals(ev.toString(), TriggerEventProcessorStage.IGNORED, ev.stage);
assertTrue(ev.toString(), ev.message.contains("cooldown"));
}
TestEvent ev = capturedEvents.get(capturedEvents.size() - 1);
CapturedEvent ev = capturedEvents.get(capturedEvents.size() - 1);
assertEquals(ev.toString(), TriggerEventProcessorStage.SUCCEEDED, ev.stage);
// the difference between timestamps of the first SUCCEEDED and the last SUCCEEDED
// must be larger than cooldown period
@ -1380,8 +1352,9 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
Map<String, Object> props = map("waitFor", 0L, "actions", Collections.singletonList(map("name","throttler", "class", ThrottlingTesterAction.class.getName())));
scheduledTriggers.add(new NodeAddedTrigger("y1", props, resourceLoader, solrCloudManager));
scheduledTriggers.add(new NodeAddedTrigger("y2", props, resourceLoader, solrCloudManager));
scheduledTriggers.resetActionThrottle();
JettySolrRunner newNode = cluster.startJettySolrRunner();
assertTrue(getTriggerFiredLatch().await(10, TimeUnit.SECONDS));
assertTrue(getTriggerFiredLatch().await(20, TimeUnit.SECONDS));
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
if (cluster.getJettySolrRunner(i) == newNode) {
cluster.stopJettySolrRunner(i);
@ -1439,7 +1412,12 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
}
@Test
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
public void testSearchRate() throws Exception {
// start a few more jetty-s
for (int i = 0; i < 3; i++) {
cluster.startJettySolrRunner();
}
CloudSolrClient solrClient = cluster.getSolrClient();
String COLL1 = "collection1";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLL1,
@ -1453,6 +1431,8 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
"'enabled' : true," +
"'rate' : 1.0," +
"'actions' : [" +
"{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
"{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
"{'name':'test','class':'" + TestSearchRateAction.class.getName() + "'}" +
"]" +
"}}";
@ -1466,6 +1446,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
"'name' : 'srt'," +
"'trigger' : 'search_rate_trigger'," +
"'stage' : ['FAILED','SUCCEEDED']," +
"'afterAction': ['compute', 'execute', 'test']," +
"'class' : '" + TestTriggerListener.class.getName() + "'" +
"}" +
"}";
@ -1481,7 +1462,7 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
// wait for listener to capture the SUCCEEDED stage
Thread.sleep(2000);
assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
TestEvent ev = listenerEvents.get("srt").get(0);
CapturedEvent ev = listenerEvents.get("srt").get(0);
long now = timeSource.getTime();
// verify waitFor
assertTrue(TimeUnit.SECONDS.convert(waitForSeconds, TimeUnit.NANOSECONDS) - WAIT_FOR_DELTA_NANOS <= now - ev.event.getEventTime());

View File

@ -0,0 +1,24 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
/**
* Interface that helps simulating action errors.
*/
public interface ActionError {
boolean shouldFail(String... args);
}

View File

@ -0,0 +1,599 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Predicate;
import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.cloud.OverseerTaskQueue;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.util.Pair;
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.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A distributed queue that uses {@link DistribStateManager} as the underlying distributed store.
* Implementation based on {@link org.apache.solr.cloud.ZkDistributedQueue}
*/
public class GenericDistributedQueue implements DistributedQueue {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
static final String PREFIX = "qn-";
/**
* Theory of operation:
* <p>
* Under ordinary circumstances we neither watch nor poll for children in ZK.
* Instead we keep an in-memory list of known child names. When the in-memory
* list is exhausted, we then fetch from ZK.
* <p>
* We only bother setting a child watcher when the queue has no children in ZK.
*/
private static final Object _IMPLEMENTATION_NOTES = null;
final String dir;
final DistribStateManager stateManager;
final Stats stats;
/**
* A lock that guards all of the mutable state that follows.
*/
private final ReentrantLock updateLock = new ReentrantLock();
/**
* Contains the last set of children fetched from ZK. Elements are removed from the head of
* this in-memory set as they are consumed from the queue. Due to the distributed nature
* of the queue, elements may appear in this set whose underlying nodes have been consumed in ZK.
* Therefore, methods like {@link #peek()} have to double-check actual node existence, and methods
* like {@link #poll()} must resolve any races by attempting to delete the underlying node.
*/
private TreeSet<String> knownChildren = new TreeSet<>();
/**
* Used to wait on ZK changes to the child list; you must hold {@link #updateLock} before waiting on this condition.
*/
private final Condition changed = updateLock.newCondition();
private boolean isDirty = true;
private int watcherCount = 0;
private final int maxQueueSize;
/**
* If {@link #maxQueueSize} is set, the number of items we can queue without rechecking the server.
*/
private final AtomicInteger offerPermits = new AtomicInteger(0);
public GenericDistributedQueue(DistribStateManager stateManager, String dir) {
this(stateManager, dir, new Stats());
}
public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats) {
this(stateManager, dir, stats, 0);
}
public GenericDistributedQueue(DistribStateManager stateManager, String dir, Stats stats, int maxQueueSize) {
this.dir = dir;
try {
if (!stateManager.hasData(dir)) {
try {
stateManager.makePath(dir);
} catch (AlreadyExistsException e) {
// ignore
}
}
} catch (IOException | KeeperException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new SolrException(ErrorCode.SERVER_ERROR, e);
}
this.stateManager = stateManager;
this.stats = stats;
this.maxQueueSize = maxQueueSize;
}
/**
* Returns the data at the first element of the queue, or null if the queue is
* empty.
*
* @return data at the first element of the queue, or null.
*/
@Override
public byte[] peek() throws Exception {
Timer.Context time = stats.time(dir + "_peek");
try {
return firstElement();
} finally {
time.stop();
}
}
/**
* Returns the data at the first element of the queue, or null if the queue is
* empty and block is false.
*
* @param block if true, blocks until an element enters the queue
* @return data at the first element of the queue, or null.
*/
@Override
public byte[] peek(boolean block) throws Exception {
return block ? peek(Long.MAX_VALUE) : peek();
}
/**
* Returns the data at the first element of the queue, or null if the queue is
* empty after wait ms.
*
* @param wait max wait time in ms.
* @return data at the first element of the queue, or null.
*/
@Override
public byte[] peek(long wait) throws Exception {
Preconditions.checkArgument(wait > 0);
Timer.Context time;
if (wait == Long.MAX_VALUE) {
time = stats.time(dir + "_peek_wait_forever");
} else {
time = stats.time(dir + "_peek_wait" + wait);
}
updateLock.lockInterruptibly();
try {
long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
while (waitNanos > 0) {
byte[] result = firstElement();
if (result != null) {
return result;
}
waitNanos = changed.awaitNanos(waitNanos);
}
return null;
} finally {
updateLock.unlock();
time.stop();
}
}
/**
* Attempts to remove the head of the queue and return it. Returns null if the
* queue is empty.
*
* @return Head of the queue or null.
*/
@Override
public byte[] poll() throws Exception {
Timer.Context time = stats.time(dir + "_poll");
try {
return removeFirst();
} finally {
time.stop();
}
}
/**
* Attempts to remove the head of the queue and return it.
*
* @return The former head of the queue
*/
@Override
public byte[] remove() throws Exception {
Timer.Context time = stats.time(dir + "_remove");
try {
byte[] result = removeFirst();
if (result == null) {
throw new NoSuchElementException();
}
return result;
} finally {
time.stop();
}
}
public void remove(Collection<String> paths) throws Exception {
if (paths.isEmpty()) return;
List<Op> ops = new ArrayList<>();
for (String path : paths) {
ops.add(Op.delete(dir + "/" + path, -1));
}
for (int from = 0; from < ops.size(); from += 1000) {
int to = Math.min(from + 1000, ops.size());
if (from < to) {
try {
stateManager.multi(ops.subList(from, to));
} catch (NoSuchElementException e) {
// don't know which nodes are not exist, so try to delete one by one node
for (int j = from; j < to; j++) {
try {
stateManager.removeData(ops.get(j).getPath(), -1);
} catch (NoSuchElementException e2) {
LOG.debug("Can not remove node which is not exist : " + ops.get(j).getPath());
}
}
}
}
}
int cacheSizeBefore = knownChildren.size();
knownChildren.removeAll(paths);
if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
stats.setQueueLength(knownChildren.size());
} else {
// There are elements get deleted but not present in the cache,
// the cache seems not valid anymore
knownChildren.clear();
isDirty = true;
}
}
/**
* Removes the head of the queue and returns it, blocks until it succeeds.
*
* @return The former head of the queue
*/
@Override
public byte[] take() throws Exception {
// Same as for element. Should refactor this.
Timer.Context timer = stats.time(dir + "_take");
updateLock.lockInterruptibly();
try {
while (true) {
byte[] result = removeFirst();
if (result != null) {
return result;
}
changed.await();
}
} finally {
updateLock.unlock();
timer.stop();
}
}
/**
* Inserts data into queue. If there are no other queue consumers, the offered element
* will be immediately visible when this method returns.
*/
@Override
public void offer(byte[] data) throws Exception {
Timer.Context time = stats.time(dir + "_offer");
try {
while (true) {
try {
if (maxQueueSize > 0) {
if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) {
// If a max queue size is set, check it before creating a new queue item.
if (!stateManager.hasData(dir)) {
// jump to the code below, which tries to create dir if it doesn't exist
throw new NoSuchElementException();
}
List<String> children = stateManager.listData(dir);
int remainingCapacity = maxQueueSize - children.size();
if (remainingCapacity <= 0) {
throw new IllegalStateException("queue is full");
}
// Allow this client to push up to 1% of the remaining queue capacity without rechecking.
offerPermits.set(remainingCapacity / 100);
}
}
// Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
// This will get set again when the watcher actually fires, but that's ok.
stateManager.createData(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL);
isDirty = true;
return;
} catch (NoSuchElementException e) {
try {
stateManager.createData(dir, new byte[0], CreateMode.PERSISTENT);
} catch (NoSuchElementException ne) {
// someone created it
}
}
}
} finally {
time.stop();
}
}
public Stats getZkStats() {
return stats;
}
@Override
public Map<String, Object> getStats() {
if (stats == null) {
return Collections.emptyMap();
}
Map<String, Object> res = new HashMap<>();
res.put("queueLength", stats.getQueueLength());
final Map<String, Object> statsMap = new HashMap<>();
res.put("stats", statsMap);
stats.getStats().forEach((op, stat) -> {
final Map<String, Object> statMap = new HashMap<>();
statMap.put("success", stat.success.get());
statMap.put("errors", stat.errors.get());
final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
statMap.put("failureDetails", failed);
stat.failureDetails.forEach(failedOp -> {
Map<String, Object> fo = new HashMap<>();
fo.put("req", failedOp.req);
fo.put("resp", failedOp.resp);
});
statsMap.put(op, statMap);
});
return res;
}
/**
* Returns the name if the first known child node, or {@code null} if the queue is empty.
* This is the only place {@link #knownChildren} is ever updated!
* The caller must double check that the actual node still exists, since the in-memory
* list is inherently stale.
*/
private String firstChild(boolean remove, boolean refetchIfDirty) throws Exception {
updateLock.lockInterruptibly();
try {
// We always return from cache first, the cache will be cleared if the node is not exist
if (!knownChildren.isEmpty() && !(isDirty && refetchIfDirty)) {
return remove ? knownChildren.pollFirst() : knownChildren.first();
}
if (!isDirty && knownChildren.isEmpty()) {
return null;
}
// Dirty, try to fetch an updated list of children from ZK.
// Only set a new watcher if there isn't already a watcher.
ChildWatcher newWatcher = (watcherCount == 0) ? new ChildWatcher() : null;
knownChildren = fetchZkChildren(newWatcher);
if (newWatcher != null) {
watcherCount++; // watcher was successfully set
}
isDirty = false;
if (knownChildren.isEmpty()) {
return null;
}
changed.signalAll();
return remove ? knownChildren.pollFirst() : knownChildren.first();
} finally {
updateLock.unlock();
}
}
/**
* Return the current set of children from ZK; does not change internal state.
*/
TreeSet<String> fetchZkChildren(Watcher watcher) throws Exception {
while (true) {
try {
TreeSet<String> orderedChildren = new TreeSet<>();
List<String> childNames = stateManager.listData(dir, watcher);
stats.setQueueLength(childNames.size());
for (String childName : childNames) {
// Check format
if (!childName.regionMatches(0, PREFIX, 0, PREFIX.length())) {
LOG.debug("Found child node with improper name: " + childName);
continue;
}
orderedChildren.add(childName);
}
return orderedChildren;
} catch (NoSuchElementException e) {
try {
stateManager.makePath(dir);
} catch (AlreadyExistsException e2) {
// ignore
}
// go back to the loop and try again
}
}
}
/**
* Return the currently-known set of elements, using child names from memory. If no children are found, or no
* children pass {@code acceptFilter}, waits up to {@code waitMillis} for at least one child to become available.
* <p>
* Package-private to support {@link OverseerTaskQueue} specifically.</p>
*/
@Override
public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
List<String> foundChildren = new ArrayList<>();
long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
boolean first = true;
while (true) {
// Trigger a refresh, but only force it if this is not the first iteration.
firstChild(false, !first);
updateLock.lockInterruptibly();
try {
for (String child : knownChildren) {
if (acceptFilter.test(child)) {
foundChildren.add(child);
}
}
if (!foundChildren.isEmpty()) {
break;
}
if (waitNanos <= 0) {
break;
}
// If this is our first time through, force a refresh before waiting.
if (first) {
first = false;
continue;
}
waitNanos = changed.awaitNanos(waitNanos);
} finally {
updateLock.unlock();
}
if (!foundChildren.isEmpty()) {
break;
}
}
// Technically we could restart the method if we fail to actually obtain any valid children
// from ZK, but this is a super rare case, and the latency of the ZK fetches would require
// much more sophisticated waitNanos tracking.
List<Pair<String, byte[]>> result = new ArrayList<>();
for (String child : foundChildren) {
if (result.size() >= max) {
break;
}
try {
VersionedData data = stateManager.getData(dir + "/" + child);
result.add(new Pair<>(child, data.getData()));
} catch (NoSuchElementException e) {
// Another client deleted the node first, remove the in-memory and continue.
updateLock.lockInterruptibly();
try {
knownChildren.remove(child);
} finally {
updateLock.unlock();
}
}
}
return result;
}
/**
* Return the head of the queue without modifying the queue.
*
* @return the data at the head of the queue.
*/
private byte[] firstElement() throws Exception {
while (true) {
String firstChild = firstChild(false, false);
if (firstChild == null) {
return null;
}
try {
VersionedData data = stateManager.getData(dir + "/" + firstChild);
return data != null ? data.getData() : null;
} catch (NoSuchElementException e) {
// Another client deleted the node first, remove the in-memory and retry.
updateLock.lockInterruptibly();
try {
// Efficient only for single-consumer
knownChildren.clear();
isDirty = true;
} finally {
updateLock.unlock();
}
}
}
}
private byte[] removeFirst() throws Exception {
while (true) {
String firstChild = firstChild(true, false);
if (firstChild == null) {
return null;
}
try {
String path = dir + "/" + firstChild;
VersionedData result = stateManager.getData(path);
stateManager.removeData(path, -1);
stats.setQueueLength(knownChildren.size());
return result.getData();
} catch (NoSuchElementException e) {
// Another client deleted the node first, remove the in-memory and retry.
updateLock.lockInterruptibly();
try {
// Efficient only for single-consumer
knownChildren.clear();
isDirty = true;
} finally {
updateLock.unlock();
}
}
}
}
@VisibleForTesting int watcherCount() throws InterruptedException {
updateLock.lockInterruptibly();
try {
return watcherCount;
} finally {
updateLock.unlock();
}
}
@VisibleForTesting boolean isDirty() throws InterruptedException {
updateLock.lockInterruptibly();
try {
return isDirty;
} finally {
updateLock.unlock();
}
}
@VisibleForTesting class ChildWatcher implements Watcher {
@Override
public void process(WatchedEvent event) {
// session events are not change events, and do not remove the watcher; except for Expired
if (Event.EventType.None.equals(event.getType()) && !Event.KeeperState.Expired.equals(event.getState())) {
return;
}
updateLock.lock();
try {
isDirty = true;
watcherCount--;
// optimistically signal any waiters that the queue may not be empty now, so they can wake up and retry
changed.signalAll();
} finally {
updateLock.unlock();
}
}
}
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
/**
* Factory for {@link GenericDistributedQueue}.
*/
public class GenericDistributedQueueFactory implements DistributedQueueFactory {
private final DistribStateManager stateManager;
public GenericDistributedQueueFactory(DistribStateManager stateManager) {
this.stateManager = stateManager;
}
@Override
public DistributedQueue makeQueue(String path) throws IOException {
return new GenericDistributedQueue(stateManager, path);
}
@Override
public void removeQueue(String path) throws IOException {
}
}

View File

@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.util.Collection;
import java.util.Collections;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.solr.common.cloud.LiveNodesListener;
/**
* This class represents a set of live nodes and allows adding listeners to track their state.
*/
public class LiveNodesSet {
private final Set<String> set = ConcurrentHashMap.newKeySet();
private final Set<LiveNodesListener> listeners = ConcurrentHashMap.newKeySet();
public Set<String> get() {
return Collections.unmodifiableSet(set);
}
public void registerLiveNodesListener(LiveNodesListener listener) {
listeners.add(listener);
}
public void removeLiveNodesListener(LiveNodesListener listener) {
listeners.remove(listener);
}
private void fireListeners(SortedSet<String> oldNodes, SortedSet<String> newNodes) {
for (LiveNodesListener listener : listeners) {
listener.onChange(oldNodes, newNodes);
}
}
public boolean isEmpty() {
return set.isEmpty();
}
public boolean contains(String id) {
return set.contains(id);
}
public synchronized boolean add(String id) {
if (set.contains(id)) {
return false;
}
TreeSet<String> oldNodes = new TreeSet<>(set);
set.add(id);
TreeSet<String> newNodes = new TreeSet<>(set);
fireListeners(oldNodes, newNodes);
return true;
}
public synchronized boolean addAll(Collection<String> nodes) {
TreeSet<String> oldNodes = new TreeSet<>(set);
boolean changed = set.addAll(nodes);
TreeSet<String> newNodes = new TreeSet<>(set);
if (changed) {
fireListeners(oldNodes, newNodes);
}
return changed;
}
public synchronized boolean remove(String id) {
if (!set.contains(id)) {
return false;
}
TreeSet<String> oldNodes = new TreeSet<>(set);
set.remove(id);
TreeSet<String> newNodes = new TreeSet<>(set);
fireListeners(oldNodes, newNodes);
return true;
}
public synchronized void clear() {
TreeSet<String> oldNodes = new TreeSet<>(set);
set.clear();
fireListeners(oldNodes, Collections.emptySortedSet());
}
}

View File

@ -0,0 +1,607 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.solr.client.solrj.SolrClient;
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.autoscaling.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.RequestWriter;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.client.solrj.response.SolrResponseBase;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
import org.apache.solr.cloud.autoscaling.OverseerTriggerThread;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CommonAdminParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ContentStreamBase;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.ObjectCache;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.REQUESTID;
/**
* Simulated {@link SolrCloudManager}.
*/
public class SimCloudManager implements SolrCloudManager {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final SimDistribStateManager stateManager;
private final SimClusterStateProvider clusterStateProvider;
private final SimNodeStateProvider nodeStateProvider;
private final AutoScalingHandler autoScalingHandler;
private final LiveNodesSet liveNodesSet = new LiveNodesSet();
private final DistributedQueueFactory queueFactory;
private final ObjectCache objectCache = new ObjectCache();
private TimeSource timeSource;
private final List<SolrInputDocument> systemColl = Collections.synchronizedList(new ArrayList<>());
private final ExecutorService simCloudManagerPool;
private final Map<String, AtomicLong> opCounts = new ConcurrentHashMap<>();
private Overseer.OverseerThread triggerThread;
private ThreadGroup triggerThreadGroup;
private SolrResourceLoader loader;
private static int nodeIdPort = 10000;
/**
* Create a simulated cluster. This cluster uses the following components:
* <ul>
* <li>{@link SimDistribStateManager} with non-shared root node.</li>
* <li>{@link SimClusterStateProvider}</li>
* <li>{@link SimNodeStateProvider}, where node values are automatically initialized when using
* {@link #simAddNode()} method.</li>
* <li>{@link GenericDistributedQueueFactory} that uses {@link SimDistribStateManager} as its storage.</li>
* <li>an instance of {@link AutoScalingHandler} for managing AutoScalingConfig.</li>
* <li>an instance of {@link OverseerTriggerThread} for managing triggers and processing events.</li>
* </ul>
* @param timeSource time source to use.
*/
public SimCloudManager(TimeSource timeSource) throws Exception {
this.stateManager = new SimDistribStateManager(SimDistribStateManager.createNewRootNode());
this.loader = new SolrResourceLoader();
// init common paths
stateManager.makePath(ZkStateReader.CLUSTER_STATE);
stateManager.makePath(ZkStateReader.CLUSTER_PROPS);
stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
stateManager.makePath(ZkStateReader.LIVE_NODES_ZKNODE);
stateManager.makePath(ZkStateReader.ROLES);
stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
this.timeSource = timeSource != null ? timeSource : TimeSource.NANO_TIME;
this.clusterStateProvider = new SimClusterStateProvider(liveNodesSet, this);
this.nodeStateProvider = new SimNodeStateProvider(liveNodesSet, this.stateManager, this.clusterStateProvider, null);
this.queueFactory = new GenericDistributedQueueFactory(stateManager);
this.simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
this.autoScalingHandler = new AutoScalingHandler(this, loader);
triggerThreadGroup = new ThreadGroup("Simulated Overseer autoscaling triggers");
OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
triggerThread.start();
}
// ---------- simulator setup methods -----------
/**
* Create a cluster with the specified number of nodes. Node metrics are pre-populated.
* @param numNodes number of nodes to create
* @param timeSource time source
* @return instance of simulated cluster
*/
public static SimCloudManager createCluster(int numNodes, TimeSource timeSource) throws Exception {
SimCloudManager cloudManager = new SimCloudManager(timeSource);
for (int i = 1; i <= numNodes; i++) {
Map<String, Object> values = createNodeValues(null);
// if (i == 1) { // designated Overseer ?
//values.put(ImplicitSnitch.NODEROLE, "overseer");
// }
String nodeId = (String)values.get(ImplicitSnitch.NODE);
cloudManager.getSimClusterStateProvider().simAddNode(nodeId);
cloudManager.getSimNodeStateProvider().simSetNodeValues(nodeId, values);
}
return cloudManager;
}
/**
* Create a cluster initialized from the provided cluster state.
* @param initialState existing cluster state
* @param timeSource time source
* @return instance of simulated cluster with the same layout as the provided cluster state.
*/
public static SimCloudManager createCluster(ClusterState initialState, TimeSource timeSource) throws Exception {
SimCloudManager cloudManager = new SimCloudManager(timeSource);
cloudManager.getSimClusterStateProvider().simSetClusterState(initialState);
for (String node : cloudManager.getClusterStateProvider().getLiveNodes()) {
cloudManager.getSimNodeStateProvider().simSetNodeValues(node, createNodeValues(node));
}
return cloudManager;
}
/**
* Create simulated node values (metrics) for a node.
* @param nodeName node name (eg. '127.0.0.1:10000_solr'). If null then a new node name will be
* created using sequentially increasing port number.
* @return node values
*/
public static Map<String, Object> createNodeValues(String nodeName) {
Map<String, Object> values = new HashMap<>();
String host, nodeId;
int port;
if (nodeName == null) {
host = "127.0.0.1";
port = nodeIdPort++;
nodeId = host + ":" + port + "_solr";
values.put("ip_1", "127");
values.put("ip_2", "0");
values.put("ip_3", "0");
values.put("ip_4", "1");
} else {
String[] hostPortCtx = nodeName.split(":");
if (hostPortCtx.length != 2) {
throw new RuntimeException("Invalid nodeName " + nodeName);
}
host = hostPortCtx[0];
String[] portCtx = hostPortCtx[1].split("_");
if (portCtx.length != 2) {
throw new RuntimeException("Invalid port_context in nodeName " + nodeName);
}
port = Integer.parseInt(portCtx[0]);
nodeId = host + ":" + port + "_" + portCtx[1];
String[] ip = host.split("\\.");
if (ip.length == 4) {
values.put("ip_1", ip[0]);
values.put("ip_2", ip[1]);
values.put("ip_3", ip[2]);
values.put("ip_4", ip[3]);
}
}
values.put(ImplicitSnitch.HOST, host);
values.put(ImplicitSnitch.PORT, port);
values.put(ImplicitSnitch.NODE, nodeId);
values.put(ImplicitSnitch.CORES, 0);
values.put(ImplicitSnitch.DISK, 1000);
values.put(ImplicitSnitch.SYSLOADAVG, 1.0);
values.put(ImplicitSnitch.HEAPUSAGE, 123450000);
values.put("sysprop.java.version", System.getProperty("java.version"));
values.put("sysprop.java.vendor", System.getProperty("java.vendor"));
// fake some metrics expected in tests
values.put("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count", 0);
values.put("metrics:solr.jvm:buffers.direct.Count", 0);
return values;
}
/**
* Get the instance of {@link SolrResourceLoader} that is used by the cluster components.
*/
public SolrResourceLoader getLoader() {
return loader;
}
/**
* Add a new node and initialize its node values (metrics). The
* /live_nodes list is updated with the new node id.
* @return new node id
*/
public String simAddNode() throws Exception {
Map<String, Object> values = createNodeValues(null);
String nodeId = (String)values.get(ImplicitSnitch.NODE);
clusterStateProvider.simAddNode(nodeId);
nodeStateProvider.simSetNodeValues(nodeId, values);
LOG.trace("-- added node " + nodeId);
return nodeId;
}
/**
* Remove a node from the cluster. This simulates a node lost scenario.
* Node id is removed from the /live_nodes list.
* @param nodeId node id
* @param withValues when true, remove also simulated node values. If false
* then node values are retained to later simulate
* a node that comes back up
*/
public void simRemoveNode(String nodeId, boolean withValues) throws Exception {
clusterStateProvider.simRemoveNode(nodeId);
if (withValues) {
nodeStateProvider.simRemoveNodeValues(nodeId);
}
LOG.trace("-- removed node " + nodeId);
}
/**
* Remove a number of randomly selected nodes
* @param number number of nodes to remove
* @param withValues when true, remove also simulated node values. If false
* then node values are retained to later simulate
* a node that comes back up
* @param random random
*/
public void simRemoveRandomNodes(int number, boolean withValues, Random random) throws Exception {
List<String> nodes = new ArrayList<>(liveNodesSet.get());
Collections.shuffle(nodes, random);
int count = Math.min(number, nodes.size());
for (int i = 0; i < count; i++) {
simRemoveNode(nodes.get(i), withValues);
}
}
/**
* Clear the (simulated) .system collection.
*/
public void simClearSystemCollection() {
systemColl.clear();
}
/**
* Get the content of (simulated) .system collection.
* @return documents in the collection, in chronological order starting from the oldest.
*/
public List<SolrInputDocument> simGetSystemCollection() {
return systemColl;
}
/**
* Get a {@link SolrClient} implementation where calls are forwarded to this
* instance of the cluster.
* @return simulated SolrClient.
*/
public SolrClient simGetSolrClient() {
return new SolrClient() {
@Override
public NamedList<Object> request(SolrRequest request, String collection) throws SolrServerException, IOException {
SolrResponse rsp = SimCloudManager.this.request(request);
return rsp.getResponse();
}
@Override
public void close() throws IOException {
}
};
}
/**
* Simulate the effect of restarting Overseer leader - in this case this means restarting the
* OverseerTriggerThread and optionally killing a node.
* @param killNodeId optional nodeId to kill. If null then don't kill any node, just restart the thread
*/
public void simRestartOverseer(String killNodeId) throws Exception {
LOG.info("=== Restarting OverseerTriggerThread and clearing object cache...");
triggerThread.interrupt();
IOUtils.closeQuietly(triggerThread);
if (killNodeId != null) {
simRemoveNode(killNodeId, true);
}
objectCache.clear();
OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
triggerThread.start();
}
/**
* Submit a task to execute in a thread pool.
* @param callable task to execute
* @return future to obtain results
*/
public <T> Future<T> submit(Callable<T> callable) {
return simCloudManagerPool.submit(callable);
}
// ---------- type-safe methods to obtain simulator components ----------
public SimClusterStateProvider getSimClusterStateProvider() {
return clusterStateProvider;
}
public SimNodeStateProvider getSimNodeStateProvider() {
return nodeStateProvider;
}
public SimDistribStateManager getSimDistribStateManager() {
return stateManager;
}
public LiveNodesSet getLiveNodesSet() {
return liveNodesSet;
}
/**
* Get the number and type of operations processed by this cluster.
*/
public Map<String, AtomicLong> simGetOpCounts() {
return opCounts;
}
/**
* Get the number of processed operations of a specified type.
* @param op operation name, eg. MOVEREPLICA
* @return number of operations
*/
public long simGetOpCount(String op) {
AtomicLong count = opCounts.get(op);
return count != null ? count.get() : 0L;
}
// --------- interface methods -----------
@Override
public ObjectCache getObjectCache() {
return objectCache;
}
@Override
public TimeSource getTimeSource() {
return timeSource;
}
@Override
public ClusterStateProvider getClusterStateProvider() {
return clusterStateProvider;
}
@Override
public NodeStateProvider getNodeStateProvider() {
return nodeStateProvider;
}
@Override
public DistribStateManager getDistribStateManager() {
return stateManager;
}
@Override
public DistributedQueueFactory getDistributedQueueFactory() {
return queueFactory;
}
@Override
public SolrResponse request(SolrRequest req) throws IOException {
try {
Future<SolrResponse> rsp = submit(() -> simHandleSolrRequest(req));
return rsp.get();
} catch (Exception e) {
throw new IOException(e);
}
}
private void incrementCount(String op) {
AtomicLong count = opCounts.computeIfAbsent(op, o -> new AtomicLong());
count.incrementAndGet();
}
/**
* Handler method for autoscaling requests. NOTE: only a specific subset of autoscaling requests is
* supported!
* @param req autoscaling request
* @return results
*/
public SolrResponse simHandleSolrRequest(SolrRequest req) throws IOException, InterruptedException {
// pay the penalty for remote request, at least 5 ms
timeSource.sleep(5);
LOG.trace("--- got SolrRequest: " + req.getMethod() + " " + req.getPath() +
(req.getParams() != null ? " " + req.getParams().toQueryString() : ""));
if (req.getPath() != null && req.getPath().startsWith("/admin/autoscaling") ||
req.getPath().startsWith("/cluster/autoscaling")) {
incrementCount("autoscaling");
ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
params.set(CommonParams.PATH, req.getPath());
LocalSolrQueryRequest queryRequest = new LocalSolrQueryRequest(null, params);
RequestWriter.ContentWriter cw = req.getContentWriter("application/json");
ByteArrayOutputStream baos = new ByteArrayOutputStream();
cw.write(baos);
String payload = baos.toString("UTF-8");
LOG.trace("-- payload: {}", payload);
queryRequest.setContentStreams(Collections.singletonList(new ContentStreamBase.StringStream(payload)));
queryRequest.getContext().put("httpMethod", req.getMethod().toString());
SolrQueryResponse queryResponse = new SolrQueryResponse();
autoScalingHandler.handleRequest(queryRequest, queryResponse);
if (queryResponse.getException() != null) {
throw new IOException(queryResponse.getException());
}
SolrResponse rsp = new SolrResponseBase();
rsp.setResponse(queryResponse.getValues());
LOG.trace("-- response: {}", rsp);
return rsp;
}
if (req instanceof UpdateRequest) {
incrementCount("update");
// support only updates to the system collection
UpdateRequest ureq = (UpdateRequest)req;
if (ureq.getCollection() == null || !ureq.getCollection().equals(CollectionAdminParams.SYSTEM_COLL)) {
throw new UnsupportedOperationException("Only .system updates are supported but got: " + req);
}
List<SolrInputDocument> docs = ureq.getDocuments();
if (docs != null) {
systemColl.addAll(docs);
}
return new UpdateResponse();
}
// support only a specific subset of collection admin ops
if (!(req instanceof CollectionAdminRequest)) {
throw new UnsupportedOperationException("Only some CollectionAdminRequest-s are supported: " + req.getClass().getName());
}
SolrParams params = req.getParams();
String a = params.get(CoreAdminParams.ACTION);
SolrResponse rsp = new SolrResponseBase();
rsp.setResponse(new NamedList<>());
if (a != null) {
CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(a);
if (action == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a);
}
LOG.debug("Invoking Collection Action :{} with params {}", action.toLower(), req.getParams().toQueryString());
NamedList results = new NamedList();
rsp.setResponse(results);
incrementCount(action.name());
switch (action) {
case REQUESTSTATUS:
// we complete all async ops immediately
String requestId = req.getParams().get(REQUESTID);
SimpleOrderedMap<String> status = new SimpleOrderedMap<>();
status.add("state", RequestStatusState.COMPLETED.getKey());
status.add("msg", "found [" + requestId + "] in completed tasks");
results.add("status", status);
results.add("success", "");
// ExecutePlanAction expects a specific response class
rsp = new CollectionAdminRequest.RequestStatusResponse();
rsp.setResponse(results);
break;
case DELETESTATUS:
requestId = req.getParams().get(REQUESTID);
results.add("status", "successfully removed stored response for [" + requestId + "]");
results.add("success", "");
break;
case CREATE:
try {
clusterStateProvider.simCreateCollection(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
break;
case DELETE:
clusterStateProvider.simDeleteCollection(req.getParams().get(CommonParams.NAME),
req.getParams().get(CommonAdminParams.ASYNC), results);
break;
case LIST:
results.add("collections", clusterStateProvider.simListCollections());
break;
case ADDREPLICA:
try {
clusterStateProvider.simAddReplica(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
break;
case MOVEREPLICA:
try {
clusterStateProvider.simMoveReplica(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
break;
case OVERSEERSTATUS:
if (req.getParams().get(CommonAdminParams.ASYNC) != null) {
results.add(REQUESTID, req.getParams().get(CommonAdminParams.ASYNC));
}
if (!liveNodesSet.get().isEmpty()) {
results.add("leader", liveNodesSet.get().iterator().next());
}
results.add("overseer_queue_size", 0);
results.add("overseer_work_queue_size", 0);
results.add("overseer_collection_queue_size", 0);
results.add("success", "");
break;
case ADDROLE:
nodeStateProvider.simAddNodeValue(req.getParams().get("node"), "nodeRole", req.getParams().get("role"));
break;
case CREATESHARD:
try {
clusterStateProvider.simCreateShard(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
break;
case SPLITSHARD:
try {
clusterStateProvider.simSplitShard(new ZkNodeProps(req.getParams().toNamedList().asMap(10)), results);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
}
break;
default:
throw new UnsupportedOperationException("Unsupported collection admin action=" + action + " in request: " + req.getParams());
}
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "action is a required param in request: " + req.getParams());
}
return rsp;
}
/**
* HTTP requests are not supported by this implementation.
*/
@Override
public byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException {
throw new UnsupportedOperationException("general HTTP requests are not supported yet");
}
@Override
public void close() throws IOException {
IOUtils.closeQuietly(clusterStateProvider);
IOUtils.closeQuietly(nodeStateProvider);
IOUtils.closeQuietly(stateManager);
triggerThread.interrupt();
IOUtils.closeQuietly(triggerThread);
IOUtils.closeQuietly(objectCache);
simCloudManagerPool.shutdownNow();
}
}

View File

@ -0,0 +1,580 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.jute.Record;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.cloud.ActionThrottle;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.IdUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.CheckVersionRequest;
import org.apache.zookeeper.proto.CreateRequest;
import org.apache.zookeeper.proto.DeleteRequest;
import org.apache.zookeeper.proto.SetDataRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simulated {@link DistribStateManager} that keeps all data locally in a static structure. Instances of this
* class are identified by their id in order to simulate the deletion of ephemeral nodes when {@link #close()} is
* invoked.
*/
public class SimDistribStateManager implements DistribStateManager {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final class Node {
ReentrantLock dataLock = new ReentrantLock();
private int version = -1;
private int seq = 0;
private final CreateMode mode;
private final String clientId;
private final String path;
private final String name;
private final Node parent;
private byte[] data = null;
private Map<String, Node> children = new ConcurrentHashMap<>();
Set<Watcher> dataWatches = ConcurrentHashMap.newKeySet();
Set<Watcher> childrenWatches = ConcurrentHashMap.newKeySet();
Node(Node parent, String name, String path, CreateMode mode, String clientId) {
this.parent = parent;
this.name = name;
this.path = path;
this.mode = mode;
this.clientId = clientId;
}
public void clear() {
dataLock.lock();
try {
children.clear();
version = 0;
seq = 0;
dataWatches.clear();
childrenWatches.clear();
data = null;
} finally {
dataLock.unlock();
}
}
public void setData(byte[] data, int version) throws BadVersionException, IOException {
Set<Watcher> currentWatchers = new HashSet<>(dataWatches);
dataLock.lock();
try {
if (version != -1 && version != this.version) {
throw new BadVersionException(version, path);
}
if (data != null) {
this.data = Arrays.copyOf(data, data.length);
} else {
this.data = null;
}
this.version++;
dataWatches.clear();
} finally {
dataLock.unlock();
}
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeDataChanged, Watcher.Event.KeeperState.SyncConnected, path));
}
}
public VersionedData getData(Watcher w) {
dataLock.lock();
try {
VersionedData res = new VersionedData(version, data, clientId);
if (w != null && !dataWatches.contains(w)) {
dataWatches.add(w);
}
return res;
} finally {
dataLock.unlock();
}
}
public void setChild(String name, Node child) {
assert child.name.equals(name);
Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
dataLock.lock();
try {
children.put(name, child);
childrenWatches.clear();
} finally {
dataLock.unlock();
}
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
}
}
public void removeChild(String name, int version) throws NoSuchElementException, BadVersionException, IOException {
Node n = children.get(name);
if (n == null) {
throw new NoSuchElementException(path + "/" + name);
}
if (version != -1 && version != n.version) {
throw new BadVersionException(version, path);
}
children.remove(name);
Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
childrenWatches.clear();
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
}
currentWatchers = new HashSet<>(n.dataWatches);
n.dataWatches.clear();
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeDeleted, Watcher.Event.KeeperState.SyncConnected, n.path));
}
// TODO: not sure if it's correct to recurse and fire watches???
Set<String> kids = new HashSet<>(n.children.keySet());
for (String kid : kids) {
n.removeChild(kid, -1);
}
}
public void removeEphemeralChildren(String id) throws NoSuchElementException, BadVersionException, IOException {
Set<String> kids = new HashSet<>(children.keySet());
for (String kid : kids) {
Node n = children.get(kid);
if (n == null) {
continue;
}
if ((CreateMode.EPHEMERAL == n.mode || CreateMode.EPHEMERAL_SEQUENTIAL == n.mode) &&
id.equals(n.clientId)) {
removeChild(n.name, -1);
} else {
n.removeEphemeralChildren(id);
}
}
}
}
private final ReentrantLock multiLock = new ReentrantLock();
public static Node createNewRootNode() {
return new Node(null, "", "/", CreateMode.PERSISTENT, "__root__");
}
private final ExecutorService watchersPool;
private final AtomicReference<ActionThrottle> throttleRef = new AtomicReference<>();
private final AtomicReference<ActionError> errorRef = new AtomicReference<>();
private final String id;
private final Node root;
public SimDistribStateManager() {
this(null);
}
/**
* Construct new state manager that uses provided root node for storing data.
* @param root if null then a new root node will be created.
*/
public SimDistribStateManager(Node root) {
this.id = IdUtils.timeRandomId();
this.root = root != null ? root : createNewRootNode();
watchersPool = ExecutorUtil.newMDCAwareFixedThreadPool(10, new DefaultSolrThreadFactory("sim-watchers"));
}
public SimDistribStateManager(ActionThrottle actionThrottle, ActionError actionError) {
this(null, actionThrottle, actionError);
}
public SimDistribStateManager(Node root, ActionThrottle actionThrottle, ActionError actionError) {
this(root);
this.throttleRef.set(actionThrottle);
this.errorRef.set(actionError);
}
private SimDistribStateManager(String id, ExecutorService watchersPool, Node root, ActionThrottle actionThrottle,
ActionError actionError) {
this.id = id;
this.watchersPool = watchersPool;
this.root = root;
this.throttleRef.set(actionThrottle);
this.errorRef.set(actionError);
}
/**
* Create a copy of this instance using a specified ephemeral owner id. This is useful when performing
* node operations that require using a specific id. Note: this instance should never be closed, it can
* be just discarded after use.
* @param id ephemeral owner id
*/
public SimDistribStateManager withEphemeralId(String id) {
return new SimDistribStateManager(id, watchersPool, root, throttleRef.get(), errorRef.get()) {
@Override
public void close() {
throw new UnsupportedOperationException("this instance should never be closed - instead close the parent instance.");
}
};
}
/**
* Get the root node of the tree used by this instance. It could be a static shared root node.
*/
public Node getRoot() {
return root;
}
/**
* Clear this instance. All nodes, watchers and data is deleted.
*/
public void clear() {
root.clear();
}
private void throttleOrError(String path) throws IOException {
ActionError err = errorRef.get();
if (err != null && err.shouldFail(path)) {
throw new IOException("Simulated error, path=" + path);
}
ActionThrottle throttle = throttleRef.get();
if (throttle != null) {
throttle.minimumWaitBetweenActions();
throttle.markAttemptingAction();
}
}
// this method should always be invoked under lock
private Node traverse(String path, boolean create, CreateMode mode) throws IOException {
if (path == null || path.isEmpty()) {
return null;
}
throttleOrError(path);
if (path.charAt(0) == '/') {
path = path.substring(1);
}
StringBuilder currentPath = new StringBuilder();
String[] elements = path.split("/");
Node parentNode = root;
Node n = null;
for (int i = 0; i < elements.length; i++) {
String currentName = elements[i];
currentPath.append('/');
LOG.info(" - parentNode=" + parentNode + ", currentName=" + currentName);
n = parentNode.children != null ? parentNode.children.get(currentName) : null;
if (n == null) {
if (create) {
if ((parentNode.mode == CreateMode.EPHEMERAL || parentNode.mode == CreateMode.EPHEMERAL_SEQUENTIAL) &&
(mode == CreateMode.EPHEMERAL || mode == CreateMode.EPHEMERAL_SEQUENTIAL)) {
throw new IOException("NoChildrenEphemerals for " + parentNode.path);
}
if (CreateMode.PERSISTENT_SEQUENTIAL == mode || CreateMode.EPHEMERAL_SEQUENTIAL == mode) {
currentName = currentName + String.format(Locale.ROOT, "%010d", parentNode.seq);
parentNode.seq++;
}
currentPath.append(currentName);
n = new Node(parentNode, currentName, currentPath.toString(), mode, id);
parentNode.setChild(currentName, n);
} else {
break;
}
} else {
currentPath.append(currentName);
}
parentNode = n;
}
return n;
}
@Override
public void close() throws IOException {
multiLock.lock();
try {
// remove all my ephemeral nodes
root.removeEphemeralChildren(id);
} catch (BadVersionException e) {
// not happening
} finally {
multiLock.unlock();
}
}
@Override
public boolean hasData(String path) throws IOException {
multiLock.lock();
try {
return traverse(path, false, CreateMode.PERSISTENT) != null;
} finally {
multiLock.unlock();
}
}
@Override
public List<String> listData(String path) throws NoSuchElementException, IOException {
multiLock.lock();
try {
Node n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
List<String> res = new ArrayList<>(n.children.keySet());
Collections.sort(res);
return res;
} finally {
multiLock.unlock();
}
}
@Override
public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException {
Node n;
List<String> res;
multiLock.lock();
try {
n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
res = new ArrayList<>(n.children.keySet());
Collections.sort(res);
} finally {
multiLock.unlock();
}
if (watcher != null) {
n.dataWatches.add(watcher);
n.childrenWatches.add(watcher);
}
return res;
}
@Override
public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException {
Node n = null;
multiLock.lock();
try {
n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
} finally {
multiLock.unlock();
}
return n.getData(watcher);
}
@Override
public void makePath(String path) throws IOException {
multiLock.lock();
try {
traverse(path, true, CreateMode.PERSISTENT);
} finally {
multiLock.unlock();
}
}
@Override
public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
Node n = null;
multiLock.lock();
try {
if (failOnExists && hasData(path)) {
throw new AlreadyExistsException(path);
}
n = traverse(path, true, createMode);
} finally {
multiLock.unlock();
}
try {
n.setData(data, -1);
} catch (BadVersionException e) {
throw new IOException("should not happen!", e);
}
}
@Override
public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, NoSuchElementException, IOException {
if ((CreateMode.EPHEMERAL == mode || CreateMode.PERSISTENT == mode) && hasData(path)) {
throw new AlreadyExistsException(path);
}
// check if parent exists
String relPath = path.charAt(0) == '/' ? path.substring(1) : path;
if (relPath.length() > 0) { // non-root path - check if parent exists
String[] elements = relPath.split("/");
StringBuilder sb = new StringBuilder();
for (int i = 0; i < elements.length - 1; i++) {
sb.append('/');
sb.append(elements[i]);
}
if (!hasData(sb.toString())) {
throw new NoSuchElementException(sb.toString());
}
}
Node n = null;
multiLock.lock();
try {
n = traverse(path, true, mode);
} finally {
multiLock.unlock();
}
try {
n.setData(data, -1);
return n.path;
} catch (BadVersionException e) {
// not happening
return null;
}
}
@Override
public void removeData(String path, int version) throws NoSuchElementException, BadVersionException, IOException {
multiLock.lock();
try {
Node n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
Node parent = n.parent;
if (parent == null) {
throw new IOException("Cannot remove root node");
}
parent.removeChild(n.name, version);
} finally {
multiLock.unlock();
}
}
@Override
public void setData(String path, byte[] data, int version) throws NoSuchElementException, BadVersionException, IOException {
multiLock.lock();
Node n = null;
try {
n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
} finally {
multiLock.unlock();
}
n.setData(data, version);
}
@Override
public List<OpResult> multi(Iterable<Op> ops) throws BadVersionException, NoSuchElementException, AlreadyExistsException, IOException, KeeperException, InterruptedException {
multiLock.lock();
List<OpResult> res = new ArrayList<>();
try {
for (Op op : ops) {
Record r = op.toRequestRecord();
try {
if (op instanceof Op.Check) {
CheckVersionRequest rr = (CheckVersionRequest)r;
Node n = traverse(rr.getPath(), false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(rr.getPath());
}
if (rr.getVersion() != -1 && n.version != rr.getVersion()) {
throw new Exception("version mismatch");
}
// everything ok
res.add(new OpResult.CheckResult());
} else if (op instanceof Op.Create) {
CreateRequest rr = (CreateRequest)r;
createData(rr.getPath(), rr.getData(), CreateMode.fromFlag(rr.getFlags()));
res.add(new OpResult.CreateResult(rr.getPath()));
} else if (op instanceof Op.Delete) {
DeleteRequest rr = (DeleteRequest)r;
removeData(rr.getPath(), rr.getVersion());
res.add(new OpResult.DeleteResult());
} else if (op instanceof Op.SetData) {
SetDataRequest rr = (SetDataRequest)r;
setData(rr.getPath(), rr.getData(), rr.getVersion());
VersionedData vd = getData(rr.getPath());
Stat s = new Stat();
s.setVersion(vd.getVersion());
res.add(new OpResult.SetDataResult(s));
} else {
throw new Exception("Unknown Op: " + op);
}
} catch (Exception e) {
res.add(new OpResult.ErrorResult(KeeperException.Code.APIERROR.intValue()));
}
}
} finally {
multiLock.unlock();
}
return res;
}
@Override
public AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws InterruptedException, IOException {
Map<String, Object> map = new HashMap<>();
int version = -1;
try {
VersionedData data = getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, watcher);
if (data != null && data.getData() != null && data.getData().length > 0) {
map = (Map<String, Object>) Utils.fromJSON(data.getData());
version = data.getVersion();
}
} catch (NoSuchElementException e) {
// ignore
}
map.put(AutoScalingParams.ZK_VERSION, version);
return new AutoScalingConfig(map);
}
// ------------ simulator methods --------------
public void simSetAutoScalingConfig(AutoScalingConfig cfg) throws Exception {
try {
makePath(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
} catch (Exception e) {
// ignore
}
setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(cfg), -1);
}
}

View File

@ -0,0 +1,284 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Predicate;
import com.codahale.metrics.Timer;
import com.google.common.base.Preconditions;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.util.Pair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simulated {@link DistributedQueueFactory} that keeps all data in memory. Unlike
* the {@link GenericDistributedQueueFactory} this queue implementation data is not
* exposed anywhere.
*/
public class SimDistributedQueueFactory implements DistributedQueueFactory {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
Map<String, SimDistributedQueue> queues = new ConcurrentHashMap<>();
public SimDistributedQueueFactory() {
}
@Override
public DistributedQueue makeQueue(final String path) throws IOException {
return queues.computeIfAbsent(path, p -> new SimDistributedQueue(path));
}
@Override
public void removeQueue(String path) throws IOException {
queues.remove(path);
}
public static class SimDistributedQueue implements DistributedQueue {
private final Queue<Pair<String, byte[]>> queue = new ConcurrentLinkedQueue<>();
private final ReentrantLock updateLock = new ReentrantLock();
private final Condition changed = updateLock.newCondition();
private final Stats stats = new Stats();
private final String dir;
private int seq = 0;
public SimDistributedQueue(String dir) {
this.dir = dir;
}
@Override
public byte[] peek() throws Exception {
Timer.Context time = stats.time(dir + "_peek");
try {
Pair<String, byte[]> pair = queue.peek();
return pair != null ? pair.second() : null;
} finally {
time.stop();
}
}
@Override
public byte[] peek(boolean block) throws Exception {
return block ? peek(Long.MAX_VALUE) : peek();
}
@Override
public byte[] peek(long wait) throws Exception {
Timer.Context time;
if (wait == Long.MAX_VALUE) {
time = stats.time(dir + "_peek_wait_forever");
} else {
time = stats.time(dir + "_peek_wait" + wait);
}
try {
Pair<String, byte[]> pair = peekInternal(wait);
return pair != null ? pair.second() : null;
} finally {
time.stop();
}
}
private Pair<String, byte[]> peekInternal(long wait) throws Exception {
Preconditions.checkArgument(wait > 0);
long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
updateLock.lockInterruptibly();
try {
while (waitNanos > 0) {
Pair<String, byte[]> pair = queue.peek();
if (pair != null) {
return pair;
}
waitNanos = changed.awaitNanos(waitNanos);
if (waitNanos < 0) { // timed out
return null;
}
}
} finally {
updateLock.unlock();
}
return null;
}
@Override
public byte[] poll() throws Exception {
Timer.Context time = stats.time(dir + "_poll");
updateLock.lockInterruptibly();
try {
Pair<String, byte[]> pair = queue.poll();
if (pair != null) {
changed.signalAll();
return pair.second();
} else {
return null;
}
} finally {
updateLock.unlock();
time.stop();
}
}
@Override
public byte[] remove() throws Exception {
Timer.Context time = stats.time(dir + "_remove");
updateLock.lockInterruptibly();
try {
byte[] res = queue.remove().second();
changed.signalAll();
return res;
} finally {
updateLock.unlock();
time.stop();
}
}
@Override
public byte[] take() throws Exception {
Timer.Context timer = stats.time(dir + "_take");
updateLock.lockInterruptibly();
try {
while (true) {
byte[] result = poll();
if (result != null) {
return result;
}
changed.await();
}
} finally {
updateLock.unlock();
timer.stop();
}
}
@Override
public void offer(byte[] data) throws Exception {
Timer.Context time = stats.time(dir + "_offer");
updateLock.lockInterruptibly();
try {
queue.offer(new Pair(String.format(Locale.ROOT, "qn-%010d", seq), data));
seq++;
LOG.trace("=== offer " + System.nanoTime());
changed.signalAll();
} finally {
updateLock.unlock();
time.stop();
}
}
@Override
public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
updateLock.lockInterruptibly();
try {
List<Pair<String, byte[]>> res = new LinkedList<>();
final int maximum = max < 0 ? Integer.MAX_VALUE : max;
final AtomicReference<Pair<String, byte[]>> pairRef = new AtomicReference<>();
queue.forEach(pair -> {
if (acceptFilter != null && !acceptFilter.test(pair.first())) {
return;
}
if (res.size() < maximum) {
pairRef.set(pair);
res.add(pair);
}
});
if (res.size() < maximum && waitMillis > 0) {
long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
waitNanos = changed.awaitNanos(waitNanos);
if (waitNanos < 0) {
return res;
}
AtomicBoolean seen = new AtomicBoolean(false);
queue.forEach(pair -> {
if (!seen.get()) {
if (pairRef.get() == null) {
seen.set(true);
} else {
if (pairRef.get().first().equals(pair.first())) {
seen.set(true);
return;
}
}
}
if (!seen.get()) {
return;
}
if (!acceptFilter.test(pair.first())) {
return;
}
if (res.size() < maximum) {
res.add(pair);
pairRef.set(pair);
} else {
return;
}
});
}
return res;
} finally {
updateLock.unlock();
}
}
public Stats getZkStats() {
return stats;
}
@Override
public Map<String, Object> getStats() {
if (stats == null) {
return Collections.emptyMap();
}
Map<String, Object> res = new HashMap<>();
res.put("queueLength", stats.getQueueLength());
final Map<String, Object> statsMap = new HashMap<>();
res.put("stats", statsMap);
stats.getStats().forEach((op, stat) -> {
final Map<String, Object> statMap = new HashMap<>();
statMap.put("success", stat.success.get());
statMap.put("errors", stat.errors.get());
final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
statMap.put("failureDetails", failed);
stat.failureDetails.forEach(failedOp -> {
Map<String, Object> fo = new HashMap<>();
fo.put("req", failedOp.req);
fo.put("resp", failedOp.resp);
});
statsMap.put(op, statMap);
});
return res;
}
}
}

View File

@ -0,0 +1,267 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.cloud.autoscaling.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simulated {@link NodeStateProvider}.
* Note: in order to setup node-level metrics use {@link #simSetNodeValues(String, Map)}. However, in order
* to setup core-level metrics use {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)}.
*/
public class SimNodeStateProvider implements NodeStateProvider {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final Map<String, Map<String, Object>> nodeValues = new ConcurrentHashMap<>();
private final SimClusterStateProvider clusterStateProvider;
private final SimDistribStateManager stateManager;
private final LiveNodesSet liveNodesSet;
public SimNodeStateProvider(LiveNodesSet liveNodesSet, SimDistribStateManager stateManager,
SimClusterStateProvider clusterStateProvider,
Map<String, Map<String, Object>> nodeValues) {
this.liveNodesSet = liveNodesSet;
this.stateManager = stateManager;
this.clusterStateProvider = clusterStateProvider;
if (nodeValues != null) {
this.nodeValues.putAll(nodeValues);
}
}
// -------- simulator setup methods ------------
/**
* Get a node value
* @param node node id
* @param key property name
* @return property value or null if property or node doesn't exist.
*/
public Object simGetNodeValue(String node, String key) {
Map<String, Object> values = nodeValues.get(node);
if (values == null) {
return null;
}
return values.get(key);
}
/**
* Set node values.
* NOTE: if values contain 'nodeRole' key then /roles.json is updated.
* @param node node id
* @param values values.
*/
public void simSetNodeValues(String node, Map<String, Object> values) {
Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
existing.clear();
if (values != null) {
existing.putAll(values);
}
if (values == null || values.isEmpty() || values.containsKey("nodeRole")) {
saveRoles();
}
}
/**
* Set a node value, replacing any previous value.
* NOTE: if key is 'nodeRole' then /roles.json is updated.
* @param node node id
* @param key property name
* @param value property value
*/
public void simSetNodeValue(String node, String key, Object value) {
Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
if (value == null) {
existing.remove(key);
} else {
existing.put(key, value);
}
if (key.equals("nodeRole")) {
saveRoles();
}
}
/**
* Add a node value, creating a list of values if necessary.
* NOTE: if key is 'nodeRole' then /roles.json is updated.
* @param node node id
* @param key property name
* @param value property value.
*/
public void simAddNodeValue(String node, String key, Object value) {
Map<String, Object> values = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
Object existing = values.get(key);
if (existing == null) {
values.put(key, value);
} else if (existing instanceof Set) {
((Set)existing).add(value);
} else {
Set<Object> vals = new HashSet<>();
vals.add(existing);
vals.add(value);
values.put(key, vals);
}
if (key.equals("nodeRole")) {
saveRoles();
}
}
/**
* Remove node values. If values contained a 'nodeRole' key then
* /roles.json is updated.
* @param node node id
*/
public void simRemoveNodeValues(String node) {
Map<String, Object> values = nodeValues.remove(node);
if (values != null && values.containsKey("nodeRole")) {
saveRoles();
}
}
/**
* Get all node values.
*/
public Map<String, Map<String, Object>> simGetAllNodeValues() {
return nodeValues;
}
private synchronized void saveRoles() {
final Map<String, Set<String>> roles = new HashMap<>();
nodeValues.forEach((n, values) -> {
String nodeRole = (String)values.get("nodeRole");
if (nodeRole != null) {
roles.computeIfAbsent(nodeRole, role -> new HashSet<>()).add(n);
}
});
try {
stateManager.setData(ZkStateReader.ROLES, Utils.toJSON(roles), -1);
} catch (Exception e) {
throw new RuntimeException("Unexpected exception saving roles " + roles, e);
}
}
/**
* Simulate getting replica metrics values. This uses per-replica properties set in
* {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean)} and
* similar methods.
* @param node node id
* @param tags metrics names
* @return map of metrics names / values
*/
public Map<String, Object> getReplicaMetricsValues(String node, Collection<String> tags) {
List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
if (replicas == null || replicas.isEmpty()) {
return Collections.emptyMap();
}
Map<String, Object> values = new HashMap<>();
for (String tag : tags) {
String[] parts = tag.split(":");
if (parts.length < 3 || !parts[0].equals("metrics")) {
LOG.warn("Invalid metrics: tag: " + tag);
continue;
}
if (!parts[1].startsWith("solr.core.")) {
// skip - this is probably solr.node or solr.jvm metric
continue;
}
String[] collParts = parts[1].substring(10).split("\\.");
if (collParts.length != 3) {
LOG.warn("Invalid registry name: " + parts[1]);
continue;
}
String collection = collParts[0];
String shard = collParts[1];
String replica = collParts[2];
String key = parts.length > 3 ? parts[2] + ":" + parts[3] : parts[2];
replicas.forEach(r -> {
if (r.getCollection().equals(collection) && r.getShard().equals(shard) && r.getCore().endsWith(replica)) {
Object value = r.getVariables().get(key);
if (value != null) {
values.put(tag, value);
} else {
value = r.getVariables().get(tag);
if (value != null) {
values.put(tag, value);
}
}
}
});
}
return values;
}
// ---------- interface methods -------------
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
LOG.trace("-- requested values for " + node + ": " + tags);
if (!liveNodesSet.contains(node)) {
nodeValues.remove(node);
return Collections.emptyMap();
}
if (tags.isEmpty()) {
return Collections.emptyMap();
}
Map<String, Object> result = new HashMap<>();
Map<String, Object> metrics = getReplicaMetricsValues(node, tags.stream().filter(s -> s.startsWith("metrics:solr.core.")).collect(Collectors.toList()));
result.putAll(metrics);
Map<String, Object> values = nodeValues.get(node);
if (values == null) {
return result;
}
result.putAll(values.entrySet().stream().filter(e -> tags.contains(e.getKey())).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
return result;
}
@Override
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
if (replicas == null || replicas.isEmpty()) {
return Collections.emptyMap();
}
Map<String, Map<String, List<ReplicaInfo>>> res = new HashMap<>();
// TODO: probably needs special treatment for "metrics:solr.core..." tags
for (ReplicaInfo r : replicas) {
Map<String, List<ReplicaInfo>> perCollection = res.computeIfAbsent(r.getCollection(), s -> new HashMap<>());
List<ReplicaInfo> perShard = perCollection.computeIfAbsent(r.getShard(), s -> new ArrayList<>());
perShard.add(r);
}
return res;
}
@Override
public void close() throws IOException {
}
}

View File

@ -0,0 +1,251 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Predicate;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.CollectionStatePredicate;
import org.apache.solr.common.cloud.DocCollection;
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.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.junit.AfterClass;
import org.junit.Before;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
/**
* Base class for simulated test cases. Tests that use this class should configure the simulated cluster
* in <code>@BeforeClass</code> like this:
* <pre>
* @BeforeClass
* public static void setupCluster() throws Exception {
* cluster = configureCluster(5, TimeSource.get("simTime:50"));
* }
* </pre>
*/
public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final int DEFAULT_TIMEOUT = 90;
/** The cluster. */
protected static SimCloudManager cluster;
protected static void configureCluster(int nodeCount, TimeSource timeSource) throws Exception {
cluster = SimCloudManager.createCluster(nodeCount, timeSource);
}
@AfterClass
public static void shutdownCluster() throws Exception {
if (cluster != null) {
cluster.close();
}
cluster = null;
}
@Override
public void setUp() throws Exception {
super.setUp();
if (cluster != null) {
// clear any persisted auto scaling configuration
cluster.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), -1);
// clean any persisted trigger state or events
removeChildren(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
removeChildren(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
removeChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
cluster.getSimClusterStateProvider().simDeleteAllCollections();
cluster.simClearSystemCollection();
cluster.getSimClusterStateProvider().simResetLeaderThrottle();
cluster.simRestartOverseer(null);
}
}
@Before
public void checkClusterConfiguration() {
if (cluster == null)
throw new RuntimeException("SimCloudManager not configured - have you called configureCluster()?");
}
protected void removeChildren(String path) throws Exception {
if (!cluster.getDistribStateManager().hasData(path)) {
return;
}
List<String> children = cluster.getDistribStateManager().listData(path);
for (String c : children) {
if (cluster.getDistribStateManager().hasData(path + "/" + c)) {
try {
cluster.getDistribStateManager().removeData(path + "/" + c, -1);
} catch (NoSuchElementException e) {
// ignore
}
}
}
}
/* Cluster helper methods ************************************/
/**
* Get the collection state for a particular collection
*/
protected DocCollection getCollectionState(String collectionName) throws IOException {
return cluster.getClusterStateProvider().getClusterState().getCollection(collectionName);
}
/**
* Wait for a particular collection state to appear in the cluster client's state reader
*
* This is a convenience method using the {@link #DEFAULT_TIMEOUT}
*
* @param message a message to report on failure
* @param collection the collection to watch
* @param predicate a predicate to match against the collection state
*/
protected long waitForState(String message, String collection, CollectionStatePredicate predicate) {
AtomicReference<DocCollection> state = new AtomicReference<>();
AtomicReference<Set<String>> liveNodesLastSeen = new AtomicReference<>();
try {
return waitForState(collection, DEFAULT_TIMEOUT, TimeUnit.SECONDS, (n, c) -> {
state.set(c);
liveNodesLastSeen.set(n);
return predicate.matches(n, c);
});
} catch (Exception e) {
throw new AssertionError(message + "\n" + "Live Nodes: " + liveNodesLastSeen.get() + "\nLast available state: " + state.get(), e);
}
}
/**
* Block until a CollectionStatePredicate returns true, or the wait times out
*
* Note that the predicate may be called again even after it has returned true, so
* implementors should avoid changing state within the predicate call itself.
*
* @param collection the collection to watch
* @param wait how long to wait
* @param unit the units of the wait parameter
* @param predicate the predicate to call on state changes
* @return number of milliseconds elapsed
* @throws InterruptedException on interrupt
* @throws TimeoutException on timeout
* @throws IOException on watcher register / unregister error
*/
public long waitForState(final String collection, long wait, TimeUnit unit, CollectionStatePredicate predicate)
throws InterruptedException, TimeoutException, IOException {
TimeOut timeout = new TimeOut(wait, unit, cluster.getTimeSource());
long timeWarn = timeout.timeLeft(TimeUnit.MILLISECONDS) / 4;
while (!timeout.hasTimedOut()) {
ClusterState state = cluster.getClusterStateProvider().getClusterState();
DocCollection coll = state.getCollectionOrNull(collection);
// due to the way we manage collections in SimClusterStateProvider a null here
// can mean that a collection is still being created but has no replicas
if (coll == null) { // does not yet exist?
timeout.sleep(50);
continue;
}
if (predicate.matches(state.getLiveNodes(), coll)) {
log.trace("-- predicate matched with state {}", state);
return timeout.timeElapsed(TimeUnit.MILLISECONDS);
}
timeout.sleep(50);
if (timeout.timeLeft(TimeUnit.MILLISECONDS) < timeWarn) {
log.trace("-- still not matching predicate: {}", state);
}
}
throw new TimeoutException();
}
/**
* Return a {@link CollectionStatePredicate} that returns true if a collection has the expected
* number of shards and replicas
*/
public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas) {
return (liveNodes, collectionState) -> {
if (collectionState == null)
return false;
if (collectionState.getSlices().size() != expectedShards)
return false;
for (Slice slice : collectionState) {
int activeReplicas = 0;
for (Replica replica : slice) {
if (replica.isActive(liveNodes))
activeReplicas++;
}
if (activeReplicas != expectedReplicas)
return false;
}
return true;
};
}
/**
* Get a (reproducibly) random shard from a {@link DocCollection}
*/
protected static Slice getRandomShard(DocCollection collection) {
List<Slice> shards = new ArrayList<>(collection.getActiveSlices());
if (shards.size() == 0)
fail("Couldn't get random shard for collection as it has no shards!\n" + collection.toString());
Collections.shuffle(shards, random());
return shards.get(0);
}
/**
* Get a (reproducibly) random replica from a {@link Slice}
*/
protected static Replica getRandomReplica(Slice slice) {
List<Replica> replicas = new ArrayList<>(slice.getReplicas());
if (replicas.size() == 0)
fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
Collections.shuffle(replicas, random());
return replicas.get(0);
}
/**
* Get a (reproducibly) random replica from a {@link Slice} matching a predicate
*/
protected static Replica getRandomReplica(Slice slice, Predicate<Replica> matchPredicate) {
List<Replica> replicas = new ArrayList<>(slice.getReplicas());
if (replicas.size() == 0)
fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
Collections.shuffle(replicas, random());
for (Replica replica : replicas) {
if (matchPredicate.test(replica))
return replica;
}
fail("Couldn't get random replica that matched conditions\n" + slice.toString());
return null; // just to keep the compiler happy - fail will always throw an Exception
}
}

View File

@ -0,0 +1,221 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
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.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.Watcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This test compares the cluster state of a real cluster and a simulated one.
*/
public class TestClusterStateProvider extends SolrCloudTestCase {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static int NODE_COUNT = 3;
private static boolean simulated;
private static SolrCloudManager cloudManager;
private static Collection<String> liveNodes;
private static Map<String, Object> clusterProperties;
private static AutoScalingConfig autoScalingConfig;
private static Map<String, Map<String, Map<String, List<ReplicaInfo>>>> replicas;
private static Map<String, Map<String, Object>> nodeValues;
private static ClusterState realState;
// set up a real cluster as the source of test data
@BeforeClass
public static void setupCluster() throws Exception {
simulated = random().nextBoolean();
LOG.info("####### Using simulated components? " + simulated);
configureCluster(NODE_COUNT)
.addConfig("conf", configset("cloud-minimal"))
.configure();
CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 2, 0, 1)
.process(cluster.getSolrClient());
init();
}
@AfterClass
public static void closeCloudManager() throws Exception {
if (simulated && cloudManager != null) {
cloudManager.close();
}
}
private static void init() throws Exception {
SolrCloudManager realManager = cluster.getJettySolrRunner(cluster.getJettySolrRunners().size() - 1).getCoreContainer()
.getZkController().getSolrCloudManager();
liveNodes = realManager.getClusterStateProvider().getLiveNodes();
clusterProperties = realManager.getClusterStateProvider().getClusterProperties();
autoScalingConfig = realManager.getDistribStateManager().getAutoScalingConfig();
replicas = new HashMap<>();
nodeValues = new HashMap<>();
liveNodes.forEach(n -> {
replicas.put(n, realManager.getNodeStateProvider().getReplicaInfo(n, Collections.emptySet()));
nodeValues.put(n, realManager.getNodeStateProvider().getNodeValues(n, ImplicitSnitch.tags));
});
realState = realManager.getClusterStateProvider().getClusterState();
if (simulated) {
// initialize simulated provider
SimCloudManager simCloudManager = new SimCloudManager(TimeSource.get("simTime:10"));
simCloudManager.getSimClusterStateProvider().simSetClusterProperties(clusterProperties);
simCloudManager.getSimDistribStateManager().simSetAutoScalingConfig(autoScalingConfig);
nodeValues.forEach((n, values) -> {
simCloudManager.getSimNodeStateProvider().simSetNodeValues(n, values);
});
simCloudManager.getSimClusterStateProvider().simSetClusterState(realState);
ClusterState simState = simCloudManager.getClusterStateProvider().getClusterState();
assertClusterStateEquals(realState, simState);
cloudManager = simCloudManager;
} else {
cloudManager = realManager;
}
}
private static void assertClusterStateEquals(ClusterState one, ClusterState two) {
assertEquals(one.getLiveNodes(), two.getLiveNodes());
assertEquals(one.getCollectionsMap().keySet(), two.getCollectionsMap().keySet());
one.forEachCollection(oneColl -> {
DocCollection twoColl = two.getCollection(oneColl.getName());
Map<String, Slice> oneSlices = oneColl.getSlicesMap();
Map<String, Slice> twoSlices = twoColl.getSlicesMap();
assertEquals(oneSlices.keySet(), twoSlices.keySet());
oneSlices.forEach((s, slice) -> {
Slice sTwo = twoSlices.get(s);
for (Replica oneReplica : slice.getReplicas()) {
Replica twoReplica = sTwo.getReplica(oneReplica.getName());
assertNotNull(twoReplica);
assertEquals(oneReplica, twoReplica);
}
});
});
}
private String addNode() throws Exception {
JettySolrRunner solr = cluster.startJettySolrRunner();
String nodeId = solr.getNodeName();
if (simulated) {
((SimCloudManager) cloudManager).getSimClusterStateProvider().simAddNode(nodeId);
}
return nodeId;
}
private String deleteNode() throws Exception {
String nodeId = cluster.getJettySolrRunner(0).getNodeName();
cluster.stopJettySolrRunner(0);
if (simulated) {
((SimCloudManager) cloudManager).getSimClusterStateProvider().simRemoveNode(nodeId);
}
return nodeId;
}
private void setAutoScalingConfig(AutoScalingConfig cfg) throws Exception {
cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getZkClient().setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH,
Utils.toJSON(cfg), -1, true);
if (simulated) {
((SimCloudManager) cloudManager).getSimDistribStateManager().simSetAutoScalingConfig(cfg);
}
}
@Test
public void testAddRemoveNode() throws Exception {
Set<String> lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
List<String> liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
assertEquals(lastNodes.size(), liveNodes.size());
liveNodes.removeAll(lastNodes);
assertTrue(liveNodes.isEmpty());
String node = addNode();
cloudManager.getTimeSource().sleep(2000);
assertFalse(lastNodes.contains(node));
lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
assertTrue(lastNodes.contains(node));
liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
assertEquals(lastNodes.size(), liveNodes.size());
liveNodes.removeAll(lastNodes);
assertTrue(liveNodes.isEmpty());
node = deleteNode();
cloudManager.getTimeSource().sleep(2000);
assertTrue(lastNodes.contains(node));
lastNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
assertFalse(lastNodes.contains(node));
liveNodes = cloudManager.getDistribStateManager().listData(ZkStateReader.LIVE_NODES_ZKNODE);
assertEquals(lastNodes.size(), liveNodes.size());
liveNodes.removeAll(lastNodes);
assertTrue(liveNodes.isEmpty()); }
@Test
public void testAutoScalingConfig() throws Exception {
final CountDownLatch triggered = new CountDownLatch(1);
Watcher w = ev -> {
if (triggered.getCount() == 0) {
fail("already triggered once!");
}
triggered.countDown();
};
AutoScalingConfig cfg = cloudManager.getDistribStateManager().getAutoScalingConfig(w);
assertEquals(autoScalingConfig, cfg);
Preference p = new Preference(Collections.singletonMap("maximize", "freedisk"));
cfg = cfg.withPolicy(cfg.getPolicy().withClusterPreferences(Collections.singletonList(p)));
setAutoScalingConfig(cfg);
if (!triggered.await(10, TimeUnit.SECONDS)) {
fail("Watch should be triggered on update!");
}
AutoScalingConfig cfg1 = cloudManager.getDistribStateManager().getAutoScalingConfig(null);
assertEquals(cfg, cfg1);
// restore
setAutoScalingConfig(autoScalingConfig);
cfg1 = cloudManager.getDistribStateManager().getAutoScalingConfig(null);
assertEquals(autoScalingConfig, cfg1);
}
}

View File

@ -0,0 +1,357 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.autoscaling.ActionContext;
import org.apache.solr.cloud.autoscaling.ComputePlanAction;
import org.apache.solr.cloud.autoscaling.ScheduledTriggers;
import org.apache.solr.cloud.autoscaling.TriggerAction;
import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.LogLevel;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
/**
* Test for {@link ComputePlanAction}
*/
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG;")
public class TestComputePlanAction extends SimSolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final AtomicBoolean fired = new AtomicBoolean(false);
private static final int NODE_COUNT = 1;
private static CountDownLatch triggerFiredLatch = new CountDownLatch(1);
private static final AtomicReference<Map> actionContextPropsRef = new AtomicReference<>();
private static final AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(1, TimeSource.get("simTime:50"));
}
@Before
public void init() throws Exception {
fired.set(false);
triggerFiredLatch = new CountDownLatch(1);
actionContextPropsRef.set(null);
if (cluster.getClusterStateProvider().getLiveNodes().size() > NODE_COUNT) {
// stop some to get to original state
int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
for (int i = 0; i < numJetties - NODE_COUNT; i++) {
String node = cluster.getSimClusterStateProvider().simGetRandomNode(random());
cluster.getSimClusterStateProvider().simRemoveNode(node);
}
}
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
SolrResponse rsp = cluster.request(req);
NamedList<Object> response = rsp.getResponse();
assertEquals(response.get("result").toString(), "success");
String setClusterPreferencesCommand = "{" +
"'set-cluster-preferences': [" +
"{'minimize': 'cores'}," +
"{'maximize': 'freedisk','precision': 100}]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPreferencesCommand);
rsp = cluster.request(req);
response = rsp.getResponse();
assertEquals(response.get("result").toString(), "success");
cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(ScheduledTriggers.DEFAULT_COOLDOWN_PERIOD_SECONDS));
}
@After
public void printState() throws Exception {
log.info("-------------_ FINAL STATE --------------");
log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
ClusterState state = cluster.getClusterStateProvider().getClusterState();
for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
log.info("* Collection " + coll + " state: " + state.getCollection(coll));
}
}
@Test
public void testNodeLost() throws Exception {
// let's start a node so that we have at least two
String node = cluster.simAddNode();
AssertingTriggerAction.expectedNode = node;
SolrClient solrClient = cluster.simGetSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '7s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeLost",
"conf",1, 2);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
"testNodeLost", clusterShape(1, 2));
ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
log.debug("-- cluster state: {}", clusterState);
DocCollection collection = clusterState.getCollection("testNodeLost");
List<Replica> replicas = collection.getReplicas(node);
assertNotNull(replicas);
assertFalse(replicas.isEmpty());
// start another node because because when the other node goes away, the cluster policy requires only
// 1 replica per node and none on the overseer
String node2 = cluster.simAddNode();
assertTrue(node2 + "is not live yet", cluster.getClusterStateProvider().getClusterState().liveNodesContain(node2) );
// stop the original node
cluster.simRemoveNode(node, false);
log.info("Stopped_node : {}", node);
assertTrue("Trigger was not fired even after 10 seconds", triggerFiredLatch.await(10, TimeUnit.SECONDS));
assertTrue(fired.get());
Map context = actionContextPropsRef.get();
assertNotNull(context);
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null , " + eventRef.get(), operations);
assertEquals("ComputePlanAction should have computed exactly 1 operation", 1, operations.size());
SolrRequest solrRequest = operations.get(0);
SolrParams params = solrRequest.getParams();
assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
String replicaToBeMoved = params.get("replica");
assertEquals("Unexpected node in computed operation", replicas.get(0).getName(), replicaToBeMoved);
// shutdown the extra node that we had started
cluster.simRemoveNode(node2, false);
}
public void testNodeWithMultipleReplicasLost() throws Exception {
AssertingTriggerAction.expectedNode = null;
// start 3 more nodes
cluster.simAddNode();
cluster.simAddNode();
cluster.simAddNode();
SolrClient solrClient = cluster.simGetSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeWithMultipleReplicasLost",
"conf",2, 3);
// create.setMaxShardsPerNode(2);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
"testNodeWithMultipleReplicasLost", clusterShape(2, 3));
ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
log.debug("-- cluster state: {}", clusterState);
DocCollection docCollection = clusterState.getCollection("testNodeWithMultipleReplicasLost");
// lets find a node with at least 2 replicas
String stoppedNodeName = null;
List<Replica> replicasToBeMoved = null;
for (String node : cluster.getClusterStateProvider().getLiveNodes()) {
List<Replica> replicas = docCollection.getReplicas(node);
if (replicas != null && replicas.size() == 2) {
stoppedNodeName = node;
replicasToBeMoved = replicas;
cluster.simRemoveNode(node, false);
break;
}
}
assertNotNull(stoppedNodeName);
assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
assertTrue(fired.get());
TriggerEvent triggerEvent = eventRef.get();
assertNotNull(triggerEvent);
assertEquals(TriggerEventType.NODELOST, triggerEvent.getEventType());
// TODO assertEquals(stoppedNodeName, triggerEvent.getProperty(TriggerEvent.NODE_NAME));
Map context = actionContextPropsRef.get();
assertNotNull(context);
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null " + actionContextPropsRef.get(), operations);
operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
for (SolrRequest solrRequest : operations) {
SolrParams params = solrRequest.getParams();
assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
String moved = params.get("replica");
assertTrue(replicasToBeMoved.stream().anyMatch(replica -> replica.getName().equals(moved)));
}
}
@Test
public void testNodeAdded() throws Exception {
AssertingTriggerAction.expectedNode = null;
SolrClient solrClient = cluster.simGetSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + TestComputePlanAction.AssertingTriggerAction.class.getName() + "'}]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// the default policy limits 1 replica per node, we need more right now
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<3', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeAdded",
"conf",1, 2);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
"testNodeAdded", (liveNodes, collectionState) -> collectionState.getReplicas().stream().allMatch(replica -> replica.isActive(liveNodes)));
// reset to the original policy which has only 1 replica per shard per node
setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// start a node so that the 'violation' created by the previous policy update is fixed
String newNode = cluster.simAddNode();
assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
assertTrue(fired.get());
Map context = actionContextPropsRef.get();
assertNotNull(context);
log.info("Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
log.info("Live nodes: " + cluster.getClusterStateProvider().getLiveNodes() + ", collection state: " + cluster.getClusterStateProvider().getClusterState().getCollection("testNodeAdded"));
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null" + context, operations);
assertEquals("ComputePlanAction should have computed exactly 1 operation, but was: " + operations, 1, operations.size());
SolrRequest request = operations.get(0);
SolrParams params = request.getParams();
assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
String nodeAdded = params.get("targetNode");
assertEquals("Unexpected node in computed operation", newNode, nodeAdded);
}
public static class AssertingTriggerAction implements TriggerAction {
static String expectedNode;
@Override
public String getName() {
return null;
}
@Override
public void process(TriggerEvent event, ActionContext context) {
if (expectedNode != null) {
Collection nodes = (Collection) event.getProperty(TriggerEvent.NODE_NAMES);
if (nodes == null || !nodes.contains(expectedNode)) return;//this is not the event we are looking for
}
if (fired.compareAndSet(false, true)) {
eventRef.set(event);
actionContextPropsRef.set(context.getProperties());
triggerFiredLatch.countDown();
}
}
@Override
public void close() throws IOException {
}
@Override
public void init(Map<String, String> args) {
}
}
}

View File

@ -0,0 +1,284 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.lang.invoke.MethodHandles;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.NoSuchElementException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.impl.ZkDistribStateManager;
import org.apache.solr.cloud.ZkTestServer;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This test compares a ZK-based {@link DistribStateManager} to the simulated one.
*/
public class TestDistribStateManager extends SolrTestCaseJ4 {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private DistribStateManager stateManager;
private ZkTestServer zkTestServer;
private SolrZkClient solrZkClient;
private boolean simulated;
private SimDistribStateManager.Node root;
@Before
public void setup() throws Exception {
simulated = random().nextBoolean();
if (simulated) {
root = SimDistribStateManager.createNewRootNode();
} else {
zkTestServer = new ZkTestServer(createTempDir("zkDir").toString());
zkTestServer.run();
}
reInit();
}
private void reInit() throws Exception {
if (stateManager != null) {
stateManager.close();
}
if (simulated) {
stateManager = new SimDistribStateManager(root);
} else {
if (solrZkClient != null) {
solrZkClient.close();
}
solrZkClient = new SolrZkClient(zkTestServer.getZkHost(), 30000);
stateManager = new ZkDistribStateManager(solrZkClient);
}
LOG.info("Using " + stateManager.getClass().getName());
}
@After
public void teardown() throws Exception {
if (solrZkClient != null) {
solrZkClient.close();
solrZkClient = null;
}
if (zkTestServer != null) {
zkTestServer.shutdown();
zkTestServer = null;
}
if (stateManager != null) {
stateManager.close();
}
stateManager = null;
}
@Test
public void testHasData() throws Exception {
assertFalse(stateManager.hasData("/hasData/foo"));
assertFalse(stateManager.hasData("/hasData/bar"));
try {
stateManager.createData("/hasData/foo", new byte[0], CreateMode.PERSISTENT);
} catch (NoSuchElementException e) {
// expected
}
stateManager.makePath("/hasData");
stateManager.createData("/hasData/foo", new byte[0], CreateMode.PERSISTENT);
stateManager.createData("/hasData/bar", new byte[0], CreateMode.PERSISTENT);
assertTrue(stateManager.hasData("/hasData/foo"));
assertTrue(stateManager.hasData("/hasData/bar"));
}
@Test
public void testListData() throws Exception {
assertFalse(stateManager.hasData("/listData/foo"));
assertFalse(stateManager.hasData("/listData/foo/bar"));
try {
stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
fail("should not succeed");
} catch (NoSuchElementException e) {
// expected
}
try {
stateManager.listData("/listData/foo");
fail("should not succeed");
} catch (NoSuchElementException e) {
// expected
}
stateManager.makePath("/listData");
List<String> kids = stateManager.listData("/listData");
assertEquals(0, kids.size());
stateManager.makePath("/listData/foo");
kids = stateManager.listData("/listData");
assertEquals(1, kids.size());
assertEquals("foo", kids.get(0));
stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
stateManager.createData("/listData/foo/baz", new byte[0], CreateMode.PERSISTENT);
kids = stateManager.listData("/listData/foo");
assertEquals(2, kids.size());
assertTrue(kids.contains("bar"));
assertTrue(kids.contains("baz"));
try {
stateManager.createData("/listData/foo/bar", new byte[0], CreateMode.PERSISTENT);
fail("should not succeed");
} catch (AlreadyExistsException e) {
// expected
}
}
static final byte[] firstData = new byte[] {
(byte)0xca, (byte)0xfe, (byte)0xba, (byte)0xbe
};
static final byte[] secondData = new byte[] {
(byte)0xbe, (byte)0xba, (byte)0xfe, (byte)0xca
};
@Test
public void testCreateMode() throws Exception {
stateManager.makePath("/createMode");
stateManager.createData("/createMode/persistent", firstData, CreateMode.PERSISTENT);
stateManager.createData("/createMode/persistent_seq", firstData, CreateMode.PERSISTENT);
for (int i = 0; i < 10; i++) {
stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
}
// check what happens with gaps
stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
stateManager.removeData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 10), -1);
stateManager.createData("/createMode/persistent_seq/data", firstData, CreateMode.PERSISTENT_SEQUENTIAL);
stateManager.createData("/createMode/ephemeral", firstData, CreateMode.EPHEMERAL);
stateManager.createData("/createMode/ephemeral_seq", firstData, CreateMode.PERSISTENT);
for (int i = 0; i < 10; i++) {
stateManager.createData("/createMode/ephemeral_seq/data", firstData, CreateMode.EPHEMERAL_SEQUENTIAL);
}
assertTrue(stateManager.hasData("/createMode"));
assertTrue(stateManager.hasData("/createMode/persistent"));
assertTrue(stateManager.hasData("/createMode/ephemeral"));
List<String> kids = stateManager.listData("/createMode/persistent_seq");
assertEquals(11, kids.size());
kids = stateManager.listData("/createMode/ephemeral_seq");
assertEquals(10, kids.size());
for (int i = 0; i < 10; i++) {
assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", i)));
}
assertFalse(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 10)));
assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 11)));
for (int i = 0; i < 10; i++) {
assertTrue(stateManager.hasData("/createMode/ephemeral_seq/data" + String.format(Locale.ROOT, "%010d", i)));
}
// check that ephemeral nodes disappear on disconnect
reInit();
assertTrue(stateManager.hasData("/createMode/persistent"));
for (int i = 0; i < 10; i++) {
assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", i)));
}
assertTrue(stateManager.hasData("/createMode/persistent_seq/data" + String.format(Locale.ROOT, "%010d", 11)));
assertFalse(stateManager.hasData("/createMode/ephemeral"));
assertTrue(stateManager.hasData("/createMode/ephemeral_seq"));
kids = stateManager.listData("/createMode/ephemeral_seq");
assertEquals(0, kids.size());
}
static class OnceWatcher implements Watcher {
CountDownLatch triggered = new CountDownLatch(1);
WatchedEvent event;
@Override
public void process(WatchedEvent event) {
if (triggered.getCount() == 0) {
fail("Watch was already triggered once!");
}
triggered.countDown();
this.event = event;
}
}
@Test
public void testGetSetRemoveData() throws Exception {
stateManager.makePath("/getData");
stateManager.createData("/getData/persistentData", firstData, CreateMode.PERSISTENT);
OnceWatcher nodeWatcher = new OnceWatcher();
VersionedData vd = stateManager.getData("/getData/persistentData", nodeWatcher);
assertNotNull(vd);
assertEquals(0, vd.getVersion());
assertTrue(Arrays.equals(firstData, vd.getData()));
// update data, test versioning
try {
stateManager.setData("/getData/persistentData", secondData, 1);
fail("should have failed");
} catch (BadVersionException e) {
// expected
}
// watch should not have fired
assertEquals(1, nodeWatcher.triggered.getCount());
stateManager.setData("/getData/persistentData", secondData, 0);
if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
fail("Node watch should have fired!");
}
// watch should not fire now because it needs to be reset
stateManager.setData("/getData/persistentData", secondData, -1);
nodeWatcher = new OnceWatcher();
stateManager.createData("/getData/ephemeralData", firstData, CreateMode.EPHEMERAL);
vd = stateManager.getData("/getData/ephemeralData", nodeWatcher);
reInit();
if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
fail("Node watch should have fired!");
}
assertTrue(stateManager.hasData("/getData/persistentData"));
assertFalse(stateManager.hasData("/getData/ephemeralData"));
nodeWatcher = new OnceWatcher();
vd = stateManager.getData("/getData/persistentData", nodeWatcher);
// try wrong version
try {
stateManager.removeData("/getData/persistentData", vd.getVersion() - 1);
fail("should have failed");
} catch (BadVersionException e) {
// expected
}
// watch should not have fired
assertEquals(1, nodeWatcher.triggered.getCount());
stateManager.removeData("/getData/persistentData", vd.getVersion());
if (!nodeWatcher.triggered.await(5, TimeUnit.SECONDS)) {
fail("Node watch should have fired!");
}
}
@Test
public void testMulti() throws Exception {
}
}

View File

@ -0,0 +1,216 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.lang.invoke.MethodHandles;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import com.google.common.collect.Lists;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.autoscaling.ActionContext;
import org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest;
import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.LogLevel;
import org.apache.solr.common.util.TimeSource;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test for {@link ExecutePlanAction}
*/
@LogLevel("org.apache.solr.cloud=DEBUG")
public class TestExecutePlanAction extends SimSolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final int NODE_COUNT = 2;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(NODE_COUNT, TimeSource.get("simTime:50"));
}
@Before
public void setUp() throws Exception {
super.setUp();
if (cluster.getClusterStateProvider().getLiveNodes().size() < NODE_COUNT) {
// start some to get to original state
int numJetties = cluster.getClusterStateProvider().getLiveNodes().size();
for (int i = 0; i < NODE_COUNT - numJetties; i++) {
cluster.simAddNode();
}
}
}
@After
public void printState() throws Exception {
log.info("-------------_ FINAL STATE --------------");
log.info("* Node values: " + Utils.toJSONString(cluster.getSimNodeStateProvider().simGetAllNodeValues()));
log.info("* Live nodes: " + cluster.getClusterStateProvider().getLiveNodes());
ClusterState state = cluster.getClusterStateProvider().getClusterState();
for (String coll: cluster.getSimClusterStateProvider().simListCollections()) {
log.info("* Collection " + coll + " state: " + state.getCollection(coll));
}
}
@Test
public void testExecute() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String collectionName = "testExecute";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
"conf", 1, 2);
create.setMaxShardsPerNode(1);
create.process(solrClient);
log.info("Collection ready after " + waitForState(collectionName, 120, TimeUnit.SECONDS, clusterShape(1, 2)) + "ms");
String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
DocCollection docCollection = clusterState.getCollection(collectionName);
List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
assertNotNull(replicas);
assertFalse(replicas.isEmpty());
List<String> otherNodes = cluster.getClusterStateProvider().getLiveNodes().stream()
.filter(node -> !node.equals(sourceNodeName)).collect(Collectors.toList());
assertFalse(otherNodes.isEmpty());
String survivor = otherNodes.get(0);
try (ExecutePlanAction action = new ExecutePlanAction()) {
action.init(Collections.singletonMap("name", "execute_plan"));
// used to signal if we found that ExecutePlanAction did in fact create the right znode before executing the operation
AtomicBoolean znodeCreated = new AtomicBoolean(false);
CollectionAdminRequest.AsyncCollectionAdminRequest moveReplica = new CollectionAdminRequest.MoveReplica(collectionName, replicas.get(0).getName(), survivor);
CollectionAdminRequest.AsyncCollectionAdminRequest mockRequest = new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.OVERSEERSTATUS) {
@Override
public void setAsyncId(String asyncId) {
super.setAsyncId(asyncId);
String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/xyz/execute_plan";
try {
if (cluster.getDistribStateManager().hasData(parentPath)) {
java.util.List<String> children = cluster.getDistribStateManager().listData(parentPath);
if (!children.isEmpty()) {
String child = children.get(0);
VersionedData data = cluster.getDistribStateManager().getData(parentPath + "/" + child);
Map m = (Map) Utils.fromJSON(data.getData());
if (m.containsKey("requestid")) {
znodeCreated.set(m.get("requestid").equals(asyncId));
}
}
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
List<CollectionAdminRequest.AsyncCollectionAdminRequest> operations = Lists.asList(moveReplica, new CollectionAdminRequest.AsyncCollectionAdminRequest[]{mockRequest});
NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST,
"mock_trigger_name", Collections.singletonList(TimeSource.CURRENT_TIME.getTime()),
Collections.singletonList(sourceNodeName));
ActionContext actionContext = new ActionContext(cluster, null,
new HashMap<>(Collections.singletonMap("operations", operations)));
action.process(nodeLostEvent, actionContext);
// assertTrue("ExecutePlanAction should have stored the requestid in ZK before executing the request", znodeCreated.get());
List<NamedList<Object>> responses = (List<NamedList<Object>>) actionContext.getProperty("responses");
assertNotNull(responses);
assertEquals(2, responses.size());
NamedList<Object> response = responses.get(0);
assertNull(response.get("failure"));
assertNotNull(response.get("success"));
}
log.info("Collection ready after " + waitForState(collectionName, 300, TimeUnit.SECONDS, clusterShape(1, 2)) + "ms");
}
@Test
public void testIntegration() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'execute_plan','class':'solr.ExecutePlanAction'}]" +
"}}";
SolrRequest req = AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String collectionName = "testIntegration";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
"conf", 1, 2);
create.setMaxShardsPerNode(1);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
collectionName, clusterShape(1, 2));
String sourceNodeName = cluster.getSimClusterStateProvider().simGetRandomNode(random());
ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
DocCollection docCollection = clusterState.getCollection(collectionName);
List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
assertNotNull(replicas);
assertFalse(replicas.isEmpty());
List<String> otherNodes = cluster.getClusterStateProvider().getLiveNodes().stream()
.filter(node -> !node.equals(sourceNodeName)).collect(Collectors.toList());
assertFalse(otherNodes.isEmpty());
String survivor = otherNodes.get(0);
cluster.simRemoveNode(sourceNodeName, false);
waitForState("Timed out waiting for replicas of collection to be 2 again",
collectionName, clusterShape(1, 2));
clusterState = cluster.getClusterStateProvider().getClusterState();
docCollection = clusterState.getCollection(collectionName);
List<Replica> replicasOnSurvivor = docCollection.getReplicas(survivor);
assertNotNull(replicasOnSurvivor);
assertEquals(2, replicasOnSurvivor.size());
}
}

View File

@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
/**
*
*/
public class TestGenericDistributedQueue extends TestSimDistributedQueue {
DistribStateManager stateManager = new SimDistribStateManager();
@Override
protected DistributedQueue makeDistributedQueue(String dqZNode) throws Exception {
return new GenericDistributedQueue(stateManager, dqZNode);
}
}

View File

@ -0,0 +1,266 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.autoscaling.ActionContext;
import org.apache.solr.cloud.autoscaling.ComputePlanAction;
import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
import org.apache.solr.cloud.autoscaling.TriggerActionBase;
import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
import org.apache.solr.cloud.autoscaling.CapturedEvent;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.LogLevel;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
/**
*
*/
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
public class TestLargeCluster extends SimSolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final int SPEED = 50;
public static final int NUM_NODES = 100;
static Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
static AtomicInteger triggerFiredCount = new AtomicInteger();
static CountDownLatch triggerFiredLatch;
static int waitForSeconds;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(NUM_NODES, TimeSource.get("simTime:" + SPEED));
}
@Before
public void setupTest() throws Exception {
waitForSeconds = 1 + random().nextInt(3);
triggerFiredCount.set(0);
triggerFiredLatch = new CountDownLatch(1);
listenerEvents.clear();
while (cluster.getClusterStateProvider().getLiveNodes().size() < NUM_NODES) {
// perhaps a test stopped a node but didn't start it back
// lets start a node
cluster.simAddNode();
}
}
public static class TestTriggerListener extends TriggerListenerBase {
@Override
public void init(SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) {
super.init(cloudManager, config);
}
@Override
public synchronized void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
ActionContext context, Throwable error, String message) {
List<CapturedEvent> lst = listenerEvents.computeIfAbsent(config.name, s -> new ArrayList<>());
lst.add(new CapturedEvent(cluster.getTimeSource().getTime(), context, config, stage, actionName, event, message));
}
}
public static class TestTriggerAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
triggerFiredCount.incrementAndGet();
triggerFiredLatch.countDown();
}
}
@Test
public void testBasic() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '" + waitForSeconds + "s'," +
"'enabled' : true," +
"'actions' : [" +
"{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
"{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
"{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
"]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setListenerCommand = "{" +
"'set-listener' : " +
"{" +
"'name' : 'foo'," +
"'trigger' : 'node_lost_trigger'," +
"'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
"'beforeAction' : ['compute', 'execute']," +
"'afterAction' : ['compute', 'execute']," +
"'class' : '" + TestTriggerListener.class.getName() + "'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
cluster.getTimeSource().sleep(5000);
// pick a few random nodes
List<String> nodes = new ArrayList<>();
int limit = 75;
for (String node : cluster.getClusterStateProvider().getLiveNodes()) {
nodes.add(node);
if (nodes.size() > limit) {
break;
}
}
Collections.shuffle(nodes, random());
// create collection on these nodes
String collectionName = "testBasic";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
"conf", 5, 5, 5, 5);
create.setMaxShardsPerNode(1);
create.setCreateNodeSet(String.join(",", nodes));
create.process(solrClient);
log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
int KILL_NODES = 8;
// kill off a number of nodes
for (int i = 0; i < KILL_NODES; i++) {
cluster.simRemoveNode(nodes.get(i), false);
}
// should fully recover
log.info("Ready after " + waitForState(collectionName, 90 * KILL_NODES, TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
log.info("OP COUNTS: " + cluster.simGetOpCounts());
long moveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
// simulate a number of flaky nodes
int FLAKY_NODES = 10;
int flakyReplicas = 0;
for (int cnt = 0; cnt < 10; cnt++) {
for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
flakyReplicas += cluster.getSimClusterStateProvider().simGetReplicaInfos(nodes.get(i))
.stream().filter(r -> r.getState().equals(Replica.State.ACTIVE)).count();
cluster.simRemoveNode(nodes.get(i), false);
}
cluster.getTimeSource().sleep(TimeUnit.SECONDS.toMillis(waitForSeconds) * 2);
for (int i = KILL_NODES; i < KILL_NODES + FLAKY_NODES; i++) {
final String nodeId = nodes.get(i);
cluster.submit(() -> cluster.getSimClusterStateProvider().simRestoreNode(nodeId));
}
}
log.info("Ready after " + waitForState(collectionName, 30 * nodes.size(), TimeUnit.SECONDS, clusterShape(5, 15)) + "ms");
log.info("OP COUNTS: " + cluster.simGetOpCounts());
long newMoveReplicaOps = cluster.simGetOpCount(CollectionParams.CollectionAction.MOVEREPLICA.name());
log.info("==== Flaky replicas: {}. Additional MOVEREPLICA count: {}", flakyReplicas, (newMoveReplicaOps - moveReplicaOps));
// flaky nodes lead to a number of MOVEREPLICA that is non-zero but lower than the number of flaky replicas
assertTrue("there should be new MOVERPLICA ops", newMoveReplicaOps - moveReplicaOps > 0);
assertTrue("there should be less than flakyReplicas=" + flakyReplicas + " MOVEREPLICA ops",
newMoveReplicaOps - moveReplicaOps < flakyReplicas);
}
@Test
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11714")
public void testSearchRate() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'search_rate_trigger'," +
"'event' : 'searchRate'," +
"'waitFor' : '" + waitForSeconds + "s'," +
"'rate' : 1.0," +
"'enabled' : true," +
"'actions' : [" +
"{'name':'compute','class':'" + ComputePlanAction.class.getName() + "'}," +
"{'name':'execute','class':'" + ExecutePlanAction.class.getName() + "'}," +
"{'name':'test','class':'" + TestTriggerAction.class.getName() + "'}" +
"]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setListenerCommand1 = "{" +
"'set-listener' : " +
"{" +
"'name' : 'srt'," +
"'trigger' : 'search_rate_trigger'," +
"'stage' : ['FAILED','SUCCEEDED']," +
"'class' : '" + TestTriggerListener.class.getName() + "'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand1);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String collectionName = "testSearchRate";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
"conf", 2, 10);
create.process(solrClient);
log.info("Ready after " + waitForState(collectionName, 300, TimeUnit.SECONDS, clusterShape(2, 10)) + " ms");
// collect the node names
Set<String> nodes = new HashSet<>();
cluster.getSimClusterStateProvider().getClusterState().getCollection(collectionName)
.getReplicas()
.forEach(r -> nodes.add(r.getNodeName()));
String metricName = "QUERY./select.requestTimes:1minRate";
// simulate search traffic
cluster.getSimClusterStateProvider().simSetShardValue(collectionName, "shard1", metricName, 40, true);
Thread.sleep(1000000000);
// boolean await = triggerFiredLatch.await(20000 / SPEED, TimeUnit.MILLISECONDS);
// assertTrue("The trigger did not fire at all", await);
// wait for listener to capture the SUCCEEDED stage
cluster.getTimeSource().sleep(2000);
assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
CapturedEvent ev = listenerEvents.get("srt").get(0);
}
}

View File

@ -0,0 +1,306 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.cloud.autoscaling.ActionContext;
import org.apache.solr.cloud.autoscaling.AutoScaling;
import org.apache.solr.cloud.autoscaling.NodeAddedTrigger;
import org.apache.solr.cloud.autoscaling.TriggerAction;
import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.common.util.TimeSource;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test for {@link NodeAddedTrigger}
*/
public class TestNodeAddedTrigger extends SimSolrCloudTestCase {
private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
fail("Did not expect the listener to fire on first run!");
return true;
};
private static int SPEED = 50;
// currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(2);
private static TimeSource timeSource;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(1, TimeSource.get("simTime:" + SPEED));
timeSource = cluster.getTimeSource();
}
@Before
public void beforeTest() throws Exception {
actionConstructorCalled = new AtomicBoolean(false);
actionInitCalled = new AtomicBoolean(false);
actionCloseCalled = new AtomicBoolean(false);
}
@Test
public void testTrigger() throws Exception {
long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
String newNode1 = cluster.simAddNode();
String newNode2 = cluster.simAddNode();
AtomicBoolean fired = new AtomicBoolean(false);
AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
trigger.setProcessor(event -> {
if (fired.compareAndSet(false, true)) {
eventRef.set(event);
long currentTimeNanos = timeSource.getTime();
long eventTimeNanos = event.getEventTime();
long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
}
} else {
fail("NodeAddedTrigger was fired more than once!");
}
return true;
});
int counter = 0;
do {
trigger.run();
timeSource.sleep(1000);
if (counter++ > 10) {
fail("Newly added node was not discovered by trigger even after 10 seconds");
}
} while (!fired.get());
TriggerEvent nodeAddedEvent = eventRef.get();
assertNotNull(nodeAddedEvent);
List<String> nodeNames = (List<String>)nodeAddedEvent.getProperty(TriggerEvent.NODE_NAMES);
assertTrue(nodeNames.contains(newNode1));
assertTrue(nodeNames.contains(newNode2));
}
// add a new node but remove it before the waitFor period expires
// and assert that the trigger doesn't fire at all
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
final long waitTime = 2;
props.put("waitFor", waitTime);
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
String newNode = cluster.simAddNode();
AtomicBoolean fired = new AtomicBoolean(false);
trigger.setProcessor(event -> {
if (fired.compareAndSet(false, true)) {
long currentTimeNanos = timeSource.getTime();
long eventTimeNanos = event.getEventTime();
long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
}
} else {
fail("NodeAddedTrigger was fired more than once!");
}
return true;
});
trigger.run(); // first run should detect the new node
cluster.simRemoveNode(newNode, true);
int counter = 0;
do {
trigger.run();
timeSource.sleep(1000);
if (counter++ > waitTime + 1) { // run it a little more than the wait time
break;
}
} while (true);
// ensure the event was not fired
assertFalse(fired.get());
}
}
public void testActionLifecycle() throws Exception {
Map<String, Object> props = createTriggerProps(0);
List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
Map<String, String> action = new HashMap<>(2);
action.put("name", "testActionInit");
action.put("class", TestNodeAddedTrigger.AssertInitTriggerAction.class.getName());
actions.add(action);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
assertEquals(true, actionConstructorCalled.get());
assertEquals(false, actionInitCalled.get());
assertEquals(false, actionCloseCalled.get());
trigger.init();
assertEquals(true, actionInitCalled.get());
assertEquals(false, actionCloseCalled.get());
}
assertEquals(true, actionCloseCalled.get());
}
public static class AssertInitTriggerAction implements TriggerAction {
public AssertInitTriggerAction() {
actionConstructorCalled.set(true);
}
@Override
public String getName() {
return "";
}
@Override
public void process(TriggerEvent event, ActionContext actionContext) {
}
@Override
public void close() throws IOException {
actionCloseCalled.compareAndSet(false, true);
}
@Override
public void init(Map<String, String> args) {
actionInitCalled.compareAndSet(false, true);
}
}
@Test
public void testListenerAcceptance() throws Exception {
Map<String, Object> props = createTriggerProps(0);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
trigger.setProcessor(noFirstRunProcessor);
trigger.run(); // starts tracking live nodes
String newNode = cluster.simAddNode();
AtomicInteger callCount = new AtomicInteger(0);
AtomicBoolean fired = new AtomicBoolean(false);
trigger.setProcessor(event -> {
if (callCount.incrementAndGet() < 2) {
return false;
} else {
fired.compareAndSet(false, true);
return true;
}
});
trigger.run(); // first run should detect the new node and fire immediately but listener isn't ready
assertEquals(1, callCount.get());
assertFalse(fired.get());
trigger.run(); // second run should again fire
assertEquals(2, callCount.get());
assertTrue(fired.get());
trigger.run(); // should not fire
assertEquals(2, callCount.get());
}
}
@Test
public void testRestoreState() throws Exception {
long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds);
// add a new node but update the trigger before the waitFor period expires
// and assert that the new trigger still fires
NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster);
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
String newNode = cluster.simAddNode();
trigger.run(); // this run should detect the new node
trigger.close(); // close the old trigger
try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name", props, cluster.getLoader(), cluster)) {
try {
newTrigger.restoreState(trigger);
fail("Trigger should only be able to restore state from an old trigger of the same name");
} catch (AssertionError e) {
// expected
}
}
try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
AtomicBoolean fired = new AtomicBoolean(false);
AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
newTrigger.setProcessor(event -> {
if (fired.compareAndSet(false, true)) {
eventRef.set(event);
long currentTimeNanos = timeSource.getTime();
long eventTimeNanos = event.getEventTime();
long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
fail("NodeAddedListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
}
} else {
fail("NodeAddedTrigger was fired more than once!");
}
return true;
});
newTrigger.restoreState(trigger); // restore state from the old trigger
int counter = 0;
do {
newTrigger.run();
timeSource.sleep(1000);
if (counter++ > 10) {
fail("Newly added node was not discovered by trigger even after 10 seconds");
}
} while (!fired.get());
// ensure the event was fired
assertTrue(fired.get());
TriggerEvent nodeAddedEvent = eventRef.get();
assertNotNull(nodeAddedEvent);
//TODO assertEquals("", newNode.getNodeName(), nodeAddedEvent.getProperty(NodeAddedTrigger.NodeAddedEvent.NODE_NAME));
}
}
private Map<String, Object> createTriggerProps(long waitForSeconds) {
Map<String, Object> props = new HashMap<>();
props.put("event", "nodeLost");
props.put("waitFor", waitForSeconds);
props.put("enabled", true);
List<Map<String, String>> actions = new ArrayList<>(3);
Map<String, String> map = new HashMap<>(2);
map.put("name", "compute_plan");
map.put("class", "solr.ComputePlanAction");
actions.add(map);
map = new HashMap<>(2);
map.put("name", "execute_plan");
map.put("class", "solr.ExecutePlanAction");
actions.add(map);
props.put("actions", actions);
return props;
}
}

View File

@ -0,0 +1,331 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.cloud.autoscaling.ActionContext;
import org.apache.solr.cloud.autoscaling.AutoScaling;
import org.apache.solr.cloud.autoscaling.NodeLostTrigger;
import org.apache.solr.cloud.autoscaling.TriggerAction;
import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.common.util.TimeSource;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test for {@link NodeLostTrigger}
*/
public class TestNodeLostTrigger extends SimSolrCloudTestCase {
private static AtomicBoolean actionConstructorCalled = new AtomicBoolean(false);
private static AtomicBoolean actionInitCalled = new AtomicBoolean(false);
private static AtomicBoolean actionCloseCalled = new AtomicBoolean(false);
private AutoScaling.TriggerEventProcessor noFirstRunProcessor = event -> {
fail("Did not expect the listener to fire on first run!");
return true;
};
private static final int SPEED = 50;
// use the same time source as the trigger
private static TimeSource timeSource;
// currentTimeMillis is not as precise so to avoid false positives while comparing time of fire, we add some delta
private static final long WAIT_FOR_DELTA_NANOS = TimeUnit.MILLISECONDS.toNanos(5);
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(5, TimeSource.get("simTime:" + SPEED));
timeSource = cluster.getTimeSource();
}
@Before
public void beforeTest() throws Exception {
actionConstructorCalled = new AtomicBoolean(false);
actionInitCalled = new AtomicBoolean(false);
actionCloseCalled = new AtomicBoolean(false);
}
@Test
public void testTrigger() throws Exception {
long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
Iterator<String> it = cluster.getLiveNodesSet().get().iterator();
String lostNodeName1 = it.next();
String lostNodeName2 = it.next();
cluster.simRemoveNode(lostNodeName1, true);
cluster.simRemoveNode(lostNodeName2, true);
timeSource.sleep(1000);
AtomicBoolean fired = new AtomicBoolean(false);
AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
trigger.setProcessor(event -> {
if (fired.compareAndSet(false, true)) {
eventRef.set(event);
long currentTimeNanos = timeSource.getTime();
long eventTimeNanos = event.getEventTime();
long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
}
} else {
fail("NodeLostListener was fired more than once!");
}
return true;
});
int counter = 0;
do {
trigger.run();
timeSource.sleep(1000);
if (counter++ > 10) {
fail("Lost node was not discovered by trigger even after 10 seconds");
}
} while (!fired.get());
TriggerEvent nodeLostEvent = eventRef.get();
assertNotNull(nodeLostEvent);
List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
assertTrue(nodeNames + " doesn't contain " + lostNodeName1, nodeNames.contains(lostNodeName1));
assertTrue(nodeNames + " doesn't contain " + lostNodeName2, nodeNames.contains(lostNodeName2));
}
// remove a node but add it back before the waitFor period expires
// and assert that the trigger doesn't fire at all
try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
final long waitTime = 2;
props.put("waitFor", waitTime);
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
String lostNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
cluster.simRemoveNode(lostNode, false);
AtomicBoolean fired = new AtomicBoolean(false);
trigger.setProcessor(event -> {
if (fired.compareAndSet(false, true)) {
long currentTimeNanos = timeSource.getTime();
long eventTimeNanos = event.getEventTime();
long waitForNanos = TimeUnit.NANOSECONDS.convert(waitTime, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
}
} else {
fail("NodeLostListener was fired more than once!");
}
return true;
});
trigger.run(); // first run should detect the lost node
int counter = 0;
do {
if (cluster.getLiveNodesSet().get().size() == 2) {
break;
}
timeSource.sleep(100);
if (counter++ > 20) {
fail("Live nodes not updated!");
}
} while (true);
counter = 0;
cluster.getSimClusterStateProvider().simRestoreNode(lostNode);
do {
trigger.run();
timeSource.sleep(1000);
if (counter++ > waitTime + 1) { // run it a little more than the wait time
break;
}
} while (true);
// ensure the event was not fired
assertFalse(fired.get());
}
}
public void testActionLifecycle() throws Exception {
Map<String, Object> props = createTriggerProps(0);
List<Map<String, String>> actions = (List<Map<String, String>>) props.get("actions");
Map<String, String> action = new HashMap<>(2);
action.put("name", "testActionInit");
action.put("class", AssertInitTriggerAction.class.getName());
actions.add(action);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
assertEquals(true, actionConstructorCalled.get());
assertEquals(false, actionInitCalled.get());
assertEquals(false, actionCloseCalled.get());
trigger.init();
assertEquals(true, actionInitCalled.get());
assertEquals(false, actionCloseCalled.get());
}
assertEquals(true, actionCloseCalled.get());
}
public static class AssertInitTriggerAction implements TriggerAction {
public AssertInitTriggerAction() {
actionConstructorCalled.set(true);
}
@Override
public String getName() {
return "";
}
@Override
public void process(TriggerEvent event, ActionContext actionContext) {
}
@Override
public void close() throws IOException {
actionCloseCalled.compareAndSet(false, true);
}
@Override
public void init(Map<String, String> args) {
actionInitCalled.compareAndSet(false, true);
}
}
@Test
public void testListenerAcceptance() throws Exception {
Map<String, Object> props = createTriggerProps(0);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, cluster.getLoader(), cluster)) {
trigger.setProcessor(noFirstRunProcessor);
String newNode = cluster.simAddNode();
trigger.run(); // starts tracking live nodes
// stop the newly created node
cluster.simRemoveNode(newNode, true);
AtomicInteger callCount = new AtomicInteger(0);
AtomicBoolean fired = new AtomicBoolean(false);
trigger.setProcessor(event -> {
if (callCount.incrementAndGet() < 2) {
return false;
} else {
fired.compareAndSet(false, true);
return true;
}
});
trigger.run(); // first run should detect the lost node and fire immediately but listener isn't ready
assertEquals(1, callCount.get());
assertFalse(fired.get());
trigger.run(); // second run should again fire
assertEquals(2, callCount.get());
assertTrue(fired.get());
trigger.run(); // should not fire
assertEquals(2, callCount.get());
}
}
@Test
public void testRestoreState() throws Exception {
long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds);
String newNode = cluster.simAddNode();
// remove a node but update the trigger before the waitFor period expires
// and assert that the new trigger still fires
NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster);
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
// stop the newly created node
cluster.simRemoveNode(newNode, true);
trigger.run(); // this run should detect the lost node
trigger.close(); // close the old trigger
try (NodeLostTrigger newTrigger = new NodeLostTrigger("some_different_name", props, cluster.getLoader(), cluster)) {
try {
newTrigger.restoreState(trigger);
fail("Trigger should only be able to restore state from an old trigger of the same name");
} catch (AssertionError e) {
// expected
}
}
try (NodeLostTrigger newTrigger = new NodeLostTrigger("node_lost_trigger", props, cluster.getLoader(), cluster)) {
AtomicBoolean fired = new AtomicBoolean(false);
AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
newTrigger.setProcessor(event -> {
if (fired.compareAndSet(false, true)) {
eventRef.set(event);
long currentTimeNanos = timeSource.getTime();
long eventTimeNanos = event.getEventTime();
long waitForNanos = TimeUnit.NANOSECONDS.convert(waitForSeconds, TimeUnit.SECONDS) - WAIT_FOR_DELTA_NANOS;
if (currentTimeNanos - eventTimeNanos <= waitForNanos) {
fail("NodeLostListener was fired before the configured waitFor period: currentTimeNanos=" + currentTimeNanos + ", eventTimeNanos=" + eventTimeNanos + ",waitForNanos=" + waitForNanos);
}
} else {
fail("NodeLostListener was fired more than once!");
}
return true;
});
newTrigger.restoreState(trigger); // restore state from the old trigger
int counter = 0;
do {
newTrigger.run();
timeSource.sleep(1000);
if (counter++ > 10) {
fail("Lost node was not discovered by trigger even after 10 seconds");
}
} while (!fired.get());
TriggerEvent nodeLostEvent = eventRef.get();
assertNotNull(nodeLostEvent);
List<String> nodeNames = (List<String>)nodeLostEvent.getProperty(TriggerEvent.NODE_NAMES);
assertTrue(nodeNames.contains(newNode));
}
}
private Map<String, Object> createTriggerProps(long waitForSeconds) {
Map<String, Object> props = new HashMap<>();
props.put("event", "nodeLost");
props.put("waitFor", waitForSeconds);
props.put("enabled", true);
List<Map<String, String>> actions = new ArrayList<>(3);
Map<String, String> map = new HashMap<>(2);
map.put("name", "compute_plan");
map.put("class", "solr.ComputePlanAction");
actions.add(map);
map = new HashMap<>(2);
map.put("name", "execute_plan");
map.put("class", "solr.ExecutePlanAction");
actions.add(map);
props.put("actions", actions);
return props;
}
}

View File

@ -0,0 +1,357 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiConsumer;
import org.apache.lucene.util.Constants;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
import org.apache.solr.client.solrj.cloud.autoscaling.Row;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggestion;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.junit.BeforeClass;
import org.junit.rules.ExpectedException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
public class TestPolicyCloud extends SimSolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@org.junit.Rule
public ExpectedException expectedException = ExpectedException.none();
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(5, TimeSource.get("simTime:50"));
}
public void testDataProviderPerReplicaDetails() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
CollectionAdminRequest.createCollection("perReplicaDataColl", "conf", 1, 5)
.process(solrClient);
waitForState("Timeout waiting for collection to become active", "perReplicaDataColl", clusterShape(1, 5));
DocCollection coll = getCollectionState("perReplicaDataColl");
String autoScaleJson = "{" +
" 'cluster-preferences': [" +
" { maximize : freedisk , precision: 50}," +
" { minimize : cores, precision: 2}" +
" ]," +
" 'cluster-policy': [" +
" { replica : '0' , 'nodeRole': 'overseer'}," +
" { 'replica': '<2', 'shard': '#ANY', 'node': '#ANY'" +
" }" +
" ]," +
" 'policies': {" +
" 'policy1': [" +
" { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
" { 'replica': '<2', 'shard': '#EACH', 'sysprop.rack': 'rack1'}" +
" ]" +
" }" +
"}";
AutoScalingConfig config = new AutoScalingConfig((Map<String, Object>) Utils.fromJSONString(autoScaleJson));
Policy.Session session = config.getPolicy().createSession(cluster);
AtomicInteger count = new AtomicInteger(0);
for (Row row : session.getSorted()) {
row.collectionVsShardVsReplicas.forEach((c, shardVsReplicas) -> shardVsReplicas.forEach((s, replicaInfos) -> {
for (ReplicaInfo replicaInfo : replicaInfos) {
if (replicaInfo.getVariables().containsKey(Suggestion.ConditionType.CORE_IDX.tagName)) count.incrementAndGet();
}
}));
}
assertTrue(count.get() > 0);
CollectionAdminRequest.deleteCollection("perReplicaDataColl").process(solrClient);
}
public void testCreateCollectionAddReplica() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode(random());
int port = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.PORT);
String commands = "{set-policy :{c1 : [{replica:0 , shard:'#EACH', port: '!" + port + "'}]}}";
solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
String collectionName = "testCreateCollectionAddReplica";
CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
.setPolicy("c1")
.process(solrClient);
waitForState("Timeout waiting for collection to become active", collectionName, clusterShape(1, 1));
getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
CollectionAdminRequest.addReplicaToShard(collectionName, "shard1").process(solrClient);
waitForState("Timed out waiting to see 2 replicas for collection: " + collectionName,
collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 2);
getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
}
public void testCreateCollectionSplitShard() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String firstNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
int firstNodePort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(firstNode, ImplicitSnitch.PORT);
String secondNode;
int secondNodePort;
while (true) {
secondNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
secondNodePort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(secondNode, ImplicitSnitch.PORT);
if (secondNodePort != firstNodePort) break;
}
String commands = "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + firstNodePort + "'}, {replica:1, shard:'#EACH', port:'" + secondNodePort + "'}]}}";
NamedList<Object> response = solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
assertEquals("success", response.get("result"));
String collectionName = "testCreateCollectionSplitShard";
CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
.setPolicy("c1")
.process(solrClient);
waitForState("Timeout waiting for collection to become active", collectionName, clusterShape(1, 2));
DocCollection docCollection = getCollectionState(collectionName);
List<Replica> list = docCollection.getReplicas(firstNode);
int replicasOnNode1 = list != null ? list.size() : 0;
list = docCollection.getReplicas(secondNode);
int replicasOnNode2 = list != null ? list.size() : 0;
assertEquals("Expected exactly one replica of collection on node with port: " + firstNodePort, 1, replicasOnNode1);
assertEquals("Expected exactly one replica of collection on node with port: " + secondNodePort, 1, replicasOnNode2);
CollectionAdminRequest.splitShard(collectionName).setShardName("shard1").process(solrClient);
waitForState("Timed out waiting to see 6 replicas for collection: " + collectionName,
collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 6);
docCollection = getCollectionState(collectionName);
list = docCollection.getReplicas(firstNode);
replicasOnNode1 = list != null ? list.size() : 0;
list = docCollection.getReplicas(secondNode);
replicasOnNode2 = list != null ? list.size() : 0;
assertEquals("Expected exactly three replica of collection on node with port: " + firstNodePort, 3, replicasOnNode1);
assertEquals("Expected exactly three replica of collection on node with port: " + secondNodePort, 3, replicasOnNode2);
CollectionAdminRequest.deleteCollection(collectionName).process(solrClient);
}
public void testMetricsTag() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'metrics:abc':'overseer', 'replica':0}" +
" ]" +
"}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
try {
solrClient.request(req);
fail("expected exception");
} catch (Exception e) {
// expected
assertTrue(e.toString().contains("Invalid metrics: param in"));
}
setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'metrics:solr.node:ADMIN./admin/authorization.clientErrors:count':'>58768765', 'replica':0}" +
" ]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
//org.eclipse.jetty.server.handler.DefaultHandler.2xx-responses
CollectionAdminRequest.createCollection("metricsTest", "conf", 1, 1)
.process(solrClient);
waitForState("Timeout waiting for collection to become active", "metricsTest", clusterShape(1, 1));
DocCollection collection = getCollectionState("metricsTest");
List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
"metrics:solr.jvm:buffers.direct.Count");
Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNodeName(), tags);
for (String tag : tags) {
assertNotNull( "missing : "+ tag , val.get(tag));
}
}
public void testCreateCollectionAddShardWithReplicaTypeUsingPolicy() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
List<String> nodes = new ArrayList<>(cluster.getClusterStateProvider().getLiveNodes());
String nrtNodeName = nodes.get(0);
int nrtPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nrtNodeName, ImplicitSnitch.PORT);
String pullNodeName = nodes.get(1);
int pullPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(pullNodeName, ImplicitSnitch.PORT);
String tlogNodeName = nodes.get(1);
int tlogPort = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(tlogNodeName, ImplicitSnitch.PORT);
log.info("NRT {} PULL {} , TLOG {} ", nrtNodeName, pullNodeName, tlogNodeName);
String commands = "{set-cluster-policy :[" +
"{replica:0 , shard:'#EACH', type: NRT, port: '!" + nrtPort + "'}" +
"{replica:0 , shard:'#EACH', type: PULL, port: '!" + pullPort + "'}" +
"{replica:0 , shard:'#EACH', type: TLOG, port: '!" + tlogPort + "'}" +
"]}";
solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
Map<String, Object> json = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
assertEquals("full json:" + Utils.toJSONString(json), "!" + nrtPort,
Utils.getObjectByPath(json, true, "cluster-policy[0]/port"));
assertEquals("full json:" + Utils.toJSONString(json), "!" + pullPort,
Utils.getObjectByPath(json, true, "cluster-policy[1]/port"));
assertEquals("full json:" + Utils.toJSONString(json), "!" + tlogPort,
Utils.getObjectByPath(json, true, "cluster-policy[2]/port"));
CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1", 1, 1, 1)
.process(solrClient);
waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(1, 3));
DocCollection coll = getCollectionState("policiesTest");
BiConsumer<String, Replica> verifyReplicas = (s, replica) -> {
switch (replica.getType()) {
case NRT: {
assertTrue("NRT replica should be in " + nrtNodeName, replica.getNodeName().equals(nrtNodeName));
break;
}
case TLOG: {
assertTrue("TLOG replica should be in " + tlogNodeName, replica.getNodeName().equals(tlogNodeName));
break;
}
case PULL: {
assertTrue("PULL replica should be in " + pullNodeName, replica.getNodeName().equals(pullNodeName));
break;
}
}
};
coll.forEachReplica(verifyReplicas);
CollectionAdminRequest.createShard("policiesTest", "s3").
process(solrClient);
coll = getCollectionState("policiesTest");
assertEquals(3, coll.getSlice("s3").getReplicas().size());
coll.forEachReplica(verifyReplicas);
}
public void testCreateCollectionAddShardUsingPolicy() throws Exception {
SolrClient solrClient = cluster.simGetSolrClient();
String nodeId = cluster.getSimClusterStateProvider().simGetRandomNode(random());
int port = (Integer)cluster.getSimNodeStateProvider().simGetNodeValue(nodeId, ImplicitSnitch.PORT);
String commands = "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + port + "'}]}}";
solrClient.request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
Map<String, Object> json = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
assertEquals("full json:"+ Utils.toJSONString(json) , "#EACH",
Utils.getObjectByPath(json, true, "/policies/c1[0]/shard"));
CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "s1,s2", 1)
.setPolicy("c1")
.process(solrClient);
waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(2, 1));
DocCollection coll = getCollectionState("policiesTest");
assertEquals("c1", coll.getPolicyName());
assertEquals(2,coll.getReplicas().size());
coll.forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
CollectionAdminRequest.createShard("policiesTest", "s3").process(solrClient);
waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(3, 1));
coll = getCollectionState("policiesTest");
assertEquals(1, coll.getSlice("s3").getReplicas().size());
coll.getSlice("s3").forEach(replica -> assertEquals(nodeId, replica.getNodeName()));
}
public void testDataProvider() throws IOException, SolrServerException, KeeperException, InterruptedException {
SolrClient solrClient = cluster.simGetSolrClient();
CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2)
.process(solrClient);
waitForState("Timeout waiting for collection to become active", "policiesTest", clusterShape(1, 2));
DocCollection rulesCollection = getCollectionState("policiesTest");
Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
"freedisk",
"cores",
"heapUsage",
"sysLoadAvg"));
assertNotNull(val.get("freedisk"));
assertNotNull(val.get("heapUsage"));
assertNotNull(val.get("sysLoadAvg"));
assertTrue(((Number) val.get("cores")).intValue() > 0);
assertTrue("freedisk value is " + ((Number) val.get("freedisk")).doubleValue(), Double.compare(((Number) val.get("freedisk")).doubleValue(), 0.0d) > 0);
assertTrue("heapUsage value is " + ((Number) val.get("heapUsage")).doubleValue(), Double.compare(((Number) val.get("heapUsage")).doubleValue(), 0.0d) > 0);
if (!Constants.WINDOWS) {
// the system load average metrics is not available on windows platform
assertTrue("sysLoadAvg value is " + ((Number) val.get("sysLoadAvg")).doubleValue(), Double.compare(((Number) val.get("sysLoadAvg")).doubleValue(), 0.0d) > 0);
}
// simulator doesn't have Overseer, so just pick a random node
String overseerNode = cluster.getSimClusterStateProvider().simGetRandomNode(random());
solrClient.request(CollectionAdminRequest.addRole(overseerNode, "overseer"));
for (int i = 0; i < 10; i++) {
Map<String, Object> data = Utils.getJson(cluster.getDistribStateManager(), ZkStateReader.ROLES);
if (i >= 9 && data.isEmpty()) {
throw new RuntimeException("NO overseer node created");
}
cluster.getTimeSource().sleep(100);
}
val = cluster.getNodeStateProvider().getNodeValues(overseerNode, Arrays.asList(
"nodeRole",
"ip_1", "ip_2", "ip_3", "ip_4",
"sysprop.java.version",
"sysprop.java.vendor"));
assertEquals("overseer", val.get("nodeRole"));
assertNotNull(val.get("ip_1"));
assertNotNull(val.get("ip_2"));
assertNotNull(val.get("ip_3"));
assertNotNull(val.get("ip_4"));
assertNotNull(val.get("sysprop.java.version"));
assertNotNull(val.get("sysprop.java.vendor"));
}
}

View File

@ -0,0 +1,220 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling.sim;
import java.nio.charset.Charset;
import java.util.NoSuchElementException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Predicate;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.junit.After;
import org.junit.Test;
/**
*
*/
public class TestSimDistributedQueue extends SolrTestCaseJ4 {
private static final Charset UTF8 = Charset.forName("UTF-8");
protected ExecutorService executor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrjNamedThreadFactory("sdqtest-"));
@Test
public void testDistributedQueue() throws Exception {
String dqZNode = "/distqueue/test1";
byte[] data = "hello world".getBytes(UTF8);
DistributedQueue dq = makeDistributedQueue(dqZNode);
// basic ops
assertNull(dq.poll());
try {
dq.remove();
fail("NoSuchElementException expected");
} catch (NoSuchElementException expected) {
// expected
}
dq.offer(data);
assertArrayEquals(dq.peek(500), data);
assertArrayEquals(dq.remove(), data);
assertNull(dq.poll());
dq.offer(data);
assertArrayEquals(dq.take(), data); // waits for data
assertNull(dq.poll());
dq.offer(data);
dq.peek(true); // wait until data is definitely there before calling remove
assertArrayEquals(dq.remove(), data);
assertNull(dq.poll());
// should block until the background thread makes the offer
(new QueueChangerThread(dq, 1000)).start();
assertNotNull(dq.peek(true));
assertNotNull(dq.remove());
assertNull(dq.poll());
// timeout scenario ... background thread won't offer until long after the peek times out
QueueChangerThread qct = new QueueChangerThread(dq, 1000);
qct.start();
assertNull(dq.peek(500));
qct.join();
}
@Test
public void testDistributedQueueBlocking() throws Exception {
String dqZNode = "/distqueue/test2";
String testData = "hello world";
DistributedQueue dq = makeDistributedQueue(dqZNode);
assertNull(dq.peek());
Future<String> future = executor.submit(() -> new String(dq.peek(true), UTF8));
try {
future.get(1000, TimeUnit.MILLISECONDS);
fail("TimeoutException expected");
} catch (TimeoutException expected) {
assertFalse(future.isDone());
}
dq.offer(testData.getBytes(UTF8));
assertEquals(testData, future.get(1000, TimeUnit.MILLISECONDS));
assertNotNull(dq.poll());
assertNull(dq.peek(100));
// Rerun the earlier test make sure updates are still seen, post reconnection.
future = executor.submit(() -> new String(dq.peek(true), UTF8));
try {
future.get(1000, TimeUnit.MILLISECONDS);
fail("TimeoutException expected");
} catch (TimeoutException expected) {
assertFalse(future.isDone());
}
dq.offer(testData.getBytes(UTF8));
assertEquals(testData, future.get(1000, TimeUnit.MILLISECONDS));
assertNotNull(dq.poll());
assertNull(dq.poll());
}
@Test
public void testLocallyOffer() throws Exception {
String dqZNode = "/distqueue/test3";
DistributedQueue dq = makeDistributedQueue(dqZNode);
dq.peekElements(1, 1, s -> true);
for (int i = 0; i < 100; i++) {
byte[] data = String.valueOf(i).getBytes(UTF8);
dq.offer(data);
assertNotNull(dq.peek());
dq.poll();
dq.peekElements(1, 1, s -> true);
}
}
@Test
public void testPeekElements() throws Exception {
String dqZNode = "/distqueue/test4";
byte[] data = "hello world".getBytes(UTF8);
DistributedQueue dq = makeDistributedQueue(dqZNode);
// Populate with data.
dq.offer(data);
dq.offer(data);
dq.offer(data);
Predicate<String> alwaysTrue = s -> true;
Predicate<String> alwaysFalse = s -> false;
// Should be able to get 0, 1, 2, or 3 instantly
for (int i = 0; i <= 3; ++i) {
assertEquals(i, dq.peekElements(i, 0, alwaysTrue).size());
}
// Asking for more should return only 3.
assertEquals(3, dq.peekElements(4, 0, alwaysTrue).size());
// If we filter everything out, we should block for the full time.
long start = System.nanoTime();
assertEquals(0, dq.peekElements(4, 1000, alwaysFalse).size());
assertTrue(System.nanoTime() - start >= TimeUnit.MILLISECONDS.toNanos(500));
// If someone adds a new matching element while we're waiting, we should return immediately.
executor.submit(() -> {
try {
Thread.sleep(500);
dq.offer(data);
} catch (Exception e) {
// ignore
}
});
start = System.nanoTime();
assertEquals(1, dq.peekElements(4, 2000, child -> {
// The 4th element in the queue will end with a "3".
return child.endsWith("3");
}).size());
long elapsed = System.nanoTime() - start;
assertTrue(elapsed < TimeUnit.MILLISECONDS.toNanos(1000));
assertTrue(elapsed >= TimeUnit.MILLISECONDS.toNanos(250));
}
protected DistributedQueue makeDistributedQueue(String dqZNode) throws Exception {
return new SimDistributedQueueFactory.SimDistributedQueue(dqZNode);
}
private static class QueueChangerThread extends Thread {
DistributedQueue dq;
long waitBeforeOfferMs;
QueueChangerThread(DistributedQueue dq, long waitBeforeOfferMs) {
this.dq = dq;
this.waitBeforeOfferMs = waitBeforeOfferMs;
}
public void run() {
try {
Thread.sleep(waitBeforeOfferMs);
dq.offer(getName().getBytes(UTF8));
} catch (InterruptedException ie) {
// do nothing
} catch (Exception exc) {
throw new RuntimeException(exc);
}
}
}
@Override
@After
public void tearDown() throws Exception {
try {
super.tearDown();
} catch (Exception exc) {
}
executor.shutdown();
}
}

View File

@ -0,0 +1,21 @@
/*
* 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.
*/
/**
* Simulated environment for autoscaling tests.
*/
package org.apache.solr.cloud.autoscaling.sim;

View File

@ -59,6 +59,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrCore;
@ -794,7 +795,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
protected void waitForBootstrapToComplete(String collectionName, String shardId) throws Exception {
NamedList rsp;// we need to wait until bootstrap is complete otherwise the replicator thread will never start
TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
rsp = invokeCdcrAction(shardToLeaderJetty.get(collectionName).get(shardId), CdcrParams.CdcrAction.BOOTSTRAP_STATUS);
if (rsp.get(RESPONSE_STATUS).toString().equals(COMPLETED)) {

View File

@ -39,6 +39,7 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.BadHdfsThreadsFilter;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
@ -221,7 +222,7 @@ public class StressHdfsTest extends BasicDistributedZkTest {
request.setPath("/admin/collections");
cloudClient.request(request);
final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (cloudClient.getZkStateReader().getClusterState().hasCollection(DELETE_DATA_DIR_COLLECTION)) {
if (timeout.hasTimedOut()) {
throw new AssertionError("Timeout waiting to see removed collection leave clusterstate");

View File

@ -33,6 +33,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
@ -213,7 +214,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
writer.writePendingUpdates();
boolean found = false;
TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS);
TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
DocCollection c1 = reader.getClusterState().getCollection("c1");
if ("y".equals(c1.getStr("x"))) {

View File

@ -200,6 +200,9 @@ public class RuleEngineTest extends SolrTestCaseJ4{
@Override
public NodeStateProvider getNodeStateProvider() {
return new NodeStateProvider() {
@Override
public void close() throws IOException { }
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return (Map<String, Object>) MockSnitch.nodeVsTags.get(node);

View File

@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.junit.After;
import org.junit.Before;
@ -318,8 +319,8 @@ class Indexer {
ArrayList<OneIndexer> _threads = new ArrayList<>();
public Indexer(OpenCloseCoreStressTest OCCST, String url, List<HttpSolrClient> clients, int numThreads, int secondsToRun, Random random) {
stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS);
nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
stopTimeout = new TimeOut(secondsToRun, TimeUnit.SECONDS, TimeSource.NANO_TIME);
nextTimeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
docsThisCycle.set(0);
qTimesAccum.set(0);
updateCounts.set(0);
@ -353,7 +354,7 @@ class Indexer {
log.info(String.format(Locale.ROOT, " s indexed: [run %,8d] [cycle %,8d] [last minute %,8d] Last core updated: %s. Seconds left in cycle %,4d",
myId, docsThisCycle.get(), myId - lastCount, core, stopTimeout.timeLeft(TimeUnit.SECONDS)));
lastCount = myId;
nextTimeout = new TimeOut(60, TimeUnit.SECONDS);
nextTimeout = new TimeOut(60, TimeUnit.SECONDS, TimeSource.NANO_TIME);
}
}

View File

@ -520,6 +520,15 @@ public class AutoScalingConfig implements MapWriter {
return withTriggerListenerConfigs(configs);
}
@Override
public Object clone() {
if (jsonMap != null) {
return new AutoScalingConfig(jsonMap);
} else {
return new AutoScalingConfig(getPolicy(), getTriggerConfigs(), getTriggerListenerConfigs(), getProperties(), zkVersion);
}
}
/**
* Return the znode version that was used to create this configuration.
*/

View File

@ -24,6 +24,7 @@ import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
import org.apache.solr.common.util.ObjectCache;
import org.apache.solr.common.util.TimeSource;
/**
* Base class for overriding some behavior of {@link SolrCloudManager}.
@ -31,6 +32,7 @@ import org.apache.solr.common.util.ObjectCache;
public class DelegatingCloudManager implements SolrCloudManager {
private final SolrCloudManager delegate;
private ObjectCache objectCache = new ObjectCache();
private TimeSource timeSource = TimeSource.NANO_TIME;
public DelegatingCloudManager(SolrCloudManager delegate) {
this.delegate = delegate;
@ -61,6 +63,16 @@ public class DelegatingCloudManager implements SolrCloudManager {
return delegate == null ? objectCache : delegate.getObjectCache();
}
@Override
public boolean isClosed() {
return delegate.isClosed();
}
@Override
public TimeSource getTimeSource() {
return delegate == null ? timeSource : delegate.getTimeSource();
}
@Override
public SolrResponse request(SolrRequest req) throws IOException {
return delegate.request(req);
@ -70,4 +82,9 @@ public class DelegatingCloudManager implements SolrCloudManager {
public byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException {
return delegate.httpRequest(url, method, headers, payload, timeout, followRedirects);
}
@Override
public void close() throws IOException {
delegate.close();
}
}

Some files were not shown because too many files have changed in this diff Show More