This commit is contained in:
Noble Paul 2017-10-04 18:53:51 +10:30
commit 0e679e4edf
113 changed files with 11564 additions and 2051 deletions

View File

@ -0,0 +1 @@
2b8c8fbd740164d220ca7d18605b8b2092e163e9

View File

@ -20,6 +20,7 @@ import java.lang.invoke.MethodHandles;
import java.util.concurrent.TimeUnit;
import org.apache.solr.util.TimeSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -31,41 +32,29 @@ public class ActionThrottle {
private volatile Long minMsBetweenActions;
private final String name;
private final TimeSource timeSource;
private final NanoTimeSource nanoTimeSource;
public interface NanoTimeSource {
long getTime();
}
private static class DefaultNanoTimeSource implements NanoTimeSource {
@Override
public long getTime() {
return System.nanoTime();
}
}
public ActionThrottle(String name, long minMsBetweenActions) {
this.name = name;
this.minMsBetweenActions = minMsBetweenActions;
this.nanoTimeSource = new DefaultNanoTimeSource();
this.timeSource = TimeSource.NANO_TIME;
}
public ActionThrottle(String name, long minMsBetweenActions, NanoTimeSource nanoTimeSource) {
public ActionThrottle(String name, long minMsBetweenActions, TimeSource timeSource) {
this.name = name;
this.minMsBetweenActions = minMsBetweenActions;
this.nanoTimeSource = nanoTimeSource;
this.timeSource = timeSource;
}
public void markAttemptingAction() {
lastActionStartedAt = nanoTimeSource.getTime();
lastActionStartedAt = timeSource.getTime();
}
public void minimumWaitBetweenActions() {
if (lastActionStartedAt == null) {
return;
}
long diff = nanoTimeSource.getTime() - lastActionStartedAt;
long diff = timeSource.getTime() - lastActionStartedAt;
int diffMs = (int) TimeUnit.MILLISECONDS.convert(diff, TimeUnit.NANOSECONDS);
long minNsBetweenActions = TimeUnit.NANOSECONDS.convert(minMsBetweenActions, TimeUnit.MILLISECONDS);
log.info("The last {} attempt started {}ms ago.", name, diffMs);

View File

@ -24,8 +24,12 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
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.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
@ -93,10 +97,31 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
final Long policyVersionBefore = PolicyHelper.REF_VERSION.get();
AtomicLong policyVersionAfter = new AtomicLong(-1);
// Kind of unnecessary, but it does put the logic of whether to override maxShardsPerNode in one place.
if (!skipCreateReplicaInClusterState) {
node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
ocmh.overseer.getZkController().getCoreContainer()).get(0).nodeName;// TODO: use replica type in this logic too
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;
if (policyVersionBefore == null && PolicyHelper.REF_VERSION.get() != null) {
policyVersionAfter.set(PolicyHelper.REF_VERSION.get());
}
}
} else {
node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
ocmh.overseer.getZkController().getCoreContainer()).get(0).nodeName;// TODO: use replica type in this logic too
}
}
log.info("Node Identified {} for creating new replica", node);
@ -172,6 +197,9 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
if (instanceDir != null) {
params.set(CoreAdminParams.INSTANCE_DIR, instanceDir);
}
if (coreNodeName != null) {
params.set(CoreAdminParams.CORE_NODE_NAME, coreNodeName);
}
ocmh.addPropertyParams(message, params);
// For tracking async calls.
@ -184,6 +212,10 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
Runnable runnable = () -> {
ocmh.processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica", asyncId, requestMap);
ocmh.waitForCoreNodeName(collection, fnode, fcoreName);
if (policyVersionAfter.get() > -1) {
PolicyHelper.REF_VERSION.remove();
ocmh.policySessionRef.decref(policyVersionAfter.get());
}
if (onComplete != null) onComplete.run();
};

View File

@ -17,6 +17,7 @@
package org.apache.solr.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@ -27,15 +28,14 @@ import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.cloud.rule.ReplicaAssigner;
import org.apache.solr.cloud.rule.Rule;
import org.apache.solr.common.SolrException;
@ -53,9 +53,10 @@ import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.NumberUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.zookeeper.data.Stat;
import static java.util.Collections.singletonMap;
import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.POLICY;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET_EMPTY;
@ -63,11 +64,9 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE_DEFAULT;
import static org.apache.solr.common.cloud.DocCollection.SNITCH;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
public class Assign {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static int incAndGetId(SolrZkClient zkClient, String collection, int defaultValue) {
String path = "/collections/"+collection;
@ -236,8 +235,7 @@ public class Assign {
return nodeList;
}
public static List<ReplicaPosition> identifyNodes(Supplier<CoreContainer> coreContainer,
ZkStateReader zkStateReader,
public static List<ReplicaPosition> identifyNodes(OverseerCollectionMessageHandler ocmh,
ClusterState clusterState,
List<String> nodeList,
String collectionName,
@ -248,13 +246,13 @@ public class Assign {
int numPullReplicas) throws KeeperException, InterruptedException {
List<Map> rulesMap = (List) message.get("rule");
String policyName = message.getStr(POLICY);
Map autoScalingJson = Utils.getJson(zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
AutoScalingConfig autoScalingConfig = ocmh.zkStateReader.getAutoScalingConfig();
if (rulesMap == null && policyName == null && autoScalingJson.get(Policy.CLUSTER_POLICY) == null) {
if (rulesMap == null && policyName == null && autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) {
log.debug("Identify nodes using default");
int i = 0;
List<ReplicaPosition> result = new ArrayList<>();
for (String aShard : shardNames) {
for (String aShard : shardNames)
for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
Replica.Type.TLOG, numTlogReplicas,
Replica.Type.PULL, numPullReplicas
@ -264,11 +262,9 @@ public class Assign {
i++;
}
}
}
return result;
} else {
if (numTlogReplicas + numPullReplicas != 0) {
if (numTlogReplicas + numPullReplicas != 0 && rulesMap != null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
Replica.Type.TLOG + " or " + Replica.Type.PULL + " replica types not supported with placement rules or cluster policies");
}
@ -285,16 +281,26 @@ public class Assign {
(List<Map>) message.get(SNITCH),
new HashMap<>(),//this is a new collection. So, there are no nodes in any shard
nodeList,
coreContainer.get(),
ocmh.overseer.getZkController().getCoreContainer(),
clusterState);
Map<ReplicaPosition, String> nodeMappings = replicaAssigner.getNodeMappings();
return nodeMappings.entrySet().stream()
.map(e -> new ReplicaPosition(e.getKey().shard, e.getKey().index, e.getKey().type, e.getValue()))
.collect(Collectors.toList());
} else {
return getPositionsUsingPolicy(collectionName,
shardNames, numNrtReplicas, policyName, zkStateReader, nodeList);
} else {
if (message.getStr(CREATE_NODE_SET) == null)
nodeList = Collections.emptyList();// unless explicitly specified do not pass node list to Policy
synchronized (ocmh) {
PolicyHelper.SESSION_REF.set(ocmh.policySessionRef);
try {
return getPositionsUsingPolicy(collectionName,
shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, ocmh.zkStateReader, nodeList);
} finally {
PolicyHelper.SESSION_REF.remove();
}
}
}
}
@ -317,10 +323,11 @@ public class Assign {
// Gets a list of candidate nodes to put the required replica(s) on. Throws errors if not enough replicas
// could be created on live nodes given maxShardsPerNode, Replication factor (if from createShard) etc.
public static List<ReplicaCount> getNodesForNewReplicas(ClusterState clusterState, String collectionName,
String shard, int numberOfNodes,
String shard, int nrtReplicas,
Object createNodeSet, CoreContainer cc) throws KeeperException, InterruptedException {
log.debug("getNodesForNewReplicas() shard: {} , replicas : {} , createNodeSet {}", shard, nrtReplicas, createNodeSet );
DocCollection coll = clusterState.getCollection(collectionName);
Integer maxShardsPerNode = coll.getInt(MAX_SHARDS_PER_NODE, 1);
Integer maxShardsPerNode = coll.getMaxShardsPerNode();
List<String> createNodeList = null;
if (createNodeSet instanceof List) {
@ -339,23 +346,23 @@ public class Assign {
availableSlots += (maxShardsPerNode - ent.getValue().thisCollectionNodes);
}
}
if (availableSlots < numberOfNodes) {
if (availableSlots < nrtReplicas) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
String.format(Locale.ROOT, "Cannot create %d new replicas for collection %s given the current number of live nodes and a maxShardsPerNode of %d",
numberOfNodes, collectionName, maxShardsPerNode));
nrtReplicas, collectionName, maxShardsPerNode));
}
}
List l = (List) coll.get(DocCollection.RULE);
List<ReplicaPosition> replicaPositions = null;
if (l != null) {
replicaPositions = getNodesViaRules(clusterState, shard, numberOfNodes, cc, coll, createNodeList, l);
replicaPositions = getNodesViaRules(clusterState, shard, nrtReplicas, cc, coll, createNodeList, l);
}
String policyName = coll.getStr(POLICY);
Map autoScalingJson = Utils.getJson(cc.getZkController().getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
if (policyName != null || autoScalingJson.get(Policy.CLUSTER_POLICY) != null) {
replicaPositions = Assign.getPositionsUsingPolicy(collectionName, Collections.singletonList(shard), numberOfNodes,
policyName, cc.getZkController().getZkStateReader(), createNodeList);
AutoScalingConfig autoScalingConfig = cc.getZkController().zkStateReader.getAutoScalingConfig();
if (policyName != null || !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) {
replicaPositions = Assign.getPositionsUsingPolicy(collectionName, Collections.singletonList(shard), nrtReplicas, 0, 0,
policyName, cc.getZkController().zkStateReader, createNodeList);
}
if(replicaPositions != null){
@ -372,27 +379,41 @@ public class Assign {
}
public static List<ReplicaPosition> getPositionsUsingPolicy(String collName, List<String> shardNames, int numReplicas,
public static List<ReplicaPosition> getPositionsUsingPolicy(String collName, List<String> shardNames,
int nrtReplicas,
int tlogReplicas,
int pullReplicas,
String policyName, ZkStateReader zkStateReader,
List<String> nodesList) throws KeeperException, InterruptedException {
log.debug("shardnames {} NRT {} TLOG {} PULL {} , policy {}, nodeList {}", shardNames, nrtReplicas, tlogReplicas, pullReplicas, policyName, nodesList);
SolrClientDataProvider clientDataProvider = null;
List<ReplicaPosition> replicaPositions = null;
AutoScalingConfig autoScalingConfig = zkStateReader.getAutoScalingConfig();
try (CloudSolrClient csc = new CloudSolrClient.Builder()
.withClusterStateProvider(new ZkClientClusterStateProvider(zkStateReader))
.build()) {
SolrClientDataProvider clientDataProvider = new SolrClientDataProvider(csc);
Map<String, Object> autoScalingJson = Utils.getJson(zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
Map<String, List<String>> locations = PolicyHelper.getReplicaLocations(collName,
autoScalingJson,
clientDataProvider, singletonMap(collName, policyName), shardNames, numReplicas, nodesList);
List<ReplicaPosition> result = new ArrayList<>();
for (Map.Entry<String, List<String>> e : locations.entrySet()) {
List<String> value = e.getValue();
for (int i = 0; i < value.size(); i++) {
result.add(new ReplicaPosition(e.getKey(), i, Replica.Type.NRT, value.get(i)));
}
}
return result;
clientDataProvider = new SolrClientDataProvider(csc);
Map<String, String> kvMap = Collections.singletonMap(collName, policyName);
replicaPositions = PolicyHelper.getReplicaLocations(
collName,
autoScalingConfig,
clientDataProvider,
kvMap,
shardNames,
nrtReplicas,
tlogReplicas,
pullReplicas,
nodesList);
return replicaPositions;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error closing CloudSolrClient",e);
} finally {
if (log.isTraceEnabled()) {
if (clientDataProvider != null) log.trace("CLUSTER_DATA_PROVIDER: " + Utils.toJSONString(clientDataProvider));
if (replicaPositions != null)
log.trace("REPLICA_POSITIONS: " + Utils.toJSONString(Utils.getDeepCopy(replicaPositions, 7, true)));
log.trace("AUTOSCALING_CONF: " + Utils.toJSONString(autoScalingConfig));
}
}
}
@ -444,7 +465,7 @@ public class Assign {
return nodeNameVsShardCount;
}
DocCollection coll = clusterState.getCollection(collectionName);
Integer maxShardsPerNode = coll.getInt(MAX_SHARDS_PER_NODE, 1);
Integer maxShardsPerNode = coll.getMaxShardsPerNode();
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
DocCollection c = entry.getValue();

View File

@ -32,7 +32,6 @@ import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrResourceLoader;
@ -50,7 +49,8 @@ public class CloudUtil {
* + throw exception if it has been.
*/
public static void checkSharedFSFailoverReplaced(CoreContainer cc, CoreDescriptor desc) {
if (!cc.isSharedFs(desc)) return;
ZkController zkController = cc.getZkController();
String thisCnn = zkController.getCoreNodeName(desc);
String thisBaseUrl = zkController.getBaseUrl();
@ -66,11 +66,10 @@ public class CloudUtil {
String cnn = replica.getName();
String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
boolean isSharedFs = replica.getStr(CoreAdminParams.DATA_DIR) != null;
log.debug("compare against coreNodeName={} baseUrl={}", cnn, baseUrl);
if (thisCnn != null && thisCnn.equals(cnn)
&& !thisBaseUrl.equals(baseUrl) && isSharedFs) {
&& !thisBaseUrl.equals(baseUrl)) {
if (cc.getLoadedCoreNames().contains(desc.getName())) {
cc.unload(desc.getName());
}

View File

@ -30,20 +30,22 @@ import java.util.Properties;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
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;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
@ -63,7 +65,12 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.COLL_CONF;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.RANDOM;
import static org.apache.solr.common.cloud.ZkStateReader.*;
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;
import static org.apache.solr.common.params.CommonParams.NAME;
@ -86,7 +93,6 @@ public class CreateCollectionCmd implements Cmd {
if (clusterState.hasCollection(collectionName)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "collection already exists: " + collectionName);
}
boolean usePolicyFramework = usePolicyFramework(ocmh.zkStateReader, message);
String configName = getConfigName(collectionName, message);
if (configName == null) {
@ -102,6 +108,9 @@ public class CreateCollectionCmd implements Cmd {
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);
@ -170,9 +179,7 @@ public class CreateCollectionCmd implements Cmd {
+ ". This requires " + requestedShardsToCreate
+ " shards to be created (higher than the allowed number)");
}
replicaPositions = Assign.identifyNodes(() -> ocmh.overseer.getZkController().getCoreContainer(),
ocmh.zkStateReader
replicaPositions = Assign.identifyNodes(ocmh
, clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
}
@ -306,6 +313,8 @@ public class CreateCollectionCmd implements Cmd {
throw ex;
} catch (Exception ex) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
} finally {
PolicyHelper.clearFlagAndDecref(ocmh.policySessionRef);
}
}
String getConfigName(String coll, ZkNodeProps message) throws KeeperException, InterruptedException {
@ -317,7 +326,7 @@ public class CreateCollectionCmd implements Cmd {
try {
configNames = ocmh.zkStateReader.getZkClient().getChildren(ZkConfigManager.CONFIGS_ZKNODE, null, true);
if (configNames.contains(ConfigSetsHandlerApi.DEFAULT_CONFIGSET_NAME)) {
if (!".system".equals(coll)) {
if (!CollectionAdminParams.SYSTEM_COLL.equals(coll)) {
copyDefaultConfigSetTo(configNames, coll);
}
return coll;

View File

@ -18,33 +18,42 @@ package org.apache.solr.cloud;
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.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.cloud.ZkNodeProps;
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.Utils;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.Assign.getNodesForNewReplicas;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.RANDOM;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
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.SHARD_ID_PROP;
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.CommonAdminParams.ASYNC;
public class CreateShardCmd implements Cmd {
@ -75,67 +84,84 @@ public class CreateShardCmd implements Cmd {
}
Object createNodeSetStr = message.get(OverseerCollectionMessageHandler.CREATE_NODE_SET);
List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas,
createNodeSetStr, ocmh.overseer.getZkController().getCoreContainer());
ZkStateReader zkStateReader = ocmh.zkStateReader;
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);
int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
CountDownLatch countDownLatch = new CountDownLatch(totalReplicas);
for (int j = 1; j <= totalReplicas; j++) {
Replica.Type typeToCreate;
if (createdNrtReplicas < numNrtReplicas) {
createdNrtReplicas++;
typeToCreate = Replica.Type.NRT;
} else if (createdTlogReplicas < numTlogReplicas) {
createdTlogReplicas++;
typeToCreate = Replica.Type.TLOG;
boolean usePolicyFramework = usePolicyFramework(collection,ocmh);
List<ReplicaPosition> positions = null;
CountDownLatch 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);
} else {
createdPullReplicas++;
typeToCreate = Replica.Type.PULL;
}
String nodeName = sortedNodeList.get(((j - 1)) % sortedNodeList.size()).nodeName;
String coreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), collection, sliceName, typeToCreate);
// String coreName = collectionName + "_" + sliceName + "_replica" + j;
log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName
+ " on " + nodeName);
// Need to create new params for each request
ZkNodeProps addReplicasProps = new ZkNodeProps(
COLLECTION_PROP, collectionName,
SHARD_ID_PROP, sliceName,
CoreAdminParams.REPLICA_TYPE, typeToCreate.name(),
CoreAdminParams.NODE, nodeName,
CoreAdminParams.NAME, coreName);
Map<String, Object> propertyParams = new HashMap<>();
ocmh.addPropertyParams(message, propertyParams);;
addReplicasProps = addReplicasProps.plus(propertyParams);
if(async!=null) addReplicasProps.getProperties().put(ASYNC, async);
final NamedList addResult = new NamedList();
ocmh.addReplica(zkStateReader.getClusterState(), addReplicasProps, addResult, ()-> {
countDownLatch.countDown();
Object addResultFailure = addResult.get("failure");
if (addResultFailure != null) {
SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
if (failure == null) {
failure = new SimpleOrderedMap();
results.add("failure", failure);
List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas,
createNodeSetStr, ocmh.overseer.getZkController().getCoreContainer());
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++;
}
failure.addAll((NamedList) addResultFailure);
} else {
SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
if (success == null) {
success = new SimpleOrderedMap();
results.add("success", success);
}
success.addAll((NamedList) addResult.get("success"));
}
});
}
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 CountDownLatch(totalReplicas);
for (ReplicaPosition position : positions) {
String nodeName = position.node;
String coreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), collection, sliceName, position.type);
log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName
+ " on " + nodeName);
// Need to create new params for each request
ZkNodeProps addReplicasProps = new ZkNodeProps(
COLLECTION_PROP, collectionName,
SHARD_ID_PROP, sliceName,
ZkStateReader.REPLICA_TYPE, position.type.name(),
CoreAdminParams.NODE, nodeName,
CoreAdminParams.NAME, coreName);
Map<String, Object> propertyParams = new HashMap<>();
ocmh.addPropertyParams(message, propertyParams);
addReplicasProps = addReplicasProps.plus(propertyParams);
if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
final NamedList addResult = new NamedList();
ocmh.addReplica(zkStateReader.getClusterState(), addReplicasProps, addResult, () -> {
countDownLatch.countDown();
Object addResultFailure = addResult.get("failure");
if (addResultFailure != null) {
SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
if (failure == null) {
failure = new SimpleOrderedMap();
results.add("failure", failure);
}
failure.addAll((NamedList) addResultFailure);
} else {
SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
if (success == null) {
success = new SimpleOrderedMap();
results.add("success", success);
}
success.addAll((NamedList) addResult.get("success"));
}
});
}
} finally {
PolicyHelper.clearFlagAndDecref(ocmh.policySessionRef);
}
log.debug("Waiting for create shard action to complete");
@ -145,4 +171,10 @@ public class CreateShardCmd implements Cmd {
log.info("Finished create command on all shards for collection: " + collectionName);
}
static boolean usePolicyFramework(DocCollection collection, OverseerCollectionMessageHandler ocmh)
throws KeeperException, InterruptedException {
Map autoScalingJson = Utils.getJson(ocmh.zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
return autoScalingJson.get(Policy.CLUSTER_POLICY) != null || collection.getPolicyName() != null;
}
}

View File

@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.solr.common.NonExistentCoreException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -57,6 +58,8 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
ZkStateReader zkStateReader = ocmh.zkStateReader;
final String collection = message.getStr(NAME);
DocCollection coll = state.getCollectionOrNull(collection);
String policy = coll == null ? null : coll.getPolicyName();
try {
// Remove the snapshots meta-data for this collection in ZK. Deleting actual index files
// should be taken care of as part of collection delete operation.

View File

@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import com.codahale.metrics.Timer;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.cloud.autoscaling.OverseerTriggerThread;
import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.cloud.overseer.CollectionMutator;
import org.apache.solr.cloud.overseer.NodeMutator;
@ -469,8 +470,8 @@ public class Overseer implements Closeable {
private OverseerThread ccThread;
private OverseerThread updaterThread;
private OverseerThread arfoThread;
private OverseerThread triggerThread;
private final ZkStateReader reader;
@ -521,16 +522,14 @@ public class Overseer implements Closeable {
overseerCollectionConfigSetProcessor = new OverseerCollectionConfigSetProcessor(reader, id, shardHandler, adminPath, stats, Overseer.this, overseerPrioritizer);
ccThread = new OverseerThread(ccTg, overseerCollectionConfigSetProcessor, "OverseerCollectionConfigSetProcessor-" + id);
ccThread.setDaemon(true);
ThreadGroup ohcfTg = new ThreadGroup("Overseer Hdfs SolrCore Failover Thread.");
OverseerAutoReplicaFailoverThread autoReplicaFailoverThread = new OverseerAutoReplicaFailoverThread(config, reader, updateShardHandler);
arfoThread = new OverseerThread(ohcfTg, autoReplicaFailoverThread, "OverseerHdfsCoreFailoverThread-" + id);
arfoThread.setDaemon(true);
ThreadGroup triggerThreadGroup = new ThreadGroup("Overseer autoscaling triggers");
OverseerTriggerThread trigger = new OverseerTriggerThread(zkController, config);
triggerThread = new OverseerThread(triggerThreadGroup, trigger, "OverseerAutoScalingTriggerThread-" + id);
updaterThread.start();
ccThread.start();
arfoThread.start();
triggerThread.start();
assert ObjectReleaseTracker.track(this);
}
@ -571,11 +570,10 @@ public class Overseer implements Closeable {
IOUtils.closeQuietly(ccThread);
ccThread.interrupt();
}
if (arfoThread != null) {
IOUtils.closeQuietly(arfoThread);
arfoThread.interrupt();
if (triggerThread != null) {
IOUtils.closeQuietly(triggerThread);
triggerThread.interrupt();
}
if (updaterThread != null) {
try {
updaterThread.join();
@ -586,15 +584,14 @@ public class Overseer implements Closeable {
ccThread.join();
} catch (InterruptedException e) {}
}
if (arfoThread != null) {
if (triggerThread != null) {
try {
arfoThread.join();
} catch (InterruptedException e) {}
triggerThread.join();
} catch (InterruptedException e) {}
}
updaterThread = null;
ccThread = null;
arfoThread = null;
triggerThread = null;
}
/**

View File

@ -1,531 +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.cloud;
import java.io.Closeable;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ClusterStateUtil;
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.core.CloudConfig;
import org.apache.solr.update.UpdateShardHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;
// TODO: how to tmp exclude nodes?
// TODO: more fine grained failover rules?
// TODO: test with lots of collections
// TODO: add config for only failover if replicas is < N
// TODO: general support for non shared filesystems
// this is specialized for a shared file system, but it should
// not be much work to generalize
// NOTE: using replication can slow down failover if a whole
// shard is lost.
/**
*
* In this simple initial implementation we are limited in how quickly we detect
* a failure by a worst case of roughly zk session timeout + WAIT_AFTER_EXPIRATION_SECONDS + WORK_LOOP_DELAY_MS
* and best case of roughly zk session timeout + WAIT_AFTER_EXPIRATION_SECONDS. Also, consider the time to
* create the SolrCore, do any recovery necessary, and warm up the readers.
*
* NOTE: this will only work with collections created via the collections api because they will have defined
* replicationFactor and maxShardsPerNode.
*
* @lucene.experimental
*/
public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private Integer lastClusterStateVersion;
private final ExecutorService updateExecutor;
private volatile boolean isClosed;
private ZkStateReader zkStateReader;
private final Cache<String,Long> baseUrlForBadNodes;
private Set<String> liveNodes = Collections.EMPTY_SET;
private final int workLoopDelay;
private final int waitAfterExpiration;
private volatile Thread thread;
public OverseerAutoReplicaFailoverThread(CloudConfig config, ZkStateReader zkStateReader,
UpdateShardHandler updateShardHandler) {
this.zkStateReader = zkStateReader;
this.workLoopDelay = config.getAutoReplicaFailoverWorkLoopDelay();
this.waitAfterExpiration = config.getAutoReplicaFailoverWaitAfterExpiration();
int badNodeExpiration = config.getAutoReplicaFailoverBadNodeExpiration();
log.debug(
"Starting "
+ this.getClass().getSimpleName()
+ " autoReplicaFailoverWorkLoopDelay={} autoReplicaFailoverWaitAfterExpiration={} autoReplicaFailoverBadNodeExpiration={}",
workLoopDelay, waitAfterExpiration, badNodeExpiration);
baseUrlForBadNodes = CacheBuilder.newBuilder()
.concurrencyLevel(1).expireAfterWrite(badNodeExpiration, TimeUnit.MILLISECONDS).build();
// TODO: Speed up our work loop when live_nodes changes??
updateExecutor = updateShardHandler.getUpdateExecutor();
// TODO: perhaps do a health ping periodically to each node (scaryish)
// And/OR work on JIRA issue around self health checks (SOLR-5805)
}
@Override
public void run() {
this.thread = Thread.currentThread();
while (!this.isClosed) {
// work loop
log.debug("do " + this.getClass().getSimpleName() + " work loop");
// every n, look at state and make add / remove calls
try {
doWork();
} catch (Exception e) {
SolrException.log(log, this.getClass().getSimpleName()
+ " had an error in its thread work loop.", e);
}
if (!this.isClosed) {
try {
Thread.sleep(workLoopDelay);
} catch (InterruptedException e) {
return;
}
}
}
}
private void doWork() {
// TODO: extract to configurable strategy class ??
ClusterState clusterState = zkStateReader.getClusterState();
//check if we have disabled autoAddReplicas cluster wide
String autoAddReplicas = zkStateReader.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, (String) null);
if (autoAddReplicas != null && autoAddReplicas.equals("false")) {
return;
}
if (clusterState != null) {
if (clusterState.getZkClusterStateVersion() != null &&
clusterState.getZkClusterStateVersion().equals(lastClusterStateVersion) && baseUrlForBadNodes.size() == 0 &&
liveNodes.equals(clusterState.getLiveNodes())) {
// nothing has changed, no work to do
return;
}
liveNodes = clusterState.getLiveNodes();
lastClusterStateVersion = clusterState.getZkClusterStateVersion();
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
log.debug("look at collection={}", entry.getKey());
DocCollection docCollection = entry.getValue();
if (!docCollection.getAutoAddReplicas()) {
log.debug("Collection {} is not setup to use autoAddReplicas, skipping..", docCollection.getName());
continue;
}
if (docCollection.getReplicationFactor() == null) {
log.debug("Skipping collection because it has no defined replicationFactor, name={}", docCollection.getName());
continue;
}
log.debug("Found collection, name={} replicationFactor={}", entry.getKey(), docCollection.getReplicationFactor());
Collection<Slice> slices = docCollection.getSlices();
for (Slice slice : slices) {
if (slice.getState() == Slice.State.ACTIVE) {
final Collection<DownReplica> downReplicas = new ArrayList<DownReplica>();
int goodReplicas = findDownReplicasInSlice(clusterState, docCollection, slice, downReplicas);
log.debug("collection={} replicationFactor={} goodReplicaCount={}", docCollection.getName(), docCollection.getReplicationFactor(), goodReplicas);
if (downReplicas.size() > 0 && goodReplicas < docCollection.getReplicationFactor()) {
// badReplicaMap.put(collection, badReplicas);
processBadReplicas(entry.getKey(), downReplicas);
} else if (goodReplicas > docCollection.getReplicationFactor()) {
log.debug("There are too many replicas");
}
}
}
}
}
}
private void processBadReplicas(final String collection, final Collection<DownReplica> badReplicas) {
for (DownReplica badReplica : badReplicas) {
log.debug("process down replica={} from collection={}", badReplica.replica.getName(), collection);
String baseUrl = badReplica.replica.getStr(ZkStateReader.BASE_URL_PROP);
Long wentBadAtNS = baseUrlForBadNodes.getIfPresent(baseUrl);
if (wentBadAtNS == null) {
log.warn("Replica {} may need to failover.",
badReplica.replica.getName());
baseUrlForBadNodes.put(baseUrl, System.nanoTime());
} else {
long elasped = System.nanoTime() - wentBadAtNS;
if (elasped < TimeUnit.NANOSECONDS.convert(waitAfterExpiration, TimeUnit.MILLISECONDS)) {
// protect against ZK 'flapping', startup and shutdown
log.debug("Looks troublesome...continue. Elapsed={}", elasped + "ns");
} else {
log.debug("We need to add a replica. Elapsed={}", elasped + "ns");
if (addReplica(collection, badReplica)) {
baseUrlForBadNodes.invalidate(baseUrl);
}
}
}
}
}
private boolean addReplica(final String collection, DownReplica badReplica) {
// first find best home - first strategy, sort by number of cores
// hosted where maxCoresPerNode is not violated
final Integer maxCoreCount = zkStateReader.getClusterProperty(ZkStateReader.MAX_CORES_PER_NODE, (Integer) null);
final String createUrl = getBestCreateUrl(zkStateReader, badReplica, maxCoreCount);
if (createUrl == null) {
log.warn("Could not find a node to create new replica on.");
return false;
}
// NOTE: we send the absolute path, which will slightly change
// behavior of these cores as they won't respond to changes
// in the solr.hdfs.home sys prop as they would have.
final String dataDir = badReplica.replica.getStr("dataDir");
final String ulogDir = badReplica.replica.getStr("ulogDir");
final String coreNodeName = badReplica.replica.getName();
final String shardId = badReplica.slice.getName();
if (dataDir != null) {
// need an async request - full shard goes down leader election
final String coreName = badReplica.replica.getStr(ZkStateReader.CORE_NAME_PROP);
log.debug("submit call to {}", createUrl);
MDC.put("OverseerAutoReplicaFailoverThread.createUrl", createUrl);
try {
updateExecutor.submit(() -> createSolrCore(collection, createUrl, dataDir, ulogDir, coreNodeName, coreName, shardId));
} finally {
MDC.remove("OverseerAutoReplicaFailoverThread.createUrl");
}
// wait to see state for core we just created
boolean success = ClusterStateUtil.waitToSeeLiveReplica(zkStateReader, collection, coreNodeName, createUrl, 30000);
if (!success) {
log.error("Creating new replica appears to have failed, timed out waiting to see created SolrCore register in the clusterstate.");
return false;
}
return true;
}
log.warn("Could not find dataDir or ulogDir in cluster state.");
return false;
}
private static int findDownReplicasInSlice(ClusterState clusterState, DocCollection collection, Slice slice, final Collection<DownReplica> badReplicas) {
int goodReplicas = 0;
Collection<Replica> replicas = slice.getReplicas();
if (replicas != null) {
for (Replica replica : replicas) {
// on a live node?
boolean live = clusterState.liveNodesContain(replica.getNodeName());
final Replica.State state = replica.getState();
final boolean okayState = state == Replica.State.DOWN
|| state == Replica.State.RECOVERING
|| state == Replica.State.ACTIVE;
log.debug("Process replica name={} live={} state={}", replica.getName(), live, state.toString());
if (live && okayState) {
goodReplicas++;
} else {
DownReplica badReplica = new DownReplica();
badReplica.replica = replica;
badReplica.slice = slice;
badReplica.collection = collection;
badReplicas.add(badReplica);
}
}
}
log.debug("bad replicas for slice {}", badReplicas);
return goodReplicas;
}
/**
*
* @return the best node to replace the badReplica on or null if there is no
* such node
*/
static String getBestCreateUrl(ZkStateReader zkStateReader, DownReplica badReplica, Integer maxCoreCount) {
assert badReplica != null;
assert badReplica.collection != null;
assert badReplica.slice != null;
log.debug("getBestCreateUrl for " + badReplica.replica);
Map<String,Counts> counts = new HashMap<>();
Set<String> unsuitableHosts = new HashSet<>();
Set<String> liveNodes = new HashSet<>(zkStateReader.getClusterState().getLiveNodes());
Map<String, Integer> coresPerNode = new HashMap<>();
ClusterState clusterState = zkStateReader.getClusterState();
if (clusterState != null) {
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
String collection = entry.getKey();
log.debug("look at collection {} as possible create candidate", collection);
DocCollection docCollection = entry.getValue();
// TODO - only operate on collections with sharedfs failover = true ??
Collection<Slice> slices = docCollection.getSlices();
for (Slice slice : slices) {
// only look at active shards
if (slice.getState() == Slice.State.ACTIVE) {
log.debug("look at slice {} for collection {} as possible create candidate", slice.getName(), collection);
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
liveNodes.remove(replica.getNodeName());
String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
if (coresPerNode.containsKey(baseUrl)) {
Integer nodeCount = coresPerNode.get(baseUrl);
coresPerNode.put(baseUrl, nodeCount++);
} else {
coresPerNode.put(baseUrl, 1);
}
if (baseUrl.equals(badReplica.replica.getStr(ZkStateReader.BASE_URL_PROP))) {
continue;
}
// on a live node?
log.debug("collection={} nodename={} livenodes={}", collection, replica.getNodeName(), clusterState.getLiveNodes());
boolean live = clusterState.liveNodesContain(replica.getNodeName());
log.debug("collection={} look at replica {} as possible create candidate, live={}", collection, replica.getName(), live);
if (live) {
Counts cnt = counts.get(baseUrl);
if (cnt == null) {
cnt = new Counts();
}
if (badReplica.collection.getName().equals(collection)) {
cnt.negRankingWeight += 3;
cnt.collectionShardsOnNode += 1;
} else {
cnt.negRankingWeight += 1;
}
if (badReplica.collection.getName().equals(collection) && badReplica.slice.getName().equals(slice.getName())) {
cnt.ourReplicas++;
}
Integer maxShardsPerNode = badReplica.collection.getMaxShardsPerNode();
if (maxShardsPerNode == null) {
log.warn("maxShardsPerNode is not defined for collection, name=" + badReplica.collection.getName());
maxShardsPerNode = Integer.MAX_VALUE;
}
log.debug("collection={} node={} maxShardsPerNode={} maxCoresPerNode={} potential hosts={}",
collection, baseUrl, maxShardsPerNode, maxCoreCount, cnt);
Collection<Replica> badSliceReplicas = null;
DocCollection c = clusterState.getCollection(badReplica.collection.getName());
if (c != null) {
Slice s = c.getSlice(badReplica.slice.getName());
if (s != null) {
badSliceReplicas = s.getReplicas();
}
}
boolean alreadyExistsOnNode = replicaAlreadyExistsOnNode(zkStateReader.getClusterState(), badSliceReplicas, badReplica, baseUrl);
if (unsuitableHosts.contains(baseUrl) || alreadyExistsOnNode || cnt.collectionShardsOnNode >= maxShardsPerNode
|| (maxCoreCount != null && coresPerNode.get(baseUrl) >= maxCoreCount) ) {
counts.remove(baseUrl);
unsuitableHosts.add(baseUrl);
log.debug("not a candidate node, collection={} node={} max shards per node={} good replicas={}", collection, baseUrl, maxShardsPerNode, cnt);
} else {
counts.put(baseUrl, cnt);
log.debug("is a candidate node, collection={} node={} max shards per node={} good replicas={}", collection, baseUrl, maxShardsPerNode, cnt);
}
}
}
}
}
}
}
for (String node : liveNodes) {
counts.put(zkStateReader.getBaseUrlForNodeName(node), new Counts(0, 0));
}
if (counts.size() == 0) {
log.debug("no suitable hosts found for getBestCreateUrl for collection={}", badReplica.collection.getName());
return null;
}
ValueComparator vc = new ValueComparator(counts);
Map<String,Counts> sortedCounts = new TreeMap<String, Counts>(vc);
sortedCounts.putAll(counts);
log.debug("empty nodes={} for collection={}", liveNodes, badReplica.collection.getName());
log.debug("sorted hosts={} for collection={}", sortedCounts, badReplica.collection.getName());
log.debug("unsuitable hosts={} for collection={}", unsuitableHosts, badReplica.collection.getName());
return sortedCounts.keySet().iterator().next();
}
private static boolean replicaAlreadyExistsOnNode(ClusterState clusterState, Collection<Replica> replicas, DownReplica badReplica, String baseUrl) {
if (replicas != null) {
log.debug("collection={} check if replica already exists on node using replicas {}", badReplica.collection.getName(), getNames(replicas));
for (Replica replica : replicas) {
final Replica.State state = replica.getState();
if (!replica.getName().equals(badReplica.replica.getName()) && replica.getStr(ZkStateReader.BASE_URL_PROP).equals(baseUrl)
&& clusterState.liveNodesContain(replica.getNodeName())
&& (state == Replica.State.ACTIVE || state == Replica.State.DOWN || state == Replica.State.RECOVERING)) {
log.debug("collection={} replica already exists on node, bad replica={}, existing replica={}, node name={}", badReplica.collection.getName(), badReplica.replica.getName(), replica.getName(), replica.getNodeName());
return true;
}
}
}
log.debug("collection={} replica does not yet exist on node: {}", badReplica.collection.getName(), baseUrl);
return false;
}
private static Object getNames(Collection<Replica> replicas) {
Set<String> names = new HashSet<>(replicas.size());
for (Replica replica : replicas) {
names.add(replica.getName());
}
return names;
}
private boolean createSolrCore(final String collection,
final String createUrl, final String dataDir, final String ulogDir,
final String coreNodeName, final String coreName, final String shardId) {
try (HttpSolrClient client = new HttpSolrClient.Builder(createUrl)
.withConnectionTimeout(30000)
.withSocketTimeout(60000)
.build()) {
log.debug("create url={}", createUrl);
Create createCmd = new Create();
createCmd.setCollection(collection);
createCmd.setCoreNodeName(coreNodeName);
// TODO: how do we ensure unique coreName
// for now, the collections API will use unique names
createCmd.setShardId(shardId);
createCmd.setCoreName(coreName);
createCmd.setDataDir(dataDir);
createCmd.setUlogDir(ulogDir.substring(0, ulogDir.length() - "/tlog".length()));
client.request(createCmd);
} catch (Exception e) {
SolrException.log(log, "Exception trying to create new replica on " + createUrl, e);
return false;
}
return true;
}
private static class ValueComparator implements Comparator<String> {
Map<String,Counts> map;
public ValueComparator(Map<String,Counts> map) {
this.map = map;
}
public int compare(String a, String b) {
if (map.get(a).negRankingWeight >= map.get(b).negRankingWeight) {
return 1;
} else {
return -1;
}
}
}
@Override
public void close() {
isClosed = true;
Thread lThread = thread;
if (lThread != null) {
lThread.interrupt();
}
}
public boolean isClosed() {
return isClosed;
}
private static class Counts {
int collectionShardsOnNode = 0;
int negRankingWeight = 0;
int ourReplicas = 0;
private Counts() {
}
private Counts(int totalReplicas, int ourReplicas) {
this.negRankingWeight = totalReplicas;
this.ourReplicas = ourReplicas;
}
@Override
public String toString() {
return "Counts [negRankingWeight=" + negRankingWeight + ", sameSliceCount="
+ ourReplicas + ", collectionShardsOnNode=" + collectionShardsOnNode + "]";
}
}
static class DownReplica {
Replica replica;
Slice slice;
DocCollection collection;
@Override
public String toString() {
return "DownReplica [replica=" + replica.getName() + ", slice="
+ slice.getName() + ", collection=" + collection.getName() + "]";
}
}
}

View File

@ -36,6 +36,7 @@ import com.google.common.collect.ImmutableMap;
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.autoscaling.PolicyHelper;
import org.apache.solr.common.cloud.DistributedQueue;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
@ -441,7 +442,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
}
}
void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws Exception {
void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws KeeperException, InterruptedException {
ZkNodeProps m = new ZkNodeProps(
Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
ZkStateReader.CORE_NAME_PROP, core,
@ -463,7 +464,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
//TODO should we not remove in the next release ?
private void migrateStateFormat(ClusterState state, ZkNodeProps message, NamedList results)
throws Exception {
throws KeeperException, InterruptedException {
final String collectionName = message.getStr(COLLECTION_PROP);
boolean firstLoop = true;
@ -634,7 +635,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
private void modifyCollection(ClusterState clusterState, ZkNodeProps message, NamedList results)
throws Exception {
throws KeeperException, InterruptedException {
final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
//the rest of the processing is based on writing cluster state properties
@ -668,6 +669,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
if (areChangesVisible) break;
Thread.sleep(100);
}
if (!areChangesVisible)
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not modify collection " + message);
}
@ -973,6 +975,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
);
}
public final PolicyHelper.SessionRef policySessionRef = new PolicyHelper.SessionRef();
@Override
public void close() throws IOException {
if (tpe != null) {

View File

@ -34,6 +34,7 @@ import java.util.Properties;
import java.util.Set;
import org.apache.solr.common.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.SolrException;
@ -215,135 +216,138 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
List<String> sliceNames = new ArrayList<>();
restoreCollection.getSlices().forEach(x -> sliceNames.add(x.getName()));
List<ReplicaPosition> replicaPositions = Assign.identifyNodes(() -> ocmh.overseer.getZkController().getCoreContainer(),
ocmh.zkStateReader, clusterState,
nodeList, restoreCollectionName,
message, sliceNames,
numNrtReplicas, numTlogReplicas, numPullReplicas);
//Create one replica per shard and copy backed up data to it
for (Slice slice : restoreCollection.getSlices()) {
log.debug("Adding replica for shard={} collection={} ", slice.getName(), restoreCollection);
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, CREATESHARD);
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
if (numNrtReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.NRT.name());
} else if (numTlogReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.TLOG.name());
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " +
Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
}
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
}
}
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), new NamedList(), null);
}
//refresh the location copy of collection state
restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
//Copy data from backed up index to each replica
for (Slice slice : restoreCollection.getSlices()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.RESTORECORE.toString());
params.set(NAME, "snapshot." + slice.getName());
params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString());
params.set(CoreAdminParams.BACKUP_REPOSITORY, repo);
ocmh.sliceCmd(clusterState, params, null, slice, shardHandler, asyncId, requestMap);
}
ocmh.processResponses(new NamedList(), shardHandler, true, "Could not restore core", asyncId, requestMap);
//Mark all shards in ACTIVE STATE
{
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
propMap.put(ZkStateReader.COLLECTION_PROP, restoreCollectionName);
for (Slice shard : restoreCollection.getSlices()) {
propMap.put(shard.getName(), Slice.State.ACTIVE.toString());
}
inQueue.offer(Utils.toJSON(new ZkNodeProps(propMap)));
}
//refresh the location copy of collection state
restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
if (totalReplicasPerShard > 1) {
log.info("Adding replicas to restored collection={}", restoreCollection);
try {
List<ReplicaPosition> replicaPositions = Assign.identifyNodes(
ocmh, clusterState,
nodeList, restoreCollectionName,
message, sliceNames,
numNrtReplicas, numTlogReplicas, numPullReplicas);
//Create one replica per shard and copy backed up data to it
for (Slice slice : restoreCollection.getSlices()) {
log.debug("Adding replica for shard={} collection={} ", slice.getName(), restoreCollection);
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, CREATESHARD);
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
//Add the remaining replicas for each shard, considering it's type
int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
// We already created either a NRT or an TLOG replica as leader
if (numNrtReplicas > 0) {
createdNrtReplicas++;
} else if (createdTlogReplicas > 0) {
createdTlogReplicas++;
if (numNrtReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.NRT.name());
} else if (numTlogReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.TLOG.name());
} else {
throw new SolrException(ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " +
Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
}
for (int i = 1; i < totalReplicasPerShard; i++) {
Replica.Type typeToCreate;
if (createdNrtReplicas < numNrtReplicas) {
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
}
}
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), new NamedList(), null);
}
//refresh the location copy of collection state
restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
//Copy data from backed up index to each replica
for (Slice slice : restoreCollection.getSlices()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.RESTORECORE.toString());
params.set(NAME, "snapshot." + slice.getName());
params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString());
params.set(CoreAdminParams.BACKUP_REPOSITORY, repo);
ocmh.sliceCmd(clusterState, params, null, slice, shardHandler, asyncId, requestMap);
}
ocmh.processResponses(new NamedList(), shardHandler, true, "Could not restore core", asyncId, requestMap);
//Mark all shards in ACTIVE STATE
{
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
propMap.put(ZkStateReader.COLLECTION_PROP, restoreCollectionName);
for (Slice shard : restoreCollection.getSlices()) {
propMap.put(shard.getName(), Slice.State.ACTIVE.toString());
}
inQueue.offer(Utils.toJSON(new ZkNodeProps(propMap)));
}
//refresh the location copy of collection state
restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
if (totalReplicasPerShard > 1) {
log.info("Adding replicas to restored collection={}", restoreCollection);
for (Slice slice : restoreCollection.getSlices()) {
//Add the remaining replicas for each shard, considering it's type
int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
// We already created either a NRT or an TLOG replica as leader
if (numNrtReplicas > 0) {
createdNrtReplicas++;
typeToCreate = Replica.Type.NRT;
} else if (createdTlogReplicas < numTlogReplicas) {
} else if (createdTlogReplicas > 0) {
createdTlogReplicas++;
typeToCreate = Replica.Type.TLOG;
} else {
createdPullReplicas++;
typeToCreate = Replica.Type.PULL;
assert createdPullReplicas <= numPullReplicas: "Unexpected number of replicas";
}
log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
propMap.put(REPLICA_TYPE, typeToCreate.name());
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
for (int i = 1; i < totalReplicasPerShard; i++) {
Replica.Type typeToCreate;
if (createdNrtReplicas < numNrtReplicas) {
createdNrtReplicas++;
typeToCreate = Replica.Type.NRT;
} else if (createdTlogReplicas < numTlogReplicas) {
createdTlogReplicas++;
typeToCreate = Replica.Type.TLOG;
} else {
createdPullReplicas++;
typeToCreate = Replica.Type.PULL;
assert createdPullReplicas <= numPullReplicas: "Unexpected number of replicas";
}
}
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
propMap.put(REPLICA_TYPE, typeToCreate.name());
ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
}
}
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
}
}
}
}
log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
} finally {
PolicyHelper.clearFlagAndDecref(ocmh.policySessionRef);
}
}
private int getInt(ZkNodeProps message, String propertyName, Integer default1, int default2) {

View File

@ -28,6 +28,7 @@ import java.util.Map;
import java.util.Set;
import org.apache.solr.common.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.cloud.overseer.OverseerAction;
@ -382,8 +383,8 @@ public class SplitShardCmd implements Cmd {
// TODO: change this to handle sharding a slice into > 2 sub-shards.
List<ReplicaPosition> replicaPositions = Assign.identifyNodes(() -> ocmh.overseer.getZkController().getCoreContainer(),
ocmh.zkStateReader, clusterState,
List<ReplicaPosition> replicaPositions = Assign.identifyNodes(ocmh,
clusterState,
new ArrayList<>(clusterState.getLiveNodes()),
collectionName,
new ZkNodeProps(collection.getProperties()),
@ -505,6 +506,8 @@ public class SplitShardCmd implements Cmd {
} catch (Exception e) {
log.error("Error executing split operation for collection: " + collectionName + " parent shard: " + slice, e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
} finally {
PolicyHelper.clearFlagAndDecref(ocmh.policySessionRef);
}
}
}

View File

@ -54,6 +54,7 @@ import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.SolrException;
@ -64,6 +65,7 @@ import org.apache.solr.common.cloud.DefaultConnectionStrategy;
import org.apache.solr.common.cloud.DefaultZkACLProvider;
import org.apache.solr.common.cloud.DefaultZkCredentialsProvider;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.LiveNodesListener;
import org.apache.solr.common.cloud.OnReconnect;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Replica.Type;
@ -188,7 +190,7 @@ public class ZkController {
private final SolrZkClient zkClient;
private final ZkCmdExecutor cmdExecutor;
private final ZkStateReader zkStateReader;
public final ZkStateReader zkStateReader;
private final String zkServerAddress; // example: 127.0.0.1:54062/solr
@ -675,6 +677,10 @@ public class ZkController {
cmdExecutor.ensureExists(ZkStateReader.LIVE_NODES_ZKNODE, zkClient);
cmdExecutor.ensureExists(ZkStateReader.COLLECTIONS_ZKNODE, zkClient);
cmdExecutor.ensureExists(ZkStateReader.ALIASES, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, zkClient);
byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
cmdExecutor.ensureExists(ZkStateReader.CLUSTER_STATE, emptyJson, CreateMode.PERSISTENT, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_SECURITY_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
@ -736,6 +742,7 @@ public class ZkController {
this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
checkForExistingEphemeralNode();
registerLiveNodesListener();
// start the overseer first as following code may need it's processing
if (!zkRunOnly) {
@ -806,6 +813,53 @@ public class ZkController {
}
}
private void registerLiveNodesListener() {
// this listener is used for generating nodeLost events, so we check only if
// some nodes went missing compared to last state
LiveNodesListener listener = (oldNodes, newNodes) -> {
oldNodes.removeAll(newNodes);
if (oldNodes.isEmpty()) { // only added nodes
return;
}
if (isClosed) {
return;
}
// if this node is in the top three then attempt to create nodeLost message
int i = 0;
for (String n : newNodes) {
if (n.equals(getNodeName())) {
break;
}
if (i > 2) {
return; // this node is not in the top three
}
i++;
}
// retrieve current trigger config - if there are no nodeLost triggers
// then don't create markers
boolean createNodes = false;
try {
createNodes = zkStateReader.getAutoScalingConfig().hasTriggerForEvents(TriggerEventType.NODELOST);
} catch (KeeperException | InterruptedException e1) {
log.warn("Unable to read autoscaling.json", e1);
}
if (createNodes) {
for (String n : oldNodes) {
String path = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + n;
try {
zkClient.create(path, null, CreateMode.PERSISTENT, true);
} catch (KeeperException.NodeExistsException e) {
// someone else already created this node - ignore
} catch (KeeperException | InterruptedException e1) {
log.warn("Unable to register nodeLost path for " + n, e1);
}
}
}
};
zkStateReader.registerLiveNodesListener(listener);
}
public void publishAndWaitForDownStates() throws KeeperException,
InterruptedException {
@ -880,8 +934,18 @@ public class ZkController {
}
String nodeName = getNodeName();
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
String nodeAddedPath = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName;
log.info("Register node as live in ZooKeeper:" + nodePath);
zkClient.makePath(nodePath, CreateMode.EPHEMERAL, true);
List<Op> ops = new ArrayList<>(2);
ops.add(Op.create(nodePath, null, zkClient.getZkACLProvider().getACLsToAdd(nodePath), CreateMode.EPHEMERAL));
// if there are nodeAdded triggers don't create nodeAdded markers
boolean createMarkerNode = zkStateReader.getAutoScalingConfig().hasTriggerForEvents(TriggerEventType.NODEADDED);
if (createMarkerNode && !zkClient.exists(nodeAddedPath, true)) {
// use EPHEMERAL so that it disappears if this node goes down
// and no other action is taken
ops.add(Op.create(nodeAddedPath, null, zkClient.getZkACLProvider().getACLsToAdd(nodeAddedPath), CreateMode.EPHEMERAL));
}
zkClient.multi(ops, true);
}
public String getNodeName() {
@ -1427,6 +1491,7 @@ public class ZkController {
if (msgNodeName.equals(nodeName) && core.equals(msgCore)) {
descriptor.getCloudDescriptor()
.setCoreNodeName(replica.getName());
getCoreContainer().getCoresLocator().persist(getCoreContainer(), descriptor);
return;
}
}
@ -1905,7 +1970,7 @@ public class ZkController {
}
}
CoreContainer getCoreContainer() {
public CoreContainer getCoreContainer() {
return cc;
}

View File

@ -0,0 +1,68 @@
/*
* 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.io.IOException;
import java.util.Map;
import org.apache.solr.common.MapWriter;
import org.apache.solr.core.CoreContainer;
/**
* Provides additional context for the TriggerAction such as the trigger instance on
* which the action is being executed as well as helper methods to pass computed information along
* to the next action
*/
public class ActionContext implements MapWriter {
private final CoreContainer coreContainer;
private final AutoScaling.Trigger source;
private final Map<String, Object> properties;
public ActionContext(CoreContainer coreContainer, AutoScaling.Trigger source, Map<String, Object> properties) {
this.coreContainer = coreContainer;
this.source = source;
this.properties = properties;
}
public CoreContainer getCoreContainer() {
return coreContainer;
}
public AutoScaling.Trigger getSource() {
return source;
}
public Map<String, Object> getProperties() {
return properties;
}
public Object getProperty(String name) {
return properties != null ? properties.get(name) : null;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put("source", source.getName());
if (properties != null) {
for (Map.Entry<String, Object> entry : properties.entrySet()) {
ew.put("properties." + entry.getKey(), entry.getValue());
}
}
}
}

View File

@ -0,0 +1,51 @@
/*
* 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 org.apache.solr.client.solrj.cloud.autoscaling.NoneSuggester;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ZkStateReader;
public class AutoAddReplicasPlanAction extends ComputePlanAction {
@Override
protected Policy.Suggester getSuggester(Policy.Session session, TriggerEvent event, ZkStateReader zkStateReader) {
// for backward compatibility
String autoAddReplicas = zkStateReader.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, (String) null);
if (autoAddReplicas != null && autoAddReplicas.equals("false")) {
return new NoneSuggester();
}
Policy.Suggester suggester = super.getSuggester(session, event, zkStateReader);
ClusterState clusterState = zkStateReader.getClusterState();
boolean anyCollections = false;
for (DocCollection collection: clusterState.getCollectionsMap().values()) {
if (collection.getAutoAddReplicas()) {
anyCollections = true;
suggester.hint(Policy.Suggester.Hint.COLL, collection.getName());
}
}
if (!anyCollections) return new NoneSuggester();
return suggester;
}
}

View File

@ -0,0 +1,170 @@
/*
* 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.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import com.google.common.base.Preconditions;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.cloud.ZkController;
public class AutoScaling {
/**
* Implementation of this interface is used for processing events generated by a trigger.
*/
public interface TriggerEventProcessor {
/**
* This method is executed for events produced by {@link Trigger#run()}.
*
* @param event a subclass of {@link TriggerEvent}
* @return true if the processor was ready to perform actions on the event, false
* otherwise. If false was returned then callers should assume the event was discarded.
*/
boolean process(TriggerEvent event);
}
/**
* Interface for a Solr trigger. Each trigger implements Runnable and Closeable interface. A trigger
* is scheduled using a {@link java.util.concurrent.ScheduledExecutorService} so it is executed as
* per a configured schedule to check whether the trigger is ready to fire. The {@link Trigger#setProcessor(TriggerEventProcessor)}
* method should be used to set a processor which is used by implementation of this class whenever
* ready.
* <p>
* As per the guarantees made by the {@link java.util.concurrent.ScheduledExecutorService} a trigger
* implementation is only ever called sequentially and therefore need not be thread safe. However, it
* is encouraged that implementations be immutable with the exception of the associated listener
* which can be get/set by a different thread than the one executing the trigger. Therefore, implementations
* should use appropriate synchronization around the listener.
* <p>
* When a trigger is ready to fire, it calls the {@link TriggerEventProcessor#process(TriggerEvent)} event
* with the proper trigger event object. If that method returns false then it should be interpreted to mean
* that Solr is not ready to process this trigger event and therefore we should retain the state and fire
* at the next invocation of the run() method.
*/
public interface Trigger extends Closeable, Runnable {
/**
* Trigger name.
*/
String getName();
/**
* Event type generated by this trigger.
*/
TriggerEventType getEventType();
/** Returns true if this trigger is enabled. */
boolean isEnabled();
/** Trigger properties. */
Map<String, Object> getProperties();
/** Number of seconds to wait between fired events ("waitFor" property). */
int getWaitForSecond();
/** Actions to execute when event is fired. */
List<TriggerAction> getActions();
/** Set event processor to call when event is fired. */
void setProcessor(TriggerEventProcessor processor);
/** Get event processor. */
TriggerEventProcessor getProcessor();
/** Return true when this trigger is closed and cannot be used. */
boolean isClosed();
/** Set internal state of this trigger from another instance. */
void restoreState(Trigger old);
/** Save internal state of this trigger in ZooKeeper. */
void saveState();
/** Restore internal state of this trigger from ZooKeeper. */
void restoreState();
/**
* Called before a trigger is scheduled. Any heavy object creation or initialisation should
* be done in this method instead of the Trigger's constructor.
*/
void init();
}
public static class TriggerFactory implements Closeable {
private final CoreContainer coreContainer;
private final ZkController zkController;
private boolean isClosed = false;
public TriggerFactory(CoreContainer coreContainer, ZkController zkController) {
Preconditions.checkNotNull(coreContainer);
Preconditions.checkNotNull(zkController);
this.coreContainer = coreContainer;
this.zkController = zkController;
}
public synchronized Trigger create(TriggerEventType type, String name, Map<String, Object> props) {
if (isClosed) {
throw new AlreadyClosedException("TriggerFactory has already been closed, cannot create new triggers");
}
switch (type) {
case NODEADDED:
return new NodeAddedTrigger(name, props, coreContainer, zkController);
case NODELOST:
return new NodeLostTrigger(name, props, coreContainer, zkController);
default:
throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
}
}
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
}
}
}
public static final String AUTO_ADD_REPLICAS_TRIGGER_DSL =
" {" +
" 'name' : '.auto_add_replicas'," +
" 'event' : 'nodeLost'," +
" 'waitFor' : -1," +
" 'enabled' : true," +
" 'actions' : [" +
" {" +
" 'name':'auto_add_replicas_plan'," +
" 'class':'solr.AutoAddReplicasPlanAction'" +
" }," +
" {" +
" 'name':'execute_plan'," +
" 'class':'solr.ExecutePlanAction'" +
" }" +
" ]" +
" }";
public static final Map<String, Object> AUTO_ADD_REPLICAS_TRIGGER_PROPS = (Map) Utils.fromJSONString(AUTO_ADD_REPLICAS_TRIGGER_DSL);
}

View File

@ -22,23 +22,35 @@ import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import com.google.common.collect.ImmutableSet;
import org.apache.solr.api.Api;
import org.apache.solr.api.ApiBag;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Cell;
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.Preference;
import org.apache.solr.client.solrj.cloud.autoscaling.Row;
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.SolrClientDataProvider;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkNodeProps;
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;
@ -50,12 +62,15 @@ 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.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.stream.Collectors.collectingAndThen;
import static java.util.stream.Collectors.toSet;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
import static org.apache.solr.common.params.AutoScalingParams.*;
import static org.apache.solr.common.params.CommonParams.JSON;
/**
@ -64,12 +79,23 @@ 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());
private static ImmutableSet<String> singletonCommands = ImmutableSet.of("set-cluster-preferences", "set-cluster-policy");
protected final CoreContainer container;
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;
public AutoScalingHandler(CoreContainer container) {
this.container = container;
Map<String, String> map = new HashMap<>(2);
map.put(NAME, "compute_plan");
map.put(CLASS, "solr.ComputePlanAction");
DEFAULT_ACTIONS.add(map);
map = new HashMap<>(2);
map.put(NAME, "execute_plan");
map.put(CLASS, "solr.ExecutePlanAction");
DEFAULT_ACTIONS.add(map);
}
@Override
@ -89,11 +115,18 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
}
Map<String, Object> map = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
if (parts.size() == 2) {
rsp.getValues().addAll(map);
} else if (parts.size() == 3 && "diagnostics".equals(parts.get(2))) {
handleDiagnostics(rsp, map);
AutoScalingConfig autoScalingConf = container.getZkController().zkStateReader.getAutoScalingConfig();
if (parts.size() == 2) {
autoScalingConf.writeMap(new MapWriter.EntryWriter() {
@Override
public MapWriter.EntryWriter put(String k, Object v) throws IOException {
rsp.getValues().add(k, v);
return this;
}
});
} else if (parts.size() == 3 && DIAGNOSTICS.equals(parts.get(2))) {
handleDiagnostics(rsp, autoScalingConf);
}
} else {
if (req.getContentStreams() == null) {
@ -104,24 +137,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
// errors have already been added to the response so there's nothing left to do
return;
}
for (CommandOperation op : ops) {
switch (op.name) {
case "set-policy":
handleSetPolicies(req, rsp, op);
break;
case "remove-policy":
handleRemovePolicy(req, rsp, op);
break;
case "set-cluster-preferences":
handleSetClusterPreferences(req, rsp, op);
break;
case "set-cluster-policy":
handleSetClusterPolicy(req, rsp, op);
break;
default:
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown command: " + op.name);
}
}
processOps(req, rsp, ops);
}
} catch (Exception e) {
rsp.getValues().add("result", "failure");
@ -131,8 +147,70 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
}
}
private void handleDiagnostics(SolrQueryResponse rsp, Map<String, Object> autoScalingConf) throws IOException {
Policy policy = new Policy(autoScalingConf);
public void processOps(SolrQueryRequest req, SolrQueryResponse rsp, List<CommandOperation> ops) throws KeeperException, InterruptedException, IOException {
while (true) {
AutoScalingConfig initialConfig = container.getZkController().zkStateReader.getAutoScalingConfig();
AutoScalingConfig currentConfig = initialConfig;
for (CommandOperation op : ops) {
switch (op.name) {
case CMD_SET_TRIGGER:
currentConfig = handleSetTrigger(req, rsp, op, currentConfig);
break;
case CMD_REMOVE_TRIGGER:
currentConfig = handleRemoveTrigger(req, rsp, op, currentConfig);
break;
case CMD_SET_LISTENER:
currentConfig = handleSetListener(req, rsp, op, currentConfig);
break;
case CMD_REMOVE_LISTENER:
currentConfig = handleRemoveListener(req, rsp, op, currentConfig);
break;
case CMD_SUSPEND_TRIGGER:
currentConfig = handleSuspendTrigger(req, rsp, op, currentConfig);
break;
case CMD_RESUME_TRIGGER:
currentConfig = handleResumeTrigger(req, rsp, op, currentConfig);
break;
case CMD_SET_POLICY:
currentConfig = handleSetPolicies(req, rsp, op, currentConfig);
break;
case CMD_REMOVE_POLICY:
currentConfig = handleRemovePolicy(req, rsp, op, currentConfig);
break;
case CMD_SET_CLUSTER_PREFERENCES:
currentConfig = handleSetClusterPreferences(req, rsp, op, currentConfig);
break;
case CMD_SET_CLUSTER_POLICY:
currentConfig = handleSetClusterPolicy(req, rsp, op, currentConfig);
break;
default:
op.addError("Unknown command: " + op.name);
}
}
List errs = CommandOperation.captureErrors(ops);
if (!errs.isEmpty()) {
throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "Error in command payload", errs);
}
if (!currentConfig.equals(initialConfig)) {
// update in ZK
if (zkSetAutoScalingConfig(container.getZkController().getZkStateReader(), currentConfig)) {
break;
} else {
// someone else updated the config, get the latest one and re-apply our ops
rsp.getValues().add("retry", "initialVersion=" + initialConfig.getZkVersion());
continue;
}
} else {
// no changes
break;
}
}
rsp.getValues().add("result", "success");
}
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()) {
@ -165,151 +243,406 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
}
}
private void handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
List clusterPolicy = (List) op.getCommandData();
private AutoScalingConfig handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
List<Map<String, Object>> clusterPolicy = (List<Map<String, Object>>) op.getCommandData();
if (clusterPolicy == null || !(clusterPolicy instanceof List)) {
op.addError("A list of cluster policies was not found");
checkErr(op);
op.addError("set-cluster-policy expects an array of objects");
return currentConfig;
}
List<Clause> cp = null;
try {
zkSetClusterPolicy(container.getZkController().getZkStateReader(), clusterPolicy);
cp = clusterPolicy.stream().map(Clause::new).collect(Collectors.toList());
} catch (Exception e) {
log.warn("error persisting policies");
op.addError(e.getMessage());
checkErr(op);
return currentConfig;
}
rsp.getValues().add("result", "success");
Policy p = currentConfig.getPolicy().withClusterPolicy(cp);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
private void checkErr(CommandOperation op) {
if (!op.hasError()) return;
throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "Error in command payload", CommandOperation.captureErrors(Collections.singletonList(op)));
}
private void handleSetClusterPreferences(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
List preferences = (List) op.getCommandData();
private AutoScalingConfig handleSetClusterPreferences(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
List<Map<String, Object>> preferences = (List<Map<String, Object>>) op.getCommandData();
if (preferences == null || !(preferences instanceof List)) {
op.addError("A list of cluster preferences not found");
checkErr(op);
return currentConfig;
}
zkSetPreferences(container.getZkController().getZkStateReader(), preferences);
rsp.getValues().add("result", "success");
List<Preference> prefs = null;
try {
prefs = preferences.stream().map(Preference::new).collect(Collectors.toList());
} catch (Exception e) {
op.addError(e.getMessage());
return currentConfig;
}
Policy p = currentConfig.getPolicy().withClusterPreferences(prefs);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
private void handleRemovePolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
String policyName = (String) op.getCommandData();
private AutoScalingConfig handleRemovePolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
String policyName = (String) op.getVal("");
if (op.hasError()) checkErr(op);
Map<String, Object> autoScalingConf = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
Map<String, Object> policies = (Map<String, Object>) autoScalingConf.get("policies");
if (op.hasError()) return currentConfig;
Map<String, List<Clause>> policies = currentConfig.getPolicy().getPolicies();
if (policies == null || !policies.containsKey(policyName)) {
op.addError("No policy exists with name: " + policyName);
return currentConfig;
}
checkErr(op);
zkSetPolicies(container.getZkController().getZkStateReader(), policyName, null);
rsp.getValues().add("result", "success");
container.getZkController().getZkStateReader().getClusterState().forEachCollection(coll -> {
if (policyName.equals(coll.getPolicyName()))
op.addError(StrUtils.formatString("policy : {0} is being used by collection {1}", policyName, coll.getName()));
});
if (op.hasError()) return currentConfig;
policies = new HashMap<>(policies);
policies.remove(policyName);
Policy p = currentConfig.getPolicy().withPolicies(policies);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
private void handleSetPolicies(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
Map<String, Object> policies = op.getDataMap();
for (Map.Entry<String, Object> policy : policies.entrySet()) {
private AutoScalingConfig handleSetPolicies(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
Map<String, Object> policiesMap = op.getDataMap();
for (Map.Entry<String, Object> policy : policiesMap.entrySet()) {
String policyName = policy.getKey();
if (policyName == null || policyName.trim().length() == 0) {
op.addError("The policy name cannot be null or empty");
return currentConfig;
}
}
checkErr(op);
List<String> params = new ArrayList<>(currentConfig.getPolicy().getParams());
Map<String, List<Clause>> mergedPolicies = new HashMap<>(currentConfig.getPolicy().getPolicies());
Map<String, List<Clause>> newPolicies = null;
try {
zkSetPolicies(container.getZkController().getZkStateReader(), null, policies);
newPolicies = Policy.policiesFromMap((Map<String, List<Map<String, Object>>>) op.getCommandData(),
params);
} catch (Exception e) {
log.warn("error persisting policies", e);
op.addError(e.getMessage());
checkErr(op);
return currentConfig;
}
rsp.getValues().add("result", "success");
mergedPolicies.putAll(newPolicies);
Policy p = currentConfig.getPolicy().withPolicies(mergedPolicies).withParams(params);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
private void zkSetPolicies(ZkStateReader reader, String policyBeRemoved, Map<String, Object> newPolicies) throws KeeperException, InterruptedException, IOException {
while (true) {
Stat stat = new Stat();
ZkNodeProps loaded = null;
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
loaded = ZkNodeProps.load(data);
Map<String, Object> policies = (Map<String, Object>) loaded.get("policies");
if (policies == null) policies = new HashMap<>(1);
if (newPolicies != null) {
policies.putAll(newPolicies);
private AutoScalingConfig handleResumeTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
String triggerName = op.getStr(NAME);
if (op.hasError()) return currentConfig;
Map<String, AutoScalingConfig.TriggerConfig> triggers = currentConfig.getTriggerConfigs();
Set<String> changed = new HashSet<>();
if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
op.addError("No trigger exists with name: " + triggerName);
return currentConfig;
}
Map<String, AutoScalingConfig.TriggerConfig> newTriggers = new HashMap<>();
for (Map.Entry<String, AutoScalingConfig.TriggerConfig> entry : triggers.entrySet()) {
if (Policy.EACH.equals(triggerName) || triggerName.equals(entry.getKey())) {
AutoScalingConfig.TriggerConfig trigger = entry.getValue();
if (!trigger.enabled) {
trigger = trigger.withEnabled(true);
newTriggers.put(entry.getKey(), trigger);
changed.add(entry.getKey());
} else {
newTriggers.put(entry.getKey(), entry.getValue());
}
} else {
policies.remove(policyBeRemoved);
newTriggers.put(entry.getKey(), entry.getValue());
}
loaded = loaded.plus("policies", policies);
verifyAutoScalingConf(loaded.getProperties());
try {
reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
} catch (KeeperException.BadVersionException bve) {
// somebody else has changed the configuration so we must retry
continue;
}
break;
}
rsp.getValues().add("changed", changed);
if (!changed.isEmpty()) {
currentConfig = currentConfig.withTriggerConfigs(newTriggers);
}
return currentConfig;
}
private void zkSetPreferences(ZkStateReader reader, List preferences) throws KeeperException, InterruptedException, IOException {
while (true) {
Stat stat = new Stat();
ZkNodeProps loaded = null;
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
loaded = ZkNodeProps.load(data);
loaded = loaded.plus("cluster-preferences", preferences);
verifyAutoScalingConf(loaded.getProperties());
private AutoScalingConfig handleSuspendTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
String triggerName = op.getStr(NAME);
if (op.hasError()) return currentConfig;
String timeout = op.getStr(TIMEOUT, null);
Date resumeTime = null;
if (timeout != null) {
try {
reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
} catch (KeeperException.BadVersionException bve) {
// somebody else has changed the configuration so we must retry
continue;
int timeoutSeconds = parseHumanTime(timeout);
resumeTime = new Date(TimeUnit.MILLISECONDS.convert(timeSource.getTime(), TimeUnit.NANOSECONDS)
+ TimeUnit.MILLISECONDS.convert(timeoutSeconds, TimeUnit.SECONDS));
} catch (IllegalArgumentException e) {
op.addError("Invalid 'timeout' value for suspend trigger: " + triggerName);
return currentConfig;
}
break;
}
Map<String, AutoScalingConfig.TriggerConfig> triggers = currentConfig.getTriggerConfigs();
Set<String> changed = new HashSet<>();
if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
op.addError("No trigger exists with name: " + triggerName);
return currentConfig;
}
Map<String, AutoScalingConfig.TriggerConfig> newTriggers = new HashMap<>();
for (Map.Entry<String, AutoScalingConfig.TriggerConfig> entry : triggers.entrySet()) {
if (Policy.EACH.equals(triggerName) || triggerName.equals(entry.getKey())) {
AutoScalingConfig.TriggerConfig trigger = entry.getValue();
if (trigger.enabled) {
trigger = trigger.withEnabled(false);
if (resumeTime != null) {
trigger = trigger.withProperty(RESUME_AT, resumeTime.getTime());
}
newTriggers.put(entry.getKey(), trigger);
changed.add(trigger.name);
} else {
newTriggers.put(entry.getKey(), entry.getValue());
}
} else {
newTriggers.put(entry.getKey(), entry.getValue());
}
}
rsp.getValues().add("changed", changed);
if (!changed.isEmpty()) {
currentConfig = currentConfig.withTriggerConfigs(newTriggers);
}
return currentConfig;
}
private void zkSetClusterPolicy(ZkStateReader reader, List clusterPolicy) throws KeeperException, InterruptedException, IOException {
while (true) {
Stat stat = new Stat();
ZkNodeProps loaded = null;
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
loaded = ZkNodeProps.load(data);
loaded = loaded.plus("cluster-policy", clusterPolicy);
verifyAutoScalingConf(loaded.getProperties());
try {
reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
} catch (KeeperException.BadVersionException bve) {
// somebody else has changed the configuration so we must retry
continue;
}
break;
private AutoScalingConfig handleRemoveListener(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
String listenerName = op.getStr(NAME);
if (op.hasError()) return currentConfig;
Map<String, AutoScalingConfig.TriggerListenerConfig> listeners = currentConfig.getTriggerListenerConfigs();
if (listeners == null || !listeners.containsKey(listenerName)) {
op.addError("No listener exists with name: " + listenerName);
return currentConfig;
}
currentConfig = currentConfig.withoutTriggerListenerConfig(listenerName);
return currentConfig;
}
private void verifyAutoScalingConf(Map<String, Object> autoScalingConf) throws IOException {
private AutoScalingConfig handleSetListener(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
String listenerName = op.getStr(NAME);
String triggerName = op.getStr(TRIGGER);
List<String> stageNames = op.getStrs(STAGE, Collections.emptyList());
String listenerClass = op.getStr(CLASS);
List<String> beforeActions = op.getStrs(BEFORE_ACTION, Collections.emptyList());
List<String> afterActions = op.getStrs(AFTER_ACTION, Collections.emptyList());
if (op.hasError()) return currentConfig;
Map<String, AutoScalingConfig.TriggerConfig> triggers = currentConfig.getTriggerConfigs();
if (triggers == null || !triggers.containsKey(triggerName)) {
op.addError("A trigger with the name " + triggerName + " does not exist");
return currentConfig;
}
AutoScalingConfig.TriggerConfig triggerConfig = triggers.get(triggerName);
if (stageNames.isEmpty() && beforeActions.isEmpty() && afterActions.isEmpty()) {
op.addError("Either 'stage' or 'beforeAction' or 'afterAction' must be specified");
return currentConfig;
}
for (String stage : stageNames) {
try {
TriggerEventProcessorStage.valueOf(stage);
} catch (IllegalArgumentException e) {
op.addError("Invalid stage name: " + stage);
}
}
if (op.hasError()) return currentConfig;
// validate that we can load the listener class
// todo allow creation from blobstore
try {
container.getResourceLoader().findClass(listenerClass, TriggerListener.class);
} catch (Exception e) {
log.warn("error loading listener class ", e);
op.addError("Listener not found: " + listenerClass + ". error message:" + e.getMessage());
return currentConfig;
}
Set<String> actionNames = new HashSet<>();
actionNames.addAll(beforeActions);
actionNames.addAll(afterActions);
for (AutoScalingConfig.ActionConfig action : triggerConfig.actions) {
actionNames.remove(action.name);
}
if (!actionNames.isEmpty()) {
op.addError("The trigger '" + triggerName + "' does not have actions named: " + actionNames);
return currentConfig;
}
AutoScalingConfig.TriggerListenerConfig listener = new AutoScalingConfig.TriggerListenerConfig(listenerName, op.getValuesExcluding("name"));
// todo - handle races between competing set-trigger and set-listener invocations
currentConfig = currentConfig.withTriggerListenerConfig(listener);
return currentConfig;
}
private AutoScalingConfig handleSetTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
// we're going to modify the op - use a copy
String triggerName = op.getStr(NAME);
String eventTypeStr = op.getStr(EVENT);
if (op.hasError()) return currentConfig;
TriggerEventType eventType = TriggerEventType.valueOf(eventTypeStr.trim().toUpperCase(Locale.ROOT));
String waitForStr = op.getStr(WAIT_FOR, null);
CommandOperation opCopy = new CommandOperation(op.name, Utils.getDeepCopy((Map) op.getCommandData(), 10));
if (waitForStr != null) {
int seconds = 0;
try {
seconds = parseHumanTime(waitForStr);
} catch (IllegalArgumentException e) {
op.addError("Invalid 'waitFor' value '" + waitForStr + "' in trigger: " + triggerName);
return currentConfig;
}
opCopy.getDataMap().put(WAIT_FOR, seconds);
}
Integer lowerBound = op.getInt(LOWER_BOUND, null);
Integer upperBound = op.getInt(UPPER_BOUND, null);
List<Map<String, String>> actions = (List<Map<String, String>>) op.getVal(ACTIONS);
if (actions == null) {
actions = DEFAULT_ACTIONS;
opCopy.getDataMap().put(ACTIONS, actions);
}
// validate that we can load all the actions
// todo allow creation from blobstore
for (Map<String, String> action : actions) {
if (!action.containsKey(NAME) || !action.containsKey(CLASS)) {
op.addError("No 'name' or 'class' specified for action: " + action);
return currentConfig;
}
String klass = action.get(CLASS);
try {
container.getResourceLoader().findClass(klass, TriggerAction.class);
} catch (Exception e) {
log.warn("Could not load class : ", e);
op.addError("Action not found: " + klass + " " + e.getMessage());
return currentConfig;
}
}
AutoScalingConfig.TriggerConfig trigger = new AutoScalingConfig.TriggerConfig(triggerName, opCopy.getValuesExcluding("name"));
currentConfig = currentConfig.withTriggerConfig(trigger);
// check that there's a default SystemLogListener, unless user specified another one
return withSystemLogListener(currentConfig, triggerName);
}
private static String fullName = SystemLogListener.class.getName();
private static String solrName = "solr." + SystemLogListener.class.getSimpleName();
static AutoScalingConfig withSystemLogListener(AutoScalingConfig autoScalingConfig, String triggerName) {
Map<String, AutoScalingConfig.TriggerListenerConfig> configs = autoScalingConfig.getTriggerListenerConfigs();
for (AutoScalingConfig.TriggerListenerConfig cfg : configs.values()) {
if (triggerName.equals(cfg.trigger)) {
// already has some listener config
return autoScalingConfig;
}
}
// need to add
Map<String, Object> properties = new HashMap<>();
properties.put(AutoScalingParams.CLASS, SystemLogListener.class.getName());
properties.put(AutoScalingParams.TRIGGER, triggerName);
properties.put(AutoScalingParams.STAGE, EnumSet.allOf(TriggerEventProcessorStage.class));
AutoScalingConfig.TriggerListenerConfig listener =
new AutoScalingConfig.TriggerListenerConfig(triggerName + CollectionAdminParams.SYSTEM_COLL, properties);
autoScalingConfig = autoScalingConfig.withTriggerListenerConfig(listener);
return autoScalingConfig;
}
private int parseHumanTime(String timeStr) {
char c = timeStr.charAt(timeStr.length() - 1);
long timeValue = Long.parseLong(timeStr.substring(0, timeStr.length() - 1));
int seconds;
switch (c) {
case 'h':
seconds = (int) TimeUnit.HOURS.toSeconds(timeValue);
break;
case 'm':
seconds = (int) TimeUnit.MINUTES.toSeconds(timeValue);
break;
case 's':
seconds = (int) timeValue;
break;
default:
throw new IllegalArgumentException("Invalid time value");
}
return seconds;
}
private AutoScalingConfig handleRemoveTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
String triggerName = op.getStr(NAME);
boolean removeListeners = op.getBoolean(REMOVE_LISTENERS, false);
if (op.hasError()) return currentConfig;
Map<String, AutoScalingConfig.TriggerConfig> triggerConfigs = currentConfig.getTriggerConfigs();
if (!triggerConfigs.containsKey(triggerName)) {
op.addError("No trigger exists with name: " + triggerName);
return currentConfig;
}
triggerConfigs = new HashMap<>(triggerConfigs);
Set<String> activeListeners = new HashSet<>();
Map<String, AutoScalingConfig.TriggerListenerConfig> listeners = currentConfig.getTriggerListenerConfigs();
for (AutoScalingConfig.TriggerListenerConfig listener : listeners.values()) {
if (triggerName.equals(listener.trigger)) {
activeListeners.add(listener.name);
}
}
if (!activeListeners.isEmpty()) {
boolean onlySystemLog = false;
if (activeListeners.size() == 1) {
AutoScalingConfig.TriggerListenerConfig cfg = listeners.get(activeListeners.iterator().next());
if (SystemLogListener.class.getName().equals(cfg.listenerClass) ||
("solr." + SystemLogListener.class.getSimpleName()).equals(cfg.listenerClass)) {
onlySystemLog = true;
}
}
if (removeListeners || onlySystemLog) {
listeners = new HashMap<>(listeners);
listeners.keySet().removeAll(activeListeners);
} else {
op.addError("Cannot remove trigger: " + triggerName + " because it has active listeners: " + activeListeners);
return currentConfig;
}
}
triggerConfigs.remove(triggerName);
currentConfig = currentConfig.withTriggerConfigs(triggerConfigs).withTriggerListenerConfigs(listeners);
return currentConfig;
}
private boolean zkSetAutoScalingConfig(ZkStateReader reader, 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) {
// somebody else has changed the configuration so we must retry
return false;
}
return true;
}
private void verifyAutoScalingConf(AutoScalingConfig autoScalingConf) throws IOException {
try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) {
Policy policy = new Policy(autoScalingConf);
Policy.Session session = policy.createSession(new SolrClientDataProvider(build));
Policy.Session session = autoScalingConf.getPolicy().createSession(new SolrClientDataProvider(build));
log.debug("Verified autoscaling configuration");
}
}
private Map<String, Object> zkReadAutoScalingConf(ZkStateReader reader) throws KeeperException, InterruptedException {
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
ZkNodeProps loaded = ZkNodeProps.load(data);
return loaded.getProperties();
}
@Override
public String getDescription() {
return "A handler for autoscaling configuration";

View File

@ -0,0 +1,115 @@
/*
* 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.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.core.CoreContainer;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class is responsible for using the configured policy and preferences
* with the hints provided by the trigger event to compute the required cluster operations.
*
* The cluster operations computed here are put into the {@link ActionContext}'s properties
* with the key name "operations". The value is a List of SolrRequest objects.
*/
public class ComputePlanAction extends TriggerActionBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Override
public void process(TriggerEvent event, ActionContext context) {
log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
CoreContainer container = context.getCoreContainer();
try {
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
.withZkHost(container.getZkController().getZkServerAddress())
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.build()) {
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
AutoScalingConfig autoScalingConf = zkStateReader.getAutoScalingConfig();
if (autoScalingConf.isEmpty()) {
log.error("Action: " + getName() + " executed but no policy is configured");
return;
}
Policy policy = autoScalingConf.getPolicy();
SolrClientDataProvider dataProvider = new SolrClientDataProvider(cloudSolrClient);
if (log.isDebugEnabled()) {
log.debug("Cluster data provider: {}", dataProvider.toMap(new HashMap<>()));
}
Policy.Session session = policy.createSession(dataProvider);
Policy.Suggester suggester = getSuggester(session, event, zkStateReader);
while (true) {
SolrRequest operation = suggester.getOperation();
if (operation == null) break;
log.info("Computed Plan: {}", operation.getParams());
Map<String, Object> props = context.getProperties();
props.compute("operations", (k, v) -> {
List<SolrRequest> operations = (List<SolrRequest>) v;
if (operations == null) operations = new ArrayList<>();
operations.add(operation);
return operations;
});
session = suggester.getSession();
suggester = getSuggester(session, event, zkStateReader);
}
}
} catch (KeeperException e) {
log.error("ZooKeeperException while processing event: " + event, e);
} catch (InterruptedException e) {
log.error("Interrupted while processing event: " + event, e);
} catch (IOException e) {
log.error("IOException while processing event: " + event, e);
} catch (Exception e) {
log.error("Unexpected exception while processing event: " + event, e);
}
}
protected Policy.Suggester getSuggester(Policy.Session session, TriggerEvent event, ZkStateReader zkStateReader) {
Policy.Suggester suggester;
switch (event.getEventType()) {
case NODEADDED:
suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
.hint(Policy.Suggester.Hint.TARGET_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
log.debug("Created suggester with targetNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
break;
case NODELOST:
suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
.hint(Policy.Suggester.Hint.SRC_NODE, event.getProperty(TriggerEvent.NODE_NAMES));
log.debug("Created suggester with srcNode: {}", event.getProperty(TriggerEvent.NODE_NAMES));
break;
default:
throw new UnsupportedOperationException("No support for events other than nodeAdded and nodeLost, received: " + event.getEventType());
}
return suggester;
}
}

View File

@ -0,0 +1,177 @@
/*
* 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.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.lang3.exception.ExceptionUtils;
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.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class is responsible for executing cluster operations read from the {@link ActionContext}'s properties
* with the key name "operations"
*/
public class ExecutePlanAction extends TriggerActionBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String PREFIX = "op-";
static final int DEFAULT_TASK_TIMEOUT_SECONDS = 120;
@Override
public void process(TriggerEvent event, ActionContext context) {
log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
CoreContainer container = context.getCoreContainer();
SolrZkClient zkClient = container.getZkController().getZkClient();
List<SolrRequest> operations = (List<SolrRequest>) context.getProperty("operations");
if (operations == null || operations.isEmpty()) {
log.info("No operations to execute for event: {}", event);
return;
}
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
.withZkHost(container.getZkController().getZkServerAddress())
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.build()) {
int counter = 0;
for (SolrRequest operation : operations) {
log.info("Executing operation: {}", operation.getParams());
try {
SolrResponse response = null;
if (operation instanceof CollectionAdminRequest.AsyncCollectionAdminRequest) {
CollectionAdminRequest.AsyncCollectionAdminRequest req = (CollectionAdminRequest.AsyncCollectionAdminRequest) operation;
String asyncId = event.getSource() + '/' + event.getId() + '/' + counter;
String znode = saveAsyncId(event, context, asyncId);
log.debug("Saved requestId: {} in znode: {}", asyncId, znode);
asyncId = req.processAsync(asyncId, cloudSolrClient);
CollectionAdminRequest.RequestStatusResponse statusResponse = waitForTaskToFinish(cloudSolrClient, 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 {
zkClient.delete(znode, -1, true);
} catch (KeeperException e) {
log.warn("Unexpected exception while trying to delete znode: " + znode, e);
}
}
response = statusResponse;
}
} else {
response = operation.process(cloudSolrClient);
}
NamedList<Object> result = response.getResponse();
context.getProperties().compute("responses", (s, o) -> {
List<NamedList<Object>> responses = (List<NamedList<Object>>) o;
if (responses == null) responses = new ArrayList<>(operations.size());
responses.add(result);
return responses;
});
} catch (SolrServerException | HttpSolrClient.RemoteSolrException 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 (KeeperException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to talk to ZooKeeper", e);
}
counter++;
}
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected IOException while processing event: " + event, e);
}
}
static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(CloudSolrClient cloudSolrClient, String requestId, long duration, TimeUnit timeUnit) throws SolrServerException, 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.requestStatus(requestId).process(cloudSolrClient);
state = statusResponse.getRequestStatus();
if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
log.info("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
CollectionAdminRequest.deleteAsyncId(requestId).process(cloudSolrClient);
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);
CollectionAdminRequest.deleteAsyncId(requestId).process(cloudSolrClient);
return statusResponse;
}
} catch (Exception e) {
Throwable rootCause = ExceptionUtils.getRootCause(e);
if (rootCause instanceof IllegalStateException && rootCause.getMessage().contains("Connection pool shut down")) {
throw e;
}
if (rootCause instanceof TimeoutException && rootCause.getMessage().contains("Could not connect to ZooKeeper")) {
throw e;
}
log.error("Unexpected Exception while querying status of requestId=" + requestId, 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);
}
log.debug("Task with requestId={} did not complete within 5 minutes. Last state={}", requestId, state);
return statusResponse;
}
/**
* Saves the given asyncId in ZK as a persistent sequential node.
*
* @return the path of the newly created node in ZooKeeper
*/
private String saveAsyncId(TriggerEvent event, ActionContext context, String asyncId) throws InterruptedException, KeeperException {
String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + context.getSource().getName() + "/" + getName();
CoreContainer container = context.getCoreContainer();
SolrZkClient zkClient = container.getZkController().getZkClient();
try {
zkClient.makePath(parentPath, new byte[0], CreateMode.PERSISTENT, true);
} catch (KeeperException.NodeExistsException e) {
// ignore
}
return zkClient.create(parentPath + "/" + PREFIX, Utils.toJSON(Collections.singletonMap("requestid", asyncId)), CreateMode.PERSISTENT_SEQUENTIAL, true);
}
}

View File

@ -0,0 +1,178 @@
/*
* 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.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.StringJoiner;
import org.apache.http.HttpEntity;
import org.apache.http.HttpResponse;
import org.apache.http.client.HttpClient;
import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.protocol.HttpClientContext;
import org.apache.http.entity.StringEntity;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.PropertiesUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simple HTTP callback that POSTs event data to a URL.
* URL, payload and headers may contain property substitution patterns, with the following properties available:
* <ul>
* <li>config.* - listener configuration</li>
* <li>event.* - event properties</li>
* <li>stage - current stage of event processing</li>
* <li>actionName - optional current action name</li>
* <li>context.* - optional {@link ActionContext} properties</li>
* <li>error - optional error string (from {@link Throwable#toString()})</li>
* <li>message - optional message</li>
* </ul>
* The following listener configuration is supported:
* <ul>
* <li>url - a URL template</li>
* <li>payload - string, optional payload template. If absent a JSON map of all properties listed above will be used.</li>
* <li>contentType - string, optional payload content type. If absent then <code>application/json</code> will be used.</li>
* <li>header.* - string, optional header template(s). The name of the property without "header." prefix defines the literal header name.</li>
* <li>timeout - int, optional connection and socket timeout in milliseconds. Default is 60 seconds.</li>
* <li>followRedirects - boolean, optional setting to follow redirects. Default is false.</li>
* </ul>
*/
public class HttpTriggerListener extends TriggerListenerBase {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private HttpClient httpClient;
private String urlTemplate;
private String payloadTemplate;
private String contentType;
private Map<String, String> headerTemplates = new HashMap<>();
private int timeout = HttpClientUtil.DEFAULT_CONNECT_TIMEOUT;
private boolean followRedirects;
@Override
public void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) {
super.init(coreContainer, config);
httpClient = coreContainer.getUpdateShardHandler().getHttpClient();
urlTemplate = (String)config.properties.get("url");
payloadTemplate = (String)config.properties.get("payload");
contentType = (String)config.properties.get("contentType");
config.properties.forEach((k, v) -> {
if (k.startsWith("header.")) {
headerTemplates.put(k.substring(7), String.valueOf(v));
}
});
timeout = PropertiesUtil.toInteger(String.valueOf(config.properties.get("timeout")), HttpClientUtil.DEFAULT_CONNECT_TIMEOUT);
followRedirects = PropertiesUtil.toBoolean(String.valueOf(config.properties.get("followRedirects")));
}
@Override
public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context, Throwable error, String message) {
Properties properties = new Properties();
properties.setProperty("stage", stage.toString());
// if configuration used "actionName" but we're in a non-action related stage then PropertiesUtil will
// throws an exception on missing value - so replace it with an empty string
if (actionName == null) {
actionName = "";
}
properties.setProperty("actionName", actionName);
if (context != null) {
context.getProperties().forEach((k, v) -> {
properties.setProperty("context." + k, String.valueOf(v));
});
}
if (error != null) {
properties.setProperty("error", error.toString());
} else {
properties.setProperty("error", "");
}
if (message != null) {
properties.setProperty("message", message);
} else {
properties.setProperty("message", "");
}
// add event properties
properties.setProperty("event.id", event.getId());
properties.setProperty("event.source", event.getSource());
properties.setProperty("event.eventTime", String.valueOf(event.eventTime));
properties.setProperty("event.eventType", event.getEventType().toString());
event.getProperties().forEach((k, v) -> {
properties.setProperty("event.properties." + k, String.valueOf(v));
});
// add config properties
properties.setProperty("config.name", config.name);
properties.setProperty("config.trigger", config.trigger);
properties.setProperty("config.listenerClass", config.listenerClass);
properties.setProperty("config.beforeActions", String.join(",", config.beforeActions));
properties.setProperty("config.afterActions", String.join(",", config.afterActions));
StringJoiner joiner = new StringJoiner(",");
config.stages.forEach(s -> joiner.add(s.toString()));
properties.setProperty("config.stages", joiner.toString());
config.properties.forEach((k, v) -> {
properties.setProperty("config.properties." + k, String.valueOf(v));
});
String url = PropertiesUtil.substituteProperty(urlTemplate, properties);
String payload;
String type;
if (payloadTemplate != null) {
payload = PropertiesUtil.substituteProperty(payloadTemplate, properties);
if (contentType != null) {
type = contentType;
} else {
type = "application/json";
}
} else {
payload = Utils.toJSONString(properties);
type = "application/json";
}
HttpPost post = new HttpPost(url);
HttpEntity entity = new StringEntity(payload, "UTF-8");
headerTemplates.forEach((k, v) -> {
String headerVal = PropertiesUtil.substituteProperty(v, properties);
if (!headerVal.isEmpty()) {
post.addHeader(k, headerVal);
}
});
post.setEntity(entity);
post.setHeader("Content-Type", type);
org.apache.http.client.config.RequestConfig.Builder requestConfigBuilder = HttpClientUtil.createDefaultRequestConfigBuilder();
requestConfigBuilder.setSocketTimeout(timeout);
requestConfigBuilder.setConnectTimeout(timeout);
requestConfigBuilder.setRedirectsEnabled(followRedirects);
post.setConfig(requestConfigBuilder.build());
try {
HttpClientContext httpClientRequestContext = HttpClientUtil.createNewHttpClientRequestContext();
HttpResponse rsp = httpClient.execute(post, httpClientRequestContext);
int statusCode = rsp.getStatusLine().getStatusCode();
if (statusCode != 200) {
LOG.warn("Error sending request for event " + event + ", HTTP response: " + rsp.toString());
}
HttpEntity responseEntity = rsp.getEntity();
Utils.consumeFully(responseEntity);
} catch (IOException e) {
LOG.warn("Exception sending request for event " + event, e);
}
}
}

View File

@ -0,0 +1,38 @@
/*
* 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.lang.invoke.MethodHandles;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implementation of {@link TriggerListener} that reports
* events to a log.
*/
public class LoggingListener extends TriggerListenerBase {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Override
public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
Throwable error, String message) {
LOG.info("{}: stage={}, actionName={}, event={}, error={}, messsage={}", config.name, stage, actionName, event, error, message);
}
}

View File

@ -0,0 +1,320 @@
/*
* 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.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.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.util.IOUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Trigger for the {@link TriggerEventType#NODEADDED} event
*/
public class NodeAddedTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final String name;
private final Map<String, Object> properties;
private final CoreContainer container;
private final ZkController zkController;
private final List<TriggerAction> actions;
private final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef;
private final boolean enabled;
private final int waitForSecond;
private final TriggerEventType eventType;
private final TimeSource timeSource;
private boolean isClosed = false;
private Set<String> lastLiveNodes;
private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>();
public NodeAddedTrigger(String name, Map<String, Object> properties,
CoreContainer container, ZkController zkController) {
super(zkController.getZkClient());
this.name = name;
this.properties = properties;
this.container = container;
this.zkController = zkController;
this.timeSource = TimeSource.CURRENT_TIME;
this.processorRef = new AtomicReference<>();
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
actions = new ArrayList<>(3);
for (Map<String, String> map : o) {
TriggerAction action = container.getResourceLoader().newInstance(map.get("class"), TriggerAction.class);
actions.add(action);
}
} else {
actions = Collections.emptyList();
}
lastLiveNodes = new HashSet<>(zkController.getZkStateReader().getClusterState().getLiveNodes());
log.debug("Initial livenodes: {}", lastLiveNodes);
this.enabled = Boolean.parseBoolean(String.valueOf(properties.getOrDefault("enabled", "true")));
this.waitForSecond = ((Long) properties.getOrDefault("waitFor", -1L)).intValue();
this.eventType = TriggerEventType.valueOf(properties.get("event").toString().toUpperCase(Locale.ROOT));
log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
}
@Override
public void init() {
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
for (int i = 0; i < o.size(); i++) {
Map<String, String> map = o.get(i);
actions.get(i).init(map);
}
}
// pick up added nodes for which marker paths were created
try {
List<String> added = zkClient.getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, null, true);
added.forEach(n -> {
// 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());
}
removeMarker(n);
});
} catch (KeeperException.NoNodeException e) {
// ignore
} catch (KeeperException | InterruptedException e) {
log.warn("Exception retrieving nodeLost markers", e);
}
}
@Override
public void setProcessor(AutoScaling.TriggerEventProcessor processor) {
processorRef.set(processor);
}
@Override
public AutoScaling.TriggerEventProcessor getProcessor() {
return processorRef.get();
}
@Override
public String getName() {
return name;
}
@Override
public TriggerEventType getEventType() {
return eventType;
}
@Override
public boolean isEnabled() {
return enabled;
}
@Override
public int getWaitForSecond() {
return waitForSecond;
}
@Override
public Map<String, Object> getProperties() {
return properties;
}
@Override
public List<TriggerAction> getActions() {
return actions;
}
@Override
public boolean equals(Object obj) {
if (obj instanceof NodeAddedTrigger) {
NodeAddedTrigger that = (NodeAddedTrigger) obj;
return this.name.equals(that.name)
&& this.properties.equals(that.properties);
}
return false;
}
@Override
public int hashCode() {
return Objects.hash(name, properties);
}
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
IOUtils.closeWhileHandlingException(actions);
}
}
@Override
public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed();
if (old instanceof NodeAddedTrigger) {
NodeAddedTrigger that = (NodeAddedTrigger) old;
assert this.name.equals(that.name);
this.lastLiveNodes = new HashSet<>(that.lastLiveNodes);
this.nodeNameVsTimeAdded = new HashMap<>(that.nodeNameVsTimeAdded);
} else {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
"Unable to restore state from an unknown type of trigger");
}
}
@Override
protected Map<String, Object> getState() {
Map<String,Object> state = new HashMap<>();
state.put("lastLiveNodes", lastLiveNodes);
state.put("nodeNameVsTimeAdded", nodeNameVsTimeAdded);
return state;
}
@Override
protected void setState(Map<String, Object> state) {
this.lastLiveNodes.clear();
this.nodeNameVsTimeAdded.clear();
Collection<String> lastLiveNodes = (Collection<String>)state.get("lastLiveNodes");
if (lastLiveNodes != null) {
this.lastLiveNodes.addAll(lastLiveNodes);
}
Map<String,Long> nodeNameVsTimeAdded = (Map<String,Long>)state.get("nodeNameVsTimeAdded");
if (nodeNameVsTimeAdded != null) {
this.nodeNameVsTimeAdded.putAll(nodeNameVsTimeAdded);
}
}
@Override
public void run() {
try {
synchronized (this) {
if (isClosed) {
log.warn("NodeAddedTrigger ran but was already closed");
throw new RuntimeException("Trigger has been closed");
}
}
log.debug("Running NodeAddedTrigger {}", name);
ZkStateReader reader = zkController.getZkStateReader();
Set<String> newLiveNodes = reader.getClusterState().getLiveNodes();
log.debug("Found livenodes: {}", newLiveNodes);
// have any nodes that we were tracking been removed from the cluster?
// if so, remove them from the tracking map
Set<String> trackingKeySet = nodeNameVsTimeAdded.keySet();
trackingKeySet.retainAll(newLiveNodes);
// have any new nodes been added?
Set<String> copyOfNew = new HashSet<>(newLiveNodes);
copyOfNew.removeAll(lastLiveNodes);
copyOfNew.forEach(n -> {
long eventTime = timeSource.getTime();
log.debug("Tracking new node: {} at time {}", n, eventTime);
nodeNameVsTimeAdded.put(n, eventTime);
});
// has enough time expired to trigger events for a node?
List<String> nodeNames = new ArrayList<>();
List<Long> times = new ArrayList<>();
for (Iterator<Map.Entry<String, Long>> it = nodeNameVsTimeAdded.entrySet().iterator(); it.hasNext(); ) {
Map.Entry<String, Long> entry = it.next();
String nodeName = entry.getKey();
Long timeAdded = entry.getValue();
long now = timeSource.getTime();
if (TimeUnit.SECONDS.convert(now - timeAdded, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
nodeNames.add(nodeName);
times.add(timeAdded);
}
}
AutoScaling.TriggerEventProcessor processor = processorRef.get();
if (!nodeNames.isEmpty()) {
if (processor != null) {
log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}", name, nodeNames, times);
if (processor.process(new NodeAddedEvent(getEventType(), getName(), times, nodeNames))) {
// remove from tracking set only if the fire was accepted
nodeNames.forEach(n -> {
nodeNameVsTimeAdded.remove(n);
removeMarker(n);
});
}
} else {
nodeNames.forEach(n -> {
nodeNameVsTimeAdded.remove(n);
removeMarker(n);
});
}
}
lastLiveNodes = new HashSet<>(newLiveNodes);
} catch (RuntimeException e) {
log.error("Unexpected exception in NodeAddedTrigger", e);
}
}
private void removeMarker(String nodeName) {
String path = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName;
try {
if (zkClient.exists(path, true)) {
zkClient.delete(path, -1, true);
}
} catch (KeeperException.NoNodeException e) {
// ignore
} catch (KeeperException | InterruptedException e) {
log.debug("Exception removing nodeAdded marker " + nodeName, e);
}
}
@Override
public boolean isClosed() {
synchronized (this) {
return isClosed;
}
}
public static class NodeAddedEvent extends TriggerEvent {
public NodeAddedEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames) {
// use the oldest time as the time of the event
super(eventType, source, times.get(0), null);
properties.put(NODE_NAMES, nodeNames);
properties.put(EVENT_TIMES, times);
}
}
}

View File

@ -0,0 +1,318 @@
/*
* 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.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.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.util.IOUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Trigger for the {@link TriggerEventType#NODELOST} event
*/
public class NodeLostTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final String name;
private final Map<String, Object> properties;
private final CoreContainer container;
private final ZkController zkController;
private final List<TriggerAction> actions;
private final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef;
private final boolean enabled;
private final int waitForSecond;
private final TriggerEventType eventType;
private final TimeSource timeSource;
private boolean isClosed = false;
private Set<String> lastLiveNodes;
private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>();
public NodeLostTrigger(String name, Map<String, Object> properties,
CoreContainer container, ZkController zkController) {
super(zkController.getZkClient());
this.name = name;
this.properties = properties;
this.container = container;
this.zkController = zkController;
this.timeSource = TimeSource.CURRENT_TIME;
this.processorRef = new AtomicReference<>();
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
actions = new ArrayList<>(3);
for (Map<String, String> map : o) {
TriggerAction action = container.getResourceLoader().newInstance(map.get("class"), TriggerAction.class);
actions.add(action);
}
} else {
actions = Collections.emptyList();
}
lastLiveNodes = new HashSet<>(zkController.getZkStateReader().getClusterState().getLiveNodes());
log.debug("Initial livenodes: {}", lastLiveNodes);
this.enabled = Boolean.parseBoolean(String.valueOf(properties.getOrDefault("enabled", "true")));
this.waitForSecond = ((Long) properties.getOrDefault("waitFor", -1L)).intValue();
this.eventType = TriggerEventType.valueOf(properties.get("event").toString().toUpperCase(Locale.ROOT));
}
@Override
public void init() {
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
for (int i = 0; i < o.size(); i++) {
Map<String, String> map = o.get(i);
actions.get(i).init(map);
}
}
// pick up lost nodes for which marker paths were created
try {
List<String> lost = zkClient.getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, null, true);
lost.forEach(n -> {
// 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());
}
removeMarker(n);
});
} catch (KeeperException.NoNodeException e) {
// ignore
} catch (KeeperException | InterruptedException e) {
log.warn("Exception retrieving nodeLost markers", e);
}
}
@Override
public void setProcessor(AutoScaling.TriggerEventProcessor processor) {
processorRef.set(processor);
}
@Override
public AutoScaling.TriggerEventProcessor getProcessor() {
return processorRef.get();
}
@Override
public String getName() {
return name;
}
@Override
public TriggerEventType getEventType() {
return eventType;
}
@Override
public boolean isEnabled() {
return enabled;
}
@Override
public int getWaitForSecond() {
return waitForSecond;
}
@Override
public Map<String, Object> getProperties() {
return properties;
}
@Override
public List<TriggerAction> getActions() {
return actions;
}
@Override
public boolean equals(Object obj) {
if (obj instanceof NodeLostTrigger) {
NodeLostTrigger that = (NodeLostTrigger) obj;
return this.name.equals(that.name)
&& this.properties.equals(that.properties);
}
return false;
}
@Override
public int hashCode() {
return Objects.hash(name, properties);
}
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
IOUtils.closeWhileHandlingException(actions);
}
}
@Override
public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed();
if (old instanceof NodeLostTrigger) {
NodeLostTrigger that = (NodeLostTrigger) old;
assert this.name.equals(that.name);
this.lastLiveNodes = new HashSet<>(that.lastLiveNodes);
this.nodeNameVsTimeRemoved = new HashMap<>(that.nodeNameVsTimeRemoved);
} else {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
"Unable to restore state from an unknown type of trigger");
}
}
@Override
protected Map<String, Object> getState() {
Map<String,Object> state = new HashMap<>();
state.put("lastLiveNodes", lastLiveNodes);
state.put("nodeNameVsTimeRemoved", nodeNameVsTimeRemoved);
return state;
}
@Override
protected void setState(Map<String, Object> state) {
this.lastLiveNodes.clear();
this.nodeNameVsTimeRemoved.clear();
Collection<String> lastLiveNodes = (Collection<String>)state.get("lastLiveNodes");
if (lastLiveNodes != null) {
this.lastLiveNodes.addAll(lastLiveNodes);
}
Map<String,Long> nodeNameVsTimeRemoved = (Map<String,Long>)state.get("nodeNameVsTimeRemoved");
if (nodeNameVsTimeRemoved != null) {
this.nodeNameVsTimeRemoved.putAll(nodeNameVsTimeRemoved);
}
}
@Override
public void run() {
try {
synchronized (this) {
if (isClosed) {
log.warn("NodeLostTrigger ran but was already closed");
throw new RuntimeException("Trigger has been closed");
}
}
ZkStateReader reader = zkController.getZkStateReader();
Set<String> newLiveNodes = reader.getClusterState().getLiveNodes();
log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes);
// have any nodes that we were tracking been added to the cluster?
// if so, remove them from the tracking map
Set<String> trackingKeySet = nodeNameVsTimeRemoved.keySet();
trackingKeySet.removeAll(newLiveNodes);
// have any nodes been removed?
Set<String> copyOfLastLiveNodes = new HashSet<>(lastLiveNodes);
copyOfLastLiveNodes.removeAll(newLiveNodes);
copyOfLastLiveNodes.forEach(n -> {
log.debug("Tracking lost node: {}", n);
nodeNameVsTimeRemoved.put(n, timeSource.getTime());
});
// has enough time expired to trigger events for a node?
List<String> nodeNames = new ArrayList<>();
List<Long> times = new ArrayList<>();
for (Iterator<Map.Entry<String, Long>> it = nodeNameVsTimeRemoved.entrySet().iterator(); it.hasNext(); ) {
Map.Entry<String, Long> entry = it.next();
String nodeName = entry.getKey();
Long timeRemoved = entry.getValue();
long now = timeSource.getTime();
if (TimeUnit.SECONDS.convert(now - timeRemoved, TimeUnit.NANOSECONDS) >= getWaitForSecond()) {
nodeNames.add(nodeName);
times.add(timeRemoved);
}
}
// fire!
AutoScaling.TriggerEventProcessor processor = processorRef.get();
if (!nodeNames.isEmpty()) {
if (processor != null) {
log.debug("NodeLostTrigger firing registered processor for lost nodes: {}", nodeNames);
if (processor.process(new NodeLostEvent(getEventType(), getName(), times, nodeNames))) {
// remove from tracking set only if the fire was accepted
nodeNames.forEach(n -> {
nodeNameVsTimeRemoved.remove(n);
removeMarker(n);
});
} else {
log.debug("NodeLostTrigger listener for lost nodes: {} is not ready, will try later", nodeNames);
}
} else {
nodeNames.forEach(n -> {
nodeNameVsTimeRemoved.remove(n);
removeMarker(n);
});
}
}
lastLiveNodes = new HashSet<>(newLiveNodes);
} catch (RuntimeException e) {
log.error("Unexpected exception in NodeLostTrigger", e);
}
}
private void removeMarker(String nodeName) {
String path = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + nodeName;
try {
if (zkClient.exists(path, true)) {
zkClient.delete(path, -1, true);
}
} catch (KeeperException.NoNodeException e) {
// ignore
} catch (KeeperException | InterruptedException e) {
log.warn("Exception removing nodeLost marker " + nodeName, e);
}
}
@Override
public boolean isClosed() {
synchronized (this) {
return isClosed;
}
}
public static class NodeLostEvent extends TriggerEvent {
public NodeLostEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames) {
// use the oldest time as the time of the event
super(eventType, source, times.get(0), null);
properties.put(NODE_NAMES, nodeNames);
properties.put(EVENT_TIMES, times);
}
}
}

View File

@ -0,0 +1,369 @@
/*
* 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.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloudConfig;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
/**
* Overseer thread responsible for reading triggers from zookeeper and
* adding/removing them from {@link ScheduledTriggers}
*/
public class OverseerTriggerThread implements Runnable, Closeable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final ZkController zkController;
private final CloudConfig cloudConfig;
private final ZkStateReader zkStateReader;
private final SolrZkClient zkClient;
private final ScheduledTriggers scheduledTriggers;
private final AutoScaling.TriggerFactory triggerFactory;
private final ReentrantLock updateLock = new ReentrantLock();
private final Condition updated = updateLock.newCondition();
/*
Following variables are only accessed or modified when updateLock is held
*/
private int znodeVersion = -1;
private Map<String, AutoScaling.Trigger> activeTriggers = new HashMap<>();
private volatile boolean isClosed = false;
private AutoScalingConfig autoScalingConfig;
public OverseerTriggerThread(ZkController zkController, CloudConfig cloudConfig) {
this.zkController = zkController;
this.cloudConfig = cloudConfig;
zkStateReader = zkController.getZkStateReader();
zkClient = zkController.getZkClient();
scheduledTriggers = new ScheduledTriggers(zkController);
triggerFactory = new AutoScaling.TriggerFactory(zkController.getCoreContainer(), zkController);
}
@Override
public void close() throws IOException {
updateLock.lock();
try {
isClosed = true;
activeTriggers.clear();
updated.signalAll();
} finally {
updateLock.unlock();
}
IOUtils.closeQuietly(triggerFactory);
IOUtils.closeQuietly(scheduledTriggers);
log.debug("OverseerTriggerThread has been closed explicitly");
}
@Override
public void run() {
int lastZnodeVersion = znodeVersion;
// we automatically add a trigger for auto add replicas if it does not exists already
while (!isClosed) {
try {
AutoScalingConfig autoScalingConfig = zkStateReader.getAutoScalingConfig();
AutoScalingConfig withAutoAddReplicasTrigger = withAutoAddReplicasTrigger(autoScalingConfig);
if (withAutoAddReplicasTrigger.equals(autoScalingConfig)) break;
log.debug("Adding .autoAddReplicas trigger");
zkClient.setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(withAutoAddReplicasTrigger), withAutoAddReplicasTrigger.getZkVersion(), true);
break;
} catch (KeeperException.BadVersionException bve) {
// somebody else has changed the configuration so we must retry
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.warn("Interrupted", e);
break;
} catch (KeeperException e) {
log.error("A ZK error has occurred", e);
}
}
if (isClosed || Thread.currentThread().isInterrupted()) return;
try {
refreshAutoScalingConf(new AutoScalingWatcher());
} catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
log.warn("ZooKeeper watch triggered for autoscaling conf, but Solr cannot talk to ZK: [{}]", e.getMessage());
} catch (KeeperException e) {
log.error("A ZK error has occurred", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.warn("Interrupted", e);
} catch (Exception e) {
log.error("Unexpected exception", e);
}
while (true) {
Map<String, AutoScaling.Trigger> copy = null;
try {
// this can throw InterruptedException and we don't want to unlock if it did, so we keep this outside
// of the try/finally block
updateLock.lockInterruptibly();
// must check for close here before we await on the condition otherwise we can only be woken up on interruption
if (isClosed) {
log.warn("OverseerTriggerThread has been closed, exiting.");
break;
}
log.debug("Current znodeVersion {}, lastZnodeVersion {}", znodeVersion, lastZnodeVersion);
try {
if (znodeVersion == lastZnodeVersion) {
updated.await();
// are we closed?
if (isClosed) {
log.warn("OverseerTriggerThread woken up but we are closed, exiting.");
break;
}
// spurious wakeup?
if (znodeVersion == lastZnodeVersion) continue;
}
copy = new HashMap<>(activeTriggers);
lastZnodeVersion = znodeVersion;
log.debug("Processed trigger updates upto znodeVersion {}", znodeVersion);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.warn("Interrupted", e);
break;
} finally {
updateLock.unlock();
}
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.warn("Interrupted", e);
break;
}
// update the current config
scheduledTriggers.setAutoScalingConfig(autoScalingConfig);
Set<String> managedTriggerNames = scheduledTriggers.getScheduledTriggerNames();
// remove the triggers which are no longer active
for (String managedTriggerName : managedTriggerNames) {
if (!copy.containsKey(managedTriggerName)) {
scheduledTriggers.remove(managedTriggerName);
}
}
// check for nodeLost triggers in the current config, and if
// absent then clean up old nodeLost / nodeAdded markers
boolean cleanOldNodeLostMarkers = true;
boolean cleanOldNodeAddedMarkers = true;
try {
// add new triggers and/or replace and close the replaced triggers
for (Map.Entry<String, AutoScaling.Trigger> entry : copy.entrySet()) {
if (entry.getValue().getEventType().equals(TriggerEventType.NODELOST)) {
cleanOldNodeLostMarkers = false;
}
if (entry.getValue().getEventType().equals(TriggerEventType.NODEADDED)) {
cleanOldNodeAddedMarkers = false;
}
scheduledTriggers.add(entry.getValue());
}
} catch (AlreadyClosedException e) {
// this _should_ mean that we're closing, complain loudly if that's not the case
if (isClosed) {
return;
} else {
throw new IllegalStateException("Caught AlreadyClosedException from ScheduledTriggers, but we're not closed yet!", e);
}
}
if (cleanOldNodeLostMarkers) {
log.debug("-- clean old nodeLost markers");
try {
List<String> markers = zkClient.getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, null, true);
markers.forEach(n -> {
removeNodeMarker(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, n);
});
} catch (KeeperException.NoNodeException e) {
// ignore
} catch (KeeperException | InterruptedException e) {
log.warn("Error removing old nodeLost markers", e);
}
}
if (cleanOldNodeAddedMarkers) {
log.debug("-- clean old nodeAdded markers");
try {
List<String> markers = zkClient.getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, null, true);
markers.forEach(n -> {
removeNodeMarker(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, n);
});
} catch (KeeperException.NoNodeException e) {
// ignore
} catch (KeeperException | InterruptedException e) {
log.warn("Error removing old nodeAdded markers", e);
}
}
}
}
private void removeNodeMarker(String path, String nodeName) {
path = path + "/" + nodeName;
try {
zkClient.delete(path, -1, true);
log.debug(" -- deleted " + path);
} catch (KeeperException.NoNodeException e) {
// ignore
} catch (KeeperException | InterruptedException e) {
log.warn("Error removing old marker " + path, e);
}
}
class AutoScalingWatcher implements Watcher {
@Override
public void process(WatchedEvent watchedEvent) {
// session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(watchedEvent.getType())) {
return;
}
try {
refreshAutoScalingConf(this);
} catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
log.warn("ZooKeeper watch triggered for autoscaling conf, but Solr cannot talk to ZK: [{}]", e.getMessage());
} catch (KeeperException e) {
log.error("A ZK error has occurred", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
} catch (InterruptedException e) {
// Restore the interrupted status
Thread.currentThread().interrupt();
log.warn("Interrupted", e);
} catch (Exception e) {
log.error("Unexpected exception", e);
}
}
}
private void refreshAutoScalingConf(Watcher watcher) throws KeeperException, InterruptedException {
updateLock.lock();
try {
if (isClosed) {
return;
}
AutoScalingConfig currentConfig = zkStateReader.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
return;
}
autoScalingConfig = currentConfig;
znodeVersion = autoScalingConfig.getZkVersion();
Map<String, AutoScaling.Trigger> triggerMap = loadTriggers(triggerFactory, autoScalingConfig);
// remove all active triggers that have been removed from ZK
Set<String> trackingKeySet = activeTriggers.keySet();
trackingKeySet.retainAll(triggerMap.keySet());
// now lets add or remove triggers which have been enabled or disabled respectively
for (Map.Entry<String, AutoScaling.Trigger> entry : triggerMap.entrySet()) {
String triggerName = entry.getKey();
AutoScaling.Trigger trigger = entry.getValue();
if (trigger.isEnabled()) {
activeTriggers.put(triggerName, trigger);
} else {
activeTriggers.remove(triggerName);
}
}
updated.signalAll();
} finally {
updateLock.unlock();
}
}
private AutoScalingConfig withAutoAddReplicasTrigger(AutoScalingConfig autoScalingConfig) {
Map<String, Object> triggerProps = AutoScaling.AUTO_ADD_REPLICAS_TRIGGER_PROPS;
String triggerName = (String) triggerProps.get("name");
Map<String, AutoScalingConfig.TriggerConfig> configs = autoScalingConfig.getTriggerConfigs();
for (AutoScalingConfig.TriggerConfig cfg : configs.values()) {
if (triggerName.equals(cfg.name)) {
// already has this trigger
return autoScalingConfig;
}
}
// need to add
triggerProps.computeIfPresent("waitFor", (k, v) -> (long) (cloudConfig.getAutoReplicaFailoverWaitAfterExpiration() / 1000));
AutoScalingConfig.TriggerConfig config = new AutoScalingConfig.TriggerConfig(triggerName, triggerProps);
autoScalingConfig = autoScalingConfig.withTriggerConfig(config);
// need to add SystemLogListener explicitly here
autoScalingConfig = AutoScalingHandler.withSystemLogListener(autoScalingConfig, triggerName);
return autoScalingConfig;
}
private static Map<String, AutoScaling.Trigger> loadTriggers(AutoScaling.TriggerFactory triggerFactory, AutoScalingConfig autoScalingConfig) {
Map<String, AutoScalingConfig.TriggerConfig> triggers = autoScalingConfig.getTriggerConfigs();
if (triggers == null) {
return Collections.emptyMap();
}
Map<String, AutoScaling.Trigger> triggerMap = new HashMap<>(triggers.size());
for (Map.Entry<String, AutoScalingConfig.TriggerConfig> entry : triggers.entrySet()) {
AutoScalingConfig.TriggerConfig cfg = entry.getValue();
TriggerEventType eventType = cfg.event;
String triggerName = entry.getKey();
triggerMap.put(triggerName, triggerFactory.create(eventType, triggerName, cfg.properties));
}
return triggerMap;
}
}

View File

@ -0,0 +1,638 @@
/*
* 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.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledFuture;
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.locks.ReentrantLock;
import java.util.stream.Collectors;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.RequestStatusResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.cloud.ActionThrottle;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.ExecutePlanAction.waitForTaskToFinish;
/**
* Responsible for scheduling active triggers, starting and stopping them and
* performing actions when they fire
*/
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_MIN_MS_BETWEEN_ACTIONS = 5000;
private final Map<String, ScheduledTrigger> scheduledTriggers = new ConcurrentHashMap<>();
/**
* Thread pool for scheduling the triggers
*/
private final ScheduledThreadPoolExecutor scheduledThreadPoolExecutor;
/**
* Single threaded executor to run the actions upon a trigger event. We rely on this being a single
* threaded executor to ensure that trigger fires do not step on each other as well as to ensure
* that we do not run scheduled trigger threads while an action has been submitted to this executor
*/
private final ExecutorService actionExecutor;
private boolean isClosed = false;
private final AtomicBoolean hasPendingActions = new AtomicBoolean(false);
private final ActionThrottle actionThrottle;
private final SolrZkClient zkClient;
private final Overseer.Stats queueStats;
private final CoreContainer coreContainer;
private final TriggerListeners listeners;
private AutoScalingConfig autoScalingConfig;
public ScheduledTriggers(ZkController zkController) {
// todo make the core pool size configurable
// it is important to use more than one because a time taking trigger can starve other scheduled triggers
// ideally we should have as many core threads as the number of triggers but firstly, we don't know beforehand
// how many triggers we have and secondly, that many threads will always be instantiated and kept around idle
// so it is wasteful as well. Hopefully 4 is a good compromise.
scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(4,
new DefaultSolrThreadFactory("ScheduledTrigger"));
scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor"));
// todo make the wait time configurable
actionThrottle = new ActionThrottle("action", DEFAULT_MIN_MS_BETWEEN_ACTIONS);
coreContainer = zkController.getCoreContainer();
zkClient = zkController.getZkClient();
queueStats = new Overseer.Stats();
listeners = new TriggerListeners();
}
/**
* Set the current autoscaling config. This is invoked by {@link OverseerTriggerThread} when autoscaling.json is updated,
* and it re-initializes trigger listeners.
* @param autoScalingConfig current autoscaling.json
*/
public void setAutoScalingConfig(AutoScalingConfig autoScalingConfig) {
this.autoScalingConfig = autoScalingConfig;
listeners.setAutoScalingConfig(autoScalingConfig);
}
/**
* 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
* operation becomes a no-op.
*
* @param newTrigger the trigger to be managed
* @throws AlreadyClosedException if this class has already been closed
*/
public synchronized void add(AutoScaling.Trigger newTrigger) {
if (isClosed) {
throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore");
}
ScheduledTrigger scheduledTrigger = new ScheduledTrigger(newTrigger, zkClient, queueStats);
ScheduledTrigger old = scheduledTriggers.putIfAbsent(newTrigger.getName(), scheduledTrigger);
if (old != null) {
if (old.trigger.equals(newTrigger)) {
// the trigger wasn't actually modified so we do nothing
return;
}
IOUtils.closeQuietly(old);
newTrigger.restoreState(old.trigger);
scheduledTrigger.setReplay(false);
scheduledTriggers.replace(newTrigger.getName(), scheduledTrigger);
}
newTrigger.setProcessor(event -> {
if (coreContainer.isShutDown()) {
String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because Solr has been shutdown.", event.toString());
log.warn(msg);
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
return false;
}
ScheduledTrigger scheduledSource = scheduledTriggers.get(event.getSource());
if (scheduledSource == null) {
String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the source trigger: %s doesn't exist.", event.toString(), event.getSource());
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.FAILED, msg);
log.warn(msg);
return false;
}
boolean replaying = event.getProperty(TriggerEvent.REPLAYING) != null ? (Boolean)event.getProperty(TriggerEvent.REPLAYING) : false;
AutoScaling.Trigger source = scheduledSource.trigger;
if (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);
// we do not want to lose this event just because the trigger was closed, perhaps a replacement will need it
return false;
}
if (hasPendingActions.compareAndSet(false, true)) {
final boolean enqueued;
if (replaying) {
enqueued = false;
} else {
enqueued = scheduledTrigger.enqueue(event);
}
// fire STARTED event listeners after enqueuing the event is successful
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.STARTED);
List<TriggerAction> actions = source.getActions();
if (actions != null) {
actionExecutor.submit(() -> {
assert hasPendingActions.get();
log.debug("-- processing actions for " + event);
try {
// let the action executor thread wait instead of the trigger thread so we use the throttle here
actionThrottle.minimumWaitBetweenActions();
actionThrottle.markAttemptingAction();
// in future, we could wait for pending tasks in a different thread and re-enqueue
// this event so that we continue processing other events and not block this action executor
waitForPendingTasks(newTrigger, actions);
ActionContext actionContext = new ActionContext(coreContainer, newTrigger, new HashMap<>());
for (TriggerAction action : actions) {
List<String> beforeActions = (List<String>)actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.BEFORE_ACTION.toString(), k -> new ArrayList<String>());
beforeActions.add(action.getName());
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.BEFORE_ACTION, action.getName(), actionContext);
try {
action.process(event, actionContext);
} catch (Exception e) {
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.FAILED, action.getName(), actionContext, e, null);
log.error("Error executing action: " + action.getName() + " for trigger event: " + event, e);
throw e;
}
List<String> afterActions = (List<String>)actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.AFTER_ACTION.toString(), k -> new ArrayList<String>());
afterActions.add(action.getName());
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.AFTER_ACTION, action.getName(), actionContext);
}
if (enqueued) {
TriggerEvent ev = scheduledTrigger.dequeue();
assert ev.getId().equals(event.getId());
}
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.SUCCEEDED);
} finally {
hasPendingActions.set(false);
}
});
} else {
if (enqueued) {
TriggerEvent ev = scheduledTrigger.dequeue();
if (!ev.getId().equals(event.getId())) {
throw new RuntimeException("Wrong event dequeued, queue of " + scheduledTrigger.trigger.getName()
+ " is broken! Expected event=" + event + " but got " + ev);
}
}
listeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.SUCCEEDED);
hasPendingActions.set(false);
}
return true;
} else {
// there is an action in the queue and we don't want to enqueue another until it is complete
return false;
}
});
newTrigger.init(); // mark as ready for scheduling
scheduledTrigger.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(scheduledTrigger, 0, DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS, TimeUnit.SECONDS);
}
private void waitForPendingTasks(AutoScaling.Trigger newTrigger, List<TriggerAction> actions) throws AlreadyClosedException {
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
.withZkHost(coreContainer.getZkController().getZkServerAddress())
.withHttpClient(coreContainer.getUpdateShardHandler().getHttpClient())
.build()) {
SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
for (TriggerAction action : actions) {
if (action instanceof ExecutePlanAction) {
String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + newTrigger.getName() + "/" + action.getName();
if (!zkClient.exists(parentPath, true)) {
break;
}
List<String> children = zkClient.getChildren(parentPath, null, true);
if (children != null) {
for (String child : children) {
String path = parentPath + '/' + child;
byte[] data = zkClient.getData(path, null, null, true);
if (data != null) {
Map map = (Map) Utils.fromJSON(data);
String requestid = (String) map.get("requestid");
try {
log.debug("Found pending task with requestid={}", requestid);
RequestStatusResponse statusResponse = waitForTaskToFinish(cloudSolrClient, requestid,
ExecutePlanAction.DEFAULT_TASK_TIMEOUT_SECONDS, TimeUnit.SECONDS);
if (statusResponse != null) {
RequestStatusState state = statusResponse.getRequestStatus();
if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED || state == RequestStatusState.NOT_FOUND) {
zkClient.delete(path, -1, true);
}
}
} catch (Exception e) {
if (coreContainer.isShutDown()) {
throw e; // propagate the abort to the caller
}
Throwable rootCause = ExceptionUtils.getRootCause(e);
if (rootCause instanceof IllegalStateException && rootCause.getMessage().contains("Connection pool shut down")) {
throw e;
}
if (rootCause instanceof TimeoutException && rootCause.getMessage().contains("Could not connect to ZooKeeper")) {
throw e;
}
log.error("Unexpected exception while waiting for pending task with requestid: " + requestid + " to finish", e);
}
}
}
}
}
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Thread interrupted", e);
} catch (Exception e) {
if (coreContainer.isShutDown()) {
throw new AlreadyClosedException("The Solr instance has been shutdown");
}
// we catch but don't rethrow because a failure to wait for pending tasks
// should not keep the actions from executing
log.error("Unexpected exception while waiting for pending tasks to finish", e);
}
}
/**
* Removes and stops the trigger with the given name. Also cleans up any leftover
* state / events in ZK.
*
* @param triggerName the name of the trigger to be removed
*/
public synchronized void remove(String triggerName) {
ScheduledTrigger removed = scheduledTriggers.remove(triggerName);
IOUtils.closeQuietly(removed);
removeTriggerZKData(triggerName);
}
private void removeTriggerZKData(String triggerName) {
String statePath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + triggerName;
String eventsPath = ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName;
try {
zkDelTree(zkClient, statePath);
} catch (KeeperException | InterruptedException e) {
log.warn("Failed to remove state for removed trigger " + statePath, e);
}
try {
zkDelTree(zkClient, eventsPath);
} catch (KeeperException | InterruptedException e) {
log.warn("Failed to remove events for removed trigger " + eventsPath, e);
}
}
static List<OpResult> zkDelTree(SolrZkClient zkClient, String znode) throws KeeperException, InterruptedException {
if (zkClient.exists(znode, true)) {
ArrayList<Op> ops = new ArrayList<>();
zkDelTree(zkClient, znode, ops);
return zkClient.multi(ops, true);
}
return Collections.emptyList();
}
private static void zkDelTree(SolrZkClient zkClient, String znode, ArrayList<Op> ops) throws KeeperException, InterruptedException {
if (zkClient.exists(znode, true)) {
List<String> children = zkClient.getChildren(znode, null, true);
if (children != null) {
for (String child : children) {
String path = znode + "/" + child;
zkDelTree(zkClient, path, ops);
}
}
ops.add(Op.delete(znode, -1));
}
}
/**
* @return an unmodifiable set of names of all triggers being managed by this class
*/
public synchronized Set<String> getScheduledTriggerNames() {
return Collections.unmodifiableSet(new HashSet<>(scheduledTriggers.keySet())); // shallow copy
}
@Override
public void close() throws IOException {
synchronized (this) {
// mark that we are closed
isClosed = true;
for (ScheduledTrigger scheduledTrigger : scheduledTriggers.values()) {
IOUtils.closeQuietly(scheduledTrigger);
}
scheduledTriggers.clear();
}
// shutdown and interrupt all running tasks because there's no longer any
// guarantee about cluster state
scheduledThreadPoolExecutor.shutdownNow();
actionExecutor.shutdownNow();
listeners.close();
}
private class ScheduledTrigger implements Runnable, Closeable {
AutoScaling.Trigger trigger;
ScheduledFuture<?> scheduledFuture;
TriggerEventQueue queue;
boolean replay;
volatile boolean isClosed;
ScheduledTrigger(AutoScaling.Trigger trigger, SolrZkClient zkClient, Overseer.Stats stats) {
this.trigger = trigger;
this.queue = new TriggerEventQueue(zkClient, trigger.getName(), stats);
this.replay = true;
this.isClosed = false;
}
public void setReplay(boolean replay) {
this.replay = replay;
}
public boolean enqueue(TriggerEvent event) {
if (isClosed) {
throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
}
return queue.offerEvent(event);
}
public TriggerEvent dequeue() {
if (isClosed) {
throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
}
TriggerEvent event = queue.pollEvent();
return event;
}
@Override
public void run() {
if (isClosed) {
throw new AlreadyClosedException("ScheduledTrigger " + trigger.getName() + " has been closed.");
}
// fire a trigger only if an action is not pending
// note this is not fool proof e.g. it does not prevent an action being executed while a trigger
// is still executing. There is additional protection against that scenario in the event listener.
if (!hasPendingActions.get()) {
// replay accumulated events on first run, if any
if (replay) {
TriggerEvent event;
// peek first without removing - we may crash before calling the listener
while ((event = queue.peekEvent()) != null) {
// override REPLAYING=true
event.getProperties().put(TriggerEvent.REPLAYING, true);
if (! trigger.getProcessor().process(event)) {
log.error("Failed to re-play event, discarding: " + event);
}
queue.pollEvent(); // always remove it from queue
}
// now restore saved state to possibly generate new events from old state on the first run
try {
trigger.restoreState();
} catch (Exception e) {
// log but don't throw - see below
log.error("Error restoring trigger state " + trigger.getName(), e);
}
replay = false;
}
try {
trigger.run();
} catch (Exception e) {
// log but do not propagate exception because an exception thrown from a scheduled operation
// will suppress future executions
log.error("Unexpected exception from trigger: " + trigger.getName(), e);
} finally {
// checkpoint after each run
trigger.saveState();
}
}
}
@Override
public void close() throws IOException {
isClosed = true;
if (scheduledFuture != null) {
scheduledFuture.cancel(true);
}
IOUtils.closeQuietly(trigger);
}
}
private class TriggerListeners {
Map<String, Map<TriggerEventProcessorStage, List<TriggerListener>>> listenersPerStage = new HashMap<>();
Map<String, TriggerListener> listenersPerName = new HashMap<>();
ReentrantLock updateLock = new ReentrantLock();
void setAutoScalingConfig(AutoScalingConfig autoScalingConfig) {
updateLock.lock();
// we will recreate this from scratch
listenersPerStage.clear();
try {
Set<String> triggerNames = autoScalingConfig.getTriggerConfigs().keySet();
Map<String, AutoScalingConfig.TriggerListenerConfig> configs = autoScalingConfig.getTriggerListenerConfigs();
Set<String> listenerNames = configs.entrySet().stream().map(entry -> entry.getValue().name).collect(Collectors.toSet());
// close those for non-existent triggers and nonexistent listener configs
for (Iterator<Map.Entry<String, TriggerListener>> it = listenersPerName.entrySet().iterator(); it.hasNext(); ) {
Map.Entry<String, TriggerListener> entry = it.next();
String name = entry.getKey();
TriggerListener listener = entry.getValue();
if (!triggerNames.contains(listener.getConfig().trigger) || !listenerNames.contains(name)) {
try {
listener.close();
} catch (Exception e) {
log.warn("Exception closing old listener " + listener.getConfig(), e);
}
it.remove();
}
}
for (Map.Entry<String, AutoScalingConfig.TriggerListenerConfig> entry : configs.entrySet()) {
AutoScalingConfig.TriggerListenerConfig config = entry.getValue();
if (!triggerNames.contains(config.trigger)) {
log.debug("-- skipping listener for non-existent trigger: {}", config);
continue;
}
// find previous instance and reuse if possible
TriggerListener oldListener = listenersPerName.get(config.name);
TriggerListener listener = null;
if (oldListener != null) {
if (!oldListener.getConfig().equals(config)) { // changed config
try {
oldListener.close();
} catch (Exception e) {
log.warn("Exception closing old listener " + oldListener.getConfig(), e);
}
} else {
listener = oldListener; // reuse
}
}
if (listener == null) { // create new instance
String clazz = config.listenerClass;
try {
listener = coreContainer.getResourceLoader().newInstance(clazz, TriggerListener.class);
} catch (Exception e) {
log.warn("Invalid TriggerListener class name '" + clazz + "', skipping...", e);
}
if (listener != null) {
try {
listener.init(coreContainer, config);
listenersPerName.put(config.name, listener);
} catch (Exception e) {
log.warn("Error initializing TriggerListener " + config, e);
IOUtils.closeQuietly(listener);
listener = null;
}
}
}
if (listener == null) {
continue;
}
// add per stage
for (TriggerEventProcessorStage stage : config.stages) {
addPerStage(config.trigger, stage, listener);
}
// add also for beforeAction / afterAction TriggerStage
if (!config.beforeActions.isEmpty()) {
addPerStage(config.trigger, TriggerEventProcessorStage.BEFORE_ACTION, listener);
}
if (!config.afterActions.isEmpty()) {
addPerStage(config.trigger, TriggerEventProcessorStage.AFTER_ACTION, listener);
}
}
} finally {
updateLock.unlock();
}
}
private void addPerStage(String triggerName, TriggerEventProcessorStage stage, TriggerListener listener) {
Map<TriggerEventProcessorStage, List<TriggerListener>> perStage =
listenersPerStage.computeIfAbsent(triggerName, k -> new HashMap<>());
List<TriggerListener> lst = perStage.computeIfAbsent(stage, k -> new ArrayList<>(3));
lst.add(listener);
}
void reset() {
updateLock.lock();
try {
listenersPerStage.clear();
for (TriggerListener listener : listenersPerName.values()) {
IOUtils.closeQuietly(listener);
}
listenersPerName.clear();
} finally {
updateLock.unlock();
}
}
void close() {
reset();
}
List<TriggerListener> getTriggerListeners(String trigger, TriggerEventProcessorStage stage) {
Map<TriggerEventProcessorStage, List<TriggerListener>> perStage = listenersPerStage.get(trigger);
if (perStage == null) {
return Collections.emptyList();
}
List<TriggerListener> lst = perStage.get(stage);
if (lst == null) {
return Collections.emptyList();
} else {
return Collections.unmodifiableList(lst);
}
}
void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage) {
fireListeners(trigger, event, stage, null, null, null, null);
}
void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage, String message) {
fireListeners(trigger, event, stage, null, null, null, message);
}
void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
ActionContext context) {
fireListeners(trigger, event, stage, actionName, context, null, null);
}
void fireListeners(String trigger, TriggerEvent event, TriggerEventProcessorStage stage, String actionName,
ActionContext context, Throwable error, String message) {
updateLock.lock();
try {
for (TriggerListener listener : getTriggerListeners(trigger, stage)) {
if (actionName != null) {
AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
if (stage == TriggerEventProcessorStage.BEFORE_ACTION) {
if (!config.beforeActions.contains(actionName)) {
continue;
}
} else if (stage == TriggerEventProcessorStage.AFTER_ACTION) {
if (!config.afterActions.contains(actionName)) {
continue;
}
}
}
try {
listener.onEvent(event, stage, actionName, context, error, message);
} catch (Exception e) {
log.warn("Exception running listener " + listener.getConfig(), e);
}
}
} finally {
updateLock.unlock();
}
}
}
}

View File

@ -0,0 +1,219 @@
/*
* 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.io.PrintWriter;
import java.io.StringWriter;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.Date;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.StringJoiner;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonParams;
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.core.CoreContainer;
import org.apache.solr.util.IdUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This listener saves events to the {@link CollectionAdminParams#SYSTEM_COLL} collection.
* <p>Configuration properties:</p>
* <ul>
* <li>collection - optional string, specifies what collection should be used for storing events. Default value
* is {@link CollectionAdminParams#SYSTEM_COLL}.</li>
* </ul>
*/
public class SystemLogListener extends TriggerListenerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String SOURCE_FIELD = "source_s";
public static final String EVENT_SOURCE_FIELD = "event.source_s";
public static final String EVENT_TYPE_FIELD = "event.type_s";
public static final String STAGE_FIELD = "stage_s";
public static final String ACTION_FIELD = "action_s";
public static final String MESSAGE_FIELD = "message_t";
public static final String BEFORE_ACTIONS_FIELD = "before.actions_ss";
public static final String AFTER_ACTIONS_FIELD = "after.actions_ss";
public static final String COLLECTIONS_FIELD = "collections_ss";
public static final String SOURCE = SystemLogListener.class.getSimpleName();
public static final String DOC_TYPE = "autoscaling_event";
private String collection = CollectionAdminParams.SYSTEM_COLL;
private boolean enabled = true;
@Override
public void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) {
super.init(coreContainer, config);
collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
}
@Override
public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
Throwable error, String message) throws Exception {
if (!enabled) {
return;
}
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
.withZkHost(coreContainer.getZkController().getZkServerAddress())
.withHttpClient(coreContainer.getUpdateShardHandler().getHttpClient())
.build()) {
SolrInputDocument doc = new SolrInputDocument();
doc.addField(CommonParams.TYPE, DOC_TYPE);
doc.addField(SOURCE_FIELD, SOURCE);
doc.addField("id", IdUtils.timeRandomId());
doc.addField("event.id_s", event.getId());
doc.addField(EVENT_TYPE_FIELD, event.getEventType().toString());
doc.addField(EVENT_SOURCE_FIELD, event.getSource());
doc.addField("event.time_l", event.getEventTime());
doc.addField("timestamp", new Date());
addMap("event.property.", doc, event.getProperties());
doc.addField(STAGE_FIELD, stage.toString());
if (actionName != null) {
doc.addField(ACTION_FIELD, actionName);
}
if (message != null) {
doc.addField(MESSAGE_FIELD, message);
}
addError(doc, error);
// add JSON versions of event and context
String eventJson = Utils.toJSONString(event);
doc.addField("event_str", eventJson);
if (context != null) {
// capture specifics of operations after compute_plan action
addOperations(doc, (List<SolrRequest>)context.getProperties().get("operations"));
// capture specifics of responses after execute_plan action
addResponses(doc, (List<NamedList<Object>>)context.getProperties().get("responses"));
addActions(BEFORE_ACTIONS_FIELD, doc, (List<String>)context.getProperties().get(TriggerEventProcessorStage.BEFORE_ACTION.toString()));
addActions(AFTER_ACTIONS_FIELD, doc, (List<String>)context.getProperties().get(TriggerEventProcessorStage.AFTER_ACTION.toString()));
String contextJson = Utils.toJSONString(context);
doc.addField("context_str", contextJson);
}
UpdateRequest req = new UpdateRequest();
req.add(doc);
cloudSolrClient.request(req, collection);
} catch (Exception e) {
if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
// relatively benign
log.info("Collection " + collection + " does not exist, disabling logging.");
enabled = false;
} else {
log.warn("Exception sending event to collection " + collection, e);
}
}
}
private void addActions(String field, SolrInputDocument doc, List<String> actions) {
if (actions == null) {
return;
}
actions.forEach(a -> doc.addField(field, a));
}
private void addMap(String prefix, SolrInputDocument doc, Map<String, Object> map) {
map.forEach((k, v) -> {
if (v instanceof Collection) {
for (Object o : (Collection)v) {
doc.addField(prefix + k + "_ss", String.valueOf(o));
}
} else {
doc.addField(prefix + k + "_ss", String.valueOf(v));
}
});
}
private void addOperations(SolrInputDocument doc, List<SolrRequest> operations) {
if (operations == null || operations.isEmpty()) {
return;
}
Set<String> collections = new HashSet<>();
for (SolrRequest req : operations) {
SolrParams params = req.getParams();
if (params == null) {
continue;
}
if (params.get(CollectionAdminParams.COLLECTION) != null) {
collections.add(params.get(CollectionAdminParams.COLLECTION));
}
// build a whitespace-separated param string
StringJoiner paramJoiner = new StringJoiner(" ");
paramJoiner.setEmptyValue("");
for (Iterator<String> it = params.getParameterNamesIterator(); it.hasNext(); ) {
final String name = it.next();
final String [] values = params.getParams(name);
for (String value : values) {
paramJoiner.add(name + "=" + value);
}
}
String paramString = paramJoiner.toString();
if (!paramString.isEmpty()) {
doc.addField("operations.params_ts", paramString);
}
}
if (!collections.isEmpty()) {
doc.addField(COLLECTIONS_FIELD, collections);
}
}
private void addResponses(SolrInputDocument doc, List<NamedList<Object>> responses) {
if (responses == null || responses.isEmpty()) {
return;
}
for (NamedList<Object> rsp : responses) {
Object o = rsp.get("success");
if (o != null) {
doc.addField("responses_ts", "success " + o);
} else {
o = rsp.get("failure");
if (o != null) {
doc.addField("responses_ts", "failure " + o);
} else { // something else
doc.addField("responses_ts", Utils.toJSONString(rsp));
}
}
}
}
private void addError(SolrInputDocument doc, Throwable error) {
if (error == null) {
return;
}
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
error.printStackTrace(pw);
pw.flush(); pw.close();
doc.addField("error.message_t", error.getMessage());
doc.addField("error.details_t", sw.toString());
}
}

View File

@ -0,0 +1,31 @@
/*
* 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.io.Closeable;
import org.apache.solr.util.plugin.MapInitializedPlugin;
/**
* Interface for actions performed in response to a trigger being activated
*/
public interface TriggerAction extends MapInitializedPlugin, Closeable {
String getName();
void process(TriggerEvent event, ActionContext context);
}

View File

@ -0,0 +1,47 @@
/*
* 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.io.IOException;
import java.util.Map;
/**
* Base class for {@link TriggerAction} implementations.
*/
public abstract class TriggerActionBase implements TriggerAction {
protected Map<String, String> initArgs;
@Override
public String getName() {
if (initArgs != null) {
return initArgs.get("name");
} else {
return getClass().getSimpleName();
}
}
@Override
public void close() throws IOException {
}
@Override
public void init(Map<String, String> args) {
this.initArgs = args;
}
}

View File

@ -0,0 +1,107 @@
/*
* 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.lang.invoke.MethodHandles;
import java.util.Arrays;
import java.util.Map;
import java.util.TreeMap;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Base class for {@link org.apache.solr.cloud.autoscaling.AutoScaling.Trigger} implementations.
* It handles state snapshot / restore in ZK.
*/
public abstract class TriggerBase implements AutoScaling.Trigger {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected SolrZkClient zkClient;
protected Map<String,Object> lastState;
protected TriggerBase(SolrZkClient zkClient) {
this.zkClient = zkClient;
try {
zkClient.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, false, true);
} catch (KeeperException | InterruptedException e) {
LOG.warn("Exception checking ZK path " + ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, e);
}
}
/**
* Prepare and return internal state of this trigger in a format suitable for persisting in ZK.
* @return map of internal state properties. Note: values must be supported by {@link Utils#toJSON(Object)}.
*/
protected abstract Map<String,Object> getState();
/**
* Restore internal state of this trigger from properties retrieved from ZK.
* @param state never null but may be empty.
*/
protected abstract void setState(Map<String,Object> state);
@Override
public void saveState() {
Map<String,Object> state = Utils.getDeepCopy(getState(), 10, false, true);
if (lastState != null && lastState.equals(state)) {
// skip saving if identical
return;
}
byte[] data = Utils.toJSON(state);
String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName();
try {
if (zkClient.exists(path, true)) {
// update
zkClient.setData(path, data, -1, true);
} else {
// create
zkClient.create(path, data, CreateMode.PERSISTENT, true);
}
lastState = state;
} catch (KeeperException | InterruptedException e) {
LOG.warn("Exception updating trigger state '" + path + "'", e);
}
}
@Override
public void restoreState() {
byte[] data = null;
String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName();
try {
if (zkClient.exists(path, true)) {
data = zkClient.getData(path, null, new Stat(), true);
}
} catch (KeeperException | InterruptedException e) {
LOG.warn("Exception getting trigger state '" + path + "'", e);
}
if (data != null) {
Map<String, Object> restoredState = (Map<String, Object>)Utils.fromJSON(data);
// make sure lastState is sorted
restoredState = Utils.getDeepCopy(restoredState, 10, false, true);
setState(restoredState);
lastState = restoredState;
}
}
}

View File

@ -0,0 +1,155 @@
/*
* 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.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.MapWriter;
import org.apache.solr.util.IdUtils;
/**
* Trigger event.
*/
public class TriggerEvent implements MapWriter {
public static final String REPLAYING = "replaying";
public static final String NODE_NAMES = "nodeNames";
public static final String EVENT_TIMES = "eventTimes";
protected final String id;
protected final String source;
protected final long eventTime;
protected final TriggerEventType eventType;
protected final Map<String, Object> properties = new HashMap<>();
public TriggerEvent(TriggerEventType eventType, String source, long eventTime,
Map<String, Object> properties) {
this(IdUtils.timeRandomId(eventTime), eventType, source, eventTime, properties);
}
public TriggerEvent(String id, TriggerEventType eventType, String source, long eventTime,
Map<String, Object> properties) {
this.id = id;
this.eventType = eventType;
this.source = source;
this.eventTime = eventTime;
if (properties != null) {
this.properties.putAll(properties);
}
}
/**
* Unique event id.
*/
public String getId() {
return id;
}
/**
* Name of the trigger that fired the event.
*/
public String getSource() {
return source;
}
/**
* Timestamp of the actual event, in nanoseconds.
* NOTE: this is NOT the timestamp when the event was fired - events may be fired
* much later than the actual condition that generated the event, due to the "waitFor" limit.
*/
public long getEventTime() {
return eventTime;
}
/**
* Get event properties (modifiable).
*/
public Map<String, Object> getProperties() {
return properties;
}
/**
* Get a named event property or null if missing.
*/
public Object getProperty(String name) {
return properties.get(name);
}
/**
* Event type.
*/
public TriggerEventType getEventType() {
return eventType;
}
/**
* Set event properties.
*
* @param properties may be null. A shallow copy of this parameter is used.
*/
public void setProperties(Map<String, Object> properties) {
this.properties.clear();
if (properties != null) {
this.properties.putAll(properties);
}
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put("id", id);
ew.put("source", source);
ew.put("eventTime", eventTime);
ew.put("eventType", eventType.toString());
ew.put("properties", properties);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TriggerEvent that = (TriggerEvent) o;
if (eventTime != that.eventTime) return false;
if (!id.equals(that.id)) return false;
if (!source.equals(that.source)) return false;
if (eventType != that.eventType) return false;
return properties.equals(that.properties);
}
@Override
public int hashCode() {
int result = id.hashCode();
result = 31 * result + source.hashCode();
result = 31 * result + (int) (eventTime ^ (eventTime >>> 32));
result = 31 * result + eventType.hashCode();
result = 31 * result + properties.hashCode();
return result;
}
@Override
public String toString() {
return this.getClass().getSimpleName() + "{" +
"id='" + id + '\'' +
", source='" + source + '\'' +
", eventTime=" + eventTime +
", properties=" + properties +
'}';
}
}

View File

@ -0,0 +1,101 @@
package org.apache.solr.cloud.autoscaling;
import java.lang.invoke.MethodHandles;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkDistributedQueue;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
*
*/
public class TriggerEventQueue extends ZkDistributedQueue {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String ENQUEUE_TIME = "_enqueue_time_";
public static final String DEQUEUE_TIME = "_dequeue_time_";
private final String triggerName;
private final TimeSource timeSource;
public TriggerEventQueue(SolrZkClient zookeeper, String triggerName, Overseer.Stats stats) {
super(zookeeper, ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName, stats);
this.triggerName = triggerName;
this.timeSource = TimeSource.CURRENT_TIME;
}
public boolean offerEvent(TriggerEvent event) {
event.getProperties().put(ENQUEUE_TIME, timeSource.getTime());
try {
byte[] data = Utils.toJSON(event);
offer(data);
return true;
} catch (KeeperException | InterruptedException e) {
LOG.warn("Exception adding event " + event + " to queue " + triggerName, e);
return false;
}
}
public TriggerEvent peekEvent() {
byte[] data;
try {
while ((data = peek()) != null) {
if (data.length == 0) {
LOG.warn("ignoring empty data...");
continue;
}
try {
Map<String, Object> map = (Map<String, Object>) Utils.fromJSON(data);
return fromMap(map);
} catch (Exception e) {
LOG.warn("Invalid event data, ignoring: " + new String(data));
continue;
}
}
} catch (KeeperException | InterruptedException e) {
LOG.warn("Exception peeking queue of trigger " + triggerName, e);
}
return null;
}
public TriggerEvent pollEvent() {
byte[] data;
try {
while ((data = poll()) != null) {
if (data.length == 0) {
LOG.warn("ignoring empty data...");
continue;
}
try {
Map<String, Object> map = (Map<String, Object>) Utils.fromJSON(data);
return fromMap(map);
} catch (Exception e) {
LOG.warn("Invalid event data, ignoring: " + new String(data));
continue;
}
}
} catch (KeeperException | InterruptedException e) {
LOG.warn("Exception polling queue of trigger " + triggerName, e);
}
return null;
}
private TriggerEvent fromMap(Map<String, Object> map) {
String id = (String)map.get("id");
String source = (String)map.get("source");
long eventTime = ((Number)map.get("eventTime")).longValue();
TriggerEventType eventType = TriggerEventType.valueOf((String)map.get("eventType"));
Map<String, Object> properties = (Map<String, Object>)map.get("properties");
TriggerEvent res = new TriggerEvent(id, eventType, source, eventTime, properties);
res.getProperties().put(DEQUEUE_TIME, timeSource.getTime());
return res;
}
}

View File

@ -0,0 +1,47 @@
/*
* 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.io.Closeable;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.core.CoreContainer;
/**
* Implementations of this interface are notified of stages in event processing that they were
* registered for. Note: instances may be closed and re-created on each auto-scaling config update.
*/
public interface TriggerListener extends Closeable {
void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) throws Exception;
AutoScalingConfig.TriggerListenerConfig getConfig();
/**
* This method is called when either a particular <code>stage</code> or
* <code>actionName</code> is reached during event processing.
* @param event current event being processed
* @param stage {@link TriggerEventProcessorStage} that this listener was registered for, or null
* @param actionName {@link TriggerAction} name that this listener was registered for, or null
* @param context optional {@link ActionContext} when the processing stage is related to an action, or null
* @param error optional {@link Throwable} error, or null
* @param message optional message
*/
void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
Throwable error, String message) throws Exception;
}

View File

@ -0,0 +1,47 @@
/*
* 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.io.IOException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.core.CoreContainer;
/**
* Base class for implementations of {@link TriggerListener}.
*/
public abstract class TriggerListenerBase implements TriggerListener {
protected AutoScalingConfig.TriggerListenerConfig config;
protected CoreContainer coreContainer;
@Override
public void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) {
this.coreContainer = coreContainer;
this.config = config;
}
@Override
public AutoScalingConfig.TriggerListenerConfig getConfig() {
return config;
}
@Override
public void close() throws IOException {
}
}

View File

@ -37,6 +37,7 @@ 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.params.CoreAdminParams;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;

View File

@ -42,7 +42,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.handler.admin.CollectionsHandler;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.util.SimplePostTool;
import org.apache.zookeeper.server.ByteBufferInputStream;
import org.slf4j.Logger;
@ -153,7 +153,7 @@ public class BlobRepository {
*/
ByteBuffer fetchBlob(String key) {
Replica replica = getSystemCollReplica();
String url = replica.getStr(BASE_URL_PROP) + "/.system/blob/" + key + "?wt=filestream";
String url = replica.getStr(BASE_URL_PROP) + "/" + CollectionAdminParams.SYSTEM_COLL + "/blob/" + key + "?wt=filestream";
HttpClient httpClient = coreContainer.getUpdateShardHandler().getHttpClient();
HttpGet httpGet = new HttpGet(url);
@ -180,10 +180,10 @@ public class BlobRepository {
private Replica getSystemCollReplica() {
ZkStateReader zkStateReader = this.coreContainer.getZkController().getZkStateReader();
ClusterState cs = zkStateReader.getClusterState();
DocCollection coll = cs.getCollectionOrNull(CollectionsHandler.SYSTEM_COLL);
if (coll == null) throw new SolrException(SERVICE_UNAVAILABLE, ".system collection not available");
DocCollection coll = cs.getCollectionOrNull(CollectionAdminParams.SYSTEM_COLL);
if (coll == null) throw new SolrException(SERVICE_UNAVAILABLE, CollectionAdminParams.SYSTEM_COLL + " collection not available");
ArrayList<Slice> slices = new ArrayList<>(coll.getActiveSlices());
if (slices.isEmpty()) throw new SolrException(SERVICE_UNAVAILABLE, "No active slices for .system collection");
if (slices.isEmpty()) throw new SolrException(SERVICE_UNAVAILABLE, "No active slices for " + CollectionAdminParams.SYSTEM_COLL + " collection");
Collections.shuffle(slices, RANDOM); //do load balancing
Replica replica = null;
@ -202,7 +202,7 @@ public class BlobRepository {
}
}
if (replica == null) {
throw new SolrException(SERVICE_UNAVAILABLE, ".no active replica available for .system collection");
throw new SolrException(SERVICE_UNAVAILABLE, "No active replica available for " + CollectionAdminParams.SYSTEM_COLL + " collection");
}
return replica;
}

View File

@ -38,10 +38,6 @@ public class CloudConfig {
private final int autoReplicaFailoverWaitAfterExpiration;
private final int autoReplicaFailoverWorkLoopDelay;
private final int autoReplicaFailoverBadNodeExpiration;
private final String zkCredentialsProviderClass;
private final String zkACLProviderClass;
@ -51,9 +47,9 @@ public class CloudConfig {
private final boolean createCollectionCheckLeaderActive;
CloudConfig(String zkHost, int zkClientTimeout, int hostPort, String hostName, String hostContext, boolean useGenericCoreNames,
int leaderVoteWait, int leaderConflictResolveWait, int autoReplicaFailoverWaitAfterExpiration,
int autoReplicaFailoverWorkLoopDelay, int autoReplicaFailoverBadNodeExpiration, String zkCredentialsProviderClass,
String zkACLProviderClass, int createCollectionWaitTimeTillActive, boolean createCollectionCheckLeaderActive) {
int leaderVoteWait, int leaderConflictResolveWait, int autoReplicaFailoverWaitAfterExpiration,
String zkCredentialsProviderClass, String zkACLProviderClass, int createCollectionWaitTimeTillActive,
boolean createCollectionCheckLeaderActive) {
this.zkHost = zkHost;
this.zkClientTimeout = zkClientTimeout;
this.hostPort = hostPort;
@ -63,8 +59,6 @@ public class CloudConfig {
this.leaderVoteWait = leaderVoteWait;
this.leaderConflictResolveWait = leaderConflictResolveWait;
this.autoReplicaFailoverWaitAfterExpiration = autoReplicaFailoverWaitAfterExpiration;
this.autoReplicaFailoverWorkLoopDelay = autoReplicaFailoverWorkLoopDelay;
this.autoReplicaFailoverBadNodeExpiration = autoReplicaFailoverBadNodeExpiration;
this.zkCredentialsProviderClass = zkCredentialsProviderClass;
this.zkACLProviderClass = zkACLProviderClass;
this.createCollectionWaitTimeTillActive = createCollectionWaitTimeTillActive;
@ -116,14 +110,6 @@ public class CloudConfig {
return autoReplicaFailoverWaitAfterExpiration;
}
public int getAutoReplicaFailoverWorkLoopDelay() {
return autoReplicaFailoverWorkLoopDelay;
}
public int getAutoReplicaFailoverBadNodeExpiration() {
return autoReplicaFailoverBadNodeExpiration;
}
public boolean getGenericCoreNodeNames() {
return useGenericCoreNames;
}
@ -146,8 +132,6 @@ public class CloudConfig {
// TODO: tune defaults
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION = 30000;
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_WORKLOOP_DELAY = 10000;
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_BAD_NODE_EXPIRATION = 60000;
private String zkHost = System.getProperty("zkHost");
private int zkClientTimeout = Integer.getInteger("zkClientTimeout", DEFAULT_ZK_CLIENT_TIMEOUT);
@ -158,8 +142,6 @@ public class CloudConfig {
private int leaderVoteWait = DEFAULT_LEADER_VOTE_WAIT;
private int leaderConflictResolveWait = DEFAULT_LEADER_CONFLICT_RESOLVE_WAIT;
private int autoReplicaFailoverWaitAfterExpiration = DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION;
private int autoReplicaFailoverWorkLoopDelay = DEFAULT_AUTO_REPLICA_FAILOVER_WORKLOOP_DELAY;
private int autoReplicaFailoverBadNodeExpiration = DEFAULT_AUTO_REPLICA_FAILOVER_BAD_NODE_EXPIRATION;
private String zkCredentialsProviderClass;
private String zkACLProviderClass;
private int createCollectionWaitTimeTillActive = DEFAULT_CREATE_COLLECTION_ACTIVE_WAIT;
@ -205,16 +187,6 @@ public class CloudConfig {
return this;
}
public CloudConfigBuilder setAutoReplicaFailoverWorkLoopDelay(int autoReplicaFailoverWorkLoopDelay) {
this.autoReplicaFailoverWorkLoopDelay = autoReplicaFailoverWorkLoopDelay;
return this;
}
public CloudConfigBuilder setAutoReplicaFailoverBadNodeExpiration(int autoReplicaFailoverBadNodeExpiration) {
this.autoReplicaFailoverBadNodeExpiration = autoReplicaFailoverBadNodeExpiration;
return this;
}
public CloudConfigBuilder setZkCredentialsProviderClass(String zkCredentialsProviderClass) {
this.zkCredentialsProviderClass = zkCredentialsProviderClass;
return this;
@ -237,8 +209,7 @@ public class CloudConfig {
public CloudConfig build() {
return new CloudConfig(zkHost, zkClientTimeout, hostPort, hostName, hostContext, useGenericCoreNames, leaderVoteWait,
leaderConflictResolveWait, autoReplicaFailoverWaitAfterExpiration, autoReplicaFailoverWorkLoopDelay,
autoReplicaFailoverBadNodeExpiration, zkCredentialsProviderClass, zkACLProviderClass, createCollectionWaitTimeTillActive,
leaderConflictResolveWait, autoReplicaFailoverWaitAfterExpiration, zkCredentialsProviderClass, zkACLProviderClass, createCollectionWaitTimeTillActive,
createCollectionCheckLeaderActive);
}
}

View File

@ -19,6 +19,7 @@ package org.apache.solr.core;
import static java.util.Objects.requireNonNull;
import static org.apache.solr.common.params.CommonParams.AUTHC_PATH;
import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH;
import static org.apache.solr.common.params.CommonParams.AUTOSCALING_HISTORY_PATH;
import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
@ -78,6 +79,7 @@ import org.apache.solr.core.backup.repository.BackupRepository;
import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.handler.SnapShooter;
import org.apache.solr.handler.admin.AutoscalingHistoryHandler;
import org.apache.solr.handler.admin.CollectionsHandler;
import org.apache.solr.handler.admin.ConfigSetsHandler;
import org.apache.solr.handler.admin.CoreAdminHandler;
@ -103,6 +105,7 @@ import org.apache.solr.security.HttpClientBuilderPlugin;
import org.apache.solr.security.PKIAuthenticationPlugin;
import org.apache.solr.security.SecurityPluginHolder;
import org.apache.solr.update.SolrCoreState;
import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.stats.MetricUtils;
@ -188,6 +191,8 @@ public class CoreContainer {
protected MetricsCollectorHandler metricsCollectorHandler;
protected AutoscalingHistoryHandler autoscalingHistoryHandler;
// Bits for the state variable.
public final static long LOAD_COMPLETE = 0x1L;
@ -531,6 +536,7 @@ public class CoreContainer {
coreAdminHandler = createHandler(CORES_HANDLER_PATH, cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
metricsHandler = createHandler(METRICS_PATH, MetricsHandler.class.getName(), MetricsHandler.class);
autoscalingHistoryHandler = createHandler(AUTOSCALING_HISTORY_PATH, AutoscalingHistoryHandler.class.getName(), AutoscalingHistoryHandler.class);
metricsCollectorHandler = createHandler(MetricsCollectorHandler.HANDLER_PATH, MetricsCollectorHandler.class.getName(), MetricsCollectorHandler.class);
// may want to add some configuration here in the future
metricsCollectorHandler.init(null);
@ -1021,7 +1027,7 @@ public class CoreContainer {
zkSys.getZkController().preRegister(dcore);
}
ConfigSet coreConfig = coreConfigService.getConfig(dcore);
ConfigSet coreConfig = getConfigSet(dcore);
dcore.setConfigSetTrusted(coreConfig.isTrusted());
log.info("Creating SolrCore '{}' using configuration from {}, trusted={}", dcore.getName(), coreConfig.getName(), dcore.isConfigSetTrusted());
try {
@ -1056,6 +1062,21 @@ public class CoreContainer {
MDCLoggingContext.clear();
}
}
public boolean isSharedFs(CoreDescriptor cd) {
try (SolrCore core = this.getCore(cd.getName())) {
if (core != null) {
return core.getDirectoryFactory().isSharedStorage();
} else {
ConfigSet configSet = getConfigSet(cd);
return DirectoryFactory.loadDirectoryFactory(configSet.getSolrConfig(), this, null).isSharedStorage();
}
}
}
private ConfigSet getConfigSet(CoreDescriptor cd) {
return coreConfigService.getConfig(cd);
}
/**
* Take action when we failed to create a SolrCore. If error is due to corrupt index, try to recover. Various recovery

View File

@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.analysis.util.ResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CollectionAdminParams;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -92,7 +93,7 @@ public class MemClassLoader extends ClassLoader implements AutoCloseable, Resour
ProtectionDomain defaultDomain = null;
//using the default protection domain, with no permissions
try {
defaultDomain = new ProtectionDomain(new CodeSource(new URL("http://localhost/.system/blob/" + jarName.get()), (Certificate[]) null),
defaultDomain = new ProtectionDomain(new CodeSource(new URL("http://localhost/" + CollectionAdminParams.SYSTEM_COLL + "/blob/" + jarName.get()), (Certificate[]) null),
null);
} catch (MalformedURLException mue) {
throw new ClassNotFoundException("Unexpected exception ", mue);

View File

@ -82,6 +82,7 @@ import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.CommonParams.EchoParamStyle;
import org.apache.solr.common.params.SolrParams;
@ -3045,7 +3046,8 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
* processing. The Decoder will only run on the first invocations, subsequent invocations will return the
* cached object.
*
* @param key A key in the format of name/version for a blob stored in the .system blob store via the Blob Store API
* @param key A key in the format of name/version for a blob stored in the
* {@link CollectionAdminParams#SYSTEM_COLL} blob store via the Blob Store API
* @param decoder a decoder with which to convert the blob into a Java Object representation (first time only)
* @return a reference to the blob that has already cached the decoded version.
*/

View File

@ -379,15 +379,13 @@ public class SolrXmlConfig {
case "zkClientTimeout":
builder.setZkClientTimeout(parseInt(name, value));
break;
case "autoReplicaFailoverBadNodeExpiration":
builder.setAutoReplicaFailoverBadNodeExpiration(parseInt(name, value));
case "autoReplicaFailoverBadNodeExpiration": case "autoReplicaFailoverWorkLoopDelay":
//TODO remove this in Solr 8.0
log.info("Configuration parameter " + name + " is ignored");
break;
case "autoReplicaFailoverWaitAfterExpiration":
builder.setAutoReplicaFailoverWaitAfterExpiration(parseInt(name, value));
break;
case "autoReplicaFailoverWorkLoopDelay":
builder.setAutoReplicaFailoverWorkLoopDelay(parseInt(name, value));
break;
case "zkHost":
builder.setZkHost(value);
break;

View File

@ -137,6 +137,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
String id = blobName + "/" + version;
Map<String, Object> doc = makeMap(
ID, id,
CommonParams.TYPE, "blob",
"md5", md5,
"blobName", blobName,
VERSION, version,

View File

@ -0,0 +1,132 @@
package org.apache.solr.handler.admin;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;
import java.util.Map;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.autoscaling.SystemLogListener;
import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.security.AuthorizationContext;
import org.apache.solr.security.PermissionNameProvider;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This handler makes it easier to retrieve a history of autoscaling events from the .system
* collection.
*/
public class AutoscalingHistoryHandler extends RequestHandlerBase implements PermissionNameProvider {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String SYSTEM_COLLECTION_PARAM = "systemCollection";
public static final String ACTION_PARAM = "action";
public static final String MESSAGE_PARAM = "message";
public static final String TRIGGER_PARAM = AutoScalingParams.TRIGGER;
public static final String TYPE_PARAM = "eventType";
public static final String NODE_PARAM = "node";
public static final String COLLECTION_PARAM = CollectionAdminParams.COLLECTION;
public static final String STAGE_PARAM = AutoScalingParams.STAGE;
public static final String BEFORE_ACTION_PARAM = AutoScalingParams.BEFORE_ACTION;
public static final String AFTER_ACTION_PARAM = AutoScalingParams.AFTER_ACTION;
private static final String EVENTS_FQ = "{!term f=" + CommonParams.TYPE + "}" + SystemLogListener.DOC_TYPE;
private static final String ACTION_FQ_FORMAT = "{!term f=" + SystemLogListener.ACTION_FIELD + "}%s";
private static final String MESSAGE_FQ_FORMAT = "{!lucene}" + SystemLogListener.MESSAGE_FIELD + ":%s";
private static final String TRIGGER_FQ_FORMAT = "{!term f=" + SystemLogListener.EVENT_SOURCE_FIELD + "}%s";
private static final String STAGE_FQ_FORMAT = "{!term f=" + SystemLogListener.STAGE_FIELD + "}%s";
private static final String COLLECTION_FQ_FORMAT = "{!term f=" + SystemLogListener.COLLECTIONS_FIELD + "}%s";
private static final String TYPE_FQ_FORMAT = "{!term f=" + SystemLogListener.EVENT_TYPE_FIELD + "}%s";
private static final String NODE_FQ_FORMAT = "{!term f=event.property." + TriggerEvent.NODE_NAMES + "_ss}%s";
private static final String BEFORE_ACTION_FQ_FORMAT = "{!term f=" + SystemLogListener.BEFORE_ACTIONS_FIELD + "}%s";
private static final String AFTER_ACTION_FQ_FORMAT = "{!term f=" + SystemLogListener.AFTER_ACTIONS_FIELD + "}%s";
private static final Map<String, String> formats = new HashMap<String, String>() {{
put(ACTION_PARAM, ACTION_FQ_FORMAT);
put(MESSAGE_PARAM, MESSAGE_FQ_FORMAT);
put(TRIGGER_PARAM, TRIGGER_FQ_FORMAT);
put(TYPE_PARAM, TYPE_FQ_FORMAT);
put(STAGE_PARAM, STAGE_FQ_FORMAT);
put(NODE_PARAM, NODE_FQ_FORMAT);
put(COLLECTION_PARAM, COLLECTION_FQ_FORMAT);
put(BEFORE_ACTION_PARAM, BEFORE_ACTION_FQ_FORMAT);
put(AFTER_ACTION_PARAM, AFTER_ACTION_FQ_FORMAT);
}};
private final CoreContainer coreContainer;
public AutoscalingHistoryHandler(CoreContainer coreContainer) {
this.coreContainer = coreContainer;
}
@Override
public Name getPermissionName(AuthorizationContext request) {
return Name.AUTOSCALING_HISTORY_READ_PERM;
}
@Override
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
String collection = params.get(SYSTEM_COLLECTION_PARAM, CollectionAdminParams.SYSTEM_COLL);
params.remove(SYSTEM_COLLECTION_PARAM);
params.remove(CommonParams.QT);
// check that we have the main query, if not then use *:*
if (params.get(CommonParams.Q) == null) {
params.add(CommonParams.Q, "*:*");
}
// sort by doc id, which are time-based, unless specified otherwise
if (params.get(CommonParams.SORT) == null) {
params.add(CommonParams.SORT, "id asc");
}
// filter query to pick only autoscaling events
params.remove(CommonParams.FQ, EVENTS_FQ);
params.add(CommonParams.FQ, EVENTS_FQ);
// add filters translated from simplified parameters
for (Map.Entry<String, String> e : formats.entrySet()) {
String[] values = params.remove(e.getKey());
if (values != null) {
for (String value : values) {
params.add(CommonParams.FQ, String.format(e.getValue(), value));
}
}
}
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
.withZkHost(coreContainer.getZkController().getZkServerAddress())
.withHttpClient(coreContainer.getUpdateShardHandler().getHttpClient())
.build()) {
QueryResponse qr = cloudSolrClient.query(collection, params);
rsp.getValues().add("response", qr.getResults());
} catch (Exception e) {
if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
// relatively benign
LOG.info("Collection " + collection + " does not exist.");
} else {
throw e;
}
}
}
@Override
public String getDescription() {
return "A handler to return autoscaling event history";
}
@Override
public Category getCategory() {
return Category.ADMIN;
}
}

View File

@ -115,7 +115,18 @@ import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;
import static org.apache.solr.common.cloud.DocCollection.RULE;
import static org.apache.solr.common.cloud.DocCollection.SNITCH;
import static org.apache.solr.common.cloud.DocCollection.STATE_FORMAT;
import static org.apache.solr.common.cloud.ZkStateReader.*;
import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
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.PROPERTY_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
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.REPLICA_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@ -348,20 +359,18 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
return Category.ADMIN;
}
public static final String SYSTEM_COLL = ".system";
private static void createSysConfigSet(CoreContainer coreContainer) throws KeeperException, InterruptedException {
SolrZkClient zk = coreContainer.getZkController().getZkStateReader().getZkClient();
ZkCmdExecutor cmdExecutor = new ZkCmdExecutor(zk.getZkClientTimeout());
cmdExecutor.ensureExists(ZkStateReader.CONFIGS_ZKNODE, zk);
cmdExecutor.ensureExists(ZkStateReader.CONFIGS_ZKNODE + "/" + SYSTEM_COLL, zk);
cmdExecutor.ensureExists(ZkStateReader.CONFIGS_ZKNODE + "/" + CollectionAdminParams.SYSTEM_COLL, zk);
try {
String path = ZkStateReader.CONFIGS_ZKNODE + "/" + SYSTEM_COLL + "/schema.xml";
String path = ZkStateReader.CONFIGS_ZKNODE + "/" + CollectionAdminParams.SYSTEM_COLL + "/schema.xml";
byte[] data = IOUtils.toByteArray(CollectionsHandler.class.getResourceAsStream("/SystemCollectionSchema.xml"));
assert data != null && data.length > 0;
cmdExecutor.ensureExists(path, data, CreateMode.PERSISTENT, zk);
path = ZkStateReader.CONFIGS_ZKNODE + "/" + SYSTEM_COLL + "/solrconfig.xml";
path = ZkStateReader.CONFIGS_ZKNODE + "/" + CollectionAdminParams.SYSTEM_COLL + "/solrconfig.xml";
data = IOUtils.toByteArray(CollectionsHandler.class.getResourceAsStream("/SystemCollectionSolrConfig.xml"));
assert data != null && data.length > 0;
cmdExecutor.ensureExists(path, data, CreateMode.PERSISTENT, zk);
@ -418,7 +427,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
if (StringUtils.isNotEmpty(shardsParam)) {
verifyShardsParam(shardsParam);
}
if (SYSTEM_COLL.equals(collectionName)) {
if (CollectionAdminParams.SYSTEM_COLL.equals(collectionName)) {
//We must always create a .system collection with only a single shard
props.put(NUM_SLICES, 1);
props.remove(SHARDS_PROP);
@ -1128,6 +1137,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
REPLICATION_FACTOR,
MAX_SHARDS_PER_NODE,
AUTO_ADD_REPLICAS,
POLICY,
COLL_CONF);
@Override

View File

@ -22,7 +22,6 @@ import java.util.Collections;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@ -140,18 +139,14 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName
continue;
}
MetricUtils.PropertyFilter propertyFilter = MetricUtils.PropertyFilter.ALL;
boolean simple = false;
if (propertyName != null) {
propertyFilter = (name) -> name.equals(propertyName);
simple = true;
// use escaped versions
key = parts[0] + ":" + parts[1];
}
MetricUtils.convertMetric(key, m, propertyFilter, false, true, true, false, ":", (k, v) -> {
if ((v instanceof Map) && propertyName != null) {
((Map)v).forEach((k1, v1) -> result.add(k + ":" + k1, v1));
} else {
result.add(k, v);
}
});
MetricUtils.convertMetric(key, m, propertyFilter, false, true, true, simple, ":", (k, v) -> result.add(k, v));
}
rsp.getValues().add("metrics", result);
if (errors.size() > 0) {

View File

@ -49,6 +49,7 @@ public interface PermissionNameProvider {
METRICS_READ_PERM("metrics-read", null),
AUTOSCALING_READ_PERM("autoscaling-read", null),
AUTOSCALING_WRITE_PERM("autoscaling-write", null),
AUTOSCALING_HISTORY_READ_PERM("autoscaling-history-read", null),
ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
;
final String name;

View File

@ -119,7 +119,7 @@ import static org.apache.solr.common.params.CollectionParams.CollectionAction.DE
import static org.apache.solr.common.params.CollectionParams.CollectionAction.RELOAD;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.params.CoreAdminParams.ACTION;
import static org.apache.solr.handler.admin.CollectionsHandler.SYSTEM_COLL;
import static org.apache.solr.common.params.CollectionAdminParams.SYSTEM_COLL;
import static org.apache.solr.servlet.SolrDispatchFilter.Action.ADMIN;
import static org.apache.solr.servlet.SolrDispatchFilter.Action.FORWARD;
import static org.apache.solr.servlet.SolrDispatchFilter.Action.PASSTHROUGH;
@ -347,7 +347,7 @@ public class HttpSolrCall {
SYSTEM_COLL.equals(corename) &&
"POST".equals(req.getMethod()) &&
!cores.getZkController().getClusterState().hasCollection(SYSTEM_COLL)) {
log.info("Going to auto-create .system collection");
log.info("Going to auto-create " + SYSTEM_COLL + " collection");
SolrQueryResponse rsp = new SolrQueryResponse();
String repFactor = String.valueOf(Math.min(3, cores.getZkController().getClusterState().getLiveNodes().size()));
cores.getCollectionsHandler().handleRequestBody(new LocalSolrQueryRequest(null,
@ -356,7 +356,7 @@ public class HttpSolrCall {
.add( NAME, SYSTEM_COLL)
.add(REPLICATION_FACTOR, repFactor)), rsp);
if (rsp.getValues().get("success") == null) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not auto-create .system collection: "+ Utils.toJSONString(rsp.getValues()));
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not auto-create " + SYSTEM_COLL + " collection: "+ Utils.toJSONString(rsp.getValues()));
}
TimeOut timeOut = new TimeOut(3, TimeUnit.SECONDS);
for (; ; ) {
@ -364,7 +364,7 @@ public class HttpSolrCall {
break;
} else {
if (timeOut.hasTimedOut()) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find .system collection even after 3 seconds");
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not find " + SYSTEM_COLL + " collection even after 3 seconds");
}
Thread.sleep(50);
}

View File

@ -0,0 +1,39 @@
package org.apache.solr.util;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.util.StringHelper;
/**
* Helper class for generating unique ID-s.
*/
public class IdUtils {
/**
* Generate a short random id (see {@link StringHelper#randomId()}).
*/
public static final String randomId() {
return StringHelper.idToString(StringHelper.randomId());
}
/**
* Generate a random id with a timestamp, in the format:
* <code>hex(timestamp) + 'T' + randomId</code>. This method
* uses {@link TimeSource#CURRENT_TIME} for timestamp values.
*/
public static final String timeRandomId() {
return timeRandomId(TimeUnit.MILLISECONDS.convert(TimeSource.CURRENT_TIME.getTime(), TimeUnit.NANOSECONDS));
}
/**
* Generate a random id with a timestamp, in the format:
* <code>hex(timestamp) + 'T' + randomId</code>.
* @param time value representing timestamp
*/
public static final String timeRandomId(long time) {
StringBuilder sb = new StringBuilder(Long.toHexString(time));
sb.append('T');
sb.append(randomId());
return sb.toString();
}
}

View File

@ -113,6 +113,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ContentStreamBase;
@ -1534,7 +1535,7 @@ public class SolrCLI {
boolean configExistsInZk = confname != null && !"".equals(confname.trim()) &&
cloudSolrClient.getZkStateReader().getZkClient().exists("/configs/" + confname, true);
if (".system".equals(collectionName)) {
if (CollectionAdminParams.SYSTEM_COLL.equals(collectionName)) {
//do nothing
} else if (configExistsInZk) {
echo("Re-using existing configuration directory "+confname);

View File

@ -0,0 +1,57 @@
/*
* 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

@ -2,16 +2,37 @@
<schema name="_system collection or core" version="1.1">
<fieldtype name="string" class="solr.StrField" sortMissingLast="true" omitNorms="true"/>
<fieldType name="long" class="solr.LongPointField" docValues="true" positionIncrementGap="0"/>
<fieldType name="double" class="solr.DoublePointField" docValues="true" positionIncrementGap="0"/>
<fieldType name="bytes" class="solr.BinaryField"/>
<fieldType name="date" class="solr.DatePointField" docValues="true"/>
<fieldType name="text_ws" class="solr.TextField" positionIncrementGap="100">
<analyzer>
<tokenizer class="solr.WhitespaceTokenizerFactory"/>
</analyzer>
</fieldType>
<uniqueKey>id</uniqueKey>
<field name="id" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
<field name="md5" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
<field name="_version_" type="long" indexed="true" stored="true"/>
<field name="type" type="string" indexed="true" stored="true" multiValued="false" required="true"/>
<!-- blob repository fields -->
<field name="md5" type="string" indexed="true" stored="true" multiValued="false"/>
<field name="blob" type="bytes" indexed="false" stored="true" multiValued="false" />
<field name="size" type="long" indexed="true" stored="true" multiValued="false" />
<field name="version" type="long" indexed="true" stored="true" multiValued="false" />
<field name="timestamp" type="date" indexed="true" stored="true" multiValued="false" />
<field name="blobName" type="string" indexed="true" stored="true" multiValued="false" />
<field name="_version_" type="long" indexed="true" stored="true"/>
<!-- general purpose fields -->
<dynamicField name="*_s" type="string" indexed="true" stored="true" />
<uniqueKey>id</uniqueKey>
<dynamicField name="*_ss" type="string" indexed="true" multiValued="true" stored="true" />
<dynamicField name="*_str" type="string" indexed="false" stored="true" />
<dynamicField name="*_strs" type="string" indexed="false" multiValued="true" stored="true" />
<dynamicField name="*_bin" type="bytes" indexed="false" multiValued="false" stored="true" />
<dynamicField name="*_t" type="text_ws" indexed="true" stored="true" />
<dynamicField name="*_ts" type="text_ws" indexed="true" multiValued="true" stored="true" />
<dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
<dynamicField name="*_l" type="long" indexed="true" stored="true"/>
<dynamicField name="*_ls" type="long" multiValued="true" indexed="true" stored="true"/>
<dynamicField name="*_d" type="double" indexed="true" stored="true"/>
<dynamicField name="*_ds" type="double" multiValued="true" indexed="true" stored="true"/>
</schema>

View File

@ -21,12 +21,12 @@ import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.cloud.ActionThrottle.NanoTimeSource;
import org.apache.solr.util.TimeSource;
import org.junit.Test;
public class ActionThrottleTest extends SolrTestCaseJ4 {
static class TestNanoTimeSource implements NanoTimeSource {
static class TestNanoTimeSource extends TimeSource {
private List<Long> returnValues;
private int index = 0;
@ -41,35 +41,38 @@ public class ActionThrottleTest extends SolrTestCaseJ4 {
}
}
// use the same time source as ActionThrottle
private static final TimeSource timeSource = TimeSource.NANO_TIME;
@Test
public void testBasics() throws Exception {
ActionThrottle at = new ActionThrottle("test", 1000);
long start = System.nanoTime();
long start = timeSource.getTime();
at.minimumWaitBetweenActions();
// should be no wait
assertTrue(TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS) < 1000);
assertTrue(TimeUnit.MILLISECONDS.convert(timeSource.getTime() - start, TimeUnit.NANOSECONDS) < 1000);
at.markAttemptingAction();
if (random().nextBoolean()) Thread.sleep(100);
at.minimumWaitBetweenActions();
long elaspsedTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS);
long elaspsedTime = TimeUnit.MILLISECONDS.convert(timeSource.getTime() - start, TimeUnit.NANOSECONDS);
assertTrue(elaspsedTime + "ms", elaspsedTime >= 995);
start = System.nanoTime();
start = timeSource.getTime();
at.markAttemptingAction();
at.minimumWaitBetweenActions();
Thread.sleep(random().nextInt(1000));
elaspsedTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS);
elaspsedTime = TimeUnit.MILLISECONDS.convert(timeSource.getTime() - start, TimeUnit.NANOSECONDS);
assertTrue(elaspsedTime + "ms", elaspsedTime >= 995);
}
@ -78,13 +81,13 @@ public class ActionThrottleTest extends SolrTestCaseJ4 {
public void testAZeroNanoTimeReturnInWait() throws Exception {
ActionThrottle at = new ActionThrottle("test", 1000, new TestNanoTimeSource(Arrays.asList(new Long[]{0L, 10L})));
long start = System.nanoTime();
long start = timeSource.getTime();
at.markAttemptingAction();
at.minimumWaitBetweenActions();
long elaspsedTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS);
long elaspsedTime = TimeUnit.MILLISECONDS.convert(timeSource.getTime() - start, TimeUnit.NANOSECONDS);
assertTrue(elaspsedTime + "ms", elaspsedTime >= 995);

View File

@ -18,13 +18,10 @@
package org.apache.solr.cloud;
import java.io.Closeable;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -40,22 +37,12 @@ public class ClusterStateMockUtil {
private final static Pattern BLUEPRINT = Pattern.compile("([a-z])(\\d+)?(?:(['A','R','D','F']))?(\\*)?");
protected static class Result implements Closeable {
OverseerAutoReplicaFailoverThread.DownReplica badReplica;
ZkStateReader reader;
@Override
public void close() throws IOException {
reader.close();
}
protected static ZkStateReader buildClusterState(String string, String ... liveNodes) {
return buildClusterState(string, 1, liveNodes);
}
protected static ClusterStateMockUtil.Result buildClusterState(List<Result> results, String string, String ... liveNodes) {
return buildClusterState(results, string, 1, liveNodes);
}
protected static ClusterStateMockUtil.Result buildClusterState(List<Result> results, String string, int replicationFactor, String ... liveNodes) {
return buildClusterState(results, string, replicationFactor, 10, liveNodes);
protected static ZkStateReader buildClusterState(String string, int replicationFactor, String ... liveNodes) {
return buildClusterState(string, replicationFactor, 10, liveNodes);
}
/**
@ -118,9 +105,7 @@ public class ClusterStateMockUtil {
*
*/
@SuppressWarnings("resource")
protected static ClusterStateMockUtil.Result buildClusterState(List<Result> results, String clusterDescription, int replicationFactor, int maxShardsPerNode, String ... liveNodes) {
ClusterStateMockUtil.Result result = new ClusterStateMockUtil.Result();
protected static ZkStateReader buildClusterState(String clusterDescription, int replicationFactor, int maxShardsPerNode, String ... liveNodes) {
Map<String,Slice> slices = null;
Map<String,Replica> replicas = null;
Map<String,Object> collectionProps = new HashMap<>();
@ -181,23 +166,12 @@ public class ClusterStateMockUtil {
String nodeName = "baseUrl" + node + "_";
String replicaName = "replica" + replicaCount++;
if ("*".equals(m.group(4))) {
replicaName += " (bad)";
}
replicaPropMap.put(ZkStateReader.NODE_NAME_PROP, nodeName);
replicaPropMap.put(ZkStateReader.BASE_URL_PROP, "http://baseUrl" + node);
replicaPropMap.put(ZkStateReader.STATE_PROP, state.toString());
replica = new Replica(replicaName, replicaPropMap);
if ("*".equals(m.group(4))) {
result.badReplica = new OverseerAutoReplicaFailoverThread.DownReplica();
result.badReplica.replica = replica;
result.badReplica.slice = slice;
result.badReplica.collection = docCollection;
}
replicas.put(replica.getName(), replica);
break;
default:
@ -216,17 +190,7 @@ public class ClusterStateMockUtil {
}
System.err.println(json);
// todo remove the limitation of always having a bad replica
assert result.badReplica != null : "Is there no bad replica?";
assert result.badReplica.slice != null : "Is there no bad replica?";
result.reader = reader;
if (results != null) {
results.add(result);
}
return result;
return reader;
}

View File

@ -42,7 +42,7 @@ import org.junit.Test;
BadHdfsThreadsFilter.class, // hdfs currently leaks thread(s)
MoveReplicaHDFSTest.ForkJoinThreadsFilter.class
})
public class MoveReplicaHDFSUlogDirTest extends SolrCloudTestCase {
public class MoveReplicaHDFSFailoverTest extends SolrCloudTestCase {
private static MiniDFSCluster dfsCluster;
@BeforeClass
@ -128,6 +128,71 @@ public class MoveReplicaHDFSUlogDirTest extends SolrCloudTestCase {
cluster.getSolrClient().commit(coll);
assertEquals(numDocs, cluster.getSolrClient().query(coll, new SolrQuery("*:*")).getResults().getNumFound());
CollectionAdminRequest.deleteCollection(coll).process(cluster.getSolrClient());
}
@Test
public void testOldReplicaIsDeleted() throws Exception {
String coll = "movereplicatest_coll3";
CollectionAdminRequest.createCollection(coll, "conf1", 1, 1)
.setCreateNodeSet(cluster.getJettySolrRunner(0).getNodeName())
.process(cluster.getSolrClient());
addDocs(coll, 2);
Replica replica = getCollectionState(coll).getReplicas().iterator().next();
cluster.getJettySolrRunners().get(0).stop();
assertTrue(ClusterStateUtil.waitForAllReplicasNotLive(cluster.getSolrClient().getZkStateReader(), 20000));
// move replica from node0 -> node1
new CollectionAdminRequest.MoveReplica(coll, replica.getName(), cluster.getJettySolrRunner(1).getNodeName())
.process(cluster.getSolrClient());
assertTrue(ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 20000));
cluster.getJettySolrRunners().get(1).stop();
assertTrue(ClusterStateUtil.waitForAllReplicasNotLive(cluster.getSolrClient().getZkStateReader(), 20000));
// node0 will delete it replica because of CloudUtil.checkSharedFSFailoverReplaced()
cluster.getJettySolrRunners().get(0).start();
Thread.sleep(5000);
assertTrue(ClusterStateUtil.waitForAllReplicasNotLive(cluster.getSolrClient().getZkStateReader(), 20000));
cluster.getJettySolrRunners().get(1).start();
assertTrue(ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 20000));
assertEquals(1, getCollectionState(coll).getReplicas().size());
assertEquals(2, cluster.getSolrClient().query(coll, new SolrQuery("*:*")).getResults().getNumFound());
CollectionAdminRequest.deleteCollection(coll).process(cluster.getSolrClient());
}
@Test
public void testOldReplicaIsDeletedInRaceCondition() throws Exception {
String coll = "movereplicatest_coll4";
CollectionAdminRequest.createCollection(coll, "conf1", 1, 1)
.setCreateNodeSet(cluster.getJettySolrRunner(0).getNodeName())
.process(cluster.getSolrClient());
addDocs(coll, 100);
Replica replica = getCollectionState(coll).getReplicas().iterator().next();
cluster.getJettySolrRunners().get(0).stop();
assertTrue(ClusterStateUtil.waitForAllReplicasNotLive(cluster.getSolrClient().getZkStateReader(), 20000));
// move replica from node0 -> node1
new CollectionAdminRequest.MoveReplica(coll, replica.getName(), cluster.getJettySolrRunner(1).getNodeName())
.process(cluster.getSolrClient());
assertTrue(ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 20000));
cluster.getJettySolrRunners().get(1).stop();
assertTrue(ClusterStateUtil.waitForAllReplicasNotLive(cluster.getSolrClient().getZkStateReader(), 20000));
cluster.getJettySolrRunners().get(1).start();
// node0 will delete it replica because of CloudUtil.checkSharedFSFailoverReplaced()
cluster.getJettySolrRunners().get(0).start();
Thread.sleep(5000);
assertTrue(ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 20000));
assertEquals(1, getCollectionState(coll).getReplicas().size());
assertEquals(100, cluster.getSolrClient().query(coll, new SolrQuery("*:*")).getResults().getNumFound());
CollectionAdminRequest.deleteCollection(coll).process(cluster.getSolrClient());
}
private void addDocs(String collection, int numDocs) throws SolrServerException, IOException {

View File

@ -42,31 +42,30 @@ public class NodeMutatorTest extends SolrTestCaseJ4Test {
@Test
public void downNodeReportsAllImpactedCollectionsAndNothingElse() throws IOException {
NodeMutator nm = new NodeMutator();
ZkNodeProps props = new ZkNodeProps(ZkStateReader.NODE_NAME_PROP, NODE1);
//We use 2 nodes with maxShardsPerNode as 1
//Collection1: 2 shards X 1 replica = replica1 on node1 and replica2 on node2
//Collection2: 1 shard X 1 replica = replica1 on node2
ClusterStateMockUtil.Result result = ClusterStateMockUtil.buildClusterState(null, "csrr2rD*csr2", 1, 1, NODE1, NODE2);
ClusterState clusterState = result.reader.getClusterState();
ZkStateReader reader = ClusterStateMockUtil.buildClusterState("csrr2rDcsr2", 1, 1, NODE1, NODE2);
ClusterState clusterState = reader.getClusterState();
assertEquals(clusterState.getCollection("collection1").getReplica("replica1").getBaseUrl(), NODE1_URL);
assertEquals(clusterState.getCollection("collection1").getReplica("replica2").getBaseUrl(), NODE2_URL);
assertEquals(clusterState.getCollection("collection2").getReplica("replica4").getBaseUrl(), NODE2_URL);
props = new ZkNodeProps(ZkStateReader.NODE_NAME_PROP, NODE1);
ZkNodeProps props = new ZkNodeProps(ZkStateReader.NODE_NAME_PROP, NODE1);
List<ZkWriteCommand> writes = nm.downNode(clusterState, props);
assertEquals(writes.size(), 1);
assertEquals(writes.get(0).name, "collection1");
assertEquals(writes.get(0).collection.getReplica("replica1").getState(), Replica.State.DOWN);
assertEquals(writes.get(0).collection.getReplica("replica2").getState(), Replica.State.ACTIVE);
result.close();
reader.close();
//We use 3 nodes with maxShardsPerNode as 1
//Collection1: 2 shards X 1 replica = replica1 on node1 and replica2 on node2
//Collection2: 1 shard X 1 replica = replica1 on node2
//Collection3: 1 shard X 3 replica = replica1 on node1 , replica2 on node2, replica3 on node3
result = ClusterStateMockUtil.buildClusterState(null, "csrr2rD*csr2csr1r2r3", 1, 1, NODE1, NODE2, NODE3);
clusterState = result.reader.getClusterState();
reader = ClusterStateMockUtil.buildClusterState("csrr2rDcsr2csr1r2r3", 1, 1, NODE1, NODE2, NODE3);
clusterState = reader.getClusterState();
assertEquals(clusterState.getCollection("collection1").getReplica("replica1").getBaseUrl(), NODE1_URL);
assertEquals(clusterState.getCollection("collection1").getReplica("replica2").getBaseUrl(), NODE2_URL);
@ -90,6 +89,6 @@ public class NodeMutatorTest extends SolrTestCaseJ4Test {
fail("No other collection needs to be changed");
}
}
result.close();
reader.close();
}
}

View File

@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.cloud.Overseer.LeaderStatus;
import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
import org.apache.solr.common.cloud.ClusterState;
@ -72,6 +73,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
private static ZkStateReader zkStateReaderMock;
private static ClusterState clusterStateMock;
private static SolrZkClient solrZkClientMock;
private static AutoScalingConfig autoScalingConfig = new AutoScalingConfig(Collections.emptyMap());
private final Map zkMap = new HashMap();
private final Map<String, ClusterState.CollectionRef> collectionsSet = new HashMap<>();
private final List<ZkNodeProps> replicas = new ArrayList<>();
@ -197,6 +199,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
when(zkStateReaderMock.getZkClient()).thenReturn(solrZkClientMock);
when(zkStateReaderMock.getClusterState()).thenReturn(clusterStateMock);
when(zkStateReaderMock.getAutoScalingConfig()).thenReturn(autoScalingConfig);
when(clusterStateMock.getCollection(anyString())).thenAnswer(invocation -> {
String key = invocation.getArgument(0);

View File

@ -48,6 +48,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.update.UpdateShardHandlerConfig;
@ -61,11 +62,14 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.xml.sax.SAXException;
import static org.apache.solr.cloud.AbstractDistribZkTestBase.verifyReplicaStatus;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@Slow
public class OverseerTest extends SolrTestCaseJ4 {
@ -607,7 +611,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
updateShardHandlers.add(updateShardHandler);
HttpShardHandlerFactory httpShardHandlerFactory = new HttpShardHandlerFactory();
httpShardHandlerFactorys.add(httpShardHandlerFactory);
Overseer overseer = new Overseer(httpShardHandlerFactory.getShardHandler(), updateShardHandler, "/admin/cores", reader, null,
MockZkController mockZkController = createMockZkController(zkClient, reader);
Overseer overseer = new Overseer(httpShardHandlerFactory.getShardHandler(), updateShardHandler, "/admin/cores", reader, mockZkController,
new CloudConfig.CloudConfigBuilder("127.0.0.1", 8983, "").build());
overseers.add(overseer);
ElectionContext ec = new OverseerElectionContext(zkClient, overseer,
@ -1174,7 +1179,10 @@ public class OverseerTest extends SolrTestCaseJ4 {
updateShardHandlers.add(updateShardHandler);
HttpShardHandlerFactory httpShardHandlerFactory = new HttpShardHandlerFactory();
httpShardHandlerFactorys.add(httpShardHandlerFactory);
Overseer overseer = new Overseer(httpShardHandlerFactory.getShardHandler(), updateShardHandler, "/admin/cores", reader, null,
MockZkController zkController = createMockZkController(zkClient, reader);
Overseer overseer = new Overseer(httpShardHandlerFactory.getShardHandler(), updateShardHandler, "/admin/cores", reader, zkController,
new CloudConfig.CloudConfigBuilder("127.0.0.1", 8983, "").build());
overseers.add(overseer);
ElectionContext ec = new OverseerElectionContext(zkClient, overseer,
@ -1183,7 +1191,19 @@ public class OverseerTest extends SolrTestCaseJ4 {
overseerElector.joinElection(ec, false);
return zkClient;
}
private MockZkController createMockZkController(SolrZkClient zkClient, ZkStateReader reader) {
CoreContainer mockAlwaysUpCoreContainer = mock(CoreContainer.class,
Mockito.withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS));
when(mockAlwaysUpCoreContainer.isShutDown()).thenReturn(Boolean.FALSE); // Allow retry on session expiry
MockZkController zkController = mock(MockZkController.class,
Mockito.withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS));
when(zkController.getCoreContainer()).thenReturn(mockAlwaysUpCoreContainer);
when(zkController.getZkClient()).thenReturn(zkClient);
when(zkController.getZkStateReader()).thenReturn(reader);
return zkController;
}
@Test
public void testRemovalOfLastReplica() throws Exception {

View File

@ -1021,4 +1021,3 @@ public class ShardSplitTest extends BasicDistributedZkTest {
return client;
}
}

View File

@ -16,12 +16,12 @@
*/
package org.apache.solr.cloud;
import static org.apache.solr.common.util.Utils.makeMap;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletionService;
@ -30,39 +30,44 @@ import java.util.concurrent.Future;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.lucene.util.LuceneTestCase.Slow;
import com.carrotsearch.randomizedtesting.annotations.Nightly;
import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.hdfs.HdfsTestUtil;
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.ClusterStateUtil;
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.params.CollectionParams;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.update.DirectUpdateHandler2;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.BadHdfsThreadsFilter;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeOut;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Nightly
@Slow
@ -70,8 +75,10 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
@ThreadLeakFilters(defaultFilters = true, filters = {
BadHdfsThreadsFilter.class // hdfs currently leaks thread(s)
})
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.*=DEBUG")
public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final boolean DEBUG = true;
private static MiniDFSCluster dfsCluster;
@ -210,9 +217,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
Thread.sleep(5000);
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cloudClient.getZkStateReader(), collection1, 120000));
assertSliceAndReplicaCount(collection1);
assertSliceAndReplicaCount(collection1, 2, 2, 120000);
assertEquals(4, ClusterStateUtil.getLiveAndActiveReplicaCount(cloudClient.getZkStateReader(), collection1));
assertTrue(ClusterStateUtil.getLiveAndActiveReplicaCount(cloudClient.getZkStateReader(), collection2) < 4);
@ -225,7 +230,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
// all docs should be queried after failover
cloudClient.commit(); // to query all docs
assertSingleReplicationAndShardSize(collection4, 5);
assertSliceAndReplicaCount(collection4, 5, 1, 120000);
queryAndAssertResultSize(collection4, numDocs, 10000);
// collection1 should still be at 4
@ -235,21 +240,22 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
assertUlogDir(collections);
ChaosMonkey.stop(jettys);
boolean allowOverseerRestart = random().nextBoolean();
List<JettySolrRunner> stoppedJetties = allowOverseerRestart
? jettys.stream().filter(jettySolrRunner -> random().nextBoolean()).collect(Collectors.toList()) : notOverseerJetties();
ChaosMonkey.stop(stoppedJetties);
ChaosMonkey.stop(controlJetty);
assertTrue("Timeout waiting for all not live", ClusterStateUtil.waitForAllReplicasNotLive(cloudClient.getZkStateReader(), 45000));
assertTrue("Timeout waiting for all not live", waitingForReplicasNotLive(cloudClient.getZkStateReader(), 45000, stoppedJetties));
ChaosMonkey.start(jettys);
ChaosMonkey.start(stoppedJetties);
ChaosMonkey.start(controlJetty);
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cloudClient.getZkStateReader(), collection1, 120000));
assertSliceAndReplicaCount(collection1);
assertSingleReplicationAndShardSize(collection3, 5);
assertSliceAndReplicaCount(collection1, 2, 2, 120000);
assertSliceAndReplicaCount(collection3, 5, 1, 120000);
// all docs should be queried
assertSingleReplicationAndShardSize(collection4, 5);
assertSliceAndReplicaCount(collection4, 5, 1, 120000);
queryAndAssertResultSize(collection4, numDocs, 10000);
assertUlogDir(collections);
@ -257,75 +263,13 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
int jettyIndex = random().nextInt(jettys.size());
ChaosMonkey.stop(jettys.get(jettyIndex));
ChaosMonkey.start(jettys.get(jettyIndex));
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cloudClient.getZkStateReader(), collection1, 60000));
assertSliceAndReplicaCount(collection1);
assertSliceAndReplicaCount(collection1, 2, 2, 120000);
assertUlogDir(collections);
assertSingleReplicationAndShardSize(collection3, 5);
ClusterStateUtil.waitForLiveAndActiveReplicaCount(cloudClient.getZkStateReader(), collection3, 5, 30000);
assertSingleReplicationAndShardSize(collection4, 5);
ClusterStateUtil.waitForLiveAndActiveReplicaCount(cloudClient.getZkStateReader(), collection4, 5, 30000);
//disable autoAddReplicas
Map m = makeMap(
"action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
"name", ZkStateReader.AUTO_ADD_REPLICAS,
"val", "false");
SolrRequest request = new QueryRequest(new MapSolrParams(m));
request.setPath("/admin/collections");
cloudClient.request(request);
int currentCount = ClusterStateUtil.getLiveAndActiveReplicaCount(cloudClient.getZkStateReader(), collection1);
ChaosMonkey.stop(jettys.get(3));
//solr.xml has defined workLoopDelay=10s and waitAfterExpiration=10s
//Hence waiting for 30 seconds to be on the safe side.
Thread.sleep(30000);
//Ensures that autoAddReplicas has not kicked in.
assertTrue(currentCount > ClusterStateUtil.getLiveAndActiveReplicaCount(cloudClient.getZkStateReader(), collection1));
//enable autoAddReplicas
m = makeMap(
"action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
"name", ZkStateReader.AUTO_ADD_REPLICAS);
request = new QueryRequest(new MapSolrParams(m));
request.setPath("/admin/collections");
cloudClient.request(request);
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cloudClient.getZkStateReader(), collection1, 90000));
assertSliceAndReplicaCount(collection1);
assertUlogDir(collections);
// restart all to test core saved state
ChaosMonkey.stop(jettys);
ChaosMonkey.stop(controlJetty);
assertTrue("Timeout waiting for all not live", ClusterStateUtil.waitForAllReplicasNotLive(cloudClient.getZkStateReader(), 45000));
ChaosMonkey.start(jettys);
ChaosMonkey.start(controlJetty);
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cloudClient.getZkStateReader(), collection1, 120000));
assertSliceAndReplicaCount(collection1);
assertUlogDir(collections);
assertSliceAndReplicaCount(collection1);
assertSingleReplicationAndShardSize(collection3, 5);
// all docs should be queried
assertSingleReplicationAndShardSize(collection4, 5);
queryAndAssertResultSize(collection4, numDocs, 10000);
assertSliceAndReplicaCount(collection3, 5, 1, 120000);
assertSliceAndReplicaCount(collection4, 5, 1, 120000);
}
private void queryAndAssertResultSize(String collection, int expectedResultSize, int timeoutMS)
@ -339,10 +283,14 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
}
SolrParams queryAll = new SolrQuery("*:*");
cloudClient.setDefaultCollection(collection);
QueryResponse queryResponse = cloudClient.query(queryAll);
actualResultSize = queryResponse.getResults().getNumFound();
if(expectedResultSize == actualResultSize) {
return;
try {
QueryResponse queryResponse = cloudClient.query(queryAll);
actualResultSize = queryResponse.getResults().getNumFound();
if(expectedResultSize == actualResultSize) {
return;
}
} catch (SolrServerException | IOException e) {
log.warn("Querying solr threw an exception. This can be expected to happen during restarts.", e);
}
Thread.sleep(1000);
@ -386,26 +334,83 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
}
}
private void assertSingleReplicationAndShardSize(String collection, int numSlices) {
Collection<Slice> slices;
slices = cloudClient.getZkStateReader().getClusterState().getCollection(collection).getActiveSlices();
assertEquals(numSlices, slices.size());
for (Slice slice : slices) {
assertEquals(1, slice.getReplicas().size());
}
// Overseer failover is not currently guaranteed with MoveReplica or Policy Framework
private List<JettySolrRunner> notOverseerJetties() throws IOException, SolrServerException {
CollectionAdminResponse response = CollectionAdminRequest.getOverseerStatus().process(cloudClient);
String overseerNode = (String) response.getResponse().get("leader");
return jettys.stream().filter(jetty -> !(jetty.getCoreContainer() != null && overseerNode.equals(jetty.getNodeName())))
.collect(Collectors.toList());
}
private void assertSliceAndReplicaCount(String collection) {
Collection<Slice> slices;
slices = cloudClient.getZkStateReader().getClusterState().getCollection(collection).getActiveSlices();
assertEquals(2, slices.size());
for (Slice slice : slices) {
assertEquals(2, slice.getReplicas().size());
private boolean waitingForReplicasNotLive(ZkStateReader zkStateReader, int timeoutInMs, List<JettySolrRunner> jetties) {
Set<String> nodeNames = jetties.stream()
.filter(jetty -> jetty.getCoreContainer() != null)
.map(JettySolrRunner::getNodeName)
.collect(Collectors.toSet());
long timeout = System.nanoTime()
+ TimeUnit.NANOSECONDS.convert(timeoutInMs, TimeUnit.MILLISECONDS);
boolean success = false;
while (!success && System.nanoTime() < timeout) {
success = true;
ClusterState clusterState = zkStateReader.getClusterState();
if (clusterState != null) {
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
DocCollection docCollection = entry.getValue();
Collection<Slice> slices = docCollection.getSlices();
for (Slice slice : slices) {
// only look at active shards
if (slice.getState() == Slice.State.ACTIVE) {
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
if (nodeNames.contains(replica.getNodeName())) {
boolean live = clusterState.liveNodesContain(replica
.getNodeName());
if (live) {
success = false;
}
}
}
}
}
}
if (!success) {
try {
Thread.sleep(500);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Interrupted");
}
}
}
}
return success;
}
@Override
public void distribTearDown() throws Exception {
super.distribTearDown();
private void assertSliceAndReplicaCount(String collection, int numSlices, int numReplicas, int timeOutInMs) throws InterruptedException {
TimeOut timeOut = new TimeOut(timeOutInMs, TimeUnit.MILLISECONDS);
while (!timeOut.hasTimedOut()) {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
Collection<Slice> slices = clusterState.getCollection(collection).getActiveSlices();
if (slices.size() == numSlices) {
boolean isMatch = true;
for (Slice slice : slices) {
int count = 0;
for (Replica replica : slice.getReplicas()) {
if (replica.getState() == Replica.State.ACTIVE && clusterState.liveNodesContain(replica.getNodeName())) {
count++;
}
}
if (count < numReplicas) {
isMatch = false;
}
}
if (isMatch) return;
}
Thread.sleep(200);
}
fail("Expected numSlices=" + numSlices + " numReplicas=" + numReplicas + " but found " + cloudClient.getZkStateReader().getClusterState().getCollection(collection) + " with /live_nodes: " + cloudClient.getZkStateReader().getClusterState().getLiveNodes());
}
}

View File

@ -1,191 +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.cloud;
import java.util.ArrayList;
import java.util.List;
import org.apache.solr.SolrTestCaseJ4;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.apache.solr.cloud.ClusterStateMockUtil.buildClusterState;
public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 {
private static final String NODE6 = "baseUrl6_";
private static final String NODE6_URL = "http://baseUrl6";
private static final String NODE5 = "baseUrl5_";
private static final String NODE5_URL = "http://baseUrl5";
private static final String NODE4 = "baseUrl4_";
private static final String NODE4_URL = "http://baseUrl4";
private static final String NODE3 = "baseUrl3_";
private static final String NODE3_URL = "http://baseUrl3";
private static final String NODE2 = "baseUrl2_";
private static final String NODE2_URL = "http://baseUrl2";
private static final String NODE1 = "baseUrl1_";
private static final String NODE1_URL = "http://baseUrl1";
private List<ClusterStateMockUtil.Result> results;
@Before
public void setUp() throws Exception {
super.setUp();
results = new ArrayList<>();
}
@After
public void tearDown() throws Exception {
super.tearDown();
for (ClusterStateMockUtil.Result result : results) {
result.close();
}
}
@Test
public void testGetBestCreateUrlBasics() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr1R*r2", NODE1);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull("Should be no live node to failover to", createUrl);
result = buildClusterState(results, "csr1R*r2", NODE1, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull("Only failover candidate node already has a replica", createUrl);
result = buildClusterState(results, "csr1R*r2sr3", NODE1, NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals("Node3 does not have a replica from the bad slice and should be the best choice", NODE3_URL, createUrl);
result = buildClusterState(results, "csr1R*r2Fsr3r4r5", NODE1, NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertTrue(createUrl.equals(NODE3_URL));
result = buildClusterState(results, "csr1*r2r3sr3r3sr4", NODE1, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE4_URL, createUrl);
result = buildClusterState(results, "csr1*r2sr3r3sr4sr4", NODE1, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertTrue(createUrl.equals(NODE3_URL) || createUrl.equals(NODE4_URL));
}
@Test
public void testGetBestCreateUrlMultipleCollections() throws Exception {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr*r2csr2", NODE1);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState(results, "csr*r2csr2", NODE1);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState(results, "csr*r2csr2", NODE1, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
}
@Test
public void testGetBestCreateUrlMultipleCollections2() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr*r2sr3cr2", NODE1);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState(results, "csr*r2sr3cr2", NODE1, NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE3_URL, createUrl);
}
@Test
public void testGetBestCreateUrlMultipleCollections3() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr5r1sr4r2sr3r6csr2*r6sr5r3sr4r3", NODE1, NODE4, NODE5, NODE6);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE1_URL, createUrl);
}
@Test
public void testGetBestCreateUrlMultipleCollections4() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr1r4sr3r5sr2r6csr5r6sr4r6sr5*r4", NODE6);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE6_URL, createUrl);
}
@Test
public void testFailOverToEmptySolrInstance() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr1*r1sr1csr1", NODE2);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
}
@Test
public void testFavorForeignSlices() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr*sr2csr3r3", NODE2, NODE3);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE3_URL, createUrl);
result = buildClusterState(results, "csr*sr2csr3r3r3r3r3r3r3", NODE2, NODE3);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
}
@Test
public void testCollectionMaxNodesPerShard() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr*sr2", 1, 1, NODE2);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertNull(createUrl);
result = buildClusterState(results, "csr*sr2", 1, 2, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
result = buildClusterState(results, "csr*csr2r2", 1, 1, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null);
assertEquals(NODE2_URL, createUrl);
}
@Test
public void testMaxCoresPerNode() {
ClusterStateMockUtil.Result result = buildClusterState(results, "csr*sr2", 1, 1, NODE2);
String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 1);
assertNull(createUrl);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertNull(createUrl);
result = buildClusterState(results, "csr*sr2", 1, 2, NODE2);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertEquals(NODE2_URL, createUrl);
result = buildClusterState(results, "csr*sr2sr3sr4", 1, 1, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 1);
assertNull(createUrl);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertNull(createUrl);
result = buildClusterState(results, "csr*sr2sr3sr4", 1, 2, NODE2, NODE3, NODE4);
createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2);
assertTrue(createUrl.equals(NODE3_URL) || createUrl.equals(NODE4_URL));
}
}

View File

@ -0,0 +1,182 @@
/*
* 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.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.ClusterStateUtil;
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.params.MapSolrParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeOut;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.apache.solr.common.util.Utils.makeMap;
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
private static final String COLLECTION1 = "testSimple1";
private static final String COLLECTION2 = "testSimple2";
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(3)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Test
public void testSimple() throws Exception {
JettySolrRunner jetty1 = cluster.getJettySolrRunner(0);
JettySolrRunner jetty2 = cluster.getJettySolrRunner(1);
JettySolrRunner jetty3 = cluster.getJettySolrRunner(2);
CollectionAdminRequest.createCollection(COLLECTION1, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(true)
.setMaxShardsPerNode(2)
.process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(COLLECTION2, "conf", 2, 2)
.setCreateNodeSet(jetty2.getNodeName()+","+jetty3.getNodeName())
.setAutoAddReplicas(false)
.setMaxShardsPerNode(2)
.process(cluster.getSolrClient());
// the number of cores in jetty1 (5) will be larger than jetty3 (1)
CollectionAdminRequest.createCollection("testSimple3", "conf", 3, 1)
.setCreateNodeSet(jetty1.getNodeName())
.setAutoAddReplicas(false)
.setMaxShardsPerNode(3)
.process(cluster.getSolrClient());
ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
// start the tests
JettySolrRunner lostJetty = random().nextBoolean() ? cluster.getJettySolrRunner(0) : cluster.getJettySolrRunner(1);
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION1, zkStateReader, lostNodeName);
lostJetty.stop();
waitForNodeLeave(lostNodeName);
waitForState("Waiting for collection " + COLLECTION1, COLLECTION1, clusterShape(2, 2));
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION1);
lostJetty.start();
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 90000));
// check cluster property is considered
disableAutoAddReplicasInCluster();
lostNodeName = jetty3.getNodeName();
jetty3.stop();
waitForNodeLeave(lostNodeName);
waitForState("Waiting for collection " + COLLECTION1, COLLECTION1, clusterShape(2, 1));
jetty3.start();
waitForState("Waiting for collection " + COLLECTION1, COLLECTION1, clusterShape(2, 2));
waitForState("Waiting for collection " + COLLECTION2, COLLECTION2, clusterShape(2, 2));
enableAutoAddReplicasInCluster();
// test for multiple collections
new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.MODIFYCOLLECTION) {
@Override
public SolrParams getParams() {
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
params.set("collection", COLLECTION2);
params.set("autoAddReplicas", true);
return params;
}
}.process(cluster.getSolrClient());
lostNodeName = jetty2.getNodeName();
replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION2, zkStateReader, lostNodeName);
jetty2.stop();
waitForNodeLeave(lostNodeName);
waitForState("Waiting for collection " + COLLECTION1, COLLECTION1, clusterShape(2, 2));
waitForState("Waiting for collection " + COLLECTION2, COLLECTION2, clusterShape(2, 2));
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION2);
// overseer failover test..
}
private void disableAutoAddReplicasInCluster() throws SolrServerException, IOException {
Map m = makeMap(
"action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
"name", ZkStateReader.AUTO_ADD_REPLICAS,
"val", "false");
QueryRequest request = new QueryRequest(new MapSolrParams(m));
request.setPath("/admin/collections");
cluster.getSolrClient().request(request);
}
private void enableAutoAddReplicasInCluster() throws SolrServerException, IOException {
Map m = makeMap(
"action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
"name", ZkStateReader.AUTO_ADD_REPLICAS);
QueryRequest request = new QueryRequest(new MapSolrParams(m));
request.setPath("/admin/collections");
cluster.getSolrClient().request(request);
}
private void checkSharedFsReplicasMovedCorrectly(List<Replica> replacedHdfsReplicas, ZkStateReader zkStateReader, String collection){
DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection);
for (Replica replica :replacedHdfsReplicas) {
boolean found = false;
String dataDir = replica.getStr("dataDir");
String ulogDir = replica.getStr("ulogDir");
for (Replica replica2 : docCollection.getReplicas()) {
if (dataDir.equals(replica2.getStr("dataDir")) && ulogDir.equals(replica2.getStr("ulogDir"))) {
found = true;
break;
}
}
if (!found) fail("Can not found a replica with same dataDir and ulogDir as " + replica + " from:" + docCollection.getReplicas());
}
}
private List<Replica> getReplacedSharedFsReplicas(String collection, ZkStateReader zkStateReader, String lostNodeName) {
List<Replica> replacedHdfsReplicas = new ArrayList<>();
for (Replica replica : zkStateReader.getClusterState().getCollection(collection).getReplicas()) {
String dataDir = replica.getStr("dataDir");
if (replica.getNodeName().equals(lostNodeName) && dataDir != null) {
replacedHdfsReplicas.add(replica);
}
}
return replacedHdfsReplicas;
}
private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS);
while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
Thread.sleep(100);
if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");
}
}
}

View File

@ -0,0 +1,195 @@
/*
* 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.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.ClusterStateUtil;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
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.util.TimeOut;
import org.junit.BeforeClass;
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)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Test
public void testSimple() throws Exception {
JettySolrRunner jetty1 = cluster.getJettySolrRunner(0);
JettySolrRunner jetty2 = cluster.getJettySolrRunner(1);
JettySolrRunner jetty3 = cluster.getJettySolrRunner(2);
String collection1 = "testSimple1";
String collection2 = "testSimple2";
CollectionAdminRequest.createCollection(collection1, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(true)
.setMaxShardsPerNode(2)
.process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection2, "conf", 1, 2)
.setCreateNodeSet(jetty2.getNodeName()+","+jetty3.getNodeName())
.setAutoAddReplicas(false)
.setMaxShardsPerNode(1)
.process(cluster.getSolrClient());
// the number of cores in jetty1 (5) will be larger than jetty3 (1)
CollectionAdminRequest.createCollection("testSimple3", "conf", 3, 1)
.setCreateNodeSet(jetty1.getNodeName())
.setAutoAddReplicas(false)
.setMaxShardsPerNode(3)
.process(cluster.getSolrClient());
// we remove the implicit created trigger, so the replicas won't be moved
String removeTriggerCommand = "{" +
"'remove-trigger' : {" +
"'name' : '.auto_add_replicas'," +
"'removeListeners': true" +
"}" +
"}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, removeTriggerCommand);
NamedList response = cluster.getSolrClient().request(req);
assertEquals(response.get("result").toString(), "success");
JettySolrRunner lostJetty = random().nextBoolean()? jetty1 : jetty2;
String lostNodeName = lostJetty.getNodeName();
List<CloudDescriptor> cloudDescriptors = lostJetty.getCoreContainer().getCores().stream()
.map(solrCore -> solrCore.getCoreDescriptor().getCloudDescriptor())
.collect(Collectors.toList());
lostJetty.stop();
waitForNodeLeave(lostNodeName);
List<SolrRequest> operations = getOperations(jetty3, lostNodeName);
assertOperations(collection1, operations, lostNodeName, cloudDescriptors, null);
lostJetty.start();
ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 30000);
String setClusterPreferencesCommand = "{" +
"'set-cluster-preferences': [" +
"{'minimize': 'cores','precision': 0}]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPreferencesCommand);
response = cluster.getSolrClient().request(req);
assertEquals(response.get("result").toString(), "success");
lostJetty = random().nextBoolean()? jetty1 : jetty2;
lostNodeName = lostJetty.getNodeName();
cloudDescriptors = lostJetty.getCoreContainer().getCores().stream()
.map(solrCore -> solrCore.getCoreDescriptor().getCloudDescriptor())
.collect(Collectors.toList());
lostJetty.stop();
waitForNodeLeave(lostNodeName);
operations = getOperations(jetty3, lostNodeName);
assertOperations(collection1, operations, lostNodeName, cloudDescriptors, jetty3);
lostJetty.start();
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 30000));
new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.MODIFYCOLLECTION) {
@Override
public SolrParams getParams() {
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
params.set("collection", collection1);
params.set("autoAddReplicas", false);
return params;
}
}.process(cluster.getSolrClient());
lostJetty = jetty1;
lostNodeName = lostJetty.getNodeName();
lostJetty.stop();
waitForNodeLeave(lostNodeName);
operations = getOperations(jetty3, lostNodeName);
assertNull(operations);
}
private void waitForNodeLeave(String lostNodeName) throws InterruptedException {
ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS);
while (reader.getClusterState().getLiveNodes().contains(lostNodeName)) {
Thread.sleep(100);
if (timeOut.hasTimedOut()) fail("Wait for " + lostNodeName + " to leave failed!");
}
}
private List<SolrRequest> getOperations(JettySolrRunner actionJetty, String lostNodeName) {
AutoAddReplicasPlanAction action = new AutoAddReplicasPlanAction();
TriggerEvent lostNode = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST, ".auto_add_replicas", Collections.singletonList(System.currentTimeMillis()), Collections.singletonList(lostNodeName));
ActionContext context = new ActionContext(actionJetty.getCoreContainer(), null, new HashMap<>());
action.process(lostNode, context);
List<SolrRequest> operations = (List) context.getProperty("operations");
return operations;
}
private void assertOperations(String collection, List<SolrRequest> operations, String lostNodeName,
List<CloudDescriptor> cloudDescriptors, JettySolrRunner destJetty) {
assertEquals("Replicas of " + collection + " is not fully moved, operations="+operations,
cloudDescriptors.stream().filter(cd -> cd.getCollectionName().equals(collection)).count(), operations.size());
for (SolrRequest solrRequest : operations) {
assertTrue(solrRequest instanceof CollectionAdminRequest.MoveReplica);
SolrParams params = solrRequest.getParams();
assertEquals(params.get("collection"), collection);
String replica = params.get("replica");
boolean found = false;
Iterator<CloudDescriptor> it = cloudDescriptors.iterator();
while (it.hasNext()) {
CloudDescriptor cd = it.next();
if (cd.getCollectionName().equals(collection) && cd.getCoreNodeName().equals(replica)) {
found = true;
it.remove();
break;
}
}
assertTrue("Can not find "+replica+ " in node " + lostNodeName, found);
String targetNode = params.get("targetNode");
assertFalse("Target node match the lost one " + lostNodeName, lostNodeName.equals(targetNode));
if (destJetty != null) {
assertEquals("Target node is not as expectation", destJetty.getNodeName(), targetNode);
}
}
for (CloudDescriptor cd : cloudDescriptors) {
if (cd.getCollectionName().equals(collection)) {
fail("Exist replica which is not moved " + cd);
}
}
}
}

View File

@ -22,10 +22,13 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
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.Policy;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@ -38,11 +41,14 @@ import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.ContentStreamBase;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.data.Stat;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
import static org.apache.solr.common.util.Utils.getObjectByPath;
/**
* Test for AutoScalingHandler
@ -54,6 +60,31 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
configureCluster(2)
.addConfig(CONFIGSET_NAME, configset("cloud-minimal"))
.configure();
testAutoAddReplicas();
}
private static void testAutoAddReplicas() throws Exception {
TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS);
while (!timeOut.hasTimedOut()) {
byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
ZkNodeProps loaded = ZkNodeProps.load(data);
Map triggers = (Map) loaded.get("triggers");
if (triggers != null && triggers.containsKey(".auto_add_replicas")) {
Map<String, Object> autoAddReplicasTrigger = (Map<String, Object>) triggers.get(".auto_add_replicas");
assertNotNull(autoAddReplicasTrigger);
List<Map<String, Object>> actions = (List<Map<String, Object>>) autoAddReplicasTrigger.get("actions");
assertNotNull(actions);
assertEquals(2, actions.size());
assertEquals("auto_add_replicas_plan", actions.get(0).get("name").toString());
assertEquals("solr.AutoAddReplicasPlanAction", actions.get(0).get("class").toString());
break;
} else {
Thread.sleep(300);
}
}
if (timeOut.hasTimedOut()) {
fail("Timeout waiting for .auto_add_replicas being created");
}
}
@Before
@ -62,6 +93,333 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
}
@Test
public void testSuspendTrigger() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String suspendEachCommand = "{\n" +
"\t\"suspend-trigger\" : {\n" +
"\t\t\"name\" : \"" + Policy.EACH + "\"\n" +
"\t}\n" +
"}";
String resumeEachCommand = "{\n" +
"\t\"resume-trigger\" : {\n" +
"\t\t\"name\" : \"" + Policy.EACH + "\"\n" +
"\t}\n" +
"}";
// these should be no-ops because there are no triggers, and it should succeed
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, suspendEachCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
assertEquals(response.get("changed").toString(), "[]");
req = createAutoScalingRequest(SolrRequest.METHOD.POST, resumeEachCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
assertEquals(response.get("changed").toString(), "[]");
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '10m'," +
"'enabled' : true}}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '10m'," +
"'enabled' : true" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String suspendTriggerCommand = "{\n" +
"\t\"suspend-trigger\" : {\n" +
"\t\t\"name\" : \"node_lost_trigger\"\n" +
"\t}\n" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, suspendTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
assertEquals(response.get("changed").toString(), "[node_lost_trigger]");
Stat stat = new Stat();
byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
ZkNodeProps loaded = ZkNodeProps.load(data);
Map<String, Object> triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(2, countNotImplicitTriggers(triggers));
assertTrue(triggers.containsKey("node_lost_trigger"));
assertTrue(triggers.containsKey("node_added_trigger"));
Map<String, Object> nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
assertEquals(4, nodeLostTrigger.size());
assertEquals("false", nodeLostTrigger.get("enabled").toString());
Map<String, Object> nodeAddedTrigger = (Map<String, Object>) triggers.get("node_added_trigger");
assertEquals(4, nodeAddedTrigger.size());
assertEquals("true", nodeAddedTrigger.get("enabled").toString());
suspendTriggerCommand = "{" +
"'suspend-trigger' : {" +
"'name' : '" + Policy.EACH + "'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, suspendTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
List<String> changed = (List<String>)response.get("changed");
assertEquals(1, changed.size());
assertTrue(changed.contains("node_added_trigger"));
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(2, countNotImplicitTriggers(triggers));
nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
assertEquals(4, nodeLostTrigger.size());
assertEquals("false", nodeLostTrigger.get("enabled").toString());
nodeAddedTrigger = (Map<String, Object>) triggers.get("node_added_trigger");
assertEquals(4, nodeAddedTrigger.size());
assertEquals("false", nodeAddedTrigger.get("enabled").toString());
String resumeTriggerCommand = "{" +
"'resume-trigger' : {" +
"'name' : 'node_added_trigger'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, resumeTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
changed = (List<String>)response.get("changed");
assertEquals(1, changed.size());
assertTrue(changed.contains("node_added_trigger"));
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(2, countNotImplicitTriggers(triggers));
nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
assertEquals(4, nodeLostTrigger.size());
assertEquals("false", nodeLostTrigger.get("enabled").toString());
nodeAddedTrigger = (Map<String, Object>) triggers.get("node_added_trigger");
assertEquals(4, nodeAddedTrigger.size());
assertEquals("true", nodeAddedTrigger.get("enabled").toString());
resumeTriggerCommand = "{" +
"'resume-trigger' : {" +
"'name' : '" + Policy.EACH + "'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, resumeTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
changed = (List<String>)response.get("changed");
assertEquals(1, changed.size());
assertTrue(changed.contains("node_lost_trigger"));
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(2, countNotImplicitTriggers(triggers));
nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
assertEquals(4, nodeLostTrigger.size());
assertEquals("true", nodeLostTrigger.get("enabled").toString());
nodeAddedTrigger = (Map<String, Object>) triggers.get("node_added_trigger");
assertEquals(4, nodeAddedTrigger.size());
assertEquals("true", nodeAddedTrigger.get("enabled").toString());
suspendTriggerCommand = "{" +
"'suspend-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'timeout' : '1h'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, suspendTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
changed = (List<String>)response.get("changed");
assertEquals(1, changed.size());
assertTrue(changed.contains("node_lost_trigger"));
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(2, countNotImplicitTriggers(triggers));
nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
assertEquals(5, nodeLostTrigger.size());
assertEquals("false", nodeLostTrigger.get("enabled").toString());
assertTrue(nodeLostTrigger.containsKey("resumeAt"));
}
@Test
public void test() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '10m'," +
"'enabled' : true," +
"'actions' : [" +
"{" +
"'name' : 'compute_plan'," +
"'class' : 'solr.ComputePlanAction'" +
"}]}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
ZkNodeProps loaded = ZkNodeProps.load(data);
Map<String, Object> triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(1, countNotImplicitTriggers(triggers));
assertTrue(triggers.containsKey("node_lost_trigger"));
Map<String, Object> nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
assertEquals(4, nodeLostTrigger.size());
List<Map<String, String>> actions = (List<Map<String, String>>) nodeLostTrigger.get("actions");
assertNotNull(actions);
assertEquals(1, actions.size());
assertEquals("600", nodeLostTrigger.get("waitFor").toString());
setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '20m'," +
"'enabled' : false" +
"}}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(1, countNotImplicitTriggers(triggers));
assertTrue(triggers.containsKey("node_lost_trigger"));
nodeLostTrigger = (Map<String, Object>) triggers.get("node_lost_trigger");
assertEquals(4, nodeLostTrigger.size());
assertEquals("1200", nodeLostTrigger.get("waitFor").toString());
assertEquals("false", nodeLostTrigger.get("enabled").toString());
actions = (List<Map<String, String>>) nodeLostTrigger.get("actions");
assertNotNull(actions);
assertEquals(2, actions.size());
String setListenerCommand = "{" +
"'set-listener' : " +
"{" +
"'name' : 'xyz'," +
"'trigger' : 'node_lost_trigger'," +
"'stage' : ['STARTED','ABORTED','SUCCEEDED']," +
"'beforeAction' : 'execute_plan'," +
"'class' : 'org.apache.solr.cloud.autoscaling.HttpTriggerListener'," +
"'url' : 'http://xyz.com/on_node_lost?node={$LOST_NODE_NAME}'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
Map<String, Object> listeners = (Map<String, Object>) loaded.get("listeners");
assertNotNull(listeners);
assertEquals(2, listeners.size());
assertTrue(listeners.containsKey("xyz"));
Map<String, Object> xyzListener = (Map<String, Object>) listeners.get("xyz");
assertEquals(6, xyzListener.size());
assertEquals("org.apache.solr.cloud.autoscaling.HttpTriggerListener", xyzListener.get("class").toString());
String removeTriggerCommand = "{" +
"'remove-trigger' : {" +
"'name' : 'node_lost_trigger'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, removeTriggerCommand);
try {
solrClient.request(req);
fail("expected exception");
} catch (HttpSolrClient.RemoteExecutionException e) {
// expected
assertTrue(String.valueOf(getObjectByPath(e.getMetaData(),
false, "error/details[0]/errorMessages[0]")).contains("Cannot remove trigger: node_lost_trigger because it has active listeners: ["));
}
String removeListenerCommand = "{\n" +
"\t\"remove-listener\" : {\n" +
"\t\t\"name\" : \"xyz\"\n" +
"\t}\n" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, removeListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
listeners = (Map<String, Object>) loaded.get("listeners");
assertNotNull(listeners);
assertEquals(1, listeners.size());
removeTriggerCommand = "{" +
"'remove-trigger' : {" +
"'name' : 'node_lost_trigger'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, removeTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
triggers = (Map<String, Object>) loaded.get("triggers");
assertNotNull(triggers);
assertEquals(0, countNotImplicitTriggers(triggers));
setListenerCommand = "{" +
"'set-listener' : {" +
"'name' : 'xyz'," +
"'trigger' : 'node_lost_trigger'," +
"'stage' : ['STARTED','ABORTED','SUCCEEDED']," +
"'beforeAction' : 'execute_plan'," +
"'class' : 'org.apache.solr.cloud.autoscaling.AutoScaling$HttpTriggerListener'," +
"'url' : 'http://xyz.com/on_node_lost?node={$LOST_NODE_NAME}'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
try {
solrClient.request(req);
fail("should have thrown Exception");
} catch (HttpSolrClient.RemoteSolrException e) {
// expected
assertTrue(String.valueOf(getObjectByPath(((HttpSolrClient.RemoteExecutionException) e).getMetaData(),
false, "error/details[0]/errorMessages[0]")).contains("A trigger with the name node_lost_trigger does not exist"));
}
}
@Test
public void testErrorHandling() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
try {
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
fail("expect exception");
} catch (HttpSolrClient.RemoteExecutionException e) {
String message = String.valueOf(Utils.getObjectByPath(e.getMetaData(), true, "error/details[0]/errorMessages[0]"));
assertTrue(message.contains("replica is required in"));
}
}
@Test
public void testPolicyAndPreferences() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
@ -177,29 +535,21 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
assertNotNull(clusterPolicy);
assertEquals(3, clusterPolicy.size());
}
public void testErrorHandling() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
try {
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
fail("expect exception");
} catch (HttpSolrClient.RemoteExecutionException e) {
String message = String.valueOf(Utils.getObjectByPath(e.getMetaData(), true, "error/details[0]/errorMessages[0]"));
assertTrue(message.contains("replica is required in"));
}
}
@Test
public void testReadApi() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
// first trigger
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger1'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '0s'," +
"'enabled' : true" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
@ -208,8 +558,8 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
NamedList<Object> response = solrClient.request(req);
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setPreferencesCommand = "{" +
@ -230,6 +580,9 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
" ]," +
" 'policy1':[" +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
" ]," +
" 'policy2':[" +
" {'replica':'<7', 'shard': '#EACH', 'node': '#ANY'}" +
" ]" +
"}}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
@ -239,6 +592,16 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
req = createAutoScalingRequest(SolrRequest.METHOD.GET, null);
response = solrClient.request(req);
Map triggers = (Map) response.get("triggers");
assertNotNull(triggers);
assertEquals(1, countNotImplicitTriggers(triggers));
assertTrue(triggers.containsKey("node_added_trigger1"));
Map node_added_trigger1 = (Map) triggers.get("node_added_trigger1");
assertEquals(4, node_added_trigger1.size());
assertEquals(0L, node_added_trigger1.get("waitFor"));
assertEquals(true, node_added_trigger1.get("enabled"));
assertEquals(2, ((List)node_added_trigger1.get("actions")).size());
List<Map> clusterPrefs = (List<Map>) response.get("cluster-preferences");
assertNotNull(clusterPrefs);
assertEquals(4, clusterPrefs.size());
@ -249,7 +612,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
Map policies = (Map) response.get("policies");
assertNotNull(policies);
assertEquals(2, policies.size());
assertEquals(3, policies.size());
assertNotNull(policies.get("xyz"));
assertNotNull(policies.get("policy1"));
@ -284,18 +647,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
assertNotNull(violations);
assertEquals(0, violations.size());
String setEmptyClusterPolicyCommand = "{" +
" 'set-cluster-policy': []" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setEmptyClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
req = createAutoScalingRequest(SolrRequest.METHOD.POST, "{set-cluster-policy : []}");
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// lets create a collection which violates the rule replicas < 2
// assert that when a cluster policy is in effect, using maxShardsPerNode throws an exception
try {
CollectionAdminRequest.Create create = CollectionAdminRequest.Create.createCollection("readApiTestViolations", CONFIGSET_NAME, 1, 6);
create.setMaxShardsPerNode(10);
@ -303,9 +655,19 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
fail();
} catch (Exception e) {
assertTrue(e.getMessage().contains("'maxShardsPerNode>0' is not supported when autoScaling policies are used"));
}
// temporarily increase replica limit in cluster policy so that we can create a collection with 6 replicas
String tempClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<4', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, tempClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// lets create a collection which violates the rule replicas < 2
CollectionAdminRequest.Create create = CollectionAdminRequest.Create.createCollection("readApiTestViolations", CONFIGSET_NAME, 1, 6);
@ -317,11 +679,6 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// get the diagnostics output again
req = createAutoScalingRequest(SolrRequest.METHOD.GET, "/diagnostics", null);
response = solrClient.request(req);
@ -335,18 +692,99 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
for (Map<String, Object> violation : violations) {
assertEquals("readApiTestViolations", violation.get("collection"));
assertEquals("shard1", violation.get("shard"));
assertEquals(Utils.makeMap("replica", "3", "delta", -1), violation.get("violation"));
assertEquals(-1l, getObjectByPath(violation, true, "violation/delta"));
assertEquals(3l, getObjectByPath(violation, true, "violation/replica/NRT"));
assertNotNull(violation.get("clause"));
}
}
@Test
public void testConcurrentUpdates() throws Exception {
int COUNT = 50;
CloudSolrClient solrClient = cluster.getSolrClient();
CountDownLatch updateLatch = new CountDownLatch(COUNT * 2);
Runnable r = () -> {
for (int i = 0; i < COUNT; i++) {
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger1'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '0s'," +
"'enabled' : true" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = null;
try {
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
} catch (Exception e) {
fail(e.toString());
} finally {
updateLatch.countDown();
}
}
};
Thread t1 = new Thread(r);
Thread t2 = new Thread(r);
t1.start();
t2.start();
boolean await = updateLatch.await(60, TimeUnit.SECONDS);
assertTrue("not all updates executed in time, remaining=" + updateLatch.getCount(), await);
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.GET, null);
NamedList<Object> response = solrClient.request(req);
Map triggers = (Map) response.get("triggers");
assertNotNull(triggers);
assertEquals(1, countNotImplicitTriggers(triggers));
assertTrue(triggers.containsKey("node_added_trigger1"));
Map node_added_trigger1 = (Map) triggers.get("node_added_trigger1");
assertEquals(4, node_added_trigger1.size());
assertEquals(0L, node_added_trigger1.get("waitFor"));
assertEquals(true, node_added_trigger1.get("enabled"));
assertEquals(2, ((List)node_added_trigger1.get("actions")).size());
}
private int countNotImplicitTriggers(Map triggers) {
if (triggers == null) return 0;
int count = 0;
for (Object trigger : triggers.keySet()) {
if (!trigger.toString().startsWith(".")) count++;
}
return count;
}
@Test
public void testDeleteUsedPolicy() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
// add multiple policies
String setPolicyCommand = "{'set-policy': {" +
" 'nodelete':[" +
" {'nodeRole':'overseer', 'replica':0}]}}";
solrClient.request(createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand));
CollectionAdminRequest.createCollection("COLL1", "conf", 1, 1)
.setPolicy("nodelete")
.process(cluster.getSolrClient());
String removePolicyCommand = "{remove-policy : nodelete}";
createAutoScalingRequest(SolrRequest.METHOD.POST, removePolicyCommand);
try {
solrClient.request(createAutoScalingRequest(SolrRequest.METHOD.POST, removePolicyCommand));
fail("should have failed");
} catch (HttpSolrClient.RemoteExecutionException e) {
assertTrue(String.valueOf(getObjectByPath(e.getMetaData(), true, "error/details[0]/errorMessages[0]"))
.contains("is being used by collection"));
} catch (Exception e) {
fail("Only RemoteExecutionException expected");
}
solrClient.request(CollectionAdminRequest.deleteCollection("COLL1"));
}
public static SolrRequest createAutoScalingRequest(SolrRequest.METHOD m, String message) {
return createAutoScalingRequest(m, null, message);
}
static SolrRequest createAutoScalingRequest(SolrRequest.METHOD m, String subPath, String message) {
boolean useV1 = random().nextBoolean();
boolean useV1 = false;
String path = useV1 ? "/admin/autoscaling" : "/cluster/autoscaling";
path += subPath != null ? subPath : "";
return useV1

View File

@ -0,0 +1,372 @@
/*
* 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.io.IOException;
import java.lang.invoke.MethodHandles;
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 com.google.common.base.Charsets;
import org.apache.solr.client.solrj.SolrRequest;
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;
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.ZkStateReader;
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.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;
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;")
public class ComputePlanActionTest extends SolrCloudTestCase {
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(NODE_COUNT)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Before
public void setUp() throws Exception {
super.setUp();
fired.set(false);
triggerFiredLatch = new CountDownLatch(1);
actionContextPropsRef.set(null);
// remove everything from autoscaling.json in ZK
zkClient().setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, "{}".getBytes(Charsets.UTF_8), true);
if (cluster.getJettySolrRunners().size() > NODE_COUNT) {
// stop some to get to original state
int numJetties = cluster.getJettySolrRunners().size();
for (int i = 0; i < numJetties - NODE_COUNT; i++) {
JettySolrRunner randomJetty = cluster.getRandomJetty(random());
List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
for (int i1 = 0; i1 < jettySolrRunners.size(); i1++) {
JettySolrRunner jettySolrRunner = jettySolrRunners.get(i1);
if (jettySolrRunner == randomJetty) {
cluster.stopJettySolrRunner(i1);
break;
}
}
}
}
CloudSolrClient solrClient = cluster.getSolrClient();
try {
CollectionAdminRequest.deleteCollection("testNodeLost").process(solrClient);
} catch (Exception e) {
// expected if testNodeLost hasn't run already
}
try {
CollectionAdminRequest.deleteCollection("testNodeAdded").process(solrClient);
} catch (Exception e) {
// expected if testNodeAdded hasn't run already
}
try {
CollectionAdminRequest.deleteCollection("testNodeWithMultipleReplicasLost").process(solrClient);
} catch (Exception e) {
// expected if testNodeWithMultipleReplicasLost hasn't run already
}
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);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setClusterPreferencesCommand = "{" +
"'set-cluster-preferences': [" +
"{'minimize': 'cores'}," +
"{'maximize': 'freedisk','precision': 100}]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPreferencesCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
}
@Test
public void testNodeLost() throws Exception {
// let's start a node so that we have at least two
JettySolrRunner runner = cluster.startJettySolrRunner();
String node = runner.getNodeName();
CloudSolrClient solrClient = cluster.getSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + ComputePlanActionTest.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.setMaxShardsPerNode(1);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
"testNodeLost", clusterShape(1, 2));
ClusterState clusterState = cluster.getSolrClient().getZkStateReader().getClusterState();
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
JettySolrRunner node2 = cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
// stop the original node
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner jettySolrRunner = cluster.getJettySolrRunners().get(i);
if (jettySolrRunner == runner) {
cluster.stopJettySolrRunner(i);
break;
}
}
cluster.waitForAllNodes(30);
assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, 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", 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
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner jettySolrRunner = cluster.getJettySolrRunners().get(i);
if (jettySolrRunner == node2) {
cluster.stopJettySolrRunner(i);
break;
}
}
}
public void testNodeWithMultipleReplicasLost() throws Exception {
// start 3 more nodes
cluster.startJettySolrRunner();
cluster.startJettySolrRunner();
cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
CloudSolrClient solrClient = cluster.getSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + ComputePlanActionTest.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.getSolrClient().getZkStateReader().getClusterState();
DocCollection docCollection = clusterState.getCollection("testNodeWithMultipleReplicasLost");
// lets find a node with at least 2 replicas
String stoppedNodeName = null;
List<Replica> replicasToBeMoved = null;
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner jettySolrRunner = cluster.getJettySolrRunners().get(i);
List<Replica> replicas = docCollection.getReplicas(jettySolrRunner.getNodeName());
if (replicas != null && replicas.size() == 2) {
stoppedNodeName = jettySolrRunner.getNodeName();
replicasToBeMoved = replicas;
cluster.stopJettySolrRunner(i);
break;
}
}
assertNotNull(stoppedNodeName);
cluster.waitForAllNodes(30);
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", 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 {
CloudSolrClient solrClient = cluster.getSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + ComputePlanActionTest.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
JettySolrRunner runner = cluster.startJettySolrRunner();
assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, 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", operations);
assertEquals("ComputePlanAction should have computed exactly 1 operation", 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", runner.getNodeName(), nodeAdded);
}
public static class AssertingTriggerAction implements TriggerAction {
@Override
public String getName() {
return null;
}
@Override
public void process(TriggerEvent event, ActionContext context) {
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,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;
import java.io.IOException;
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.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.SolrServerException;
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;
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.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeSource;
import org.apache.zookeeper.data.Stat;
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.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
/**
* Test for {@link ExecutePlanAction}
*/
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
public class ExecutePlanActionTest extends SolrCloudTestCase {
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)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Before
public void setUp() throws Exception {
super.setUp();
// clear any persisted auto scaling configuration
Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
if (cluster.getJettySolrRunners().size() < NODE_COUNT) {
// start some to get to original state
int numJetties = cluster.getJettySolrRunners().size();
for (int i = 0; i < NODE_COUNT - numJetties; i++) {
cluster.startJettySolrRunner();
}
}
cluster.waitForAllNodes(30);
}
@Test
public void testExecute() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String collectionName = "testExecute";
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));
JettySolrRunner sourceNode = cluster.getRandomJetty(random());
String sourceNodeName = sourceNode.getNodeName();
ClusterState clusterState = solrClient.getZkStateReader().getClusterState();
DocCollection docCollection = clusterState.getCollection(collectionName);
List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
assertNotNull(replicas);
assertFalse(replicas.isEmpty());
List<JettySolrRunner> otherJetties = cluster.getJettySolrRunners().stream()
.filter(jettySolrRunner -> jettySolrRunner != sourceNode).collect(Collectors.toList());
assertFalse(otherJetties.isEmpty());
JettySolrRunner survivor = otherJetties.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.getNodeName());
CollectionAdminRequest.AsyncCollectionAdminRequest mockRequest = new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.LIST) {
@Override
public String processAsync(String asyncId, SolrClient client) throws IOException, SolrServerException {
String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/xyz/execute_plan";
try {
if (zkClient().exists(parentPath, true)) {
java.util.List<String> children = zkClient().getChildren(parentPath, null, true);
if (!children.isEmpty()) {
String child = children.get(0);
byte[] data = zkClient().getData(parentPath + "/" + child, null, null, true);
Map m = (Map) Utils.fromJSON(data);
if (m.containsKey("requestid")) {
znodeCreated.set(m.get("requestid").equals(asyncId));
}
}
}
} catch (Exception e) {
throw new RuntimeException(e);
}
super.processAsync(asyncId, client);
return asyncId;
}
};
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(survivor.getCoreContainer(), new NodeLostTrigger("xyz", Collections.singletonMap("event", TriggerEventType.NODELOST.name()), survivor.getCoreContainer(), survivor.getCoreContainer().getZkController()),
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"));
}
waitForState("Timed out waiting for replicas of new collection to be active",
collectionName, clusterShape(1, 2));
}
@Test
public void testIntegration() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
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));
JettySolrRunner sourceNode = cluster.getRandomJetty(random());
String sourceNodeName = sourceNode.getNodeName();
ClusterState clusterState = solrClient.getZkStateReader().getClusterState();
DocCollection docCollection = clusterState.getCollection(collectionName);
List<Replica> replicas = docCollection.getReplicas(sourceNodeName);
assertNotNull(replicas);
assertFalse(replicas.isEmpty());
List<JettySolrRunner> otherJetties = cluster.getJettySolrRunners().stream()
.filter(jettySolrRunner -> jettySolrRunner != sourceNode).collect(Collectors.toList());
assertFalse(otherJetties.isEmpty());
JettySolrRunner survivor = otherJetties.get(0);
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner runner = cluster.getJettySolrRunner(i);
if (runner == sourceNode) {
cluster.stopJettySolrRunner(i);
}
}
cluster.waitForAllNodes(30);
waitForState("Timed out waiting for replicas of collection to be 2 again",
collectionName, clusterShape(1, 2));
clusterState = solrClient.getZkStateReader().getClusterState();
docCollection = clusterState.getCollection(collectionName);
List<Replica> replicasOnSurvivor = docCollection.getReplicas(survivor.getNodeName());
assertNotNull(replicasOnSurvivor);
assertEquals(2, replicasOnSurvivor.size());
}
}

View File

@ -0,0 +1,56 @@
/*
* 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 com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.cloud.MoveReplicaHDFSTest;
import org.apache.solr.cloud.hdfs.HdfsTestUtil;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.util.BadHdfsThreadsFilter;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@LuceneTestCase.Slow
@ThreadLeakFilters(defaultFilters = true, filters = {
BadHdfsThreadsFilter.class, // hdfs currently leaks thread(s)
MoveReplicaHDFSTest.ForkJoinThreadsFilter.class
})
public class HdfsAutoAddReplicasIntegrationTest extends AutoAddReplicasIntegrationTest {
private static MiniDFSCluster dfsCluster;
@BeforeClass
public static void setupClass() throws Exception {
System.setProperty("solr.hdfs.blockcache.enabled", "false");
dfsCluster = HdfsTestUtil.setupClass(createTempDir().toFile().getAbsolutePath());
ZkConfigManager configManager = new ZkConfigManager(zkClient());
configManager.uploadConfigDir(configset("cloud-hdfs"), "conf");
System.setProperty("solr.hdfs.home", HdfsTestUtil.getDataDir(dfsCluster, "data"));
}
@AfterClass
public static void teardownClass() throws Exception {
cluster.shutdown(); // need to close before the MiniDFSCluster
HdfsTestUtil.teardownClass(dfsCluster);
dfsCluster = null;
}
}

View File

@ -0,0 +1,209 @@
/*
* 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 javax.servlet.ServletException;
import javax.servlet.ServletInputStream;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.LogLevel;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.handler.AbstractHandler;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
/**
*
*/
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
@SolrTestCaseJ4.SuppressSSL
public class HttpTriggerListenerTest extends SolrCloudTestCase {
private static CountDownLatch triggerFiredLatch;
private MockService mockService;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(2)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Before
public void setupTest() throws Exception {
mockService = new MockService();
mockService.start();
triggerFiredLatch = new CountDownLatch(1);
}
@After
public void teardownTest() throws Exception {
if (mockService != null) {
mockService.close();
}
}
@Test
public void testHttpListenerIntegration() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '0s'," +
"'enabled' : true," +
"'actions' : [" +
"{'name':'test','class':'" + TestDummyAction.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_added_trigger'," +
"'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
"'beforeAction' : 'test'," +
"'afterAction' : ['test']," +
"'class' : '" + HttpTriggerListener.class.getName() + "'," +
"'url' : '" + mockService.server.getURI().toString() + "/${config.name:invalid}/${config.properties.xyz:invalid}/${stage}'," +
"'payload': 'actionName=${actionName}, source=${event.source}, type=${event.eventType}'," +
"'header.X-Foo' : '${config.name:invalid}'," +
"'xyz': 'foo'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
assertEquals(requests.toString(), 0, requests.size());
cluster.startJettySolrRunner();
boolean await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
assertTrue("The trigger did not fire at all", await);
Thread.sleep(5000);
assertEquals(requests.toString(), 4, requests.size());
requests.forEach(s -> assertTrue(s.contains("Content-Type: application/json")));
requests.forEach(s -> assertTrue(s.contains("X-Foo: foo")));
requests.forEach(s -> assertTrue(s.contains("source=node_added_trigger")));
requests.forEach(s -> assertTrue(s.contains("type=NODEADDED")));
String request = requests.get(0);
assertTrue(request, request.startsWith("/foo/foo/STARTED"));
assertTrue(request, request.contains("actionName=,")); // empty actionName
request = requests.get(1);
assertTrue(request, request.startsWith("/foo/foo/BEFORE_ACTION"));
assertTrue(request, request.contains("actionName=test,")); // actionName
request = requests.get(2);
assertTrue(request, request.startsWith("/foo/foo/AFTER_ACTION"));
assertTrue(request, request.contains("actionName=test,")); // actionName
request = requests.get(3);
assertTrue(request, request.startsWith("/foo/foo/SUCCEEDED"));
assertTrue(request, request.contains("actionName=,")); // empty actionName
}
public static class TestDummyAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) {
triggerFiredLatch.countDown();
}
}
static List<String> requests = new ArrayList<>();
private static class MockService extends Thread {
Server server;
public void start() {
server = new Server(new InetSocketAddress("localhost", 0));
server.setHandler(new AbstractHandler() {
@Override
public void handle(String s, Request request, HttpServletRequest httpServletRequest, HttpServletResponse httpServletResponse) throws IOException, ServletException {
StringBuilder stringBuilder = new StringBuilder();
stringBuilder.append(httpServletRequest.getRequestURI());
Enumeration<String> headerNames = httpServletRequest.getHeaderNames();
while (headerNames.hasMoreElements()) {
stringBuilder.append('\n');
String name = headerNames.nextElement();
stringBuilder.append(name);
stringBuilder.append(": ");
stringBuilder.append(httpServletRequest.getHeader(name));
}
stringBuilder.append("\n\n");
ServletInputStream is = request.getInputStream();
byte[] httpInData = new byte[request.getContentLength()];
int len = -1;
while ((len = is.read(httpInData)) != -1) {
stringBuilder.append(new String(httpInData, 0, len, "UTF-8"));
}
requests.add(stringBuilder.toString());
httpServletResponse.setStatus(HttpServletResponse.SC_OK);
request.setHandled(true);
}
});
try {
server.start();
for (int i = 0; i < 30; i++) {
Thread.sleep(1000);
if (server.isRunning()) {
break;
}
if (server.isFailed()) {
throw new Exception("MockService startup failed - the test will fail...");
}
}
} catch (Exception e) {
throw new RuntimeException("Exception starting MockService", e);
}
}
void close() throws Exception {
if (server != null) {
server.stop();
}
}
}
}

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;
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.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test for {@link NodeAddedTrigger}
*/
public class NodeAddedTriggerTest extends SolrCloudTestCase {
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 TimeSource timeSource = TimeSource.CURRENT_TIME;
// 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);
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(1)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Before
public void beforeTest() throws Exception {
actionConstructorCalled = new AtomicBoolean(false);
actionInitCalled = new AtomicBoolean(false);
actionCloseCalled = new AtomicBoolean(false);
}
@Test
public void testTrigger() throws Exception {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container, container.getZkController())) {
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
JettySolrRunner newNode1 = cluster.startJettySolrRunner();
JettySolrRunner newNode2 = cluster.startJettySolrRunner();
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();
Thread.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.getNodeName()));
assertTrue(nodeNames.contains(newNode2.getNodeName()));
}
// 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, container, container.getZkController())) {
final long waitTime = 2;
props.put("waitFor", waitTime);
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
JettySolrRunner newNode = cluster.startJettySolrRunner();
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
newNode.stop(); // stop the new jetty
int counter = 0;
do {
trigger.run();
Thread.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 {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
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", NodeAddedTriggerTest.AssertInitTriggerAction.class.getName());
actions.add(action);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container, container.getZkController())) {
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 {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
Map<String, Object> props = createTriggerProps(0);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container, container.getZkController())) {
trigger.setProcessor(noFirstRunProcessor);
trigger.run(); // starts tracking live nodes
JettySolrRunner newNode = cluster.startJettySolrRunner();
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 {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
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, container, container.getZkController());
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
JettySolrRunner newNode = cluster.startJettySolrRunner();
trigger.run(); // this run should detect the new node
trigger.close(); // close the old trigger
try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name", props, container, container.getZkController())) {
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, container, container.getZkController())) {
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();
Thread.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,348 @@
/*
* 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.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.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test for {@link NodeLostTrigger}
*/
public class NodeLostTriggerTest extends SolrCloudTestCase {
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;
};
// use the same time source as the trigger
private final TimeSource timeSource = TimeSource.CURRENT_TIME;
// 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)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Before
public void beforeTest() throws Exception {
actionConstructorCalled = new AtomicBoolean(false);
actionInitCalled = new AtomicBoolean(false);
actionCloseCalled = new AtomicBoolean(false);
}
@Test
public void testTrigger() throws Exception {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container, container.getZkController())) {
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
String lostNodeName1 = cluster.getJettySolrRunner(1).getNodeName();
cluster.stopJettySolrRunner(1);
String lostNodeName2 = cluster.getJettySolrRunner(1).getNodeName();
cluster.stopJettySolrRunner(1);
Thread.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();
Thread.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, container, container.getZkController())) {
final long waitTime = 2;
props.put("waitFor", waitTime);
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
JettySolrRunner lostNode = cluster.getJettySolrRunner(1);
lostNode.stop();
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 (container.getZkController().getZkStateReader().getClusterState().getLiveNodes().size() == 2) {
break;
}
Thread.sleep(100);
if (counter++ > 20) {
fail("Live nodes not updated!");
}
} while (true);
counter = 0;
lostNode.start();
do {
trigger.run();
Thread.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 {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
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, container, container.getZkController())) {
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 {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
Map<String, Object> props = createTriggerProps(0);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, container, container.getZkController())) {
trigger.setProcessor(noFirstRunProcessor);
JettySolrRunner newNode = cluster.startJettySolrRunner();
cluster.waitForAllNodes(5);
trigger.run(); // starts tracking live nodes
// stop the newly created node
List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
for (int i = 0; i < jettySolrRunners.size(); i++) {
JettySolrRunner jettySolrRunner = jettySolrRunners.get(i);
if (newNode == jettySolrRunner) {
cluster.stopJettySolrRunner(i);
break;
}
}
cluster.waitForAllNodes(5);
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 {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds);
JettySolrRunner newNode = cluster.startJettySolrRunner();
String lostNodeName = newNode.getNodeName();
// 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, container, container.getZkController());
trigger.setProcessor(noFirstRunProcessor);
trigger.run();
// stop the newly created node
List<JettySolrRunner> jettySolrRunners = cluster.getJettySolrRunners();
for (int i = 0; i < jettySolrRunners.size(); i++) {
JettySolrRunner jettySolrRunner = jettySolrRunners.get(i);
if (newNode == jettySolrRunner) {
cluster.stopJettySolrRunner(i);
break;
}
}
trigger.run(); // this run should detect the lost node
trigger.close(); // close the old trigger
try (NodeLostTrigger newTrigger = new NodeLostTrigger("some_different_name", props, container, container.getZkController())) {
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, container, container.getZkController())) {
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();
Thread.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(lostNodeName));
}
}
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,239 @@
/*
* 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.lang.invoke.MethodHandles;
import java.util.Collection;
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.SolrRequest;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.LogLevel;
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;
/**
* Test for {@link SystemLogListener}
*/
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG")
public class SystemLogListenerTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final AtomicBoolean fired = new AtomicBoolean(false);
private static final int NODE_COUNT = 3;
private static CountDownLatch triggerFiredLatch = new CountDownLatch(1);
private static final AtomicReference<Map> actionContextPropsRef = new AtomicReference<>();
private static final AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
public static class AssertingTriggerAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) {
if (fired.compareAndSet(false, true)) {
eventRef.set(event);
actionContextPropsRef.set(context.getProperties());
triggerFiredLatch.countDown();
}
}
}
public static class ErrorTriggerAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) {
throw new RuntimeException("failure from ErrorTriggerAction");
}
}
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(NODE_COUNT)
.addConfig("conf", configset("cloud-minimal"))
.configure();
CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 3)
.process(cluster.getSolrClient());
}
@Test
public void test() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
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'}," +
"{'name':'test','class':'" + AssertingTriggerAction.class.getName() + "'}," +
"{'name':'error','class':'" + ErrorTriggerAction.class.getName() + "'}]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// remove default listener
String removeListenerCommand = "{\n" +
"\t\"remove-listener\" : {\n" +
"\t\t\"name\" : \"node_lost_trigger.system\"\n" +
"\t}\n" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, removeListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("test",
"conf",3, 2);
create.setMaxShardsPerNode(3);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
"test", clusterShape(3, 2));
String setListenerCommand = "{" +
"'set-listener' : " +
"{" +
"'name' : 'foo'," +
"'trigger' : 'node_lost_trigger'," +
"'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
"'beforeAction' : ['compute_plan','execute_plan','test','error']," +
"'afterAction' : ['compute_plan','execute_plan','test','error']," +
"'class' : '" + SystemLogListener.class.getName() + "'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// stop non-overseer node
NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
String overseerLeader = (String) overSeerStatus.get("leader");
int nonOverseerLeaderIndex = 0;
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner jetty = cluster.getJettySolrRunner(i);
if (!jetty.getNodeName().equals(overseerLeader)) {
nonOverseerLeaderIndex = i;
}
}
cluster.stopJettySolrRunner(nonOverseerLeaderIndex);
cluster.waitForAllNodes(30);
assertTrue("Trigger was not fired ", triggerFiredLatch.await(30, TimeUnit.SECONDS));
assertTrue(fired.get());
Map context = actionContextPropsRef.get();
assertNotNull(context);
// make sure the event docs are replicated and committed
Thread.sleep(5000);
cluster.getSolrClient().commit(CollectionAdminParams.SYSTEM_COLL, true, true);
ModifiableSolrParams query = new ModifiableSolrParams();
query.add(CommonParams.Q, "type:" + SystemLogListener.DOC_TYPE);
query.add(CommonParams.SORT, "id asc");
QueryResponse resp = cluster.getSolrClient().query(CollectionAdminParams.SYSTEM_COLL, query);
SolrDocumentList docs = resp.getResults();
assertNotNull(docs);
assertEquals("wrong number of events added to .system", 9, docs.size());
docs.forEach(doc -> assertCommonFields(doc));
// STARTED
SolrDocument doc = docs.get(0);
assertEquals("STARTED", doc.getFieldValue("stage_s"));
// BEFORE_ACTION compute_plan
doc = docs.get(1);
assertEquals("BEFORE_ACTION", doc.getFieldValue("stage_s"));
assertEquals("compute_plan", doc.getFieldValue("action_s"));
// AFTER_ACTION compute_plan
doc = docs.get(2);
assertEquals("AFTER_ACTION", doc.getFieldValue("stage_s"));
assertEquals("compute_plan", doc.getFieldValue("action_s"));
Collection<Object> vals = doc.getFieldValues("operations.params_ts");
assertEquals(3, vals.size());
for (Object val : vals) {
assertTrue(val.toString(), String.valueOf(val).contains("action=MOVEREPLICA"));
}
// BEFORE_ACTION execute_plan
doc = docs.get(3);
assertEquals("BEFORE_ACTION", doc.getFieldValue("stage_s"));
assertEquals("execute_plan", doc.getFieldValue("action_s"));
vals = doc.getFieldValues("operations.params_ts");
assertEquals(3, vals.size());
// AFTER_ACTION execute_plan
doc = docs.get(4);
assertEquals("AFTER_ACTION", doc.getFieldValue("stage_s"));
assertEquals("execute_plan", doc.getFieldValue("action_s"));
vals = doc.getFieldValues("operations.params_ts");
assertNotNull(vals);
assertEquals(3, vals.size());
vals = doc.getFieldValues("responses_ts");
assertNotNull(vals);
assertEquals(3, vals.size());
vals.forEach(s -> assertTrue(s.toString(), s.toString().startsWith("success MOVEREPLICA action completed successfully")));
// BEFORE_ACTION test
doc = docs.get(5);
assertEquals("BEFORE_ACTION", doc.getFieldValue("stage_s"));
assertEquals("test", doc.getFieldValue("action_s"));
// AFTER_ACTION test
doc = docs.get(6);
assertEquals("AFTER_ACTION", doc.getFieldValue("stage_s"));
assertEquals("test", doc.getFieldValue("action_s"));
// BEFORE_ACTION error
doc = docs.get(7);
assertEquals("BEFORE_ACTION", doc.getFieldValue("stage_s"));
assertEquals("error", doc.getFieldValue("action_s"));
// FAILED error
doc = docs.get(8);
assertEquals("FAILED", doc.getFieldValue("stage_s"));
assertEquals("error", doc.getFieldValue("action_s"));
assertEquals("failure from ErrorTriggerAction", doc.getFieldValue("error.message_t"));
assertTrue(doc.getFieldValue("error.details_t").toString().contains("RuntimeException"));
}
private void assertCommonFields(SolrDocument doc) {
assertEquals(SystemLogListener.class.getSimpleName(), doc.getFieldValue(SystemLogListener.SOURCE_FIELD));
assertEquals(SystemLogListener.DOC_TYPE, doc.getFieldValue(CommonParams.TYPE));
assertEquals("node_lost_trigger", doc.getFieldValue("event.source_s"));
assertNotNull(doc.getFieldValue("event.time_l"));
assertNotNull(doc.getFieldValue("timestamp"));
assertNotNull(doc.getFieldValue("event.property.nodeNames_ss"));
assertNotNull(doc.getFieldValue("event_str"));
assertEquals("NODELOST", doc.getFieldValue("event.type_s"));
}
}

View File

@ -22,12 +22,15 @@ import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.function.BiConsumer;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.OverseerTaskProcessor;
@ -44,6 +47,9 @@ import org.junit.rules.ExpectedException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.AutoScalingHandlerTest.createAutoScalingRequest;
import static org.apache.solr.common.util.Utils.getObjectByPath;
@LuceneTestCase.Slow
public class TestPolicyCloud extends SolrCloudTestCase {
@ -69,7 +75,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
JettySolrRunner jetty = cluster.getRandomJetty(random());
int port = jetty.getLocalPort();
String commands = "{set-policy :{c1 : [{replica:2 , shard:'#EACH', port: '" + port + "'}]}}";
String commands = "{set-policy :{c1 : [{replica:0 , shard:'#EACH', port: '!" + port + "'}]}}";
cluster.getSolrClient().request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
String collectionName = "testCreateCollectionAddReplica";
@ -130,6 +136,111 @@ public class TestPolicyCloud extends SolrCloudTestCase {
assertEquals("Expected exactly three replica of collection on node with port: " + secondNodePort, 3, replicasOnNode2);
}
public void testMetricsTag() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
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 (HttpSolrClient.RemoteExecutionException e) {
// expected
assertTrue(String.valueOf(getObjectByPath(e.getMetaData(),
false, "error/details[0]/errorMessages[0]")).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(cluster.getSolrClient());
DocCollection collection = getCollectionState("metricsTest");
SolrClientDataProvider provider = new SolrClientDataProvider(solrClient);
List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
"metrics:solr.jvm:buffers.direct.Count");
Map<String, Object> val = provider.getNodeValues(collection .getReplicas().get(0).getNodeName(), tags);
for (String tag : tags) {
assertNotNull( "missing : "+ tag , val.get(tag));
}
}
public void testCreateCollectionAddShardWithReplicaTypeUsingPolicy() throws Exception {
JettySolrRunner jetty = cluster.getJettySolrRunners().get(0);
String nrtNodeName = jetty.getNodeName();
int nrtPort = jetty.getLocalPort();
jetty = cluster.getJettySolrRunners().get(1);
String pullNodeName = jetty.getNodeName();
int pullPort = jetty.getLocalPort();
jetty = cluster.getJettySolrRunners().get(2);
String tlogNodeName = jetty.getNodeName();
int tlogPort = jetty.getLocalPort();
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 + "'}" +
"]}";
cluster.getSolrClient().request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
Map<String, Object> json = Utils.getJson(cluster.getZkClient(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, true);
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(cluster.getSolrClient());
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(cluster.getSolrClient());
coll = getCollectionState("policiesTest");
assertEquals(3, coll.getSlice("s3").getReplicas().size());
coll.forEachReplica(verifyReplicas);
}
public void testCreateCollectionAddShardUsingPolicy() throws Exception {
JettySolrRunner jetty = cluster.getRandomJetty(random());
int port = jetty.getLocalPort();

File diff suppressed because it is too large Load Diff

View File

@ -31,6 +31,7 @@ import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.handler.TestBlobHandler;
import org.junit.BeforeClass;
import org.junit.Test;
@ -46,7 +47,7 @@ public class BlobRepositoryCloudTest extends SolrCloudTestCase {
.configure();
// Thread.sleep(2000);
HashMap<String, String> params = new HashMap<>();
CollectionAdminRequest.createCollection(".system", null, 1, 1)
CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 1)
.process(cluster.getSolrClient());
// test component will fail if it cant' find a blob with this data by this name
TestBlobHandler.postData(cluster.getSolrClient(), findLiveNodeURI(), "testResource", ByteBuffer.wrap("foo,bar\nbaz,bam".getBytes(StandardCharsets.UTF_8)));

View File

@ -0,0 +1,294 @@
package org.apache.solr.handler.admin;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.cloud.autoscaling.ActionContext;
import org.apache.solr.cloud.autoscaling.SystemLogListener;
import org.apache.solr.cloud.autoscaling.TriggerActionBase;
import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
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.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
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;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;")
public class AutoscalingHistoryHandlerTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static CountDownLatch actionFiredLatch;
private static CloudSolrClient solrClient;
private static String PREFIX = AutoscalingHistoryHandlerTest.class.getSimpleName();
private static CountDownLatch getActionFiredLatch() {
return actionFiredLatch;
}
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(3)
.addConfig("conf", configset("cloud-minimal"))
.configure();
solrClient = cluster.getSolrClient();
CollectionAdminRequest.createCollection(PREFIX + "_collection", null, 1, 3)
.process(solrClient);
CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 3)
.process(solrClient);
}
public static class TesterAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) {
getActionFiredLatch().countDown();
}
}
@Before
public void setupTest() throws Exception {
actionFiredLatch = new CountDownLatch(1);
// first trigger
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : '" + PREFIX + "_node_added_trigger'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '0s'," +
"'enabled' : true," +
"'actions' : [" +
"{'name':'compute_plan','class':'solr.ComputePlanAction'}," +
"{'name':'execute_plan','class':'solr.ExecutePlanAction'}," +
"{'name':'test','class':'" + TesterAction.class.getName() + "'}" +
"]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// second trigger
setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : '" + PREFIX + "_node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '0s'," +
"'enabled' : true," +
"'actions' : [" +
"{'name':'compute_plan','class':'solr.ComputePlanAction'}," +
"{'name':'execute_plan','class':'solr.ExecutePlanAction'}," +
"{'name':'test','class':'" + TesterAction.class.getName() + "'}" +
"]" +
"}}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setTriggerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// remove default listeners
String removeListenerCommand = "{\n" +
"\t\"remove-listener\" : {\n" +
"\t\t\"name\" : \"" + PREFIX + "_node_lost_trigger.system\"\n" +
"\t}\n" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, removeListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
removeListenerCommand = "{\n" +
"\t\"remove-listener\" : {\n" +
"\t\t\"name\" : \"" + PREFIX + "_node_added_trigger.system\"\n" +
"\t}\n" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, removeListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
// set up our own listeners
String setListenerCommand = "{" +
"'set-listener' : " +
"{" +
"'name' : 'node_added'," +
"'trigger' : '" + PREFIX + "_node_added_trigger'," +
"'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
"'beforeAction' : ['compute_plan','execute_plan','test']," +
"'afterAction' : ['compute_plan','execute_plan','test']," +
"'class' : '" + SystemLogListener.class.getName() + "'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
setListenerCommand = "{" +
"'set-listener' : " +
"{" +
"'name' : 'node_lost'," +
"'trigger' : '" + PREFIX + "_node_lost_trigger'," +
"'stage' : ['STARTED','ABORTED','SUCCEEDED', 'FAILED']," +
"'beforeAction' : ['compute_plan','execute_plan','test']," +
"'afterAction' : ['compute_plan','execute_plan','test']," +
"'class' : '" + SystemLogListener.class.getName() + "'" +
"}" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
}
private void resetLatch() {
actionFiredLatch = new CountDownLatch(1);
}
@Test
public void testHistory() throws Exception {
waitForState("Timed out wait for collection be active", PREFIX + "_collection",
clusterShape(1, 3));
waitForState("Timed out wait for collection be active", CollectionAdminParams.SYSTEM_COLL,
clusterShape(1, 3));
// todo remove this workaround after SOLR-9440
cluster.getSolrClient().getZkStateReader().registerCore(".system");
cluster.getSolrClient().getZkStateReader().registerCore(PREFIX + "_collection");
JettySolrRunner jetty = cluster.startJettySolrRunner();
String nodeAddedName = jetty.getNodeName();
boolean await = actionFiredLatch.await(60, TimeUnit.SECONDS);
assertTrue("action did not execute", await);
// commit on the history collection
Thread.sleep(2000);
solrClient.commit(CollectionAdminParams.SYSTEM_COLL);
Thread.sleep(2000);
// verify that new docs exist
ModifiableSolrParams query = params(CommonParams.Q, "type:" + SystemLogListener.DOC_TYPE,
CommonParams.FQ, "event.source_s:" + PREFIX + "_node_added_trigger");
QueryResponse resp = solrClient.query(CollectionAdminParams.SYSTEM_COLL, query);
SolrDocumentList docs = resp.getResults();
assertNotNull(docs);
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.TRIGGER_PARAM, PREFIX + "_node_added_trigger");
QueryResponse rsp = solrClient.query(query);
docs = rsp.getResults();
if (docs.size() != 8) {
log.info("Cluster state: " + solrClient.getZkStateReader().getClusterState());
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH);
log.info("Wrong response: ", rsp);
log.info("Full response: " + solrClient.query(query));
}
assertEquals(8, docs.size());
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.STAGE_PARAM, "STARTED");
docs = solrClient.query(query).getResults();
assertEquals(1, docs.size());
assertEquals("NODEADDED", docs.get(0).getFieldValue("event.type_s"));
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.NODE_PARAM, nodeAddedName);
docs = solrClient.query(query).getResults();
assertEquals(8, docs.size());
for (SolrDocument doc : docs) {
assertTrue(doc.getFieldValues("event.property.nodeNames_ss").contains(nodeAddedName));
}
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.ACTION_PARAM, "test");
docs = solrClient.query(query).getResults();
assertEquals(2, docs.size());
assertEquals("BEFORE_ACTION", docs.get(0).getFieldValue("stage_s"));
assertEquals("AFTER_ACTION", docs.get(1).getFieldValue("stage_s"));
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.ACTION_PARAM, "test");
docs = solrClient.query(query).getResults();
assertEquals(2, docs.size());
assertEquals("BEFORE_ACTION", docs.get(0).getFieldValue("stage_s"));
assertEquals("AFTER_ACTION", docs.get(1).getFieldValue("stage_s"));
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.COLLECTION_PARAM, CollectionAdminParams.SYSTEM_COLL);
docs = solrClient.query(query).getResults();
assertEquals(5, docs.size());
assertEquals("AFTER_ACTION", docs.get(0).getFieldValue("stage_s"));
assertEquals("compute_plan", docs.get(0).getFieldValue("action_s"));
// reset latch
resetLatch();
// kill a node
cluster.stopJettySolrRunner(0);
await = actionFiredLatch.await(60, TimeUnit.SECONDS);
// wait for recovery
waitForRecovery(PREFIX + "_collection");
Thread.sleep(5000);
// commit on the history collection
solrClient.commit(CollectionAdminParams.SYSTEM_COLL);
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.TRIGGER_PARAM, PREFIX + "_node_lost_trigger");
docs = solrClient.query(query).getResults();
assertEquals(8, docs.size());
query = params(CommonParams.QT, CommonParams.AUTOSCALING_HISTORY_PATH,
AutoscalingHistoryHandler.TRIGGER_PARAM, PREFIX + "_node_lost_trigger",
AutoscalingHistoryHandler.COLLECTION_PARAM, PREFIX + "_collection");
docs = solrClient.query(query).getResults();
assertEquals(5, docs.size());
}
private void waitForRecovery(String collection) throws Exception {
boolean recovered = false;
for (int i = 0; i < 300; i++) {
ClusterState state = solrClient.getZkStateReader().getClusterState();
DocCollection collState = getCollectionState(collection);
log.debug("###### " + collState);
Collection<Replica> replicas = collState.getReplicas();
boolean allActive = true;
boolean hasLeaders = true;
if (replicas != null && !replicas.isEmpty()) {
for (Replica r : replicas) {
if (!r.isActive(state.getLiveNodes())) {
log.info("Not active: " + r);
allActive = false;
}
}
} else {
allActive = false;
}
for (Slice slice : collState.getSlices()) {
if (slice.getLeader() == null) {
hasLeaders = false;
}
}
if (allActive && hasLeaders) {
recovered = true;
break;
} else {
log.info("--- waiting, allActive=" + allActive + ", hasLeaders=" + hasLeaders);
Thread.sleep(1000);
}
}
assertTrue("replica never fully recovered", recovered);
}
}

View File

@ -277,15 +277,6 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 {
val = values.findRecursive("metrics", key3);
assertNotNull(val);
String key4 = "solr.core.collection1:QUERY./select.requestTimes:1minRate";
resp = new SolrQueryResponse();
handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json",
MetricsHandler.KEY_PARAM, key4), resp);
values = resp.getValues();
val = values.findRecursive("metrics", key4);
assertNotNull(val);
assertTrue(val instanceof Number);
// test errors
// invalid keys

View File

@ -175,6 +175,7 @@ The attributes that can be modified are:
* collection.configName
* rule
* snitch
* policy
+
See the <<create,CREATE action>> section above for details on these attributes.

View File

@ -1,202 +1,202 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.

View File

@ -324,3 +324,31 @@ The `remove-policy` command accepts a policy name to be removed from Solr. The p
----
If you attempt to remove a policy that is being used by a collection, this command will fail to delete the policy until the collection itself is deleted.
== History API
History of autoscaling events is available at `/admin/autoscaling/history`. It returs information
about past autoscaling events and details about their processing. This history is kept in
the `.system` collection, and populated by a trigger listener `SystemLogListener` - by default this
listener is added to all new triggers.
History events are regular Solr documents so they can be also accessed directly by
searching on the `.system` collection. History handler acts as a regular search handler, so all
query parameters supported by `/select` handler for that collection are supported here too.
However, the history handler makes this
process easier by offering a simpler syntax and knowledge of field names
used by `SystemLogListener` for serialization of event data.
History documents contain also the action context, if it was available, which gives
further insight into eg. exact operations that were computed and/or executed.
Specifically, the following query parameters can be used (they are turned into
filter queries, so an implicit AND is applied):
* `trigger` - trigger name
* `eventType` - event type / trigger type (eg. `nodeAdded`)
* `collection` - collection name involved in event processing
* `stage` - event processing stage
* `action` - trigger action
* `node` - node name that the event refers to
* `beforeAction` - beforeAction stage
* `afterAction` - afterAction stage

View File

@ -17,11 +17,15 @@
package org.apache.solr.client.solrj.cloud.autoscaling;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy.Suggester;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.util.Pair;
class AddReplicaSuggester extends Suggester {
@ -32,34 +36,37 @@ class AddReplicaSuggester extends Suggester {
}
SolrRequest tryEachNode(boolean strict) {
String coll = (String) hints.get(Hint.COLL);
String shard = (String) hints.get(Hint.SHARD);
if (coll == null || shard == null)
Set<Pair<String, String>> shards = (Set<Pair<String, String>>) hints.getOrDefault(Hint.COLL_SHARD, Collections.emptySet());
if (shards.isEmpty()) {
throw new RuntimeException("add-replica requires 'collection' and 'shard'");
//iterate through elements and identify the least loaded
}
for (Pair<String,String> shard : shards) {
Replica.Type type = Replica.Type.get((String) hints.get(Hint.REPLICATYPE));
//iterate through elements and identify the least loaded
List<Clause.Violation> leastSeriousViolation = null;
Integer targetNodeIndex = null;
for (int i = getMatrix().size() - 1; i >= 0; i--) {
Row row = getMatrix().get(i);
if (!row.isLive) continue;
if (!isAllowed(row.node, Hint.TARGET_NODE)) continue;
Row tmpRow = row.addReplica(shard.first(), shard.second(), type);
List<Clause.Violation> leastSeriousViolation = null;
Integer targetNodeIndex = null;
for (int i = getMatrix().size() - 1; i >= 0; i--) {
Row row = getMatrix().get(i);
if (!isAllowed(row.node, Hint.TARGET_NODE)) continue;
Row tmpRow = row.addReplica(coll, shard);
tmpRow.violations.clear();
List<Clause.Violation> errs = testChangedMatrix(strict, getModifiedMatrix(getMatrix(), tmpRow, i));
if(!containsNewErrors(errs)) {
if(isLessSerious(errs, leastSeriousViolation)){
leastSeriousViolation = errs;
targetNodeIndex = i;
List<Clause.Violation> errs = testChangedMatrix(strict, getModifiedMatrix(getMatrix(), tmpRow, i));
if (!containsNewErrors(errs)) {
if (isLessSerious(errs, leastSeriousViolation)) {
leastSeriousViolation = errs;
targetNodeIndex = i;
}
}
}
}
if (targetNodeIndex != null) {// there are no rule violations
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(coll, shard));
return CollectionAdminRequest
.addReplicaToShard(coll, shard)
.setNode(getMatrix().get(targetNodeIndex).node);
if (targetNodeIndex != null) {// there are no rule violations
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(shard.first(), shard.second(), type));
return CollectionAdminRequest
.addReplicaToShard(shard.first(), shard.second())
.setType(type)
.setNode(getMatrix().get(targetNodeIndex).node);
}
}
return null;

View File

@ -0,0 +1,531 @@
/*
* 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.client.solrj.cloud.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.stream.Collectors.collectingAndThen;
import static java.util.stream.Collectors.toList;
/**
* Bean representation of <code>autoscaling.json</code>, which parses data
* lazily.
*/
public class AutoScalingConfig implements MapWriter {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final Map<String, Object> jsonMap;
private final boolean empty;
private Policy policy;
private Map<String, TriggerConfig> triggers;
private Map<String, TriggerListenerConfig> listeners;
private final int zkVersion;
/**
* Bean representation of trigger listener config.
*/
public static class TriggerListenerConfig implements MapWriter {
public final String name;
public final String trigger;
public final EnumSet<TriggerEventProcessorStage> stages = EnumSet.noneOf(TriggerEventProcessorStage.class);
public final String listenerClass;
public final Set<String> beforeActions;
public final Set<String> afterActions;
public final Map<String, Object> properties;
public TriggerListenerConfig(String name, Map<String, Object> properties) {
this.name = name;
if (properties == null) {
this.properties = Collections.emptyMap();
} else {
this.properties = Collections.unmodifiableMap(new HashMap<>(properties));
}
trigger = (String)this.properties.get(AutoScalingParams.TRIGGER);
List<Object> stageNames = getList(AutoScalingParams.STAGE, this.properties);
for (Object stageName : stageNames) {
try {
TriggerEventProcessorStage stage = TriggerEventProcessorStage.valueOf(String.valueOf(stageName).toUpperCase(Locale.ROOT));
stages.add(stage);
} catch (Exception e) {
LOG.warn("Invalid stage name '" + name + "' in listener config, skipping: " + properties);
}
}
listenerClass = (String)this.properties.get(AutoScalingParams.CLASS);
Set<String> bActions = new HashSet<>();
getList(AutoScalingParams.BEFORE_ACTION, this.properties).forEach(o -> bActions.add(String.valueOf(o)));
beforeActions = Collections.unmodifiableSet(bActions);
Set<String> aActions = new HashSet<>();
getList(AutoScalingParams.AFTER_ACTION, this.properties).forEach(o -> aActions.add(String.valueOf(o)));
afterActions = Collections.unmodifiableSet(aActions);
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
// don't write duplicate entries - skip explicit fields if their values
// are already contained in properties
// if (!properties.containsKey(AutoScalingParams.NAME)) {
// ew.put(AutoScalingParams.NAME, name);
// }
if (!properties.containsKey(AutoScalingParams.CLASS)) {
ew.put(AutoScalingParams.CLASS, listenerClass);
}
if (!properties.containsKey(AutoScalingParams.TRIGGER)) {
ew.put(AutoScalingParams.TRIGGER, trigger);
}
if (!properties.containsKey(AutoScalingParams.STAGE)) {
ew.put(AutoScalingParams.STAGE, stages);
}
if (!properties.containsKey(AutoScalingParams.BEFORE_ACTION)) {
ew.put(AutoScalingParams.BEFORE_ACTION, beforeActions);
}
if (!properties.containsKey(AutoScalingParams.AFTER_ACTION)) {
ew.put(AutoScalingParams.AFTER_ACTION, afterActions);
}
// forEach doesn't allow throwing exceptions...
for (Map.Entry<String, Object> entry : properties.entrySet()) {
ew.put(entry.getKey(), entry.getValue());
}
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TriggerListenerConfig that = (TriggerListenerConfig) o;
if (name != null ? !name.equals(that.name) : that.name != null) return false;
if (trigger != null ? !trigger.equals(that.trigger) : that.trigger != null) return false;
if (!stages.equals(that.stages)) return false;
if (listenerClass != null ? !listenerClass.equals(that.listenerClass) : that.listenerClass != null) return false;
if (!beforeActions.equals(that.beforeActions)) return false;
if (!afterActions.equals(that.afterActions)) return false;
return properties.equals(that.properties);
}
}
/**
* Bean representation of trigger config.
*/
public static class TriggerConfig implements MapWriter {
/** Trigger name. */
public final String name;
/** Trigger event type. */
public final TriggerEventType event;
/** Enabled flag. */
public final boolean enabled;
/** List of configured actions, never null. */
public final List<ActionConfig> actions;
/** Map of additional trigger properties, never null. */
public final Map<String, Object> properties;
public TriggerConfig(String name, Map<String, Object> properties) {
this.name = name;
if (properties != null) {
this.properties = Collections.unmodifiableMap(new HashMap<>(properties));
} else {
this.properties = Collections.emptyMap();
}
String event = (String) this.properties.get(AutoScalingParams.EVENT);
if (event != null) {
TriggerEventType type = null;
try {
type = TriggerEventType.valueOf(event.toUpperCase(Locale.ROOT));
} catch (Exception e) {
}
if (type == null) {
this.event = TriggerEventType.INVALID;
} else {
this.event = type;
}
} else {
this.event = TriggerEventType.INVALID;
}
enabled = Boolean.parseBoolean(String.valueOf(this.properties.getOrDefault("enabled", "true")));
List<Map<String, Object>> newActions = (List<Map<String, Object>>)this.properties.get("actions");
if (newActions != null) {
this.actions = newActions.stream().map(ActionConfig::new).collect(collectingAndThen(toList(), Collections::unmodifiableList));
} else {
this.actions = Collections.emptyList();
}
}
/**
* Create a copy of this config with specified enabled flag.
* @param enabled true when enabled, false otherwise.
* @return modified copy of the configuration
*/
public TriggerConfig withEnabled(boolean enabled) {
Map<String, Object> props = new HashMap<>(properties);
props.put(AutoScalingParams.ENABLED, String.valueOf(enabled));
return new TriggerConfig(name, props);
}
/**
* Create a copy of this config with specified property.
* @param key property name
* @param value property value
* @return modified copy of the configuration
*/
public TriggerConfig withProperty(String key, Object value) {
Map<String, Object> props = new HashMap<>(properties);
props.put(key, String.valueOf(value));
return new TriggerConfig(name, props);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TriggerConfig that = (TriggerConfig) o;
if (name != null ? !name.equals(that.name) : that.name != null) return false;
if (event != that.event) return false;
return properties.equals(that.properties);
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
// if (!properties.containsKey(AutoScalingParams.NAME)) {
// ew.put(AutoScalingParams.NAME, name);
// }
if (!properties.containsKey(AutoScalingParams.EVENT)) {
ew.put(AutoScalingParams.EVENT, event.toString());
}
// forEach doesn't allow throwing exceptions...
for (Map.Entry<String, Object> entry : properties.entrySet()) {
ew.put(entry.getKey(), entry.getValue());
}
}
}
/**
* Bean representation of trigger action configuration.
*/
public static class ActionConfig implements MapWriter {
/** Action name. */
public final String name;
/** Class name of action implementtion. */
public final String actionClass;
/** Additional action properties. */
public final Map<String, Object> properties;
/**
* Construct from a JSON map.
* @param properties JSON map with properties - selected properties will be
* used for setting the values of <code>name</code> and
* <code>actionClass</code>.
*/
public ActionConfig(Map<String, Object> properties) {
if (properties != null) {
this.properties = Collections.unmodifiableMap(new HashMap<>(properties));
} else {
this.properties = Collections.emptyMap();
}
this.name = (String)this.properties.get(AutoScalingParams.NAME);
this.actionClass = (String)this.properties.get(AutoScalingParams.CLASS);
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
// forEach doesn't allow throwing exceptions...
for (Map.Entry<String, Object> entry : properties.entrySet()) {
ew.put(entry.getKey(), entry.getValue());
}
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ActionConfig that = (ActionConfig) o;
return properties.equals(that.properties);
}
}
/**
* Construct from bytes that represent a UTF-8 JSON string.
* @param utf8 config data
*/
public AutoScalingConfig(byte[] utf8) {
this(utf8 != null && utf8.length > 0 ? (Map<String, Object>)Utils.fromJSON(utf8) : Collections.emptyMap());
}
/**
* Construct from a JSON map representation.
* @param jsonMap JSON map representation of the config. Note that this map is evaluated lazily, and
* outside modifications may cause unpredictable behavior.
*/
public AutoScalingConfig(Map<String, Object> jsonMap) {
this.jsonMap = jsonMap;
int version = -1;
if (jsonMap.containsKey(AutoScalingParams.ZK_VERSION)) {
try {
version = (Integer)jsonMap.get(AutoScalingParams.ZK_VERSION);
} catch (Exception e) {
// ignore
}
}
zkVersion = version;
jsonMap.remove(AutoScalingParams.ZK_VERSION);
empty = jsonMap.isEmpty();
}
private AutoScalingConfig(Policy policy, Map<String, TriggerConfig> triggerConfigs, Map<String,
TriggerListenerConfig> listenerConfigs, int zkVersion) {
this.policy = policy;
this.triggers = triggerConfigs != null ? Collections.unmodifiableMap(triggerConfigs) : null;
this.listeners = listenerConfigs != null ? Collections.unmodifiableMap(listenerConfigs) : null;
this.jsonMap = null;
this.zkVersion = zkVersion;
this.empty = policy == null &&
(triggerConfigs == null || triggerConfigs.isEmpty()) &&
(listenerConfigs == null || listenerConfigs.isEmpty());
}
/**
* Return true if the source <code>autoscaling.json</code> was empty, false otherwise.
*/
public boolean isEmpty() {
return empty;
}
/**
* Get {@link Policy} configuration.
*/
public Policy getPolicy() {
if (policy == null) {
if (jsonMap != null) {
policy = new Policy(jsonMap);
} else {
policy = new Policy();
}
}
return policy;
}
/**
* Get trigger configurations.
*/
public Map<String, TriggerConfig> getTriggerConfigs() {
if (triggers == null) {
if (jsonMap != null) {
Map<String, Object> trigMap = (Map<String, Object>)jsonMap.get("triggers");
if (trigMap == null) {
triggers = Collections.emptyMap();
} else {
HashMap<String, TriggerConfig> newTriggers = new HashMap<>(trigMap.size());
for (Map.Entry<String, Object> entry : trigMap.entrySet()) {
newTriggers.put(entry.getKey(), new TriggerConfig(entry.getKey(), (Map<String, Object>)entry.getValue()));
}
triggers = Collections.unmodifiableMap(newTriggers);
}
} else {
triggers = Collections.emptyMap();
}
}
return triggers;
}
/**
* Check whether triggers for specific event type exist.
* @param types list of event types
* @return true if there's at least one trigger matching at least one event type,
* false otherwise,
*/
public boolean hasTriggerForEvents(TriggerEventType... types) {
if (types == null || types.length == 0) {
return false;
}
for (TriggerConfig config : getTriggerConfigs().values()) {
for (TriggerEventType type : types) {
if (config.event.equals(type)) {
return true;
}
}
}
return false;
}
/**
* Get listener configurations.
*/
public Map<String, TriggerListenerConfig> getTriggerListenerConfigs() {
if (listeners == null) {
if (jsonMap != null) {
Map<String, Object> map = (Map<String, Object>)jsonMap.get("listeners");
if (map == null) {
listeners = Collections.emptyMap();
} else {
HashMap<String, TriggerListenerConfig> newListeners = new HashMap<>(map.size());
for (Map.Entry<String, Object> entry : map.entrySet()) {
newListeners.put(entry.getKey(), new TriggerListenerConfig(entry.getKey(), (Map<String, Object>)entry.getValue()));
}
this.listeners = Collections.unmodifiableMap(newListeners);
}
} else {
listeners = Collections.emptyMap();
}
}
return listeners;
}
/**
* Create a copy of the config with replaced policy.
* @param policy new policy
* @return modified copy of the configuration
*/
public AutoScalingConfig withPolicy(Policy policy) {
return new AutoScalingConfig(policy, getTriggerConfigs(), getTriggerListenerConfigs(), zkVersion);
}
/**
* Create a copy of the config with replaced trigger configurations.
* @param configs new trigger configurations
* @return modified copy of the configuration
*/
public AutoScalingConfig withTriggerConfigs(Map<String, TriggerConfig> configs) {
return new AutoScalingConfig(getPolicy(), configs, getTriggerListenerConfigs(), zkVersion);
}
/**
* Create a copy of the config with replaced trigger configuration
* @param config new trigger configuration
* @return modified copy of the configuration
*/
public AutoScalingConfig withTriggerConfig(TriggerConfig config) {
Map<String, TriggerConfig> configs = new HashMap<>(getTriggerConfigs());
configs.put(config.name, config);
return withTriggerConfigs(configs);
}
/**
* Create a copy of the config without a trigger configuration.
* @param name trigger configuration name
* @return modified copy of the configuration, even if the specified config name didn't exist.
*/
public AutoScalingConfig withoutTriggerConfig(String name) {
Map<String, TriggerConfig> configs = new HashMap<>(getTriggerConfigs());
configs.remove(name);
return withTriggerConfigs(configs);
}
/**
* Create a copy of the config with replaced trigger listener configurations.
* @param configs new trigger listener configurations
* @return modified copy of the configuration
*/
public AutoScalingConfig withTriggerListenerConfigs(Map<String, TriggerListenerConfig> configs) {
return new AutoScalingConfig(getPolicy(), getTriggerConfigs(), configs, zkVersion);
}
/**
* Create a copy of the config with replaced trigger listener configuration.
* @param config new trigger listener configuration
* @return modified copy of the configuration
*/
public AutoScalingConfig withTriggerListenerConfig(TriggerListenerConfig config) {
Map<String, TriggerListenerConfig> configs = new HashMap<>(getTriggerListenerConfigs());
configs.put(config.name, config);
return withTriggerListenerConfigs(configs);
}
/**
* Create a copy of the config without a trigger listener configuration.
* @param name trigger listener configuration name
* @return modified copy of the configuration, even if the specified config name didn't exist.
*/
public AutoScalingConfig withoutTriggerListenerConfig(String name) {
Map<String, TriggerListenerConfig> configs = new HashMap<>(getTriggerListenerConfigs());
configs.remove(name);
return withTriggerListenerConfigs(configs);
}
/**
* Return the znode version that was used to create this configuration.
*/
public int getZkVersion() {
return zkVersion;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
Policy policy = getPolicy();
// properties of Policy are expected at top level
policy.writeMap(ew);
ew.put("triggers", getTriggerConfigs());
ew.put("listeners", getTriggerListenerConfigs());
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
AutoScalingConfig that = (AutoScalingConfig) o;
if (!getPolicy().equals(that.getPolicy())) return false;
if (!triggers.equals(that.triggers)) return false;
return listeners.equals(that.listeners);
}
private static List<Object> getList(String key, Map<String, Object> properties) {
return getList(key, properties, null);
}
private static List<Object> getList(String key, Map<String, Object> properties, List<Object> defaultList) {
if (defaultList == null) {
defaultList = Collections.emptyList();
}
Object o = properties.get(key);
if (o == null) {
return defaultList;
}
if (o instanceof List) {
return (List)o;
} else if (o instanceof Collection) {
return new ArrayList<>((Collection) o);
} else {
return Collections.singletonList(String.valueOf(o));
}
}
}

View File

@ -20,18 +20,20 @@ package org.apache.solr.client.solrj.cloud.autoscaling;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy.ReplicaInfo;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
@ -50,13 +52,16 @@ import static org.apache.solr.common.params.CoreAdminParams.SHARD;
// a set of conditions in a policy
public class Clause implements MapWriter, Comparable<Clause> {
public Map<String, Object> original;
public Condition collection, shard, replica, tag, globalTag;
final Map<String, Object> original;
Condition collection, shard, replica, tag, globalTag;
final Replica.Type type;
boolean strict = true;
public Clause(Map<String, Object> m) {
this.original = m;
this.original = Utils.getDeepCopy(m, 10);
String type = (String) m.get("type");
this.type = type == null || ANY.equals(type) ? null : Replica.Type.valueOf(type.toUpperCase(Locale.ROOT));
strict = Boolean.parseBoolean(String.valueOf(m.getOrDefault("strict", "true")));
Optional<String> globalTagName = m.keySet().stream().filter(Policy.GLOBAL_ONLY_TAGS::contains).findFirst();
if (globalTagName.isPresent()) {
@ -70,7 +75,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
} else {
collection = parse(COLLECTION, m);
shard = parse(SHARD, m);
if(m.get(REPLICA) == null){
if (m.get(REPLICA) == null) {
throw new RuntimeException(StrUtils.formatString("'replica' is required in {0}", Utils.toJSONString(m)));
}
this.replica = parse(REPLICA, m);
@ -79,6 +84,12 @@ public class Clause implements MapWriter, Comparable<Clause> {
}
if (tag == null)
throw new RuntimeException("Invalid op, must have one and only one tag other than collection, shard,replica " + Utils.toJSONString(m));
if (tag.name.startsWith(Clause.METRICS_PREFIX)) {
List<String> ss = StrUtils.splitSmart(tag.name, ':');
if (ss.size() < 3 || ss.size() > 4) {
throw new RuntimeException("Invalid metrics: param in " + Utils.toJSONString(m) + " must have at 2 or 3 segments after 'metrics:' separated by ':'");
}
}
}
@ -100,32 +111,44 @@ public class Clause implements MapWriter, Comparable<Clause> {
tag = parse(s, singletonMap(s, o));
}
private int compareTypes(Replica.Type t1, Replica.Type t2) {
if (t1 == null && t2 == null) return 0;
if (t1 != null && t2 == null) return -1;
if (t1 == null) return 1;
return 0;
}
@Override
public int compareTo(Clause that) {
try {
int v = Integer.compare(this.tag.op.priority, that.tag.op.priority);
if (v != 0) return v;
if (this.isPerCollectiontag() && that.isPerCollectiontag()) {
v = Integer.compare(this.replica.op.priority, that.replica.op.priority);
if (v == 0) {
v = Long.compare((Long) this.replica.val, (Long) that.replica.val);
v = this.replica.op == LESS_THAN ? v : v * -1;
}
return v;
} else {
return 0;
int v = Integer.compare(this.tag.op.priority, that.tag.op.priority);
if (v != 0) return v;
if (this.isPerCollectiontag() && that.isPerCollectiontag()) {
v = Integer.compare(this.replica.op.priority, that.replica.op.priority);
if (v == 0) {// higher the number of replicas , harder to satisfy
v = Long.compare((Long) this.replica.val, (Long) that.replica.val);
v = this.replica.op == LESS_THAN ? v : v * -1;
}
} catch (NullPointerException e) {
throw e;
if (v == 0) v = compareTypes(this.type, that.type);
return v;
} else {
return 0;
}
}
void addTags(List<String> params) {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Clause that = (Clause)o;
return compareTo(that) == 0;
}
void addTags(Collection<String> params) {
if (globalTag != null && !params.contains(globalTag.name)) params.add(globalTag.name);
if (tag != null && !params.contains(tag.name)) params.add(tag.name);
}
public static class Condition {
class Condition {
final String name;
final Object val;
final Operand op;
@ -136,19 +159,18 @@ public class Clause implements MapWriter, Comparable<Clause> {
this.op = op;
}
TestStatus match(Row row) {
return op.match(val, row.getVal(name));
boolean isPass(Object inputVal) {
if (inputVal instanceof ReplicaCount) inputVal = ((ReplicaCount) inputVal).getVal(type);
ValidateInfo validator = getValidator(name);
if (validator == LazyValidator.INST) { // we don't know the type
return op.match(parseString(val), parseString(inputVal)) == PASS;
} else {
return op.match(val, validate(name, inputVal, false)) == PASS;
}
}
TestStatus match(Object testVal) {
return op.match(this.val, testVal);
}
public boolean isPass(Object inputVal) {
return op.match(val, validate(name, inputVal, false)) == PASS;
}
public boolean isPass(Row row) {
boolean isPass(Row row) {
return op.match(val, row.getVal(name)) == PASS;
}
@ -161,8 +183,14 @@ public class Clause implements MapWriter, Comparable<Clause> {
return false;
}
public Integer delta(Object val) {
return op.delta(this.val, val);
public Long delta(Object val) {
if (this.val instanceof String) {
if (op == LESS_THAN || op == GREATER_THAN) {
return op.delta(Clause.parseDouble(name, this.val), Clause.parseDouble(name, val));
} else {
return 0l;
}
} else return op.delta(this.val, val);
}
public String getName() {
@ -172,13 +200,13 @@ public class Clause implements MapWriter, Comparable<Clause> {
public Object getValue() {
return val;
}
public Operand getOperand() {
return op;
}
}
static Condition parse(String s, Map m) {
Condition parse(String s, Map m) {
Object expectedVal = null;
Object val = m.get(s);
try {
@ -209,12 +237,12 @@ public class Clause implements MapWriter, Comparable<Clause> {
public class Violation implements MapWriter {
final String shard, coll, node;
final Object actualVal;
final Integer delta;//how far is the actual value from the expected value
final Long delta;//how far is the actual value from the expected value
final Object tagKey;
private final int hash;
private Violation(String coll, String shard, String node, Object actualVal, Integer delta, Object tagKey) {
private Violation(String coll, String shard, String node, Object actualVal, Long delta, Object tagKey) {
this.shard = shard;
this.coll = coll;
this.node = node;
@ -251,6 +279,11 @@ public class Clause implements MapWriter, Comparable<Clause> {
return false;
}
@Override
public String toString() {
return Utils.toJSONString(Utils.getDeepCopy(toMap(new LinkedHashMap<>()), 5));
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.putIfNotNull("collection", coll);
@ -258,8 +291,8 @@ public class Clause implements MapWriter, Comparable<Clause> {
ew.putIfNotNull("node", node);
ew.putIfNotNull("tagKey", String.valueOf(tagKey));
ew.putIfNotNull("violation", (MapWriter) ew1 -> {
ew1.put(getClause().isPerCollectiontag() ? "replica" : tag.name,
String.valueOf(actualVal));
if (getClause().isPerCollectiontag()) ew1.put("replica", actualVal);
else ew1.put(tag.name, String.valueOf(actualVal));
ew1.putIfNotNull("delta", delta);
});
ew.put("clause", getClause());
@ -270,12 +303,12 @@ public class Clause implements MapWriter, Comparable<Clause> {
public List<Violation> test(List<Row> allRows) {
List<Violation> violations = new ArrayList<>();
if (isPerCollectiontag()) {
Map<String, Map<String, Map<String, AtomicInteger>>> replicaCount = computeReplicaCounts(allRows);
for (Map.Entry<String, Map<String, Map<String, AtomicInteger>>> e : replicaCount.entrySet()) {
Map<String, Map<String, Map<String, ReplicaCount>>> replicaCount = computeReplicaCounts(allRows);
for (Map.Entry<String, Map<String, Map<String, ReplicaCount>>> e : replicaCount.entrySet()) {
if (!collection.isPass(e.getKey())) continue;
for (Map.Entry<String, Map<String, AtomicInteger>> shardVsCount : e.getValue().entrySet()) {
for (Map.Entry<String, Map<String, ReplicaCount>> shardVsCount : e.getValue().entrySet()) {
if (!shard.isPass(shardVsCount.getKey())) continue;
for (Map.Entry<String, AtomicInteger> counts : shardVsCount.getValue().entrySet()) {
for (Map.Entry<String, ReplicaCount> counts : shardVsCount.getValue().entrySet()) {
if (!replica.isPass(counts.getValue())) {
violations.add(new Violation(
e.getKey(),
@ -301,24 +334,23 @@ public class Clause implements MapWriter, Comparable<Clause> {
}
private Map<String, Map<String, Map<String, AtomicInteger>>> computeReplicaCounts(List<Row> allRows) {
Map<String, Map<String, Map<String, AtomicInteger>>> collVsShardVsTagVsCount = new HashMap<>();
for (Row row : allRows)
private Map<String, Map<String, Map<String, ReplicaCount>>> computeReplicaCounts(List<Row> allRows) {
Map<String, Map<String, Map<String, ReplicaCount>>> collVsShardVsTagVsCount = new HashMap<>();
for (Row row : allRows) {
for (Map.Entry<String, Map<String, List<ReplicaInfo>>> colls : row.collectionVsShardVsReplicas.entrySet()) {
String collectionName = colls.getKey();
if (!collection.isPass(collectionName)) continue;
collVsShardVsTagVsCount.putIfAbsent(collectionName, new HashMap<>());
Map<String, Map<String, AtomicInteger>> collMap = collVsShardVsTagVsCount.get(collectionName);
Map<String, Map<String, ReplicaCount>> collMap = collVsShardVsTagVsCount.computeIfAbsent(collectionName, s -> new HashMap<>());
for (Map.Entry<String, List<ReplicaInfo>> shards : colls.getValue().entrySet()) {
String shardName = shards.getKey();
if (ANY.equals(shard.val)) shardName = ANY;
if (!shard.isPass(shardName)) break;
collMap.putIfAbsent(shardName, new HashMap<>());
Map<String, AtomicInteger> tagVsCount = collMap.get(shardName);
Map<String, ReplicaCount> tagVsCount = collMap.computeIfAbsent(shardName, s -> new HashMap<>());
Object tagVal = row.getVal(tag.name);
tagVsCount.putIfAbsent(tag.isPass(tagVal) ? String.valueOf(tagVal) : "", new AtomicInteger());
tagVsCount.computeIfAbsent(tag.isPass(tagVal) ? String.valueOf(tagVal) : "", s -> new ReplicaCount());
if (tag.isPass(tagVal)) {
tagVsCount.get(String.valueOf(tagVal)).addAndGet(shards.getValue().size());
tagVsCount.get(String.valueOf(tagVal)).increment(shards.getValue());
}
}
}
}
@ -343,7 +375,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
NOT_APPLICABLE, FAIL, PASS
}
private static final Set<String> IGNORE_TAGS = new HashSet<>(Arrays.asList(REPLICA, COLLECTION, SHARD, "strict"));
private static final Set<String> IGNORE_TAGS = new HashSet<>(Arrays.asList(REPLICA, COLLECTION, SHARD, "strict", "type"));
static class ValidateInfo {
final Class type;
@ -356,54 +388,82 @@ public class Clause implements MapWriter, Comparable<Clause> {
this.type = type;
this.vals = vals;
this.min = min;
if(min != null && !type.isInstance(min)) throw new RuntimeException("wrong min value type, expected: " + type.getName() + " actual: " + min.getClass().getName());
if (min != null && !type.isInstance(min))
throw new RuntimeException("wrong min value type, expected: " + type.getName() + " actual: " + min.getClass().getName());
this.max = max;
if(max != null && !type.isInstance(max)) throw new RuntimeException("wrong max value type, expected: " + type.getName() + " actual: " + max.getClass().getName());
if (max != null && !type.isInstance(max))
throw new RuntimeException("wrong max value type, expected: " + type.getName() + " actual: " + max.getClass().getName());
}
public Object validate(String name, Object val, boolean isRuleVal) {
if (type == Double.class) {
Double num = parseDouble(name, val);
if (isRuleVal) {
if (min != null)
if (Double.compare(num, (Double) min) == -1)
throw new RuntimeException(name + ": " + val + " must be greater than " + min);
if (max != null)
if (Double.compare(num, (Double) max) == 1)
throw new RuntimeException(name + ": " + val + " must be less than " + max);
}
return num;
} else if (type == Long.class) {
Long num = parseLong(name, val);
if (isRuleVal) {
if (min != null)
if (num < min.longValue())
throw new RuntimeException(name + ": " + val + " must be greater than " + min);
if (max != null)
if (num > max.longValue())
throw new RuntimeException(name + ": " + val + " must be less than " + max);
}
return num;
} else if (type == String.class) {
if (isRuleVal && vals != null && !vals.contains(val))
throw new RuntimeException(name + ": " + val + " must be one of " + StrUtils.join(vals, ','));
return val;
} else {
throw new RuntimeException("Invalid type ");
}
}
}
static class LazyValidator extends ValidateInfo {
static final LazyValidator INST = new LazyValidator();
LazyValidator() {
super(null, null, null, null);
}
@Override
public Object validate(String name, Object val, boolean isRuleVal) {
return parseString(val);
}
}
public static String parseString(Object val) {
return val == null ? null : String.valueOf(val);
}
/**
*
* @param name name of the condition
* @param val value of the condition
* @param name name of the condition
* @param val value of the condition
* @param isRuleVal is this provided in the rule
* @return actual validated value
*/
public static Object validate(String name, Object val, boolean isRuleVal) {
public static Object validate(String name, Object val, boolean isRuleVal) {
if (val == null) return null;
ValidateInfo info = getValidator(name);
if (info == null) throw new RuntimeException("Unknown type :" + name);
return info.validate(name, val, isRuleVal);
}
private static ValidateInfo getValidator(String name) {
ValidateInfo info = validatetypes.get(name);
if (info == null && name.startsWith(ImplicitSnitch.SYSPROP)) info = validatetypes.get("STRING");
if (info == null) throw new RuntimeException("Unknown type :" + name);
if (info.type == Double.class) {
Double num = parseDouble(name, val);
if (isRuleVal) {
if (info.min != null)
if (Double.compare(num, (Double) info.min) == -1)
throw new RuntimeException(name + ": " + val + " must be greater than " + info.min);
if (info.max != null)
if (Double.compare(num, (Double) info.max) == 1)
throw new RuntimeException(name + ": " + val + " must be less than " + info.max);
}
return num;
} else if (info.type == Long.class) {
Long num = parseLong(name, val);
if (isRuleVal) {
if (info.min != null)
if (num < info.min.longValue())
throw new RuntimeException(name + ": " + val + " must be greater than " + info.min);
if (info.max != null)
if (num > info.max.longValue())
throw new RuntimeException(name + ": " + val + " must be less than " + info.max);
}
return num;
} else if (info.type == String.class) {
if (isRuleVal && info.vals != null && !info.vals.contains(val))
throw new RuntimeException(name + ": " + val + " must be one of " + StrUtils.join(info.vals, ','));
return val;
} else {
throw new RuntimeException("Invalid type ");
}
if (info == null && name.startsWith(METRICS_PREFIX)) info = LazyValidator.INST;
return info;
}
public static Long parseLong(String name, Object val) {
@ -425,7 +485,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
num = (Number) val;
}
if (num != null) {
if (num != null) {
return num.longValue();
}
throw new RuntimeException(name + ": " + val + "not a valid number");
@ -446,12 +506,14 @@ public class Clause implements MapWriter, Comparable<Clause> {
num = (Number) val;
}
if (num != null) {
if (num != null) {
return num.doubleValue();
}
throw new RuntimeException(name + ": " + val + "not a valid number");
}
public static final String METRICS_PREFIX = "metrics:";
private static final Map<String, ValidateInfo> validatetypes = new HashMap<>();
static {
@ -467,6 +529,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
validatetypes.put("NUMBER", new ValidateInfo(Long.class, null, 0L, Long.MAX_VALUE));//generic number validation
validatetypes.put("STRING", new ValidateInfo(String.class, null, null, null));//generic string validation
validatetypes.put("node", new ValidateInfo(String.class, null, null, null));
validatetypes.put("LAZY", LazyValidator.INST);
for (String ip : ImplicitSnitch.IP_SNITCHES) validatetypes.put(ip, new ValidateInfo(Long.class, null, 0L, 255L));
}
}

View File

@ -38,7 +38,7 @@ public interface ClusterDataProvider extends Closeable {
* <p>
* the format is {collection:shard :[{replicadetails}]}
*/
Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys);
Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys);
Collection<String> getNodes();

View File

@ -21,7 +21,6 @@ import java.util.List;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.Clause.Violation;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy.ReplicaInfo;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy.Suggester;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.util.Pair;
@ -40,41 +39,41 @@ public class MoveReplicaSuggester extends Suggester {
List<Clause.Violation> leastSeriousViolation = null;
Integer targetNodeIndex = null;
Integer sourceNodeIndex = null;
ReplicaInfo fromReplicaInfo = null;
ReplicaInfo sourceReplicaInfo = null;
for (Pair<ReplicaInfo, Row> fromReplica : getValidReplicas(true, true, -1)) {
Row fromRow = fromReplica.second();
ReplicaInfo replicaInfo = fromReplica.first();
String coll = replicaInfo.collection;
String shard = replicaInfo.shard;
Pair<Row, ReplicaInfo> pair = fromRow.removeReplica(coll, shard);
Row tmpRow = pair.first();
if (tmpRow == null) {
Pair<Row, ReplicaInfo> pair = fromRow.removeReplica(coll, shard, replicaInfo.type);
Row srcTmpRow = pair.first();
if (srcTmpRow == null) {
//no such replica available
continue;
}
tmpRow.violations.clear();
final int i = getMatrix().indexOf(fromRow);
for (int j = getMatrix().size() - 1; j > i; j--) {
Row targetRow = getMatrix().get(j);
if(!targetRow.isLive) continue;
if (!isAllowed(targetRow.node, Hint.TARGET_NODE)) continue;
targetRow = targetRow.addReplica(coll, shard);
targetRow.violations.clear();
List<Violation> errs = testChangedMatrix(strict, getModifiedMatrix(getModifiedMatrix(getMatrix(), tmpRow, i), targetRow, j));
if (!containsNewErrors(errs) && isLessSerious(errs, leastSeriousViolation)) {
targetRow = targetRow.addReplica(coll, shard, replicaInfo.type);
List<Violation> errs = testChangedMatrix(strict, getModifiedMatrix(getModifiedMatrix(getMatrix(), srcTmpRow, i), targetRow, j));
if (!containsNewErrors(errs) && isLessSerious(errs, leastSeriousViolation) &&
Policy.compareRows(srcTmpRow, targetRow, session.getPolicy()) < 1) {
leastSeriousViolation = errs;
targetNodeIndex = j;
sourceNodeIndex = i;
fromReplicaInfo = replicaInfo;
sourceReplicaInfo = replicaInfo;
}
}
}
if (targetNodeIndex != null && sourceNodeIndex != null) {
getMatrix().set(sourceNodeIndex, getMatrix().get(sourceNodeIndex).removeReplica(fromReplicaInfo.collection, fromReplicaInfo.shard).first());
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(fromReplicaInfo.collection, fromReplicaInfo.shard));
getMatrix().set(sourceNodeIndex, getMatrix().get(sourceNodeIndex).removeReplica(sourceReplicaInfo.collection, sourceReplicaInfo.shard, sourceReplicaInfo.type).first());
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(sourceReplicaInfo.collection, sourceReplicaInfo.shard, sourceReplicaInfo.type));
return new CollectionAdminRequest.MoveReplica(
fromReplicaInfo.collection,
fromReplicaInfo.name,
sourceReplicaInfo.collection,
sourceReplicaInfo.name,
getMatrix().get(targetNodeIndex).node);
}
return null;

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.client.solrj.cloud.autoscaling;
import org.apache.solr.client.solrj.SolrRequest;
public class NoneSuggester extends Policy.Suggester{
@Override
SolrRequest init() {
return null;
}
@Override
public SolrRequest getOperation() {
return null;
}
}

View File

@ -25,6 +25,7 @@ import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.ANY;
import java.util.Objects;
import org.apache.solr.client.solrj.cloud.autoscaling.Clause.TestStatus;
import org.apache.solr.common.params.CoreAdminParams;
public enum Operand {
@ -37,7 +38,7 @@ public enum Operand {
},
EQUAL("", 0) {
@Override
public int _delta(int expected, int actual) {
public long _delta(long expected, long actual) {
return expected - actual;
}
},
@ -48,7 +49,7 @@ public enum Operand {
}
@Override
public int _delta(int expected, int actual) {
public long _delta(long expected, long actual) {
return expected - actual;
}
@ -57,6 +58,7 @@ public enum Operand {
@Override
public TestStatus match(Object ruleVal, Object testVal) {
if (testVal == null) return NOT_APPLICABLE;
if (ruleVal instanceof String) ruleVal = Clause.parseDouble("", ruleVal);
if (ruleVal instanceof Double) {
return Double.compare(Clause.parseDouble("", testVal), (Double) ruleVal) == 1 ? PASS : FAIL;
}
@ -64,7 +66,7 @@ public enum Operand {
}
@Override
protected int _delta(int expected, int actual) {
protected long _delta(long expected, long actual) {
return actual > expected ? 0 : (expected + 1) - actual;
}
},
@ -72,6 +74,7 @@ public enum Operand {
@Override
public TestStatus match(Object ruleVal, Object testVal) {
if (testVal == null) return NOT_APPLICABLE;
if (ruleVal instanceof String) ruleVal = Clause.parseDouble("", ruleVal);
if (ruleVal instanceof Double) {
return Double.compare(Clause.parseDouble("", testVal), (Double) ruleVal) == -1 ? PASS : FAIL;
}
@ -79,7 +82,7 @@ public enum Operand {
}
@Override
protected int _delta(int expected, int actual) {
protected long _delta(long expected, long actual) {
return actual < expected ? 0 : (expected ) - actual;
}
@ -92,10 +95,6 @@ public enum Operand {
this.priority = priority;
}
public String toStr(Object expectedVal) {
return operand + expectedVal.toString();
}
public TestStatus match(Object ruleVal, Object testVal) {
return Objects.equals(ruleVal, testVal) ? PASS : FAIL;
}
@ -107,17 +106,17 @@ public enum Operand {
}
public Integer delta(Object expected, Object actual) {
public Long delta(Object expected, Object actual) {
try {
Integer expectedInt = Integer.parseInt(String.valueOf(expected));
Integer actualInt = Integer.parseInt(String.valueOf(actual));
Long expectedInt = (Long) Clause.validate(CoreAdminParams.REPLICA, expected, false);
Long actualInt = (Long) Clause.validate(CoreAdminParams.REPLICA, actual, false);
return _delta(expectedInt, actualInt);
} catch (Exception e) {
return null;
}
}
protected int _delta(int expected, int actual) {
protected long _delta(long expected, long actual) {
return 0;
}
}

View File

@ -32,6 +32,7 @@ import java.util.Objects;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.function.Consumer;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@ -39,6 +40,7 @@ import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.Clause.Violation;
import org.apache.solr.common.IteratorWriter;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.StrUtils;
@ -46,6 +48,7 @@ import org.apache.solr.common.util.Utils;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.stream.Collectors.collectingAndThen;
import static java.util.stream.Collectors.toList;
/*The class that reads, parses and applies policies specified in
@ -61,55 +64,78 @@ public class Policy implements MapWriter {
public static final String POLICY = "policy";
public static final String EACH = "#EACH";
public static final String ANY = "#ANY";
public static final String POLICIES = "policies";
public static final String CLUSTER_POLICY = "cluster-policy";
public static final String CLUSTER_PREFERENCE = "cluster-preferences";
public static final String CLUSTER_PREFERENCES = "cluster-preferences";
public static final Set<String> GLOBAL_ONLY_TAGS = Collections.singleton("cores");
final Map<String, List<Clause>> policies = new HashMap<>();
public static final Preference DEFAULT_PREFERENCE = new Preference((Map<String, Object>) Utils.fromJSONString("{minimize : cores, precision:1}"));
final Map<String, List<Clause>> policies;
final List<Clause> clusterPolicy;
final List<Preference> clusterPreferences;
final List<String> params;
public Policy() {
this(Collections.emptyMap());
}
public Policy(Map<String, Object> jsonMap) {
clusterPreferences = ((List<Map<String, Object>>) jsonMap.getOrDefault(CLUSTER_PREFERENCE, emptyList())).stream()
.map(Preference::new)
int[] idx = new int[1];
List<Preference> initialClusterPreferences = ((List<Map<String, Object>>) jsonMap.getOrDefault(CLUSTER_PREFERENCES, emptyList())).stream()
.map(m -> new Preference(m, idx[0]++))
.collect(toList());
for (int i = 0; i < clusterPreferences.size() - 1; i++) {
Preference preference = clusterPreferences.get(i);
preference.next = clusterPreferences.get(i + 1);
for (int i = 0; i < initialClusterPreferences.size() - 1; i++) {
Preference preference = initialClusterPreferences.get(i);
preference.next = initialClusterPreferences.get(i + 1);
}
if (clusterPreferences.isEmpty()) {
clusterPreferences.add(new Preference((Map<String, Object>) Utils.fromJSONString("{minimize : cores, precision:1}")));
if (initialClusterPreferences.isEmpty()) {
initialClusterPreferences.add(DEFAULT_PREFERENCE);
}
SortedSet<String> paramsOfInterest = new TreeSet<>();
this.clusterPreferences = Collections.unmodifiableList(initialClusterPreferences);
final SortedSet<String> paramsOfInterest = new TreeSet<>();
for (Preference preference : clusterPreferences) {
if (paramsOfInterest.contains(preference.name.name())) {
throw new RuntimeException(preference.name + " is repeated");
}
paramsOfInterest.add(preference.name.toString());
}
this.params = new ArrayList<>(paramsOfInterest);
List<String> newParams = new ArrayList<>(paramsOfInterest);
clusterPolicy = ((List<Map<String, Object>>) jsonMap.getOrDefault(CLUSTER_POLICY, emptyList())).stream()
.map(Clause::new)
.filter(clause -> {
clause.addTags(params);
clause.addTags(newParams);
return true;
})
.collect(Collectors.toList());
.collect(collectingAndThen(toList(), Collections::unmodifiableList));
((Map<String, List<Map<String, Object>>>) jsonMap.getOrDefault("policies", emptyMap())).forEach((s, l1) ->
this.policies.put(s, l1.stream()
.map(Clause::new)
.filter(clause -> {
if (!clause.isPerCollectiontag())
throw new RuntimeException(clause.globalTag.name + " is only allowed in 'cluster-policy'");
clause.addTags(params);
return true;
})
.sorted()
.collect(toList())));
this.policies = Collections.unmodifiableMap(
policiesFromMap((Map<String, List<Map<String, Object>>>)jsonMap.getOrDefault(POLICIES, emptyMap()), newParams));
this.params = Collections.unmodifiableList(newParams);
}
private Policy(Map<String, List<Clause>> policies, List<Clause> clusterPolicy, List<Preference> clusterPreferences,
List<String> params) {
this.policies = policies != null ? Collections.unmodifiableMap(policies) : Collections.emptyMap();
this.clusterPolicy = clusterPolicy != null ? Collections.unmodifiableList(clusterPolicy) : Collections.emptyList();
this.clusterPreferences = clusterPreferences != null ? Collections.unmodifiableList(clusterPreferences) :
Collections.singletonList(DEFAULT_PREFERENCE);
this.params = params != null ? Collections.unmodifiableList(params) : Collections.emptyList();
}
public Policy withPolicies(Map<String, List<Clause>> policies) {
return new Policy(policies, clusterPolicy, clusterPreferences, params);
}
public Policy withClusterPreferences(List<Preference> clusterPreferences) {
return new Policy(policies, clusterPolicy, clusterPreferences, params);
}
public Policy withClusterPolicy(List<Clause> clusterPolicy) {
return new Policy(policies, clusterPolicy, clusterPreferences, params);
}
public Policy withParams(List<String> params) {
return new Policy(policies, clusterPolicy, clusterPreferences, params);
}
public List<Clause> getClusterPolicy() {
@ -123,24 +149,43 @@ public class Policy implements MapWriter {
@Override
public void writeMap(EntryWriter ew) throws IOException {
if (!policies.isEmpty()) {
ew.put("policies", (MapWriter) ew1 -> {
ew.put(POLICIES, (MapWriter) ew1 -> {
for (Map.Entry<String, List<Clause>> e : policies.entrySet()) {
ew1.put(e.getKey(), e.getValue());
}
});
}
if (!clusterPreferences.isEmpty()) {
ew.put("preferences", (IteratorWriter) iw -> {
ew.put(CLUSTER_PREFERENCES, (IteratorWriter) iw -> {
for (Preference p : clusterPreferences) iw.add(p);
});
}
if (!clusterPolicy.isEmpty()) {
ew.put(CLUSTER_POLICY, (IteratorWriter) iw -> {
for (Clause c : clusterPolicy) {
iw.add(c);
}
});
}
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Policy policy = (Policy) o;
if (!getPolicies().equals(policy.getPolicies())) return false;
if (!getClusterPolicy().equals(policy.getClusterPolicy())) return false;
if (!getClusterPreferences().equals(policy.getClusterPreferences())) return false;
return params.equals(policy.params);
}
/*This stores the logical state of the system, given a policy and
* a cluster state.
*
*/
* a cluster state.
*
*/
public class Session implements MapWriter {
final List<String> nodes;
final ClusterDataProvider dataProvider;
@ -208,20 +253,7 @@ public class Policy implements MapWriter {
* Apply the preferences and conditions
*/
private void applyRules() {
if (!clusterPreferences.isEmpty()) {
//this is to set the approximate value according to the precision
ArrayList<Row> tmpMatrix = new ArrayList<>(matrix);
for (Preference p : clusterPreferences) {
Collections.sort(tmpMatrix, (r1, r2) -> p.compare(r1, r2, false));
p.setApproxVal(tmpMatrix);
}
//approximate values are set now. Let's do recursive sorting
Collections.sort(matrix, (Row r1, Row r2) -> {
int result = clusterPreferences.get(0).compare(r1, r2, true);
if (result == 0) result = clusterPreferences.get(0).compare(r1, r2, false);
return result;
});
}
setApproxValuesAndSortNodes(clusterPreferences, matrix);
for (Clause clause : expandedClauses) {
List<Violation> errs = clause.test(matrix);
@ -229,6 +261,8 @@ public class Policy implements MapWriter {
}
}
public List<Violation> getViolations() {
return violations;
}
@ -258,6 +292,22 @@ public class Policy implements MapWriter {
}
}
static void setApproxValuesAndSortNodes(List<Preference> clusterPreferences, List<Row> matrix) {
if (!clusterPreferences.isEmpty()) {
//this is to set the approximate value according to the precision
ArrayList<Row> tmpMatrix = new ArrayList<>(matrix);
for (Preference p : clusterPreferences) {
Collections.sort(tmpMatrix, (r1, r2) -> p.compare(r1, r2, false));
p.setApproxVal(tmpMatrix);
}
//approximate values are set now. Let's do recursive sorting
Collections.sort(matrix, (Row r1, Row r2) -> {
int result = clusterPreferences.get(0).compare(r1, r2, true);
if (result == 0) result = clusterPreferences.get(0).compare(r1, r2, false);
return result;
});
}
}
public Session createSession(ClusterDataProvider dataProvider) {
return new Session(dataProvider);
@ -298,37 +348,6 @@ public class Policy implements MapWriter {
}
public static class ReplicaInfo implements MapWriter {
final String name;
String core, collection, shard;
Map<String, Object> variables;
public ReplicaInfo(String name, String coll, String shard, Map<String, Object> vals) {
this.name = name;
this.variables = vals;
this.collection = coll;
this.shard = shard;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put(name, variables);
}
public String getCore() {
return core;
}
public String getCollection() {
return collection;
}
public String getShard() {
return shard;
}
}
/* A suggester is capable of suggesting a collection operation
* given a particular session. Before it suggests a new operation,
* it ensures that ,
@ -348,10 +367,12 @@ public class Policy implements MapWriter {
}
public Suggester hint(Hint hint, Object value) {
if (hint == Hint.TARGET_NODE || hint == Hint.SRC_NODE) {
((Set) hints.computeIfAbsent(hint, h -> new HashSet<>())).add(value);
hint.validator.accept(value);
if (hint.multiValued) {
Collection<?> values = value instanceof Collection ? (Collection)value : Collections.singletonList(value);
((Set) hints.computeIfAbsent(hint, h -> new HashSet<>())).addAll(values);
} else {
hints.put(hint, value);
hints.put(hint, value == null ? null : String.valueOf(value));
}
return this;
}
@ -361,21 +382,30 @@ public class Policy implements MapWriter {
public SolrRequest getOperation() {
if (!isInitialized) {
String coll = (String) hints.get(Hint.COLL);
String shard = (String) hints.get(Hint.SHARD);
// if this is not a known collection from the existing clusterstate,
// then add it
if (session.matrix.stream().noneMatch(row -> row.collectionVsShardVsReplicas.containsKey(coll))) {
session.addClausesForCollection(session.dataProvider, coll);
Set<String> collections = (Set<String>) hints.getOrDefault(Hint.COLL, Collections.emptySet());
Set<Pair<String, String>> s = (Set<Pair<String, String>>) hints.getOrDefault(Hint.COLL_SHARD, Collections.emptySet());
if (!collections.isEmpty() || !s.isEmpty()) {
HashSet<Pair<String, String>> shards = new HashSet<>(s);
collections.stream().forEach(c -> shards.add(new Pair<>(c, null)));
for (Pair<String, String> shard : shards) {
// if this is not a known collection from the existing clusterstate,
// then add it
if (session.matrix.stream().noneMatch(row -> row.collectionVsShardVsReplicas.containsKey(shard.first()))) {
session.addClausesForCollection(session.dataProvider, shard.first());
}
for (Row row : session.matrix) {
Map<String, List<ReplicaInfo>> shardInfo = row.collectionVsShardVsReplicas.computeIfAbsent(shard.first(), it -> new HashMap<>());
if (shard.second() != null) shardInfo.computeIfAbsent(shard.second(), it -> new ArrayList<>());
}
}
Collections.sort(session.expandedClauses);
}
if (coll != null) {
for (Row row : session.matrix) {
if (!row.collectionVsShardVsReplicas.containsKey(coll)) row.collectionVsShardVsReplicas.put(coll, new HashMap<>());
if (shard != null) {
Map<String, List<ReplicaInfo>> shardInfo = row.collectionVsShardVsReplicas.get(coll);
if (!shardInfo.containsKey(shard)) shardInfo.put(shard, new ArrayList<>());
}
Set<String> srcNodes = (Set<String>) hints.get(Hint.SRC_NODE);
if (srcNodes != null && !srcNodes.isEmpty()) {
// the source node is dead so live nodes may not have it
for (String srcNode : srcNodes) {
if(session.matrix.stream().noneMatch(row -> row.node.equals(srcNode)))
session.matrix.add(new Row(srcNode, session.getPolicy().params, session.dataProvider));
}
}
session.applyRules();
@ -420,7 +450,7 @@ public class Policy implements MapWriter {
}
List<Pair<ReplicaInfo, Row>> getValidReplicas(boolean sortDesc, boolean isSource, int until) {
List<Pair<Policy.ReplicaInfo, Row>> allPossibleReplicas = new ArrayList<>();
List<Pair<ReplicaInfo, Row>> allPossibleReplicas = new ArrayList<>();
if (sortDesc) {
if (until == -1) until = getMatrix().size();
@ -433,18 +463,20 @@ public class Policy implements MapWriter {
return allPossibleReplicas;
}
void addReplicaToList(Row r, boolean isSource, List<Pair<Policy.ReplicaInfo, Row>> replicaList) {
void addReplicaToList(Row r, boolean isSource, List<Pair<ReplicaInfo, Row>> replicaList) {
if (!isAllowed(r.node, isSource ? Hint.SRC_NODE : Hint.TARGET_NODE)) return;
for (Map.Entry<String, Map<String, List<Policy.ReplicaInfo>>> e : r.collectionVsShardVsReplicas.entrySet()) {
for (Map.Entry<String, Map<String, List<ReplicaInfo>>> e : r.collectionVsShardVsReplicas.entrySet()) {
if (!isAllowed(e.getKey(), Hint.COLL)) continue;
for (Map.Entry<String, List<Policy.ReplicaInfo>> shard : e.getValue().entrySet()) {
if (!isAllowed(e.getKey(), Hint.SHARD)) continue;
for (Map.Entry<String, List<ReplicaInfo>> shard : e.getValue().entrySet()) {
if (!isAllowed(new Pair<>(e.getKey(), shard.getKey()), Hint.COLL_SHARD)) continue;//todo fix
if(shard.getValue() == null || shard.getValue().isEmpty()) continue;
replicaList.add(new Pair<>(shard.getValue().get(0), r));
}
}
}
protected List<Violation> testChangedMatrix(boolean strict, List<Row> rows) {
List<Violation> testChangedMatrix(boolean strict, List<Row> rows) {
setApproxValuesAndSortNodes(session.getPolicy().clusterPreferences,rows);
List<Violation> errors = new ArrayList<>();
for (Clause clause : session.expandedClauses) {
if (strict || clause.strict) {
@ -465,7 +497,7 @@ public class Policy implements MapWriter {
protected boolean isAllowed(Object v, Hint hint) {
Object hintVal = hints.get(hint);
if (hint == Hint.TARGET_NODE || hint == Hint.SRC_NODE) {
if (hint.multiValued) {
Set set = (Set) hintVal;
return set == null || set.contains(v);
} else {
@ -474,12 +506,68 @@ public class Policy implements MapWriter {
}
public enum Hint {
COLL, SHARD, SRC_NODE, TARGET_NODE
COLL(true),
// collection shard pair
// this should be a Pair<String, String> , (collection,shard)
COLL_SHARD(true, v -> {
Collection c = v instanceof Collection ? (Collection) v : Collections.singleton(v);
for (Object o : c) {
if (!(o instanceof Pair)) {
throw new RuntimeException("SHARD hint must use a Pair");
}
Pair p = (Pair) o;
if (p.first() == null || p.second() == null) {
throw new RuntimeException("Both collection and shard must not be null");
}
}
}),
SRC_NODE(true),
TARGET_NODE(true),
REPLICATYPE(false, o -> {
if (!(o instanceof Replica.Type)) {
throw new RuntimeException("REPLICATYPE hint must use a ReplicaType");
}
});
public final boolean multiValued;
public final Consumer<Object> validator;
Hint(boolean multiValued) {
this(multiValued, v -> {
Collection c = v instanceof Collection ? (Collection) v : Collections.singleton(v);
for (Object o : c) {
if (!(o instanceof String)) throw new RuntimeException("hint must be of type String");
}
});
}
Hint(boolean multiValued, Consumer<Object> c) {
this.multiValued = multiValued;
this.validator = c;
}
}
}
public static Map<String, List<Clause>> policiesFromMap(Map<String, List<Map<String, Object>>> map, List<String> newParams) {
Map<String, List<Clause>> newPolicies = new HashMap<>();
map.forEach((s, l1) ->
newPolicies.put(s, l1.stream()
.map(Clause::new)
.filter(clause -> {
if (!clause.isPerCollectiontag())
throw new RuntimeException(clause.globalTag.name + " is only allowed in 'cluster-policy'");
clause.addTags(newParams);
return true;
})
.sorted()
.collect(collectingAndThen(toList(), Collections::unmodifiableList))));
return newPolicies;
}
public static List<Clause> mergePolicies(String coll,
List<Clause> collPolicy,
List<Clause> globalPolicy) {
@ -518,4 +606,21 @@ public class Policy implements MapWriter {
public Map<String, List<Clause>> getPolicies() {
return policies;
}
public List<String> getParams() {
return params;
}
/**
* Compares two {@link Row} loads according to a policy.
*
* @param r1 the first {@link Row} to compare
* @param r2 the second {@link Row} to compare
* @return the value {@code 0} if r1 and r2 are equally loaded
* a value {@code -1} if r1 is more loaded than r2
* a value {@code 1} if r1 is less loaded than r2
*/
static int compareRows(Row r1, Row r2, Policy policy) {
return policy.clusterPreferences.get(0).compare(r1, r2, true);
}
}

View File

@ -20,28 +20,35 @@ package org.apache.solr.client.solrj.cloud.autoscaling;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.EnumMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy.Suggester.Hint;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.Utils;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CoreAdminParams.NODE;
public class PolicyHelper {
public static Map<String, List<String>> getReplicaLocations(String collName, Map<String, Object> autoScalingJson,
ClusterDataProvider cdp,
Map<String, String> optionalPolicyMapping,
List<String> shardNames,
int repFactor,
List<String> nodesList) {
Map<String, List<String>> positionMapping = new HashMap<>();
for (String shardName : shardNames) positionMapping.put(shardName, new ArrayList<>(repFactor));
if (optionalPolicyMapping != null) {
private static ThreadLocal<Map<String, String>> policyMapping = new ThreadLocal<>();
public static List<ReplicaPosition> getReplicaLocations(String collName, AutoScalingConfig autoScalingConfig,
ClusterDataProvider cdp,
Map<String, String> optionalPolicyMapping,
List<String> shardNames,
int nrtReplicas,
int tlogReplicas,
int pullReplicas,
List<String> nodesList) {
List<ReplicaPosition> positions = new ArrayList<>();
final ClusterDataProvider delegate = cdp;
cdp = new ClusterDataProvider() {
@Override
@ -50,7 +57,7 @@ public class PolicyHelper {
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return delegate.getReplicaInfo(node, keys);
}
@ -61,36 +68,126 @@ public class PolicyHelper {
@Override
public String getPolicyNameByCollection(String coll) {
return optionalPolicyMapping.containsKey(coll) ?
return policyMapping.get() != null && policyMapping.get().containsKey(coll) ?
optionalPolicyMapping.get(coll) :
delegate.getPolicyNameByCollection(coll);
}
};
policyMapping.set(optionalPolicyMapping);
Policy.Session session = null;
try {
session = SESSION_REF.get() != null ?
SESSION_REF.get().initOrGet(cdp, autoScalingConfig.getPolicy()) :
autoScalingConfig.getPolicy().createSession(cdp);
Map<Replica.Type, Integer> typeVsCount = new EnumMap<>(Replica.Type.class);
typeVsCount.put(Replica.Type.NRT, nrtReplicas);
typeVsCount.put(Replica.Type.TLOG, tlogReplicas);
typeVsCount.put(Replica.Type.PULL, pullReplicas);
for (String shardName : shardNames) {
int idx = 0;
for (Map.Entry<Replica.Type, Integer> e : typeVsCount.entrySet()) {
for (int i = 0; i < e.getValue(); i++) {
Policy.Suggester suggester = session.getSuggester(ADDREPLICA)
.hint(Hint.REPLICATYPE, e.getKey())
.hint(Hint.COLL_SHARD, new Pair<>(collName, shardName));
if (nodesList != null) {
for (String nodeName : nodesList) {
suggester = suggester.hint(Hint.TARGET_NODE, nodeName);
}
}
SolrRequest op = suggester.getOperation();
if (op == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No node can satisfy the rules " +
Utils.toJSONString(Utils.getDeepCopy(session.expandedClauses, 4, true)));
}
session = suggester.getSession();
positions.add(new ReplicaPosition(shardName, ++idx, e.getKey(), op.getParams().get(NODE)));
}
}
}
} finally {
if (session != null && SESSION_REF.get() != null) SESSION_REF.get().updateSession(session);
policyMapping.remove();
}
return positions;
}
public static final int SESSION_EXPIRY = 180;//3 seconds
public static ThreadLocal<Long> REF_VERSION = new ThreadLocal<>();
public static class SessionRef {
private final AtomicLong myVersion = new AtomicLong(0);
AtomicInteger refCount = new AtomicInteger();
private Policy.Session session;
long lastUsedTime;
public SessionRef() {
}
public long getRefVersion(){
return myVersion.get();
}
Policy policy = new Policy(autoScalingJson);
Policy.Session session = policy.createSession(cdp);
for (String shardName : shardNames) {
for (int i = 0; i < repFactor; i++) {
Policy.Suggester suggester = session.getSuggester(ADDREPLICA)
.hint(Hint.COLL, collName)
.hint(Hint.SHARD, shardName);
if (nodesList != null) {
for (String nodeName : nodesList) {
suggester = suggester.hint(Hint.TARGET_NODE, nodeName);
}
public void decref(long version) {
synchronized (SessionRef.class) {
if (session == null) return;
if(myVersion.get() != version) return;
if (refCount.decrementAndGet() <= 0) {
session = null;
lastUsedTime = 0;
}
SolrRequest op = suggester.getOperation();
if (op == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No node can satisfy the rules "+ Utils.toJSONString(Utils.getDeepCopy(session.expandedClauses, 4, true)));
}
session = suggester.getSession();
positionMapping.get(shardName).add(op.getParams().get(CoreAdminParams.NODE));
}
}
return positionMapping;
public int getRefCount() {
return refCount.get();
}
public Policy.Session get() {
synchronized (SessionRef.class) {
if (session == null) return null;
if (TimeUnit.SECONDS.convert(System.nanoTime() - lastUsedTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
session = null;
return null;
} else {
REF_VERSION.set(myVersion.get());
refCount.incrementAndGet();
return session;
}
}
}
public Policy.Session initOrGet(ClusterDataProvider cdp, Policy policy) {
synchronized (SessionRef.class) {
Policy.Session session = get();
if (session != null) return session;
this.session = policy.createSession(cdp);
myVersion.incrementAndGet();
lastUsedTime = System.nanoTime();
REF_VERSION.set(myVersion.get());
refCount.set(1);
return this.session;
}
}
private void updateSession(Policy.Session session) {
this.session = session;
lastUsedTime = System.nanoTime();
}
}
public static void clearFlagAndDecref(SessionRef policySessionRef) {
Long refVersion = REF_VERSION.get();
if (refVersion != null) policySessionRef.decref(refVersion);
REF_VERSION.remove();
}
public static ThreadLocal<SessionRef> SESSION_REF = new ThreadLocal<>();
}

View File

@ -18,22 +18,32 @@
package org.apache.solr.client.solrj.cloud.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.List;
import java.util.Map;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class Preference implements MapWriter {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
final Policy.SortParam name;
Integer precision;
final Policy.Sort sort;
Preference next;
public int idx;
final int idx;
private final Map original;
Preference(Map<String, Object> m) {
public Preference(Map<String, Object> m) {
this(m, 0);
}
public Preference(Map<String, Object> m, int idx) {
this.idx = idx;
this.original = Utils.getDeepCopy(m,3);
sort = Policy.Sort.get(m);
name = Policy.SortParam.get(m.get(sort.name()).toString());
@ -43,7 +53,7 @@ public class Preference implements MapWriter {
throw new RuntimeException("precision must be a positive value ");
}
if(precision< name.min || precision> name.max){
throw new RuntimeException(StrUtils.formatString("invalid precision value {0} must lie between {1} and {1}",
throw new RuntimeException(StrUtils.formatString("invalid precision value {0} , must lie between {1} and {2}",
precision, name.min, name.max ) );
}
@ -53,6 +63,9 @@ public class Preference implements MapWriter {
// recursive, it uses the precision to tie & when there is a tie use the next preference to compare
// in non-recursive mode, precision is not taken into consideration and sort is done on actual value
int compare(Row r1, Row r2, boolean useApprox) {
if (!r1.isLive && !r2.isLive) return 0;
if (!r1.isLive) return -1;
if (!r2.isLive) return 1;
Object o1 = useApprox ? r1.cells[idx].approxVal : r1.cells[idx].val;
Object o2 = useApprox ? r2.cells[idx].approxVal : r2.cells[idx].val;
int result = 0;
@ -68,6 +81,9 @@ public class Preference implements MapWriter {
void setApproxVal(List<Row> tmpMatrix) {
Object prevVal = null;
for (Row row : tmpMatrix) {
if (!row.isLive) {
continue;
}
prevVal = row.cells[idx].approxVal =
(prevVal == null || Double.compare(Math.abs(((Number) prevVal).doubleValue() - ((Number) row.cells[idx].val).doubleValue()), precision) > 0) ?
row.cells[idx].val :
@ -83,7 +99,27 @@ public class Preference implements MapWriter {
}
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Preference that = (Preference) o;
if (idx != that.idx) return false;
if (getName() != that.getName()) return false;
if (precision != null ? !precision.equals(that.precision) : that.precision != null) return false;
if (sort != that.sort) return false;
if (next != null ? !next.equals(that.next) : that.next != null) return false;
return original.equals(that.original);
}
public Policy.SortParam getName() {
return name;
}
@Override
public String toString() {
return Utils.toJSONString(this);
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.client.solrj.cloud.autoscaling;
import java.io.IOException;
import java.util.List;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.Replica;
class ReplicaCount extends Number implements MapWriter {
long nrt, tlog, pull;
public long total() {
return nrt + tlog + pull;
}
@Override
public int intValue() {
return (int) total();
}
@Override
public long longValue() {
return total();
}
@Override
public float floatValue() {
return total();
}
@Override
public double doubleValue() {
return total();
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put(Replica.Type.NRT.name(), nrt);
ew.put(Replica.Type.PULL.name(), pull);
ew.put(Replica.Type.TLOG.name(), tlog);
ew.put("count", total());
}
public Long getVal(Replica.Type type) {
if (type == null) return total();
switch (type) {
case NRT:
return nrt;
case PULL:
return pull;
case TLOG:
return tlog;
}
return total();
}
public void increment(List<ReplicaInfo> infos) {
if (infos == null) return;
for (ReplicaInfo info : infos) {
switch (info.type) {
case NRT:
nrt++;
break;
case PULL:
pull++;
break;
case TLOG:
tlog++;
break;
default:
nrt++;
}
}
}
}

View File

@ -0,0 +1,64 @@
/*
* 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.client.solrj.cloud.autoscaling;
import java.io.IOException;
import java.util.Map;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.Replica;
public class ReplicaInfo implements MapWriter {
final String name;
String core, collection, shard;
Replica.Type type;
Map<String, Object> variables;
public ReplicaInfo(String name, String coll, String shard, Replica.Type type, Map<String, Object> vals) {
this.name = name;
this.variables = vals;
this.collection = coll;
this.shard = shard;
this.type = type;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put(name, (MapWriter) ew1 -> {
if (variables != null) {
for (Map.Entry<String, Object> e : variables.entrySet()) {
ew1.put(e.getKey(), e.getValue());
}
}
if (type != null) ew1.put("type", type.toString());
});
}
public String getCore() {
return core;
}
public String getCollection() {
return collection;
}
public String getShard() {
return shard;
}
}

View File

@ -19,14 +19,15 @@ package org.apache.solr.client.solrj.cloud.autoscaling;
import java.io.IOException;
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 org.apache.solr.client.solrj.cloud.autoscaling.Policy.ReplicaInfo;
import org.apache.solr.common.IteratorWriter;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.Utils;
@ -37,15 +38,16 @@ public class Row implements MapWriter {
public final String node;
final Cell[] cells;
public Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas;
List<Clause> violations = new ArrayList<>();
boolean anyValueMissing = false;
boolean isLive = true;
public Row(String node, List<String> params, ClusterDataProvider dataProvider) {
collectionVsShardVsReplicas = dataProvider.getReplicaInfo(node, params);
if (collectionVsShardVsReplicas == null) collectionVsShardVsReplicas = new HashMap<>();
this.node = node;
cells = new Cell[params.size()];
Map<String, Object> vals = dataProvider.getNodeValues(node, params);
isLive = dataProvider.getNodes().contains(node);
Map<String, Object> vals = isLive ? dataProvider.getNodeValues(node, params) : Collections.emptyMap();
for (int i = 0; i < params.size(); i++) {
String s = params.get(i);
cells[i] = new Cell(i, s, Clause.validate(s,vals.get(s), false));
@ -54,8 +56,10 @@ public class Row implements MapWriter {
}
}
public Row(String node, Cell[] cells, boolean anyValueMissing, Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas, List<Clause> violations) {
public Row(String node, Cell[] cells, boolean anyValueMissing, Map<String,
Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas, boolean isLive) {
this.node = node;
this.isLive = isLive;
this.cells = new Cell[cells.length];
for (int i = 0; i < this.cells.length; i++) {
this.cells[i] = cells[i].copy();
@ -63,7 +67,6 @@ public class Row implements MapWriter {
}
this.anyValueMissing = anyValueMissing;
this.collectionVsShardVsReplicas = collectionVsShardVsReplicas;
this.violations = violations;
}
@Override
@ -75,7 +78,7 @@ public class Row implements MapWriter {
}
Row copy() {
return new Row(node, cells, anyValueMissing, Utils.getDeepCopy(collectionVsShardVsReplicas, 3), new ArrayList<>(violations));
return new Row(node, cells, anyValueMissing, Utils.getDeepCopy(collectionVsShardVsReplicas, 3), isLive);
}
Object getVal(String name) {
@ -89,28 +92,42 @@ public class Row implements MapWriter {
}
// this adds a replica to the replica info
public Row addReplica(String coll, String shard) {
public Row addReplica(String coll, String shard, Replica.Type type) {
Row row = copy();
Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.computeIfAbsent(coll, k -> new HashMap<>());
List<ReplicaInfo> replicas = c.computeIfAbsent(shard, k -> new ArrayList<>());
replicas.add(new ReplicaInfo("" + new Random().nextInt(1000) + 1000, coll, shard, new HashMap<>()));
replicas.add(new ReplicaInfo("" + new Random().nextInt(1000) + 1000, coll, shard, type, new HashMap<>()));
for (Cell cell : row.cells) {
if (cell.name.equals("cores")) cell.val = ((Number) cell.val).longValue() + 1;
if (cell.name.equals("cores")) {
cell.val = cell.val == null ? 0 : ((Number) cell.val).longValue() + 1;
}
}
return row;
}
public Pair<Row, ReplicaInfo> removeReplica(String coll, String shard) {
public Pair<Row, ReplicaInfo> removeReplica(String coll, String shard, Replica.Type type) {
Row row = copy();
Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.get(coll);
if (c == null) return null;
List<ReplicaInfo> s = c.get(shard);
if (s == null || s.isEmpty()) return null;
for (Cell cell : row.cells) {
if (cell.name.equals("cores")) cell.val = ((Number) cell.val).longValue() -1;
List<ReplicaInfo> r = c.get(shard);
if (r == null) return null;
int idx = -1;
for (int i = 0; i < r.size(); i++) {
ReplicaInfo info = r.get(i);
if (type == null || info.type == type) {
idx = i;
break;
}
}
return new Pair(row, s.remove(0));
if(idx == -1) return null;
for (Cell cell : row.cells) {
if (cell.name.equals("cores")) {
cell.val = cell.val == null ? 0 : ((Number) cell.val).longValue() - 1;
}
}
return new Pair(row, r.remove(idx));
}

View File

@ -0,0 +1,30 @@
/*
* 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.client.solrj.cloud.autoscaling;
/**
* Enum that represents the stages of trigger event processing.
*/
public enum TriggerEventProcessorStage {
WAITING,
STARTED,
ABORTED,
SUCCEEDED,
FAILED,
BEFORE_ACTION,
AFTER_ACTION
}

View File

@ -0,0 +1,31 @@
/*
* 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.client.solrj.cloud.autoscaling;
/**
* Enum that represents trigger event types.
*/
public enum TriggerEventType {
NODEADDED,
NODELOST,
REPLICALOST,
MANUAL,
SCHEDULED,
SEARCHRATE,
INDEXRATE,
INVALID
}

View File

@ -73,8 +73,8 @@ public class HttpClientUtil {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final int DEFAULT_CONNECT_TIMEOUT = 60000;
private static final int DEFAULT_SO_TIMEOUT = 600000;
public static final int DEFAULT_CONNECT_TIMEOUT = 60000;
public static final int DEFAULT_SO_TIMEOUT = 600000;
private static final int VALIDATE_AFTER_INACTIVITY_DEFAULT = 3000;
private static final int EVICT_IDLE_CONNECTIONS_DEFAULT = 50000;

View File

@ -498,7 +498,6 @@ public class HttpSolrClient extends SolrClient {
throw new SolrServerException("Unsupported method: " + request.getMethod());
}
private static final List<String> errPath = Arrays.asList("metadata", "error-class");//Utils.getObjectByPath(err, false,"metadata/error-class")
protected NamedList<Object> executeMethod(HttpRequestBase method, final ResponseParser processor, final boolean isV2Api) throws SolrServerException {

View File

@ -33,7 +33,7 @@ import java.util.Set;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy.ReplicaInfo;
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
import org.apache.solr.client.solrj.request.GenericSolrRequest;
import org.apache.solr.client.solrj.response.SimpleSolrResponse;
import org.apache.solr.common.MapWriter;
@ -54,10 +54,13 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.client.solrj.cloud.autoscaling.Clause.METRICS_PREFIX;
/**
* Class that implements {@link ClusterStateProvider} accepting a SolrClient
*/
public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final CloudSolrClient solrClient;
private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> data = new HashMap<>();
@ -75,13 +78,10 @@ public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
DocCollection coll = ref.get();
if (coll == null) return;
coll.forEachReplica((shard, replica) -> {
Map<String, Map<String, List<ReplicaInfo>>> nodeData = data.get(replica.getNodeName());
if (nodeData == null) data.put(replica.getNodeName(), nodeData = new HashMap<>());
Map<String, List<ReplicaInfo>> collData = nodeData.get(collName);
if (collData == null) nodeData.put(collName, collData = new HashMap<>());
List<ReplicaInfo> replicas = collData.get(shard);
if (replicas == null) collData.put(shard, replicas = new ArrayList<>());
replicas.add(new ReplicaInfo(replica.getName(), collName, shard, new HashMap<>()));
Map<String, Map<String, List<ReplicaInfo>>> nodeData = data.computeIfAbsent(replica.getNodeName(), k -> new HashMap<>());
Map<String, List<ReplicaInfo>> collData = nodeData.computeIfAbsent(collName, k -> new HashMap<>());
List<ReplicaInfo> replicas = collData.computeIfAbsent(shard, k -> new ArrayList<>());
replicas.add(new ReplicaInfo(replica.getName(), collName, shard, replica.getType(), new HashMap<>()));
});
});
}
@ -103,7 +103,7 @@ public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
@Override
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return data.getOrDefault(node, Collections.emptyMap());//todo fill other details
return data.computeIfAbsent(node, s -> Collections.emptyMap());//todo fill other details
}
@Override
@ -167,7 +167,28 @@ public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
@Override
protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
ClientSnitchCtx snitchContext = (ClientSnitchCtx) ctx;
readSysProps(solrNode, requestedTags, snitchContext);
Map<String, String> metricsKeyVsTag = new HashMap<>();
for (String tag : requestedTags) {
if (tag.startsWith(SYSPROP)) {
metricsKeyVsTag.put("solr.jvm:system.properties:" + tag.substring(SYSPROP.length()), tag);
} else if (tag.startsWith(METRICS_PREFIX)) {
metricsKeyVsTag.put(tag.substring(METRICS_PREFIX.length()), tag);
}
}
if (!metricsKeyVsTag.isEmpty()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.add("key", metricsKeyVsTag.keySet().toArray(new String[metricsKeyVsTag.size()]));
try {
SimpleSolrResponse rsp = snitchContext.invoke(solrNode, CommonParams.METRICS_PATH, params);
metricsKeyVsTag.forEach((key, tag) -> {
Object v = Utils.getObjectByPath(rsp.nl, true, Arrays.asList("metrics", key));
if (v != null) snitchContext.getTags().put(tag, v);
});
} catch (Exception e) {
log.warn("could not get tags from node " + solrNode, e);
}
}
Set<String> groups = new HashSet<>();
List<String> prefixes = new ArrayList<>();
if (requestedTags.contains(DISK)) {
@ -219,38 +240,5 @@ public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
private void readSysProps(String solrNode, Set<String> requestedTags, ClientSnitchCtx snitchContext) {
List<String> prefixes = null;
ModifiableSolrParams params;
List<String> sysProp = null;
for (String tag : requestedTags) {
if (!tag.startsWith(SYSPROP)) continue;
if (sysProp == null) {
prefixes = new ArrayList<>();
sysProp = new ArrayList<>();
prefixes.add("system.properties");
}
sysProp.add(tag.substring(SYSPROP.length()));
}
if (sysProp == null) return;
params = new ModifiableSolrParams();
params.add("prefix", StrUtils.join(prefixes, ','));
for (String s : sysProp) params.add("property", s);
try {
SimpleSolrResponse rsp = snitchContext.invoke(solrNode, CommonParams.METRICS_PATH, params);
Map m = rsp.nl.asMap(6);
for (String s : sysProp) {
Object v = Utils.getObjectByPath(m, true,
Arrays.asList("metrics", "solr.jvm", "system.properties", s));
if (v != null) snitchContext.getTags().put("sysprop." + s, v);
}
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
}
}

View File

@ -18,6 +18,7 @@ package org.apache.solr.client.solrj.request;
import java.io.IOException;
import java.util.Collection;
import java.util.Iterator;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
@ -32,6 +33,7 @@ import org.apache.solr.client.solrj.V2RequestSupport;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.Replica;
@ -57,7 +59,7 @@ import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SE
*
* @since solr 4.5
*/
public abstract class CollectionAdminRequest<T extends CollectionAdminResponse> extends SolrRequest<T> implements V2RequestSupport {
public abstract class CollectionAdminRequest<T extends CollectionAdminResponse> extends SolrRequest<T> implements V2RequestSupport, MapWriter {
protected final CollectionAction action;
@ -97,6 +99,22 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
}
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put("class", this.getClass().getName());
ew.put("method", getMethod().toString());
SolrParams params = getParams();
if (params != null) {
for (Iterator<String> it = params.getParameterNamesIterator(); it.hasNext(); ) {
final String name = it.next();
final String [] values = params.getParams(name);
for (String value : values) {
ew.put("params." + name, value);
}
}
}
}
/**
* Base class for asynchronous collection admin requests
*/

View File

@ -20,7 +20,9 @@ package org.apache.solr.common;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
/**
@ -38,6 +40,25 @@ public interface MapWriter extends MapSerializable {
public EntryWriter put(String k, Object v) throws IOException {
if (v instanceof MapWriter) v = ((MapWriter) v).toMap(new LinkedHashMap<>());
if (v instanceof IteratorWriter) v = ((IteratorWriter) v).toList(new ArrayList<>());
if (v instanceof Iterable) {
List lst = new ArrayList();
for (Object vv : (Iterable)v) {
if (vv instanceof MapWriter) vv = ((MapWriter) vv).toMap(new LinkedHashMap<>());
if (vv instanceof IteratorWriter) vv = ((IteratorWriter) vv).toList(new ArrayList<>());
lst.add(vv);
}
v = lst;
}
if (v instanceof Map) {
Map map = new LinkedHashMap();
for (Map.Entry<?, ?> entry : ((Map<?, ?>)v).entrySet()) {
Object vv = entry.getValue();
if (vv instanceof MapWriter) vv = ((MapWriter) vv).toMap(new LinkedHashMap<>());
if (vv instanceof IteratorWriter) vv = ((IteratorWriter) vv).toList(new ArrayList<>());
map.put(entry.getKey(), vv);
}
v = map;
}
map.put(k, v);
return this;
}

View File

@ -25,10 +25,12 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
import org.noggit.JSONWriter;
@ -339,7 +341,20 @@ public class ClusterState implements JSONWriter.Writable {
public Map<String, CollectionRef> getCollectionStates() {
return immutableCollectionStates;
}
public void forEachCollection(Consumer<DocCollection> consumer) {
collectionStates.forEach((s, collectionRef) -> {
try {
consumer.accept(collectionRef.get());
} catch (SolrException e) {
if (e.getCause() instanceof KeeperException.NoNodeException) {
//don't do anything. This collection does not exist
} else{
throw e;
}
}
});
}
public static class CollectionRef {
protected final AtomicInteger gets = new AtomicInteger();
private final DocCollection coll;

View File

@ -253,16 +253,5 @@ public class ClusterStateUtil {
return success;
}
public static boolean isAutoAddReplicas(ZkStateReader reader, String collection) {
ClusterState clusterState = reader.getClusterState();
if (clusterState != null) {
DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection != null) {
return docCollection.getAutoAddReplicas();
}
}
return false;
}
}

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