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 NanoTimeSource nanoTimeSource;
public interface NanoTimeSource {
long getTime();
}
private static class DefaultNanoTimeSource implements NanoTimeSource {
@Override
public long getTime() {
return System.nanoTime();
}
}
private final TimeSource timeSource;
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,11 +97,32 @@ 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) {
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);
if (!clusterState.liveNodesContain(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,7 +281,7 @@ public class Assign {
(List<Map>) message.get(SNITCH),
new HashMap<>(),//this is a new collection. So, there are no nodes in any shard
nodeList,
coreContainer.get(),
ocmh.overseer.getZkController().getCoreContainer(),
clusterState);
Map<ReplicaPosition, String> nodeMappings = replicaAssigner.getNodeMappings();
@ -293,8 +289,18 @@ public class Assign {
.map(e -> new ReplicaPosition(e.getKey().shard, e.getKey().index, e.getKey().type, e.getValue()))
.collect(Collectors.toList());
} 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, policyName, zkStateReader, nodeList);
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,6 +49,7 @@ 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);
@ -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,33 +84,47 @@ 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;
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 {
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++;
}
}
}
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;
} 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;
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);
@ -109,15 +132,15 @@ public class CreateShardCmd implements Cmd {
ZkNodeProps addReplicasProps = new ZkNodeProps(
COLLECTION_PROP, collectionName,
SHARD_ID_PROP, sliceName,
CoreAdminParams.REPLICA_TYPE, typeToCreate.name(),
ZkStateReader.REPLICA_TYPE, position.type.name(),
CoreAdminParams.NODE, nodeName,
CoreAdminParams.NAME, coreName);
Map<String, Object> propertyParams = new HashMap<>();
ocmh.addPropertyParams(message, propertyParams);;
ocmh.addPropertyParams(message, propertyParams);
addReplicasProps = addReplicasProps.plus(propertyParams);
if(async!=null) addReplicasProps.getProperties().put(ASYNC, async);
if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
final NamedList addResult = new NamedList();
ocmh.addReplica(zkStateReader.getClusterState(), addReplicasProps, addResult, ()-> {
ocmh.addReplica(zkStateReader.getClusterState(), addReplicasProps, addResult, () -> {
countDownLatch.countDown();
Object addResultFailure = addResult.get("failure");
if (addResultFailure != null) {
@ -137,6 +160,9 @@ public class CreateShardCmd implements Cmd {
}
});
}
} finally {
PolicyHelper.clearFlagAndDecref(ocmh.policySessionRef);
}
log.debug("Waiting for create shard action to complete");
countDownLatch.await(5, TimeUnit.MINUTES);
@ -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;
@ -470,7 +471,7 @@ public class Overseer implements Closeable {
private OverseerThread updaterThread;
private OverseerThread arfoThread;
private OverseerThread triggerThread;
private final ZkStateReader reader;
@ -522,15 +523,13 @@ public class Overseer implements Closeable {
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();
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,12 +216,12 @@ 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,
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);
@ -234,7 +235,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
} 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, " +
throw new SolrException(ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " +
Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
}
@ -344,6 +345,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
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());
AutoScalingConfig autoScalingConf = container.getZkController().zkStateReader.getAutoScalingConfig();
if (parts.size() == 2) {
rsp.getValues().addAll(map);
} else if (parts.size() == 3 && "diagnostics".equals(parts.get(2))) {
handleDiagnostics(rsp, map);
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);
}
checkErr(op);
zkSetPolicies(container.getZkController().getZkStateReader(), policyName, null);
rsp.getValues().add("result", "success");
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()) {
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 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;
}
mergedPolicies.putAll(newPolicies);
Policy p = currentConfig.getPolicy().withPolicies(mergedPolicies).withParams(params);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
rsp.getValues().add("result", "success");
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;
}
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);
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 {
policies.remove(policyBeRemoved);
newTriggers.put(entry.getKey(), entry.getValue());
}
loaded = loaded.plus("policies", policies);
verifyAutoScalingConf(loaded.getProperties());
} else {
newTriggers.put(entry.getKey(), entry.getValue());
}
}
rsp.getValues().add("changed", changed);
if (!changed.isEmpty()) {
currentConfig = currentConfig.withTriggerConfigs(newTriggers);
}
return currentConfig;
}
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;
}
break;
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;
}
}
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());
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;
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;
}
break;
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;
@ -52,8 +48,8 @@ public class CloudConfig {
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) {
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 {
@ -1057,6 +1063,21 @@ public class CoreContainer {
}
}
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
* strategies can be specified via system properties "-DCoreInitFailedAction={fromleader, none}"

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;
@ -42,34 +42,37 @@ 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,
@ -1184,6 +1192,18 @@ public class OverseerTest extends SolrTestCaseJ4 {
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,7 +75,9 @@ 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);
@ -258,74 +264,12 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
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,11 +283,15 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
}
SolrParams queryAll = new SolrQuery("*:*");
cloudClient.setDefaultCollection(collection);
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());
// 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 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) {
assertEquals(1, slice.getReplicas().size());
// 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");
}
}
}
}
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());
}
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

@ -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,23 +36,24 @@ 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'");
}
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(coll, shard);
tmpRow.violations.clear();
Row tmpRow = row.addReplica(shard.first(), shard.second(), type);
List<Clause.Violation> errs = testChangedMatrix(strict, getModifiedMatrix(getMatrix(), tmpRow, i));
if(!containsNewErrors(errs)) {
if(isLessSerious(errs, leastSeriousViolation)){
if (!containsNewErrors(errs)) {
if (isLessSerious(errs, leastSeriousViolation)) {
leastSeriousViolation = errs;
targetNodeIndex = i;
}
@ -56,11 +61,13 @@ class AddReplicaSuggester extends Suggester {
}
if (targetNodeIndex != null) {// there are no rule violations
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(coll, shard));
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(shard.first(), shard.second(), type));
return CollectionAdminRequest
.addReplicaToShard(coll, shard)
.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) {
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;
}
if (v == 0) v = compareTypes(this.type, that.type);
return v;
} else {
return 0;
}
} catch (NullPointerException e) {
throw e;
}
}
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));
}
TestStatus match(Object testVal) {
return op.match(this.val, testVal);
}
public boolean isPass(Object inputVal) {
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;
}
}
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() {
@ -178,7 +206,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
}
}
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,15 +388,65 @@ 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 isRuleVal is this provided in the rule
@ -372,38 +454,16 @@ public class Clause implements MapWriter, Comparable<Clause> {
*/
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) {
@ -452,6 +512,8 @@ public class Clause implements MapWriter, Comparable<Clause> {
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,18 +149,37 @@ 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
@ -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);
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(coll))) {
session.addClausesForCollection(session.dataProvider, coll);
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,
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 repFactor,
int nrtReplicas,
int tlogReplicas,
int pullReplicas,
List<String> nodesList) {
Map<String, List<String>> positionMapping = new HashMap<>();
for (String shardName : shardNames) positionMapping.put(shardName, new ArrayList<>(repFactor));
if (optionalPolicyMapping != null) {
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,22 +68,30 @@ 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);
Policy policy = new Policy(autoScalingJson);
Policy.Session session = policy.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) {
for (int i = 0; i < repFactor; i++) {
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.COLL, collName)
.hint(Hint.SHARD, shardName);
.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);
@ -84,13 +99,95 @@ public class PolicyHelper {
}
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)));
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));
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();
}
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;
}
}
}
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

@ -254,15 +254,4 @@ 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;
}
}

View File

@ -36,10 +36,10 @@ import org.noggit.JSONWriter;
import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_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.TLOG_REPLICAS;
/**
* Models a Collection in zookeeper (but that Java name is obviously taken, hence "DocCollection")
@ -244,7 +244,8 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
if (maxShardsPerNode == null) {
throw new SolrException(ErrorCode.BAD_REQUEST, MAX_SHARDS_PER_NODE + " is not in the cluster state.");
}
return maxShardsPerNode;
//maxShardsPerNode=0 when policy is used. This variable is not important then
return maxShardsPerNode == 0 ? Integer.MAX_VALUE : maxShardsPerNode;
}
public String getZNode(){

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