mirror of https://github.com/apache/lucene.git
Merge branch 'feature/autoscaling' of https://git-wip-us.apache.org/repos/asf/lucene-solr
This commit is contained in:
commit
0e679e4edf
|
@ -0,0 +1 @@
|
|||
2b8c8fbd740164d220ca7d18605b8b2092e163e9
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
};
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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() + "]";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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";
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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 {
|
||||
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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}"
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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 {
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -1021,4 +1021,3 @@ public class ShardSplitTest extends BasicDistributedZkTest {
|
|||
return client;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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!");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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"));
|
||||
}
|
||||
}
|
|
@ -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
|
@ -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)));
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<>();
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue