SOLR-14656: Removing Autoscaling Framework

The following were removed:
 *  Autoscaling policy, triggers etc.
 *  withCollection handling
 *  UTILIZENODE command
 *  Sim framework
 *  Suggestions tab in UI
 *  Reference guide pages for autoscaling
 *  autoAddReplicas feature
 *  UTILIZENODE
This commit is contained in:
Ishan Chattopadhyaya 2020-07-20 17:47:01 +05:30
parent 1d143744bd
commit cc0c111949
287 changed files with 606 additions and 80223 deletions

View File

@ -36,6 +36,8 @@ Improvements
Other Changes
----------------------
* SOLR-14656: Autoscaling framework removed (Ishan Chattopadhyaya, noble, Ilan Ginzburg)
* LUCENE-9391: Upgrade HPPC to 0.8.2. (Haoyu Zhai)
* SOLR-10288: Remove non-minified JavaScript from the webapp. (Erik Hatcher, marcussorealheis)

View File

@ -330,7 +330,7 @@ function print_usage() {
if [ -z "$CMD" ]; then
echo ""
echo "Usage: solr COMMAND OPTIONS"
echo " where COMMAND is one of: start, stop, restart, status, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, autoscaling, export"
echo " where COMMAND is one of: start, stop, restart, status, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, export"
echo ""
echo " Standalone server example (start Solr running in the background on port 8984):"
echo ""
@ -1417,11 +1417,6 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then
fi
if [[ "$SCRIPT_CMD" == "autoscaling" ]]; then
run_tool autoscaling $@
exit $?
fi
if [[ "$SCRIPT_CMD" == "export" ]]; then
run_tool export $@
exit $?

View File

@ -224,7 +224,6 @@ IF "%1"=="version" goto get_version
IF "%1"=="-v" goto get_version
IF "%1"=="-version" goto get_version
IF "%1"=="assert" goto run_assert
IF "%1"=="autoscaling" goto run_autoscaling
IF "%1"=="export" goto run_export
IF "%1"=="package" goto run_package
@ -303,7 +302,7 @@ goto done
:script_usage
@echo.
@echo Usage: solr COMMAND OPTIONS
@echo where COMMAND is one of: start, stop, restart, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, autoscaling, export
@echo where COMMAND is one of: start, stop, restart, healthcheck, create, create_core, create_collection, delete, version, zk, auth, assert, config, export
@echo.
@echo Standalone server example (start Solr running in the background on port 8984):
@echo.
@ -1454,13 +1453,6 @@ if errorlevel 1 (
)
goto done
:run_autoscaling
"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
-Dlog4j.configurationFile="file:///%DEFAULT_SERVER_DIR%\resources\log4j2-console.xml" ^
-classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^
org.apache.solr.util.SolrCLI %*
goto done:
:run_export
"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^
-Dlog4j.configurationFile="file:///%DEFAULT_SERVER_DIR%\resources\log4j2-console.xml" ^

View File

@ -235,7 +235,6 @@ public class CloudUtil {
* Return a {@link CollectionStatePredicate} that returns true if a collection has the expected
* number of shards and replicas.
* <p>Note: for shards marked as inactive the current Solr behavior is that replicas remain active.
* {@link org.apache.solr.cloud.autoscaling.sim.SimCloudManager} follows this behavior.</p>
* @param expectedShards expected number of shards
* @param expectedReplicas expected number of active replicas per shard
* @param withInactive if true then count also inactive shards

View File

@ -39,7 +39,6 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
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;
@ -83,16 +82,14 @@ import com.codahale.metrics.Timer;
* collections, shards, replicas and setting various properties.</p>
*
* <p>The <b>Overseer</b> is a single elected node in the SolrCloud cluster that is in charge of interactions with
* ZooKeeper that require global synchronization. It also hosts the Collection API implementation and the
* Autoscaling framework.</p>
* ZooKeeper that require global synchronization. </p>
*
* <p>The Overseer deals with:</p>
* <ul>
* <li>Cluster State updates, i.e. updating Collections' <code>state.json</code> files in ZooKeeper, see {@link ClusterStateUpdater},</li>
* <li>Collection API implementation, including Autoscaling replica placement computation, see
* <li>Collection API implementation, see
* {@link OverseerCollectionConfigSetProcessor} and {@link OverseerCollectionMessageHandler} (and the example below),</li>
* <li>Updating Config Sets, see {@link OverseerCollectionConfigSetProcessor} and {@link OverseerConfigSetMessageHandler},</li>
* <li>Autoscaling triggers, see {@link org.apache.solr.cloud.autoscaling.OverseerTriggerThread}.</li>
* </ul>
*
* <p>The nodes in the cluster communicate with the Overseer over queues implemented in ZooKeeper. There are essentially
@ -644,14 +641,8 @@ public class Overseer implements SolrCloseable {
ccThread = new OverseerThread(ccTg, overseerCollectionConfigSetProcessor, "OverseerCollectionConfigSetProcessor-" + id);
ccThread.setDaemon(true);
ThreadGroup triggerThreadGroup = new ThreadGroup("Overseer autoscaling triggers");
OverseerTriggerThread trigger = new OverseerTriggerThread(zkController.getCoreContainer().getResourceLoader(),
zkController.getSolrCloudManager());
triggerThread = new OverseerThread(triggerThreadGroup, trigger, "OverseerAutoScalingTriggerThread-" + id);
updaterThread.start();
ccThread.start();
triggerThread.start();
systemCollectionCompatCheck(new BiConsumer<String, Object>() {
boolean firstPair = true;

View File

@ -60,7 +60,6 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
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.AlreadyClosedException;
@ -99,7 +98,6 @@ import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.URLUtil;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloseHook;
@ -897,13 +895,8 @@ public class ZkController implements Closeable {
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.SOLR_SECURITY_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
bootstrapDefaultConfigSet(zkClient);
}
@ -1044,29 +1037,6 @@ public class ZkController implements Closeable {
}
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) {
byte[] json = Utils.toJSON(Collections.singletonMap("timestamp", getSolrCloudManager().getTimeSource().getEpochTimeNs()));
for (String n : oldNodes) {
String path = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + n;
try {
zkClient.create(path, json, 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);
}
}
}
return false;
};
zkStateReader.registerLiveNodesListener(listener);
@ -1152,18 +1122,9 @@ public class ZkController implements Closeable {
}
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);
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
byte[] json = Utils.toJSON(Collections.singletonMap("timestamp", TimeSource.NANO_TIME.getEpochTimeNs()));
ops.add(Op.create(nodeAddedPath, json, zkClient.getZkACLProvider().getACLsToAdd(nodeAddedPath), CreateMode.EPHEMERAL));
}
zkClient.multi(ops, true);
}
@ -1173,11 +1134,9 @@ public class ZkController implements Closeable {
}
String nodeName = getNodeName();
String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;
String nodeAddedPath = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName;
log.info("Remove node as live in ZooKeeper:{}", nodePath);
List<Op> ops = new ArrayList<>(2);
ops.add(Op.delete(nodePath, -1));
ops.add(Op.delete(nodeAddedPath, -1));
try {
zkClient.multi(ops, true);

View File

@ -44,12 +44,10 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.ActiveReplicaWatcher;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
@ -148,18 +146,11 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
List<CreateReplica> createReplicas;
try {
createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount, sessionWrapper)
List<CreateReplica> createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
.stream()
.map(replicaPosition -> assignReplicaDetails(ocmh.cloudManager, clusterState, message, replicaPosition))
.collect(Collectors.toList());
} finally {
if (sessionWrapper.get() != null) {
sessionWrapper.get().release();
}
}
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
ZkStateReader zkStateReader = ocmh.zkStateReader;
@ -339,8 +330,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
String collectionName, ZkNodeProps message,
EnumMap<Replica.Type, Integer> replicaTypeVsCount,
AtomicReference< PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException {
EnumMap<Replica.Type, Integer> replicaTypeVsCount) throws IOException, InterruptedException {
boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
boolean skipNodeAssignment = message.getBool(CollectionAdminParams.SKIP_NODE_ASSIGNMENT, false);
String sliceName = message.getStr(SHARD_ID_PROP);
@ -365,7 +355,6 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
positions = Assign.getNodesForNewReplicas(clusterState, collection.getName(), sliceName, numNrtReplicas,
numTlogReplicas, numPullReplicas, createNodeSetStr, cloudManager);
sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
}
if (positions == null) {

View File

@ -16,6 +16,10 @@
*/
package org.apache.solr.cloud.api.collections;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
import static org.apache.solr.common.cloud.DocCollection.SNITCH;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
@ -29,19 +33,15 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Random;
import java.util.Set;
import java.util.stream.Collectors;
import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.BadVersionException;
import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.cloud.rule.ReplicaAssigner;
import org.apache.solr.cloud.rule.Rule;
import org.apache.solr.common.SolrException;
@ -52,19 +52,14 @@ import org.apache.solr.common.cloud.ReplicaPosition;
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.CollectionAdminParams;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
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 static org.apache.solr.client.solrj.cloud.autoscaling.Policy.POLICY;
import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.CREATE_NODE_SET;
import static org.apache.solr.common.cloud.DocCollection.SNITCH;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import com.google.common.collect.ImmutableMap;
public class Assign {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@ -255,57 +250,6 @@ public class Assign {
return nodeList;
}
/**
* <b>Note:</b> where possible, the {@link #usePolicyFramework(DocCollection, SolrCloudManager)} method should
* be used instead of this method
*
* @return true if autoscaling policy framework should be used for replica placement
*/
public static boolean usePolicyFramework(SolrCloudManager cloudManager) throws IOException, InterruptedException {
Objects.requireNonNull(cloudManager, "The SolrCloudManager instance cannot be null");
return usePolicyFramework(Optional.empty(), cloudManager);
}
/**
* @return true if auto scaling policy framework should be used for replica placement
* for this collection, otherwise false
*/
public static boolean usePolicyFramework(DocCollection collection, SolrCloudManager cloudManager)
throws IOException, InterruptedException {
Objects.requireNonNull(collection, "The DocCollection instance cannot be null");
Objects.requireNonNull(cloudManager, "The SolrCloudManager instance cannot be null");
return usePolicyFramework(Optional.of(collection), cloudManager);
}
@SuppressWarnings({"unchecked"})
private static boolean usePolicyFramework(Optional<DocCollection> collection, SolrCloudManager cloudManager) throws IOException, InterruptedException {
boolean useLegacyAssignment = true;
Map<String, Object> clusterProperties = cloudManager.getClusterStateProvider().getClusterProperties();
if (clusterProperties.containsKey(CollectionAdminParams.DEFAULTS)) {
Map<String, Object> defaults = (Map<String, Object>) clusterProperties.get(CollectionAdminParams.DEFAULTS);
Map<String, Object> collectionDefaults = (Map<String, Object>) defaults.getOrDefault(CollectionAdminParams.CLUSTER, Collections.emptyMap());
useLegacyAssignment = Boolean.parseBoolean(collectionDefaults.getOrDefault(CollectionAdminParams.USE_LEGACY_REPLICA_ASSIGNMENT, "true").toString());
}
if (!useLegacyAssignment) {
// if legacy assignment is not selected then autoscaling is always available through the implicit policy/preferences
return true;
}
// legacy assignment is turned on, which means we must look at the actual autoscaling config
// to determine whether policy framework can be used or not for this collection
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
// if no autoscaling configuration exists then obviously we cannot use the policy framework
if (autoScalingConfig.getPolicy().isEmpty()) return false;
// do custom preferences exist
if (!autoScalingConfig.getPolicy().hasEmptyPreferences()) return true;
// does a cluster policy exist
if (!autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) return true;
// finally we check if the current collection has a policy
return !collection.isPresent() || collection.get().getPolicyName() != null;
}
static class ReplicaCount {
public final String nodeName;
public int thisCollectionNodes = 0;
@ -358,44 +302,6 @@ public class Assign {
return assignStrategy.assign(cloudManager, assignRequest);
}
public static List<ReplicaPosition> getPositionsUsingPolicy(String collName, List<String> shardNames,
int nrtReplicas,
int tlogReplicas,
int pullReplicas,
String policyName, SolrCloudManager cloudManager,
List<String> nodesList) throws IOException, InterruptedException, AssignmentException {
log.debug("shardnames {} NRT {} TLOG {} PULL {} , policy {}, nodeList {}", shardNames, nrtReplicas, tlogReplicas, pullReplicas, policyName, nodesList);
List<ReplicaPosition> replicaPositions = null;
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
try {
Map<String, String> kvMap = Collections.singletonMap(collName, policyName);
replicaPositions = PolicyHelper.getReplicaLocations(
collName,
autoScalingConfig,
cloudManager,
kvMap,
shardNames,
nrtReplicas,
tlogReplicas,
pullReplicas,
nodesList);
return replicaPositions;
} catch (Exception e) {
throw new AssignmentException("Error getting replica locations : " + e.getMessage(), e);
} finally {
if (log.isTraceEnabled()) {
if (replicaPositions != null) {
if (log.isTraceEnabled()) {
log.trace("REPLICA_POSITIONS: {}", Utils.toJSONString(Utils.getDeepCopy(replicaPositions, 7, true)));
}
}
if (log.isTraceEnabled()) {
log.trace("AUTOSCALING_CONF: {}", Utils.toJSONString(autoScalingConfig));
}
}
}
}
static HashMap<String, ReplicaCount> getNodeNameVsShardCount(String collectionName,
ClusterState clusterState, List<String> createNodeList) {
HashMap<String, ReplicaCount> nodeNameVsShardCount = new HashMap<>();
@ -637,22 +543,6 @@ public class Assign {
}
}
public static class PolicyBasedAssignStrategy implements AssignStrategy {
public String policyName;
public PolicyBasedAssignStrategy(String policyName) {
this.policyName = policyName;
}
@Override
public List<ReplicaPosition> assign(SolrCloudManager solrCloudManager, AssignRequest assignRequest) throws Assign.AssignmentException, IOException, InterruptedException {
return Assign.getPositionsUsingPolicy(assignRequest.collectionName,
assignRequest.shardNames, assignRequest.numNrtReplicas,
assignRequest.numTlogReplicas, assignRequest.numPullReplicas,
policyName, solrCloudManager, assignRequest.nodes);
}
}
public static class AssignStrategyFactory {
public SolrCloudManager solrCloudManager;
@ -663,19 +553,16 @@ public class Assign {
public AssignStrategy create(ClusterState clusterState, DocCollection collection) throws IOException, InterruptedException {
@SuppressWarnings({"unchecked", "rawtypes"})
List<Map> ruleMaps = (List<Map>) collection.get("rule");
String policyName = collection.getStr(POLICY);
@SuppressWarnings({"rawtypes"})
List snitches = (List) collection.get(SNITCH);
Strategy strategy = null;
if ((ruleMaps == null || ruleMaps.isEmpty()) && !usePolicyFramework(collection, solrCloudManager)) {
strategy = Strategy.LEGACY;
} else if (ruleMaps != null && !ruleMaps.isEmpty()) {
if (ruleMaps != null && !ruleMaps.isEmpty()) {
strategy = Strategy.RULES;
} else {
strategy = Strategy.POLICY;
strategy = Strategy.LEGACY;
}
switch (strategy) {
case LEGACY:
return new LegacyAssignStrategy();
@ -683,15 +570,13 @@ public class Assign {
List<Rule> rules = new ArrayList<>();
for (Object map : ruleMaps) rules.add(new Rule((Map) map));
return new RulesBasedAssignStrategy(rules, snitches, clusterState);
case POLICY:
return new PolicyBasedAssignStrategy(policyName);
default:
throw new Assign.AssignmentException("Unknown strategy type: " + strategy);
}
}
private enum Strategy {
LEGACY, RULES, POLICY;
LEGACY, RULES;
}
}
}

View File

@ -31,15 +31,13 @@ import java.util.NoSuchElementException;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.NotEmptyException;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.BadVersionException;
import org.apache.solr.client.solrj.cloud.NotEmptyException;
import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
@ -147,7 +145,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
List<String> shardNames = populateShardNames(message, router);
checkReplicaTypes(message);
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper = new AtomicReference<>();
try {
@ -187,7 +185,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
List<ReplicaPosition> replicaPositions = null;
try {
replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, clusterState.getCollection(collectionName), message, shardNames, sessionWrapper);
replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState, clusterState.getCollection(collectionName), message, shardNames);
} catch (Assign.AssignmentException e) {
ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
@ -335,16 +333,13 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
throw ex;
} catch (Exception ex) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
} finally {
if (sessionWrapper.get() != null) sessionWrapper.get().release();
}
}
public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
DocCollection docCollection,
ZkNodeProps message,
List<String> shardNames,
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper) throws IOException, InterruptedException, Assign.AssignmentException {
List<String> shardNames) throws IOException, InterruptedException, Assign.AssignmentException {
final String collectionName = message.getStr(NAME);
// look at the replication factor and see if it matches reality
// if it does not, find best nodes to create more cores
@ -386,7 +381,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(cloudManager);
Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, docCollection);
replicaPositions = assignStrategy.assign(cloudManager, assignRequest);
sessionWrapper.set(PolicyHelper.getLastSessionWrapper(true));
}
return replicaPositions;
}

View File

@ -38,8 +38,8 @@ import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.BadVersionException;
import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
@ -91,7 +91,6 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.POLICY;
import static org.apache.solr.common.cloud.DocCollection.SNITCH;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
@ -146,9 +145,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
ZkStateReader.NRT_REPLICAS, "1",
ZkStateReader.TLOG_REPLICAS, "0",
ZkStateReader.PULL_REPLICAS, "0",
ZkStateReader.AUTO_ADD_REPLICAS, "false",
DocCollection.RULE, null,
POLICY, null,
SNITCH, null,
WITH_COLLECTION, null,
COLOCATED_WITH, null));
@ -238,7 +235,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
.put(ADDREPLICA, new AddReplicaCmd(this))
.put(MOVEREPLICA, new MoveReplicaCmd(this))
.put(REINDEXCOLLECTION, new ReindexCollectionCmd(this))
.put(UTILIZENODE, new UtilizeNodeCmd(this))
.put(RENAME, new RenameCmd(this))
.build()
;

View File

@ -35,7 +35,6 @@ import com.google.common.annotations.VisibleForTesting;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
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;
@ -110,10 +109,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
ZkStateReader.TLOG_REPLICAS,
ZkStateReader.REPLICATION_FACTOR,
"shards",
Policy.POLICY,
CollectionAdminParams.CREATE_NODE_SET_PARAM,
CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM,
ZkStateReader.AUTO_ADD_REPLICAS
CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM
);
private final OverseerCollectionMessageHandler ocmh;

View File

@ -27,9 +27,6 @@ import java.util.Locale;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.ActiveReplicaWatcher;
import org.apache.solr.common.SolrCloseableLatch;
import org.apache.solr.common.SolrException;
@ -101,7 +98,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
SolrCloseableLatch countDownLatch = new SolrCloseableLatch(sourceReplicas.size(), ocmh);
SolrCloseableLatch replicasToRecover = new SolrCloseableLatch(numLeaders, ocmh);
AtomicReference<PolicyHelper.SessionWrapper> sessionWrapperRef = new AtomicReference<>();
try {
for (ZkNodeProps sourceReplica : sourceReplicas) {
@SuppressWarnings({"rawtypes"})
@ -127,7 +123,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, clusterState.getCollection(sourceCollection));
targetNode = assignStrategy.assign(ocmh.cloudManager, assignRequest).get(0).node;
sessionWrapperRef.set(PolicyHelper.getLastSessionWrapper(true));
}
ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
if (async != null) msg.getProperties().put(ASYNC, async);
@ -185,8 +180,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
log.debug("Finished waiting for replicas to be added");
}
} finally {
PolicyHelper.SessionWrapper sw = sessionWrapperRef.get();
if (sw != null) sw.release();
}
// now wait for leader replicas to recover
log.debug("Waiting for {} leader replicas to recover", numLeaders);

View File

@ -35,7 +35,6 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.overseer.OverseerAction;
@ -110,15 +109,15 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
Object format = properties.get("stateFormat");
if (format != null && !"2".equals(format)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Collection " + backupCollection + " is in stateFormat=" + format +
" no longer supported in Solr 9 and above. It can't be restored. If it originates in Solr 8 you can restore" +
" it there, migrate it to stateFormat=2 and backup again, it will then be restorable on Solr 9");
" no longer supported in Solr 9 and above. It can't be restored. If it originates in Solr 8 you can restore" +
" it there, migrate it to stateFormat=2 and backup again, it will then be restorable on Solr 9");
}
String backupCollectionAlias = properties.getProperty(BackupManager.COLLECTION_ALIAS_PROP);
DocCollection backupCollectionState = backupMgr.readCollectionState(location, backupName, backupCollection);
// Get the Solr nodes to restore a collection.
final List<String> nodeList = Assign.getLiveOrLiveAndCreateNodeSetList(
zkStateReader.getClusterState().getLiveNodes(), message, OverseerCollectionMessageHandler.RANDOM);
zkStateReader.getClusterState().getLiveNodes(), message, OverseerCollectionMessageHandler.RANDOM);
int numShards = backupCollectionState.getActiveSlices().size();
@ -136,7 +135,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
int numPullReplicas = getInt(message, PULL_REPLICAS, backupCollectionState.getNumPullReplicas(), 0);
int totalReplicasPerShard = numNrtReplicas + numTlogReplicas + numPullReplicas;
assert totalReplicasPerShard > 0;
//Upload the configs
String configName = (String) properties.get(CollectionAdminParams.COLL_CONF);
String restoreConfigName = message.getStr(CollectionAdminParams.COLL_CONF, configName);
@ -149,7 +148,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
log.info("Starting restore into collection={} with backup_name={} at location={}", restoreCollectionName, backupName,
location);
location);
//Create core-less collection
{
@ -190,7 +189,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
Map<String, Slice> newSlices = new LinkedHashMap<>(backupSlices.size());
for (Slice backupSlice : backupSlices) {
newSlices.put(backupSlice.getName(),
new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(),restoreCollectionName));
new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(), restoreCollectionName));
}
propMap.put(OverseerCollectionMessageHandler.SHARDS_PROP, newSlices);
}
@ -222,216 +221,211 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
List<String> sliceNames = new ArrayList<>();
restoreCollection.getSlices().forEach(x -> sliceNames.add(x.getName()));
PolicyHelper.SessionWrapper sessionWrapper = null;
try {
Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
.forCollection(restoreCollectionName)
.forShard(sliceNames)
.assignNrtReplicas(numNrtReplicas)
.assignTlogReplicas(numTlogReplicas)
.assignPullReplicas(numPullReplicas)
.onNodes(nodeList)
.build();
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, restoreCollection);
List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
.forCollection(restoreCollectionName)
.forShard(sliceNames)
.assignNrtReplicas(numNrtReplicas)
.assignTlogReplicas(numTlogReplicas)
.assignPullReplicas(numPullReplicas)
.onNodes(nodeList)
.build();
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, restoreCollection);
List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
CountDownLatch countDownLatch = new CountDownLatch(restoreCollection.getSlices().size());
CountDownLatch countDownLatch = new CountDownLatch(restoreCollection.getSlices().size());
//Create one replica per shard and copy backed up data to it
for (Slice slice : restoreCollection.getSlices()) {
if (log.isInfoEnabled()) {
log.info("Adding replica for shard={} collection={} ", slice.getName(), restoreCollection);
//Create one replica per shard and copy backed up data to it
for (Slice slice : restoreCollection.getSlices()) {
if (log.isInfoEnabled()) {
log.info("Adding replica for shard={} collection={} ", slice.getName(), restoreCollection);
}
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, CREATESHARD);
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
if (numNrtReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.NRT.name());
} else if (numTlogReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.TLOG.name());
} else {
throw new SolrException(ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " +
Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
}
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
}
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, CREATESHARD);
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
}
if (numNrtReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.NRT.name());
} else if (numTlogReplicas >= 1) {
propMap.put(REPLICA_TYPE, Replica.Type.TLOG.name());
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
final NamedList addReplicaResult = new NamedList();
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult, () -> {
Object addResultFailure = addReplicaResult.get("failure");
if (addResultFailure != null) {
SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
if (failure == null) {
failure = new SimpleOrderedMap();
results.add("failure", failure);
}
failure.addAll((NamedList) addResultFailure);
} else {
throw new SolrException(ErrorCode.BAD_REQUEST, "Unexpected number of replicas, replicationFactor, " +
Replica.Type.NRT + " or " + Replica.Type.TLOG + " must be greater than 0");
}
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
if (success == null) {
success = new SimpleOrderedMap();
results.add("success", success);
}
success.addAll((NamedList) addReplicaResult.get("success"));
}
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
final NamedList addReplicaResult = new NamedList();
ocmh.addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult, () -> {
Object addResultFailure = addReplicaResult.get("failure");
if (addResultFailure != null) {
SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
if (failure == null) {
failure = new SimpleOrderedMap();
results.add("failure", failure);
}
failure.addAll((NamedList) addResultFailure);
} else {
SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
if (success == null) {
success = new SimpleOrderedMap();
results.add("success", success);
}
success.addAll((NamedList) addReplicaResult.get("success"));
}
countDownLatch.countDown();
});
}
boolean allIsDone = countDownLatch.await(1, TimeUnit.HOURS);
if (!allIsDone) {
throw new TimeoutException("Initial replicas were not created within 1 hour. Timing out.");
}
Object failures = results.get("failure");
if (failures != null && ((SimpleOrderedMap) failures).size() > 0) {
log.error("Restore failed to create initial replicas.");
ocmh.cleanupCollection(restoreCollectionName, new NamedList<Object>());
return;
}
//refresh the location copy of collection state
restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
{
ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
// Copy data from backed up index to each replica
for (Slice slice : restoreCollection.getSlices()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.RESTORECORE.toString());
params.set(NAME, "snapshot." + slice.getName());
params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString());
params.set(CoreAdminParams.BACKUP_REPOSITORY, repo);
shardRequestTracker.sliceCmd(clusterState, params, null, slice, shardHandler);
}
shardRequestTracker.processResponses(new NamedList(), shardHandler, true, "Could not restore core");
}
{
ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
for (Slice s : restoreCollection.getSlices()) {
for (Replica r : s.getReplicas()) {
String nodeName = r.getNodeName();
String coreNodeName = r.getCoreName();
Replica.State stateRep = r.getState();
if (log.isDebugEnabled()) {
log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}", nodeName, coreNodeName,
stateRep.name());
}
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
params.set(CoreAdminParams.NAME, coreNodeName);
shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
}
shardRequestTracker.processResponses(new NamedList(), shardHandler, true,
"REQUESTAPPLYUPDATES calls did not succeed");
}
}
//Mark all shards in ACTIVE STATE
{
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
propMap.put(ZkStateReader.COLLECTION_PROP, restoreCollectionName);
for (Slice shard : restoreCollection.getSlices()) {
propMap.put(shard.getName(), Slice.State.ACTIVE.toString());
}
ocmh.overseer.offerStateUpdate((Utils.toJSON(new ZkNodeProps(propMap))));
}
if (totalReplicasPerShard > 1) {
if (log.isInfoEnabled()) {
log.info("Adding replicas to restored collection={}", restoreCollection.getName());
}
for (Slice slice : restoreCollection.getSlices()) {
//Add the remaining replicas for each shard, considering it's type
int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
// We already created either a NRT or an TLOG replica as leader
if (numNrtReplicas > 0) {
createdNrtReplicas++;
} else if (createdTlogReplicas > 0) {
createdTlogReplicas++;
}
for (int i = 1; i < totalReplicasPerShard; i++) {
Replica.Type typeToCreate;
if (createdNrtReplicas < numNrtReplicas) {
createdNrtReplicas++;
typeToCreate = Replica.Type.NRT;
} else if (createdTlogReplicas < numTlogReplicas) {
createdTlogReplicas++;
typeToCreate = Replica.Type.TLOG;
} else {
createdPullReplicas++;
typeToCreate = Replica.Type.PULL;
assert createdPullReplicas <= numPullReplicas: "Unexpected number of replicas";
}
if (log.isDebugEnabled()) {
log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
}
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
propMap.put(REPLICA_TYPE, typeToCreate.name());
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
}
}
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
}
}
}
if (backupCollectionAlias != null && !backupCollectionAlias.equals(backupCollection)) {
log.debug("Restoring alias {} -> {}", backupCollectionAlias, backupCollection);
ocmh.zkStateReader.aliasesManager
.applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(backupCollectionAlias, backupCollection));
}
log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
} finally {
if (sessionWrapper != null) sessionWrapper.release();
countDownLatch.countDown();
});
}
boolean allIsDone = countDownLatch.await(1, TimeUnit.HOURS);
if (!allIsDone) {
throw new TimeoutException("Initial replicas were not created within 1 hour. Timing out.");
}
Object failures = results.get("failure");
if (failures != null && ((SimpleOrderedMap) failures).size() > 0) {
log.error("Restore failed to create initial replicas.");
ocmh.cleanupCollection(restoreCollectionName, new NamedList<Object>());
return;
}
//refresh the location copy of collection state
restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
{
ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
// Copy data from backed up index to each replica
for (Slice slice : restoreCollection.getSlices()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.RESTORECORE.toString());
params.set(NAME, "snapshot." + slice.getName());
params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString());
params.set(CoreAdminParams.BACKUP_REPOSITORY, repo);
shardRequestTracker.sliceCmd(clusterState, params, null, slice, shardHandler);
}
shardRequestTracker.processResponses(new NamedList(), shardHandler, true, "Could not restore core");
}
{
ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
for (Slice s : restoreCollection.getSlices()) {
for (Replica r : s.getReplicas()) {
String nodeName = r.getNodeName();
String coreNodeName = r.getCoreName();
Replica.State stateRep = r.getState();
if (log.isDebugEnabled()) {
log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}", nodeName, coreNodeName,
stateRep.name());
}
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
params.set(CoreAdminParams.NAME, coreNodeName);
shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
}
shardRequestTracker.processResponses(new NamedList(), shardHandler, true,
"REQUESTAPPLYUPDATES calls did not succeed");
}
}
//Mark all shards in ACTIVE STATE
{
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
propMap.put(ZkStateReader.COLLECTION_PROP, restoreCollectionName);
for (Slice shard : restoreCollection.getSlices()) {
propMap.put(shard.getName(), Slice.State.ACTIVE.toString());
}
ocmh.overseer.offerStateUpdate((Utils.toJSON(new ZkNodeProps(propMap))));
}
if (totalReplicasPerShard > 1) {
if (log.isInfoEnabled()) {
log.info("Adding replicas to restored collection={}", restoreCollection.getName());
}
for (Slice slice : restoreCollection.getSlices()) {
//Add the remaining replicas for each shard, considering it's type
int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
// We already created either a NRT or an TLOG replica as leader
if (numNrtReplicas > 0) {
createdNrtReplicas++;
} else if (createdTlogReplicas > 0) {
createdTlogReplicas++;
}
for (int i = 1; i < totalReplicasPerShard; i++) {
Replica.Type typeToCreate;
if (createdNrtReplicas < numNrtReplicas) {
createdNrtReplicas++;
typeToCreate = Replica.Type.NRT;
} else if (createdTlogReplicas < numTlogReplicas) {
createdTlogReplicas++;
typeToCreate = Replica.Type.TLOG;
} else {
createdPullReplicas++;
typeToCreate = Replica.Type.PULL;
assert createdPullReplicas <= numPullReplicas : "Unexpected number of replicas";
}
if (log.isDebugEnabled()) {
log.debug("Adding replica for shard={} collection={} of type {} ", slice.getName(), restoreCollection, typeToCreate);
}
HashMap<String, Object> propMap = new HashMap<>();
propMap.put(COLLECTION_PROP, restoreCollectionName);
propMap.put(SHARD_ID_PROP, slice.getName());
propMap.put(REPLICA_TYPE, typeToCreate.name());
// Get the first node matching the shard to restore in
String node;
for (ReplicaPosition replicaPosition : replicaPositions) {
if (Objects.equals(replicaPosition.shard, slice.getName())) {
node = replicaPosition.node;
propMap.put(CoreAdminParams.NODE, node);
replicaPositions.remove(replicaPosition);
break;
}
}
// add async param
if (asyncId != null) {
propMap.put(ASYNC, asyncId);
}
ocmh.addPropertyParams(message, propMap);
ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
}
}
}
if (backupCollectionAlias != null && !backupCollectionAlias.equals(backupCollection)) {
log.debug("Restoring alias {} -> {}", backupCollectionAlias, backupCollection);
ocmh.zkStateReader.aliasesManager
.applyModificationAndExportToZk(a -> a.cloneWithCollectionAlias(backupCollectionAlias, backupCollection));
}
log.info("Completed restoring collection={} backupName={}", restoreCollection, backupName);
}
private int getInt(ZkNodeProps message, String propertyName, Integer count, int defaultValue) {

View File

@ -18,41 +18,17 @@
package org.apache.solr.cloud.api.collections;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.CompositeIdRouter;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.PlainIdRouter;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ReplicaPosition;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.*;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.params.CommonAdminParams;
import org.apache.solr.common.params.CommonParams;
@ -71,13 +47,15 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import java.lang.invoke.MethodHandles;
import java.util.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import static org.apache.solr.client.solrj.impl.SolrClientNodeStateProvider.Variable.CORE_IDX;
import static org.apache.solr.common.cloud.ZkStateReader.*;
import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonAdminParams.NUM_SUB_SHARDS;
@ -135,7 +113,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
String splitKey = message.getStr("split.key");
DocCollection collection = clusterState.getCollection(collectionName);
PolicyHelper.SessionWrapper sessionWrapper = null;
Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
if (parentSlice.getState() != Slice.State.ACTIVE) {
@ -427,7 +404,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
log.debug("Successfully applied buffered updates on : {}", subShardNames);
// Replica creation for the new Slices
// replica placement is controlled by the autoscaling policy framework
Set<String> nodes = clusterState.getLiveNodes();
List<String> nodeList = new ArrayList<>(nodes.size());
@ -458,7 +434,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, collection);
List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
sessionWrapper = PolicyHelper.getLastSessionWrapper(true);
t.stop();
t = timings.sub("createReplicaPlaceholders");
@ -611,7 +586,6 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
log.error("Error executing split operation for collection: {} parent shard: {}", collectionName, slice, e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
} finally {
if (sessionWrapper != null) sessionWrapper.release();
if (!success) {
cleanupAfterFailure(zkStateReader, collectionName, parentSlice.getName(), subSlices, offlineSlices);
unlockForSplit(ocmh.cloudManager, collectionName, parentSlice.getName());
@ -638,7 +612,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
Map<String, Object> nodeValues = nodeStateProvider.getNodeValues(parentShardLeader.getNodeName(),
Collections.singletonList(ImplicitSnitch.DISK));
Map<String, Map<String, List<Replica>>> infos = nodeStateProvider.getReplicaInfo(parentShardLeader.getNodeName(),
Collections.singletonList(Type.CORE_IDX.metricsAttribute));
Collections.singletonList(CORE_IDX.metricsAttribute));
if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing replica information for parent shard leader");
}
@ -647,11 +621,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
Double indexSize = null;
for (Replica info : lst) {
if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
Number size = (Number)info.get(Type.CORE_IDX.metricsAttribute);
Number size = (Number)info.get( CORE_IDX.metricsAttribute);
if (size == null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing index size information for parent shard leader");
}
indexSize = (Double) Type.CORE_IDX.convertVal(size);
indexSize = (Double) CORE_IDX.convertVal(size);
break;
}
}

View File

@ -1,139 +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.api.collections;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
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.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
import static org.apache.solr.common.params.AutoScalingParams.NODE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
public class UtilizeNodeCmd implements OverseerCollectionMessageHandler.Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final OverseerCollectionMessageHandler ocmh;
public UtilizeNodeCmd(OverseerCollectionMessageHandler ocmh) {
this.ocmh = ocmh;
}
@Override
public void call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
ocmh.checkRequired(message, NODE);
String nodeName = message.getStr(NODE);
String async = message.getStr(ASYNC);
AutoScalingConfig autoScalingConfig = ocmh.overseer.getSolrCloudManager().getDistribStateManager().getAutoScalingConfig();
//first look for any violation that may use this replica
List<ZkNodeProps> requests = new ArrayList<>();
//first look for suggestions if any
List<Suggester.SuggestionInfo> suggestions = PolicyHelper.getSuggestions(autoScalingConfig, ocmh.overseer.getSolrCloudManager());
for (Suggester.SuggestionInfo suggestionInfo : suggestions) {
if (log.isInfoEnabled()) {
log.info("op: {}", suggestionInfo.getOperation());
}
String coll = null;
List<String> pieces = StrUtils.splitSmart(suggestionInfo.getOperation().getPath(), '/');
if (pieces.size() > 1) {
coll = pieces.get(2);
} else {
continue;
}
log.info("coll: {}", coll);
if (suggestionInfo.getOperation() instanceof V2Request) {
String targetNode = (String) Utils.getObjectByPath(suggestionInfo.getOperation(), true, "command/move-replica/targetNode");
if (Objects.equals(targetNode, nodeName)) {
String replica = (String) Utils.getObjectByPath(suggestionInfo.getOperation(), true, "command/move-replica/replica");
requests.add(new ZkNodeProps(COLLECTION_PROP, coll,
CollectionParams.TARGET_NODE, targetNode,
ASYNC, async,
REPLICA_PROP, replica));
}
}
}
executeAll(requests);
PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getSession(ocmh.overseer.getSolrCloudManager());
Policy.Session session = sessionWrapper.get();
Suggester initialsuggester = session.getSuggester(MOVEREPLICA)
.hint(Suggester.Hint.TARGET_NODE, nodeName);
Suggester suggester = null;
for (; ; ) {
suggester = session.getSuggester(MOVEREPLICA)
.hint(Suggester.Hint.TARGET_NODE, nodeName);
@SuppressWarnings({"rawtypes"})
SolrRequest request = suggester.getSuggestion();
if (requests.size() > 10) {
log.info("too_many_suggestions");
PolicyHelper.logState(ocmh.overseer.getSolrCloudManager(), initialsuggester);
break;
}
log.info("SUGGESTION: {}", request);
if (request == null) break;
session = suggester.getSession();
requests.add(new ZkNodeProps(COLLECTION_PROP, request.getParams().get(COLLECTION_PROP),
CollectionParams.TARGET_NODE, request.getParams().get(CollectionParams.TARGET_NODE),
REPLICA_PROP, request.getParams().get(REPLICA_PROP),
ASYNC, request.getParams().get(ASYNC)));
}
if (log.isInfoEnabled()) {
log.info("total_suggestions: {}", requests.size());
}
if (requests.size() == 0) {
PolicyHelper.logState(ocmh.overseer.getSolrCloudManager(), initialsuggester);
}
sessionWrapper.returnSession(session);
try {
executeAll(requests);
} finally {
sessionWrapper.release();
}
}
private void executeAll(List<ZkNodeProps> requests) throws Exception {
if (requests.isEmpty()) return;
for (ZkNodeProps props : requests) {
@SuppressWarnings({"rawtypes"})
NamedList result = new NamedList();
ocmh.commandMap.get(MOVEREPLICA)
.call(ocmh.overseer.getSolrCloudManager().getClusterStateProvider().getClusterState(),
props,
result);
}
requests.clear();
}
}

View File

@ -1,68 +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.autoscaling;
import java.io.IOException;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.common.MapWriter;
/**
* 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 SolrCloudManager cloudManager;
private final AutoScaling.Trigger source;
private final Map<String, Object> properties;
public ActionContext(SolrCloudManager cloudManager, AutoScaling.Trigger source, Map<String, Object> properties) {
this.cloudManager = cloudManager;
this.source = source;
this.properties = properties;
}
public SolrCloudManager getCloudManager() {
return cloudManager;
}
public AutoScaling.Trigger getSource() {
return source;
}
public Map<String, Object> getProperties() {
return properties;
}
public Object getProperty(String name) {
return properties != null ? properties.get(name) : null;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put("source", source.getName());
if (properties != null) {
for (Map.Entry<String, Object> entry : properties.entrySet()) {
ew.put("properties." + entry.getKey(), entry.getValue());
}
}
}
}

View File

@ -1,40 +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.autoscaling;
import java.util.Collections;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.core.SolrResourceLoader;
import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
/**
* This class configures the parent ComputePlanAction to compute plan
* only for collections which have autoAddReplicas=true.
*/
public class AutoAddReplicasPlanAction extends ComputePlanAction {
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
properties.put("collections", Collections.singletonMap(AUTO_ADD_REPLICAS, "true"));
super.configure(loader, cloudManager, properties);
}
}

View File

@ -1,246 +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.autoscaling;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
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 AutoScaling.Trigger#setProcessor(AutoScaling.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 when trigger is created but before it's initialized or scheduled for use.
* This method should also verify that the trigger configuration parameters are correct. It may
* be called multiple times.
* @param properties configuration properties
* @throws TriggerValidationException contains details of invalid configuration parameters.
*/
void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException;
/**
* 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() throws Exception;
}
/**
* Factory to produce instances of {@link Trigger}.
*/
public static abstract class TriggerFactory implements Closeable {
protected boolean isClosed = false;
public abstract Trigger create(TriggerEventType type, String name, Map<String, Object> props) throws TriggerValidationException;
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
}
}
}
/**
* Default implementation of {@link TriggerFactory}.
*/
public static class TriggerFactoryImpl extends TriggerFactory {
private final SolrCloudManager cloudManager;
private final SolrResourceLoader loader;
public TriggerFactoryImpl(SolrResourceLoader loader, SolrCloudManager cloudManager) {
Objects.requireNonNull(cloudManager);
Objects.requireNonNull(loader);
this.cloudManager = cloudManager;
this.loader = loader;
}
@Override
public synchronized Trigger create(TriggerEventType type, String name, Map<String, Object> props) throws TriggerValidationException {
if (isClosed) {
throw new AlreadyClosedException("TriggerFactory has already been closed, cannot create new triggers");
}
if (type == null) {
throw new IllegalArgumentException("Trigger type must not be null");
}
if (name == null || name.isEmpty()) {
throw new IllegalArgumentException("Trigger name must not be empty");
}
Trigger t;
switch (type) {
case NODEADDED:
t = new NodeAddedTrigger(name);
break;
case NODELOST:
t = new NodeLostTrigger(name);
break;
case SEARCHRATE:
t = new SearchRateTrigger(name);
break;
case METRIC:
t = new MetricTrigger(name);
break;
case SCHEDULED:
t = new ScheduledTrigger(name);
break;
case INDEXSIZE:
t = new IndexSizeTrigger(name);
break;
default:
throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
}
t.configure(loader, cloudManager, props);
return t;
}
}
public static final String AUTO_ADD_REPLICAS_TRIGGER_NAME = ".auto_add_replicas";
public static final String AUTO_ADD_REPLICAS_TRIGGER_DSL =
" {" +
" 'name' : '" + AUTO_ADD_REPLICAS_TRIGGER_NAME + "'," +
" 'event' : 'nodeLost'," +
" 'waitFor' : -1," +
" 'enabled' : true," +
" 'actions' : [" +
" {" +
" 'name':'auto_add_replicas_plan'," +
" 'class':'solr.AutoAddReplicasPlanAction'" +
" }," +
" {" +
" 'name':'execute_plan'," +
" 'class':'solr.ExecutePlanAction'" +
" }" +
" ]" +
" }";
@SuppressWarnings({"unchecked"})
public static final Map<String, Object> AUTO_ADD_REPLICAS_TRIGGER_PROPS = (Map) Utils.fromJSONString(AUTO_ADD_REPLICAS_TRIGGER_DSL);
public static final String SCHEDULED_MAINTENANCE_TRIGGER_NAME = ".scheduled_maintenance";
public static final String SCHEDULED_MAINTENANCE_TRIGGER_DSL =
" {" +
" 'name' : '" + SCHEDULED_MAINTENANCE_TRIGGER_NAME + "'," +
" 'event' : 'scheduled'," +
" 'startTime' : 'NOW'," +
" 'every' : '+1DAY'," +
" 'enabled' : true," +
" 'actions' : [" +
" {" +
" 'name':'inactive_shard_plan'," +
" 'class':'solr.InactiveShardPlanAction'" +
" }," +
" {" +
" 'name':'inactive_markers_plan'," +
" 'class':'solr.InactiveMarkersPlanAction'" +
" }," +
" {" +
" 'name':'execute_plan'," +
" 'class':'solr.ExecutePlanAction'" +
" }" +
" ]" +
" }";
@SuppressWarnings({"unchecked"})
public static final Map<String, Object> SCHEDULED_MAINTENANCE_TRIGGER_PROPS = (Map) Utils.fromJSONString(SCHEDULED_MAINTENANCE_TRIGGER_DSL);
}

View File

@ -1,742 +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.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.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.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.solr.api.Api;
import org.apache.solr.api.ApiBag;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.MapWriter;
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.SolrParams;
import org.apache.solr.common.util.CommandOperation;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.handler.RequestHandlerUtils;
import org.apache.solr.request.SolrQueryRequest;
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.zookeeper.KeeperException;
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;
/**
* Handler for /cluster/autoscaling
*/
public class AutoScalingHandler extends RequestHandlerBase implements PermissionNameProvider {
public static final String HANDLER_PATH = "/admin/autoscaling";
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected final SolrCloudManager cloudManager;
protected final SolrResourceLoader loader;
protected final AutoScaling.TriggerFactory triggerFactory;
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 final TimeSource timeSource;
public AutoScalingHandler(SolrCloudManager cloudManager, SolrResourceLoader loader) {
this.cloudManager = cloudManager;
this.loader = loader;
this.triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
this.timeSource = cloudManager.getTimeSource();
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);
}
Optional<BiConsumer<SolrQueryResponse, AutoScalingConfig>> getSubpathExecutor(List<String> path, SolrQueryRequest request) {
if (path.size() == 3) {
if (DIAGNOSTICS.equals(path.get(2))) {
return Optional.of((rsp, autoScalingConf) -> handleDiagnostics(rsp, autoScalingConf));
} else if (SUGGESTIONS.equals(path.get(2))) {
return Optional.of((rsp, autoScalingConf) -> handleSuggestions(rsp, autoScalingConf, request.getParams()));
} else {
return Optional.empty();
}
}
return Optional.empty();
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
try {
String httpMethod = (String) req.getContext().get("httpMethod");
RequestHandlerUtils.setWt(req, JSON);
if ("GET".equals(httpMethod)) {
String path = (String) req.getContext().get("path");
if (path == null) path = "/cluster/autoscaling";
List<String> parts = StrUtils.splitSmart(path, '/', true);
if (parts.size() < 2 || parts.size() > 3) {
// invalid
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
}
AutoScalingConfig autoScalingConf = cloudManager.getDistribStateManager().getAutoScalingConfig();
if (parts.size() == 2) {
autoScalingConf.writeMap(new MapWriter.EntryWriter() {
@Override
public MapWriter.EntryWriter put(CharSequence k, Object v) {
rsp.getValues().add(k.toString(), v);
return this;
}
});
} else {
getSubpathExecutor(parts, req).ifPresent(it -> it.accept(rsp, autoScalingConf));
}
} else {
if (req.getContentStreams() == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No commands specified for autoscaling");
}
String path = (String) req.getContext().get("path");
if (path != null) {
List<String> parts = StrUtils.splitSmart(path, '/', true);
if(parts.size() == 3){
getSubpathExecutor(parts, req).ifPresent(it -> {
Map map = null;
try {
map = (Map) Utils.fromJSON(req.getContentStreams().iterator().next().getStream());
} catch (IOException e1) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "error parsing payload", e1);
}
it.accept(rsp, new AutoScalingConfig(map));
});
return;
}
}
List<CommandOperation> ops = CommandOperation.readCommands(req.getContentStreams(), rsp.getValues(), singletonCommands);
if (ops == null) {
// errors have already been added to the response so there's nothing left to do
return;
}
processOps(req, rsp, ops);
}
} catch (Exception e) {
rsp.getValues().add("result", "failure");
throw e;
} finally {
RequestHandlerUtils.addExperimentalFormatWarning(rsp);
}
}
@SuppressWarnings({"unchecked"})
private void handleSuggestions(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf, SolrParams params) {
rsp.getValues().add("suggestions",
PolicyHelper.getSuggestions(autoScalingConf, cloudManager, params));
}
@SuppressWarnings({"unchecked", "rawtypes"})
public void processOps(SolrQueryRequest req, SolrQueryResponse rsp, List<CommandOperation> ops)
throws KeeperException, InterruptedException, IOException {
while (true) {
AutoScalingConfig initialConfig = cloudManager.getDistribStateManager().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;
case CMD_SET_PROPERTIES:
currentConfig = handleSetProperties(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 (setAutoScalingConfig(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 AutoScalingConfig handleSetProperties(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op, AutoScalingConfig currentConfig) {
Map<String, Object> map = op.getDataMap() == null ? Collections.emptyMap() : op.getDataMap();
Map<String, Object> configProps = new HashMap<>(currentConfig.getProperties());
configProps.putAll(map);
// remove a key which is set to null
map.forEach((k, v) -> {
if (v == null) configProps.remove(k);
});
return currentConfig.withProperties(configProps);
}
@SuppressWarnings({"unchecked"})
private void handleDiagnostics(SolrQueryResponse rsp, AutoScalingConfig autoScalingConf) {
Policy policy = autoScalingConf.getPolicy();
rsp.getValues().add("diagnostics", PolicyHelper.getDiagnostics(policy, cloudManager));
}
@SuppressWarnings({"unchecked"})
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("set-cluster-policy expects an array of objects");
return currentConfig;
}
List<Clause> cp = null;
try {
cp = clusterPolicy.stream().map(Clause::create).collect(Collectors.toList());
} catch (Exception e) {
op.addError(e.getMessage());
return currentConfig;
}
Policy p = currentConfig.getPolicy().withClusterPolicy(cp);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
@SuppressWarnings({"unchecked"})
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");
return currentConfig;
}
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 AutoScalingConfig handleRemovePolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
String policyName = (String) op.getVal("");
if (op.hasError()) return currentConfig;
Map<String, List<Clause>> policies = currentConfig.getPolicy().getPolicies();
if (policies == null || !policies.containsKey(policyName)) {
op.addError("No policy exists with name: " + policyName);
return currentConfig;
}
cloudManager.getClusterStateProvider().getClusterState().forEachCollection(coll -> {
if (policyName.equals(coll.getPolicyName()))
op.addError(StrUtils.formatString("policy : {0} is being used by collection {1}", policyName, coll.getName()));
});
if (op.hasError()) return currentConfig;
policies = new HashMap<>(policies);
policies.remove(policyName);
Policy p = currentConfig.getPolicy().withPolicies(policies);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
@SuppressWarnings({"unchecked"})
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;
}
}
Map<String, List<Clause>> currentClauses = new HashMap<>(currentConfig.getPolicy().getPolicies());
Map<String, List<Clause>> newClauses = null;
try {
newClauses = Policy.clausesFromMap((Map<String, List<Map<String, Object>>>) op.getCommandData(),
new ArrayList<>() );
} catch (Exception e) {
op.addError(e.getMessage());
return currentConfig;
}
currentClauses.putAll(newClauses);
Policy p = currentConfig.getPolicy().withPolicies(currentClauses);
currentConfig = currentConfig.withPolicy(p);
return currentConfig;
}
@SuppressWarnings({"unchecked"})
private AutoScalingConfig handleResumeTrigger(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op,
AutoScalingConfig currentConfig) throws KeeperException, InterruptedException {
String triggerName = op.getStr(NAME);
if (op.hasError()) return currentConfig;
Map<String, AutoScalingConfig.TriggerConfig> triggers = currentConfig.getTriggerConfigs();
Set<String> changed = new HashSet<>();
if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
op.addError("No trigger exists with name: " + triggerName);
return currentConfig;
}
Map<String, AutoScalingConfig.TriggerConfig> newTriggers = new HashMap<>();
for (Map.Entry<String, AutoScalingConfig.TriggerConfig> entry : triggers.entrySet()) {
if (Policy.EACH.equals(triggerName) || triggerName.equals(entry.getKey())) {
AutoScalingConfig.TriggerConfig trigger = entry.getValue();
if (!trigger.enabled) {
trigger = trigger.withEnabled(true);
newTriggers.put(entry.getKey(), trigger);
changed.add(entry.getKey());
} else {
newTriggers.put(entry.getKey(), entry.getValue());
}
} else {
newTriggers.put(entry.getKey(), entry.getValue());
}
}
rsp.getValues().add("changed", changed);
if (!changed.isEmpty()) {
currentConfig = currentConfig.withTriggerConfigs(newTriggers);
}
return currentConfig;
}
@SuppressWarnings({"unchecked"})
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 {
int timeoutSeconds = parseHumanTime(timeout);
resumeTime = new Date(TimeUnit.MILLISECONDS.convert(timeSource.getTimeNs(), TimeUnit.NANOSECONDS)
+ TimeUnit.MILLISECONDS.convert(timeoutSeconds, TimeUnit.SECONDS));
} catch (IllegalArgumentException e) {
op.addError("Invalid 'timeout' value for suspend trigger: " + triggerName);
return currentConfig;
}
}
Map<String, AutoScalingConfig.TriggerConfig> triggers = currentConfig.getTriggerConfigs();
Set<String> changed = new HashSet<>();
if (!Policy.EACH.equals(triggerName) && !triggers.containsKey(triggerName)) {
op.addError("No trigger exists with name: " + triggerName);
return currentConfig;
}
Map<String, AutoScalingConfig.TriggerConfig> newTriggers = new HashMap<>();
for (Map.Entry<String, AutoScalingConfig.TriggerConfig> entry : triggers.entrySet()) {
if (Policy.EACH.equals(triggerName) || triggerName.equals(entry.getKey())) {
AutoScalingConfig.TriggerConfig trigger = entry.getValue();
if (trigger.enabled) {
trigger = trigger.withEnabled(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 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 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;
AutoScalingConfig.TriggerListenerConfig listenerConfig = new AutoScalingConfig.TriggerListenerConfig(listenerName, op.getValuesExcluding("name"));
// validate that we can load the listener class
// todo allow creation from blobstore
TriggerListener listener = null;
try {
listener = loader.newInstance(listenerClass, TriggerListener.class);
listener.configure(loader, cloudManager, listenerConfig);
} catch (TriggerValidationException e) {
log.warn("invalid listener configuration", e);
op.addError("invalid listener configuration: " + e.toString());
return currentConfig;
} catch (Exception e) {
log.warn("error loading listener class ", e);
op.addError("Listener not found: " + listenerClass + ". error message:" + e.getMessage());
return currentConfig;
} finally {
if (listener != null) {
IOUtils.closeQuietly(listener);
}
}
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;
}
// todo - handle races between competing set-trigger and set-listener invocations
currentConfig = currentConfig.withTriggerListenerConfig(listenerConfig);
return currentConfig;
}
@SuppressWarnings({"unchecked"})
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 {
loader.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"));
// validate trigger config
AutoScaling.Trigger t = null;
try {
t = triggerFactory.create(trigger.event, trigger.name, trigger.properties);
} catch (Exception e) {
op.addError("Error validating trigger config " + trigger.name + ": " + e.toString());
return currentConfig;
} finally {
if (t != null) {
IOUtils.closeQuietly(t);
}
}
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();
public 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 setAutoScalingConfig(AutoScalingConfig currentConfig) throws KeeperException, InterruptedException, IOException {
verifyAutoScalingConf(currentConfig);
try {
cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(currentConfig), currentConfig.getZkVersion());
} catch (BadVersionException bve) {
// somebody else has changed the configuration so we must retry
return false;
}
//log.debug("-- saved version " + currentConfig.getZkVersion() + ": " + currentConfig);
return true;
}
private void verifyAutoScalingConf(AutoScalingConfig autoScalingConf) throws IOException {
Policy.Session session = autoScalingConf.getPolicy()
.createSession(cloudManager);
log.debug("Verified autoscaling configuration");
}
@Override
public String getDescription() {
return "A handler for autoscaling configuration";
}
@Override
public Name getPermissionName(AuthorizationContext request) {
switch (request.getHttpMethod()) {
case "GET":
return Name.AUTOSCALING_READ_PERM;
case "POST": {
return StrUtils.splitSmart(request.getResource(), '/', true).size() == 3 ?
Name.AUTOSCALING_READ_PERM :
Name.AUTOSCALING_WRITE_PERM;
}
default:
return null;
}
}
@Override
public Collection<Api> getApis() {
return ApiBag.wrapRequestHandlers(this, "autoscaling.Commands");
}
@Override
public Boolean registerV2() {
return Boolean.TRUE;
}
@Override
public SolrRequestHandler getSubHandler(String path) {
if (path.equals("/diagnostics") || path.equals("/suggestions")) return this;
return null;
}
}

View File

@ -1,355 +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.autoscaling;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.*;
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.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.SolrResourceLoader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import static org.apache.solr.cloud.autoscaling.TriggerEvent.NODE_NAMES;
/**
* 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.
* <p>
* 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());
// accept all collections by default
Predicate<String> collectionsPredicate = s -> true;
public ComputePlanAction() {
super();
TriggerUtils.validProperties(validProperties, "collections");
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
Object value = properties.get("collections");
if (value instanceof String) {
String colString = (String) value;
if (!colString.isEmpty()) {
List<String> whiteListedCollections = StrUtils.splitSmart(colString, ',');
collectionsPredicate = whiteListedCollections::contains;
}
} else if (value instanceof Map) {
@SuppressWarnings({"unchecked"})
Map<String, String> matchConditions = (Map<String, String>) value;
collectionsPredicate = collectionName -> {
try {
DocCollection collection = cloudManager.getClusterStateProvider().getCollection(collectionName);
if (collection == null) {
log.debug("Collection: {} was not found while evaluating conditions", collectionName);
return false;
}
for (Map.Entry<String, String> entry : matchConditions.entrySet()) {
if (!entry.getValue().equals(collection.get(entry.getKey()))) {
if (log.isDebugEnabled()) {
log.debug("Collection: {} does not match condition: {}:{}", collectionName, entry.getKey(), entry.getValue());
}
return false;
}
}
return true;
} catch (IOException e) {
log.error("Exception fetching collection information for: {}", collectionName, e);
return false;
}
};
}
}
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
if (log.isDebugEnabled()) {
log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
}
SolrCloudManager cloudManager = context.getCloudManager();
try {
AutoScalingConfig autoScalingConf = cloudManager.getDistribStateManager().getAutoScalingConfig();
if (autoScalingConf.isEmpty()) {
throw new Exception("Action: " + getName() + " executed but no policy is configured");
}
PolicyHelper.SessionWrapper sessionWrapper = PolicyHelper.getSession(cloudManager);
Policy.Session session = sessionWrapper.get();
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
if (log.isTraceEnabled()) {
log.trace("-- session: {}", session);
log.trace("-- state: {}", clusterState);
}
try {
Suggester suggester = getSuggester(session, event, context, cloudManager);
int maxOperations = getMaxNumOps(event, autoScalingConf, clusterState);
int requestedOperations = getRequestedNumOps(event);
if (requestedOperations > maxOperations) {
log.warn("Requested number of operations {} higher than maximum {}, adjusting...",
requestedOperations, maxOperations);
}
int opCount = 0;
int opLimit = maxOperations;
if (requestedOperations > 0) {
opLimit = requestedOperations;
}
do {
// computing changes in large clusters may take a long time
if (Thread.currentThread().isInterrupted()) {
throw new InterruptedException("stopping - thread was interrupted");
}
@SuppressWarnings({"rawtypes"})
SolrRequest operation = suggester.getSuggestion();
opCount++;
// prepare suggester for the next iteration
if (suggester.getSession() != null) {
session = suggester.getSession();
}
suggester = getSuggester(session, event, context, cloudManager);
// break on first null op
// unless a specific number of ops was requested
// uncomment the following to log too many operations
/*if (opCount > 10) {
PolicyHelper.logState(cloudManager, initialSuggester);
}*/
if (operation == null) {
if (requestedOperations < 0) {
//uncomment the following to log zero operations
// PolicyHelper.logState(cloudManager, initialSuggester);
break;
} else {
log.info("Computed plan empty, remained {} requested ops to try.", opCount - opLimit);
continue;
}
}
if (log.isDebugEnabled()) {
log.debug("Computed Plan: {}", operation.getParams());
}
Map<String, Object> props = context.getProperties();
props.compute("operations", (k, v) -> {
@SuppressWarnings({"unchecked", "rawtypes"})
List<SolrRequest> operations = (List<SolrRequest>) v;
if (operations == null) operations = new ArrayList<>();
operations.add(operation);
return operations;
});
} while (opCount < opLimit);
} finally {
releasePolicySession(sessionWrapper, session);
}
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected exception while processing event: " + event, e);
}
}
private void releasePolicySession(PolicyHelper.SessionWrapper sessionWrapper, Policy.Session session) {
sessionWrapper.returnSession(session);
sessionWrapper.release();
}
protected int getMaxNumOps(TriggerEvent event, AutoScalingConfig autoScalingConfig, ClusterState clusterState) {
// estimate a maximum default limit that should be sufficient for most purposes:
// number of nodes * total number of replicas * 3
AtomicInteger totalRF = new AtomicInteger();
clusterState.forEachCollection(coll -> {
Integer rf = coll.getReplicationFactor();
if (rf == null) {
if (coll.getSlices().isEmpty()) {
rf = 1; // ???
} else {
rf = coll.getReplicas().size() / coll.getSlices().size();
}
}
totalRF.addAndGet(rf * coll.getSlices().size());
});
int totalMax = clusterState.getLiveNodes().size() * totalRF.get() * 3;
int maxOp = (Integer) autoScalingConfig.getProperties().getOrDefault(AutoScalingParams.MAX_COMPUTE_OPERATIONS, totalMax);
Object o = event.getProperty(AutoScalingParams.MAX_COMPUTE_OPERATIONS, maxOp);
try {
return Integer.parseInt(String.valueOf(o));
} catch (Exception e) {
log.warn("Invalid '{}' event property: {}, using default {}", AutoScalingParams.MAX_COMPUTE_OPERATIONS, o, maxOp);
return maxOp;
}
}
protected int getRequestedNumOps(TriggerEvent event) {
@SuppressWarnings({"unchecked"})
Collection<TriggerEvent.Op> ops = (Collection<TriggerEvent.Op>) event.getProperty(TriggerEvent.REQUESTED_OPS, Collections.emptyList());
if (ops.isEmpty()) {
return -1;
} else {
return ops.size();
}
}
private static final String START = "__start__";
protected Suggester getSuggester(Policy.Session session, TriggerEvent event, ActionContext context, SolrCloudManager cloudManager) throws IOException {
Suggester suggester;
switch (event.getEventType()) {
case NODEADDED:
suggester = getNodeAddedSuggester(cloudManager, session, event);
break;
case NODELOST:
suggester = getNodeLostSuggester(cloudManager, session, event);
break;
case SEARCHRATE:
case METRIC:
case INDEXSIZE:
@SuppressWarnings({"unchecked"})
List<TriggerEvent.Op> ops = (List<TriggerEvent.Op>)event.getProperty(TriggerEvent.REQUESTED_OPS, Collections.emptyList());
int start = (Integer)event.getProperty(START, 0);
if (ops.isEmpty() || start >= ops.size()) {
return NoneSuggester.get(session);
}
TriggerEvent.Op op = ops.get(start);
suggester = session.getSuggester(op.getAction());
if (suggester instanceof UnsupportedSuggester) {
@SuppressWarnings({"unchecked"})
List<TriggerEvent.Op> unsupportedOps = (List<TriggerEvent.Op>)context.getProperties().computeIfAbsent(TriggerEvent.UNSUPPORTED_OPS, k -> new ArrayList<TriggerEvent.Op>());
unsupportedOps.add(op);
}
for (Map.Entry<Suggester.Hint, Object> e : op.getHints().entrySet()) {
suggester = suggester.hint(e.getKey(), e.getValue());
}
if (applyCollectionHints(cloudManager, suggester) == 0) return NoneSuggester.get(session);
suggester = suggester.forceOperation(true);
event.getProperties().put(START, ++start);
break;
case SCHEDULED:
String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
suggester = session.getSuggester(action);
if (applyCollectionHints(cloudManager, suggester) == 0) return NoneSuggester.get(session);
break;
default:
throw new UnsupportedOperationException("No support for events other than nodeAdded, nodeLost, searchRate, metric, scheduled and indexSize. Received: " + event.getEventType());
}
return suggester;
}
private Suggester getNodeLostSuggester(SolrCloudManager cloudManager, Policy.Session session, TriggerEvent event) throws IOException {
String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
switch (action) {
case MOVEREPLICA:
Suggester s = session.getSuggester(action)
.hint(Suggester.Hint.SRC_NODE, event.getProperty(NODE_NAMES));
if (applyCollectionHints(cloudManager, s) == 0) return NoneSuggester.get(session);
return s;
case DELETENODE:
int start = (Integer)event.getProperty(START, 0);
@SuppressWarnings({"unchecked"})
List<String> srcNodes = (List<String>) event.getProperty(NODE_NAMES);
if (srcNodes.isEmpty() || start >= srcNodes.size()) {
return NoneSuggester.get(session);
}
String sourceNode = srcNodes.get(start);
s = session.getSuggester(action)
.hint(Suggester.Hint.SRC_NODE, event.getProperty(NODE_NAMES));
if (applyCollectionHints(cloudManager, s) == 0) return NoneSuggester.get(session);
s.hint(Suggester.Hint.SRC_NODE, Collections.singletonList(sourceNode));
event.getProperties().put(START, ++start);
return s;
case NONE:
return NoneSuggester.get(session);
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unsupported preferredOperation: " + action.toLower() + " specified for node lost trigger");
}
}
/**
* Applies collection hints for all collections that match the {@link #collectionsPredicate}
* and returns the number of collections that matched.
* @return number of collections that match the {@link #collectionsPredicate}
* @throws IOException if {@link org.apache.solr.client.solrj.impl.ClusterStateProvider} throws IOException
*/
private int applyCollectionHints(SolrCloudManager cloudManager, Suggester s) throws IOException {
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
Set<String> set = clusterState.getCollectionStates().keySet().stream()
.filter(collectionRef -> collectionsPredicate.test(collectionRef))
.collect(Collectors.toSet());
if (set.size() < clusterState.getCollectionStates().size()) {
// apply hints only if a subset of collections are selected
set.forEach(c -> s.hint(Suggester.Hint.COLL, c));
}
return set.size();
}
private Suggester getNodeAddedSuggester(SolrCloudManager cloudManager, Policy.Session session, TriggerEvent event) throws IOException {
String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
Replica.Type replicaType = (Replica.Type) event.getProperty(AutoScalingParams.REPLICA_TYPE, Replica.Type.NRT);
CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
Suggester suggester = session.getSuggester(action)
.hint(Suggester.Hint.TARGET_NODE, event.getProperty(NODE_NAMES));
switch (action) {
case ADDREPLICA:
// add all collection/shard pairs and let policy engine figure out which one
// to place on the target node
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
Set<Pair<String, String>> collShards = new HashSet<>();
clusterState.getCollectionStates().entrySet().stream()
.filter(e -> collectionsPredicate.test(e.getKey()))
.forEach(entry -> {
DocCollection docCollection = entry.getValue().get();
if (docCollection != null) {
docCollection.getActiveSlices().stream()
.map(slice -> new Pair<>(entry.getKey(), slice.getName()))
.forEach(collShards::add);
}
});
suggester.hint(Suggester.Hint.COLL_SHARD, collShards);
suggester.hint(Suggester.Hint.REPLICATYPE, replicaType);
break;
case MOVEREPLICA:
case NONE:
break;
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unsupported preferredOperation=" + preferredOp + " for node added event");
}
return suggester;
}
}

View File

@ -1,250 +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.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.Locale;
import java.util.Map;
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.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.common.SolrException;
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.SolrResourceLoader;
import org.apache.solr.util.TestInjection;
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;
public static final String TASK_TIMEOUT_SECONDS = "taskTimeoutSeconds";
public static final String TASK_TIMEOUT_FAIL = "taskTimeoutFail";
int taskTimeoutSeconds;
boolean taskTimeoutFail;
public ExecutePlanAction() {
TriggerUtils.validProperties(validProperties, TASK_TIMEOUT_SECONDS, TASK_TIMEOUT_FAIL);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
String str = String.valueOf(properties.getOrDefault(TASK_TIMEOUT_SECONDS, DEFAULT_TASK_TIMEOUT_SECONDS));
taskTimeoutSeconds = Integer.parseInt(str);
str = String.valueOf(properties.getOrDefault(TASK_TIMEOUT_FAIL, false));
taskTimeoutFail = Boolean.parseBoolean(str);
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void process(TriggerEvent event, ActionContext context) throws Exception {
if (log.isDebugEnabled()) {
log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
}
SolrCloudManager cloudManager = context.getCloudManager();
List<SolrRequest> operations = (List<SolrRequest>) context.getProperty("operations");
if (operations == null || operations.isEmpty()) {
log.info("No operations to execute for event: {}", event);
return;
}
try {
int counter = 0;
for (SolrRequest operation : operations) {
if (log.isDebugEnabled()) {
log.debug("Executing operation: {}", operation.getParams());
}
try {
SolrResponse response = null;
if (operation instanceof CollectionAdminRequest.AsyncCollectionAdminRequest) {
CollectionAdminRequest.AsyncCollectionAdminRequest req = (CollectionAdminRequest.AsyncCollectionAdminRequest) operation;
// waitForFinalState so that the end effects of operations are visible
req.setWaitForFinalState(true);
String asyncId = event.getSource() + '/' + event.getId() + '/' + counter;
String znode = saveAsyncId(cloudManager.getDistribStateManager(), event, asyncId);
log.trace("Saved requestId: {} in znode: {}", asyncId, znode);
// TODO: find a better way of using async calls using dataProvider API !!!
req.setAsyncId(asyncId);
if (TestInjection.delayInExecutePlanAction != null) {
cloudManager.getTimeSource().sleep(TestInjection.delayInExecutePlanAction);
}
CollectionAdminRequest.RequestStatusResponse statusResponse = null;
RequestStatusState state = RequestStatusState.FAILED;
if (!TestInjection.failInExecutePlanAction) {
SolrResponse asyncResponse = cloudManager.request(req);
if (asyncResponse.getResponse().get("error") != null) {
throw new IOException("" + asyncResponse.getResponse().get("error"));
}
asyncId = (String)asyncResponse.getResponse().get("requestid");
statusResponse = waitForTaskToFinish(cloudManager, asyncId,
taskTimeoutSeconds, TimeUnit.SECONDS);
}
if (statusResponse != null) {
state = statusResponse.getRequestStatus();
// overwrite to test a long-running task
if (TestInjection.delayInExecutePlanAction != null &&
TestInjection.delayInExecutePlanAction > TimeUnit.MILLISECONDS.convert(taskTimeoutSeconds, TimeUnit.SECONDS)) {
state = RequestStatusState.RUNNING;
}
if (TestInjection.failInExecutePlanAction) {
state = RequestStatusState.FAILED;
}
// should we accept partial success here? i.e. some operations won't be completed
// successfully but the event processing will still be declared a success
if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED || state == RequestStatusState.NOT_FOUND) {
// remove pending task marker for this request
try {
cloudManager.getDistribStateManager().removeData(znode, -1);
} catch (Exception e) {
log.warn("Unexpected exception while trying to delete znode: {}", znode, e);
}
}
response = statusResponse;
}
if (state == RequestStatusState.RUNNING || state == RequestStatusState.SUBMITTED) {
String msg = String.format(Locale.ROOT, "Task %s is still running after " + taskTimeoutSeconds + " seconds. Consider increasing " +
TASK_TIMEOUT_SECONDS + " action property or `waitFor` of the trigger %s. Operation: %s",
asyncId, event.source, req);
if (taskTimeoutFail) {
throw new IOException(msg);
} else {
log.warn(msg);
}
} else if (state == RequestStatusState.FAILED) {
// remove it as a pending task
try {
cloudManager.getDistribStateManager().removeData(znode, -1);
} catch (Exception e) {
log.warn("Unexpected exception while trying to delete znode: {}", znode, e);
}
throw new IOException("Task " + asyncId + " failed: " + (statusResponse != null ? statusResponse : " timed out. Operation: " + req));
}
} else {
// generic response - can't easily determine success or failure
response = cloudManager.request(operation);
}
NamedList<Object> result = response.getResponse();
context.getProperties().compute("responses", (s, o) -> {
@SuppressWarnings({"unchecked"})
List<NamedList<Object>> responses = (List<NamedList<Object>>) o;
if (responses == null) responses = new ArrayList<>(operations.size());
responses.add(result);
return responses;
});
counter++;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected exception executing operation: " + operation.getParams(), e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "ExecutePlanAction was interrupted", e);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected exception executing operation: " + operation.getParams(), e);
}
}
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected exception while processing event: " + event, e);
}
}
static CollectionAdminRequest.RequestStatusResponse waitForTaskToFinish(SolrCloudManager cloudManager, String requestId, long duration, TimeUnit timeUnit) throws IOException, InterruptedException {
long timeoutSeconds = timeUnit.toSeconds(duration);
RequestStatusState state = RequestStatusState.NOT_FOUND;
CollectionAdminRequest.RequestStatusResponse statusResponse = null;
for (int i = 0; i < timeoutSeconds; i++) {
try {
statusResponse = (CollectionAdminRequest.RequestStatusResponse)cloudManager.request(CollectionAdminRequest.requestStatus(requestId));
state = statusResponse.getRequestStatus();
if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
log.trace("Task with requestId={} finished with state={} in {}s", requestId, state, i * 5);
cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
return statusResponse;
} else if (state == RequestStatusState.NOT_FOUND) {
// the request for this id was never actually submitted! no harm done, just bail out
log.warn("Task with requestId={} was not found on overseer", requestId);
cloudManager.request(CollectionAdminRequest.deleteAsyncId(requestId));
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;
}
if (rootCause instanceof SolrServerException) {
throw e;
}
log.error("Unexpected Exception while querying status of requestId={}", requestId, e);
throw e;
}
if (i > 0 && i % 5 == 0) {
log.trace("Task with requestId={} still not complete after {}s. Last state={}", requestId, i * 5, state);
}
cloudManager.getTimeSource().sleep(5000);
}
log.debug("Task with requestId={} did not complete within {} seconds. Last state={}", timeoutSeconds, requestId, state);
return statusResponse;
}
/**
* Saves the given asyncId in ZK as a persistent sequential node. This allows us to wait for the completion
* of pending tasks from this event in {@link ScheduledTriggers}
* before starting the actions of the next event.
*
* @return the path of the newly created node in ZooKeeper
*/
private String saveAsyncId(DistribStateManager stateManager, TriggerEvent event, String asyncId) throws InterruptedException, AlreadyExistsException, IOException, KeeperException {
String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + event.getSource() + "/" + getName();
try {
stateManager.makePath(parentPath);
} catch (AlreadyExistsException e) {
// ignore
}
return stateManager.createData(parentPath + "/" + PREFIX, Utils.toJSON(Collections.singletonMap("requestid", asyncId)), CreateMode.PERSISTENT_SEQUENTIAL);
}
}

View File

@ -1,164 +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.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.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
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.SolrResourceLoader;
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 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;
public HttpTriggerListener() {
super();
TriggerUtils.requiredProperties(requiredProperties, validProperties, "url");
TriggerUtils.validProperties(validProperties, "payload", "contentType", "timeout", "followRedirects");
validPropertyPrefixes.add("header.");
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
super.configure(loader, cloudManager, config);
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";
}
Map<String, String> headers = new HashMap<>();
headerTemplates.forEach((k, v) -> {
String headerVal = PropertiesUtil.substituteProperty(v, properties);
if (!headerVal.isEmpty()) {
headers.put(k, headerVal);
}
});
headers.put("Content-Type", type);
try {
cloudManager.httpRequest(url, SolrRequest.METHOD.POST, headers, payload, timeout, followRedirects);
} catch (IOException e) {
log.warn("Exception sending request for event {}", event, e);
}
}
}

View File

@ -1,141 +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.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.NotEmptyException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_ACTIVE;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_STATE;
/**
* This plan simply removes nodeAdded and nodeLost markers from Zookeeper if their TTL has
* expired. These markers are used by {@link NodeAddedTrigger} and {@link NodeLostTrigger} to
* ensure fault tolerance in case of Overseer leader crash.
*/
public class InactiveMarkersPlanAction extends TriggerActionBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String TTL_PROP = "ttl";
public static final int DEFAULT_TTL_SECONDS = 3600 * 24 * 2;
private int cleanupTTL;
public InactiveMarkersPlanAction() {
super();
TriggerUtils.validProperties(validProperties, TTL_PROP);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
String cleanupStr = String.valueOf(properties.getOrDefault(TTL_PROP, String.valueOf(DEFAULT_TTL_SECONDS)));
try {
cleanupTTL = Integer.parseInt(cleanupStr);
} catch (Exception e) {
throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "': " + e.toString());
}
if (cleanupTTL < 0) {
throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "', should be > 0. ");
}
}
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
if (log.isTraceEnabled()) {
log.trace("-- {} cleaning markers", getName());
}
// use epoch time to track this across JVMs and nodes
long currentTimeNs = cloudManager.getTimeSource().getEpochTimeNs();
Map<String, Object> results = new LinkedHashMap<>();
Set<String> cleanedUp = new TreeSet<>();
cleanupMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, currentTimeNs, cleanedUp);
if (!cleanedUp.isEmpty()) {
results.put("nodeAdded", cleanedUp);
cleanedUp = new TreeSet<>();
}
cleanupMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, currentTimeNs, cleanedUp);
if (!cleanedUp.isEmpty()) {
results.put("nodeLost", cleanedUp);
}
if (!results.isEmpty()) {
context.getProperties().put(getName(), results);
}
}
private void cleanupMarkers(String path, long currentTimeNs, Set<String> cleanedUp) throws Exception {
DistribStateManager stateManager = cloudManager.getDistribStateManager();
if (!stateManager.hasData(path)) {
return;
}
List<String> markers = stateManager.listData(path);
markers.forEach(m -> {
String markerPath = path + "/" + m;
try {
Map<String, Object> payload = Utils.getJson(stateManager, markerPath);
if (payload.isEmpty()) {
log.trace(" -- ignore {}: either missing or unsupported format", markerPath);
return;
}
boolean activeMarker = payload.getOrDefault(MARKER_STATE, MARKER_ACTIVE)
.equals(MARKER_ACTIVE);
long timestamp = ((Number)payload.get("timestamp")).longValue();
long delta = TimeUnit.NANOSECONDS.toSeconds(currentTimeNs - timestamp);
if (delta > cleanupTTL || !activeMarker) {
try {
stateManager.removeData(markerPath, -1);
log.trace(" -- remove {}, delta={}, ttl={}, active={}", markerPath, delta, cleanupTTL, activeMarker);
cleanedUp.add(m);
} catch (NoSuchElementException nse) {
// someone already removed it - ignore
return;
} catch (BadVersionException be) {
throw new RuntimeException("should never happen", be);
} catch (NotEmptyException ne) {
log.error("Marker znode should be empty but it's not! Ignoring {} ({})", markerPath, ne);
}
} else {
log.trace(" -- keep {}, delta={}, ttl={}, active={}", markerPath, delta, cleanupTTL, activeMarker);
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return;
} catch (IOException | KeeperException e) {
log.warn("Could not cleanup marker at {}, skipping... ", markerPath, e);
}
});
}
}

View File

@ -1,163 +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.autoscaling;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class checks whether there are shards that have been inactive for a long
* time (which usually means they are left-overs from shard splitting) and requests their removal
* after their cleanup TTL period elapsed.
* <p>Shard delete requests are put into the {@link ActionContext}'s properties
* with the key name "operations". The value is a List of SolrRequest objects.</p>
*/
public class InactiveShardPlanAction extends TriggerActionBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String TTL_PROP = "ttl";
public static final int DEFAULT_TTL_SECONDS = 3600 * 24 * 2;
private int cleanupTTL;
public InactiveShardPlanAction() {
super();
TriggerUtils.validProperties(validProperties, TTL_PROP);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
String cleanupStr = String.valueOf(properties.getOrDefault(TTL_PROP, String.valueOf(DEFAULT_TTL_SECONDS)));
try {
cleanupTTL = Integer.parseInt(cleanupStr);
} catch (Exception e) {
throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "': " + e.toString());
}
if (cleanupTTL < 0) {
throw new TriggerValidationException(getName(), TTL_PROP, "invalid value '" + cleanupStr + "', should be > 0. ");
}
}
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
SolrCloudManager cloudManager = context.getCloudManager();
ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
Map<String, List<String>> cleanup = new LinkedHashMap<>();
Map<String, List<String>> inactive = new LinkedHashMap<>();
Map<String, Map<String, Object>> staleLocks = new LinkedHashMap<>();
state.forEachCollection(coll ->
coll.getSlices().forEach(s -> {
if (Slice.State.INACTIVE.equals(s.getState())) {
inactive.computeIfAbsent(coll.getName(), c -> new ArrayList<>()).add(s.getName());
String tstampStr = s.getStr(ZkStateReader.STATE_TIMESTAMP_PROP);
if (tstampStr == null || tstampStr.isEmpty()) {
return;
}
long timestamp = Long.parseLong(tstampStr);
// this timestamp uses epoch time
long currentTime = cloudManager.getTimeSource().getEpochTimeNs();
long delta = TimeUnit.NANOSECONDS.toSeconds(currentTime - timestamp);
if (log.isDebugEnabled()) {
log.debug("{}/{}: tstamp={}, time={}, delta={}", coll.getName(), s.getName(), timestamp, currentTime, delta);
}
if (delta > cleanupTTL) {
if (log.isDebugEnabled()) {
log.debug("-- delete inactive {} / {}", coll.getName(), s.getName());
}
@SuppressWarnings({"unchecked", "rawtypes"})
List<SolrRequest> operations = (List<SolrRequest>)context.getProperties().computeIfAbsent("operations", k -> new ArrayList<>());
operations.add(CollectionAdminRequest.deleteShard(coll.getName(), s.getName()));
cleanup.computeIfAbsent(coll.getName(), c -> new ArrayList<>()).add(s.getName());
}
}
// check for stale shard split locks
String parentPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll.getName();
List<String> locks;
try {
locks = cloudManager.getDistribStateManager().listData(parentPath).stream()
.filter(name -> name.endsWith("-splitting"))
.collect(Collectors.toList());
for (String lock : locks) {
try {
String lockPath = parentPath + "/" + lock;
Map<String, Object> lockData = Utils.getJson(cloudManager.getDistribStateManager(), lockPath);
String tstampStr = (String)lockData.get(ZkStateReader.STATE_TIMESTAMP_PROP);
if (tstampStr == null || tstampStr.isEmpty()) {
return;
}
long timestamp = Long.parseLong(tstampStr);
// this timestamp uses epoch time
long currentTime = cloudManager.getTimeSource().getEpochTimeNs();
long delta = TimeUnit.NANOSECONDS.toSeconds(currentTime - timestamp);
if (log.isDebugEnabled()) {
log.debug("{}/{}: locktstamp={}, time={}, delta={}", coll.getName(), lock, timestamp, currentTime, delta);
}
if (delta > cleanupTTL) {
if (log.isDebugEnabled()) {
log.debug("-- delete inactive split lock for {}/{}, delta={}", coll.getName(), lock, delta);
}
cloudManager.getDistribStateManager().removeData(lockPath, -1);
lockData.put("currentTimeNs", currentTime);
lockData.put("deltaSec", delta);
lockData.put("ttlSec", cleanupTTL);
staleLocks.put(coll.getName() + "/" + lock, lockData);
} else {
if (log.isDebugEnabled()) {
log.debug("-- lock {}/{} still active (delta={})", coll.getName(), lock, delta);
}
}
} catch (NoSuchElementException nse) {
// already removed by someone else - ignore
}
}
} catch (Exception e) {
log.warn("Exception checking for inactive shard split locks in {}", parentPath, e);
}
})
);
Map<String, Object> results = new LinkedHashMap<>();
if (!cleanup.isEmpty()) {
results.put("inactive", inactive);
results.put("cleanup", cleanup);
}
if (!staleLocks.isEmpty()) {
results.put("staleLocks", staleLocks);
}
if (!results.isEmpty()) {
context.getProperties().put(getName(), results);
}
}
}

View File

@ -1,562 +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.autoscaling;
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.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.Locale;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
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.Slice;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CommonAdminParams;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.metrics.SolrCoreMetricManager;
import org.apache.solr.update.SolrIndexSplitter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.CORE_IDX;
/**
*
*/
public class IndexSizeTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
// configuration properties
public static final String ABOVE_BYTES_PROP = "aboveBytes";
public static final String ABOVE_DOCS_PROP = "aboveDocs";
public static final String ABOVE_OP_PROP = "aboveOp";
public static final String BELOW_BYTES_PROP = "belowBytes";
public static final String BELOW_DOCS_PROP = "belowDocs";
public static final String BELOW_OP_PROP = "belowOp";
public static final String COLLECTIONS_PROP = "collections";
public static final String MAX_OPS_PROP = "maxOps";
public static final String SPLIT_FUZZ_PROP = CommonAdminParams.SPLIT_FUZZ;
public static final String SPLIT_METHOD_PROP = CommonAdminParams.SPLIT_METHOD;
public static final String SPLIT_BY_PREFIX = CommonAdminParams.SPLIT_BY_PREFIX;
// event properties
public static final String BYTES_SIZE_KEY = "__bytes__";
public static final String TOTAL_BYTES_SIZE_KEY = "__total_bytes__";
public static final String DOCS_SIZE_KEY = "__docs__";
public static final String MAX_DOC_KEY = "__maxDoc__";
public static final String COMMIT_SIZE_KEY = "__commitBytes__";
public static final String ABOVE_SIZE_KEY = "aboveSize";
public static final String BELOW_SIZE_KEY = "belowSize";
public static final String VIOLATION_KEY = "violationType";
public static final int DEFAULT_MAX_OPS = 10;
public enum Unit { bytes, docs }
private long aboveBytes, aboveDocs, belowBytes, belowDocs;
private int maxOps;
private SolrIndexSplitter.SplitMethod splitMethod;
private boolean splitByPrefix;
private float splitFuzz;
private CollectionParams.CollectionAction aboveOp, belowOp;
private final Set<String> collections = new HashSet<>();
private final Map<String, Long> lastAboveEventMap = new ConcurrentHashMap<>();
private final Map<String, Long> lastBelowEventMap = new ConcurrentHashMap<>();
public IndexSizeTrigger(String name) {
super(TriggerEventType.INDEXSIZE, name);
TriggerUtils.validProperties(validProperties,
ABOVE_BYTES_PROP, ABOVE_DOCS_PROP, ABOVE_OP_PROP,
BELOW_BYTES_PROP, BELOW_DOCS_PROP, BELOW_OP_PROP,
COLLECTIONS_PROP, MAX_OPS_PROP,
SPLIT_METHOD_PROP, SPLIT_FUZZ_PROP, SPLIT_BY_PREFIX);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
String aboveStr = String.valueOf(properties.getOrDefault(ABOVE_BYTES_PROP, Long.MAX_VALUE));
String belowStr = String.valueOf(properties.getOrDefault(BELOW_BYTES_PROP, -1));
try {
aboveBytes = Long.parseLong(aboveStr);
if (aboveBytes <= 0) {
throw new Exception("value must be > 0");
}
} catch (Exception e) {
throw new TriggerValidationException(getName(), ABOVE_BYTES_PROP, "invalid value '" + aboveStr + "': " + e.toString());
}
try {
belowBytes = Long.parseLong(belowStr);
if (belowBytes < 0) {
belowBytes = -1;
}
} catch (Exception e) {
throw new TriggerValidationException(getName(), BELOW_BYTES_PROP, "invalid value '" + belowStr + "': " + e.toString());
}
// below must be at least 2x smaller than above, otherwise splitting a shard
// would immediately put the shard below the threshold and cause the mergeshards action
if (belowBytes > 0 && (belowBytes * 2 > aboveBytes)) {
throw new TriggerValidationException(getName(), BELOW_BYTES_PROP,
"invalid value " + belowBytes + ", should be less than half of '" + ABOVE_BYTES_PROP + "' value, which is " + aboveBytes);
}
// do the same for docs bounds
aboveStr = String.valueOf(properties.getOrDefault(ABOVE_DOCS_PROP, Long.MAX_VALUE));
belowStr = String.valueOf(properties.getOrDefault(BELOW_DOCS_PROP, -1));
try {
aboveDocs = Long.parseLong(aboveStr);
if (aboveDocs <= 0) {
throw new Exception("value must be > 0");
}
} catch (Exception e) {
throw new TriggerValidationException(getName(), ABOVE_DOCS_PROP, "invalid value '" + aboveStr + "': " + e.toString());
}
try {
belowDocs = Long.parseLong(belowStr);
if (belowDocs < 0) {
belowDocs = -1;
}
} catch (Exception e) {
throw new TriggerValidationException(getName(), BELOW_DOCS_PROP, "invalid value '" + belowStr + "': " + e.toString());
}
// below must be at least 2x smaller than above, otherwise splitting a shard
// would immediately put the shard below the threshold and cause the mergeshards action
if (belowDocs > 0 && (belowDocs * 2 > aboveDocs)) {
throw new TriggerValidationException(getName(), BELOW_DOCS_PROP,
"invalid value " + belowDocs + ", should be less than half of '" + ABOVE_DOCS_PROP + "' value, which is " + aboveDocs);
}
String collectionsString = (String) properties.get(COLLECTIONS_PROP);
if (collectionsString != null && !collectionsString.isEmpty()) {
collections.addAll(StrUtils.splitSmart(collectionsString, ','));
}
String aboveOpStr = String.valueOf(properties.getOrDefault(ABOVE_OP_PROP, CollectionParams.CollectionAction.SPLITSHARD.toLower()));
// TODO: this is a placeholder until SOLR-9407 is implemented
String belowOpStr = String.valueOf(properties.getOrDefault(BELOW_OP_PROP, CollectionParams.CollectionAction.MERGESHARDS.toLower()));
aboveOp = CollectionParams.CollectionAction.get(aboveOpStr);
if (aboveOp == null) {
throw new TriggerValidationException(getName(), ABOVE_OP_PROP, "unrecognized value of: '" + aboveOpStr + "'");
}
belowOp = CollectionParams.CollectionAction.get(belowOpStr);
if (belowOp == null) {
throw new TriggerValidationException(getName(), BELOW_OP_PROP, "unrecognized value of: '" + belowOpStr + "'");
}
String maxOpsStr = String.valueOf(properties.getOrDefault(MAX_OPS_PROP, DEFAULT_MAX_OPS));
try {
maxOps = Integer.parseInt(maxOpsStr);
if (maxOps < 1) {
throw new Exception("must be > 1");
}
} catch (Exception e) {
throw new TriggerValidationException(getName(), MAX_OPS_PROP, "invalid value: '" + maxOpsStr + "': " + e.getMessage());
}
String methodStr = (String)properties.getOrDefault(SPLIT_METHOD_PROP, SolrIndexSplitter.SplitMethod.LINK.toLower());
splitMethod = SolrIndexSplitter.SplitMethod.get(methodStr);
if (splitMethod == null) {
throw new TriggerValidationException(getName(), SPLIT_METHOD_PROP, "unrecognized value of: '" + methodStr + "'");
}
String fuzzStr = String.valueOf(properties.getOrDefault(SPLIT_FUZZ_PROP, 0.0f));
try {
splitFuzz = Float.parseFloat(fuzzStr);
} catch (Exception e) {
throw new TriggerValidationException(getName(), SPLIT_FUZZ_PROP, "invalid value: '" + fuzzStr + "': " + e.getMessage());
}
String splitByPrefixStr = String.valueOf(properties.getOrDefault(SPLIT_BY_PREFIX, false));
try {
splitByPrefix = getValidBool(splitByPrefixStr);
} catch (Exception e) {
throw new TriggerValidationException(getName(), SPLIT_BY_PREFIX, "invalid value: '" + splitByPrefixStr + "': " + e.getMessage());
}
}
private boolean getValidBool(String str) throws Exception {
if (str != null && (str.toLowerCase(Locale.ROOT).equals("true") || str.toLowerCase(Locale.ROOT).equals("false"))) {
return Boolean.parseBoolean(str);
}
throw new IllegalArgumentException("Expected a valid boolean value but got " + str);
}
@Override
protected Map<String, Object> getState() {
Map<String, Object> state = new HashMap<>();
state.put("lastAboveEventMap", lastAboveEventMap);
state.put("lastBelowEventMap", lastBelowEventMap);
return state;
}
@Override
@SuppressWarnings({"unchecked"})
protected void setState(Map<String, Object> state) {
this.lastAboveEventMap.clear();
this.lastBelowEventMap.clear();
Map<String, Long> replicaVsTime = (Map<String, Long>)state.get("lastAboveEventMap");
if (replicaVsTime != null) {
this.lastAboveEventMap.putAll(replicaVsTime);
}
replicaVsTime = (Map<String, Long>)state.get("lastBelowEventMap");
if (replicaVsTime != null) {
this.lastBelowEventMap.putAll(replicaVsTime);
}
}
@Override
public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed();
if (old instanceof IndexSizeTrigger) {
IndexSizeTrigger that = (IndexSizeTrigger)old;
assert this.name.equals(that.name);
this.lastAboveEventMap.clear();
this.lastBelowEventMap.clear();
this.lastAboveEventMap.putAll(that.lastAboveEventMap);
this.lastBelowEventMap.putAll(that.lastBelowEventMap);
} else {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
"Unable to restore state from an unknown type of trigger");
}
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void run() {
synchronized(this) {
if (isClosed) {
log.warn("{} ran but was already closed", getName());
return;
}
}
AutoScaling.TriggerEventProcessor processor = processorRef.get();
if (processor == null) {
return;
}
// replica name / info + size, retrieved from leaders only
Map<String, Replica> currentSizes = new HashMap<>();
try {
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
for (String node : clusterState.getLiveNodes()) {
Map<String, Replica> metricTags = new HashMap<>();
// coll, shard, replica
Map<String, Map<String, List<Replica>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
infos.forEach((coll, shards) -> {
if (!collections.isEmpty() && !collections.contains(coll)) {
return;
}
DocCollection docCollection = clusterState.getCollection(coll);
shards.forEach((sh, replicas) -> {
// check only the leader replica in an active shard
Slice s = docCollection.getSlice(sh);
if (s.getState() != Slice.State.ACTIVE) {
return;
}
Replica r = s.getLeader();
// no leader - don't do anything
if (r == null) {
return;
}
// not on this node
if (!r.getNodeName().equals(node)) {
return;
}
// find ReplicaInfo
Replica info = null;
for (Replica ri : replicas) {
if (r.getCoreName().equals(ri.getCoreName())) {
info = ri;
break;
}
}
if (info == null) {
// probably replica is not on this node?
return;
}
// we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
String replicaName = Utils.parseMetricsReplicaName(coll, info.getCoreName());
if (replicaName == null) { // should never happen???
replicaName = info.getName(); // which is actually coreNode name...
}
String registry = SolrCoreMetricManager.createRegistryName(true, coll, sh, replicaName, null);
String tag = "metrics:" + registry + ":" + CORE_IDX.metricsAttribute;
metricTags.put(tag, info);
tag = "metrics:" + registry + ":SEARCHER.searcher.numDocs";
metricTags.put(tag, info);
tag = "metrics:" + registry + ":SEARCHER.searcher.maxDoc";
metricTags.put(tag, info);
tag = "metrics:" + registry + ":SEARCHER.searcher.indexCommitSize";
metricTags.put(tag, info);
});
});
if (metricTags.isEmpty()) {
continue;
}
Map<String, Object> sizes = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
sizes.forEach((tag, size) -> {
final Replica info = metricTags.get(tag);
if (info == null) {
log.warn("Missing replica info for response tag {}", tag);
} else {
// verify that it's a Number
if (!(size instanceof Number)) {
log.warn("invalid size value for tag {} - not a number: '{}' is {}", tag, size, size.getClass().getName());
return;
}
Replica currentInfo = currentSizes.computeIfAbsent(info.getCoreName(), k -> (Replica) info.clone());
if (tag.contains("INDEX")) {
currentInfo.getProperties().put(TOTAL_BYTES_SIZE_KEY, ((Number) size).longValue());
} else if (tag.endsWith("SEARCHER.searcher.numDocs")) {
currentInfo.getProperties().put(DOCS_SIZE_KEY, ((Number) size).longValue());
} else if (tag.endsWith("SEARCHER.searcher.maxDoc")) {
currentInfo.getProperties().put(MAX_DOC_KEY, ((Number) size).longValue());
} else if (tag.endsWith("SEARCHER.searcher.indexCommitSize")) {
currentInfo.getProperties().put(COMMIT_SIZE_KEY, ((Number) size).longValue());
}
}
});
}
} catch (IOException e) {
log.warn("Error running trigger {}", getName(), e);
return;
}
long now = cloudManager.getTimeSource().getTimeNs();
// now check thresholds
// collection / list(info)
Map<String, List<Replica>> aboveSize = new HashMap<>();
Set<String> splittable = new HashSet<>();
currentSizes.forEach((coreName, info) -> {
// calculate estimated bytes
long maxDoc = (Long)info.get(MAX_DOC_KEY);
long numDocs = (Long)info.get(DOCS_SIZE_KEY);
long commitSize = (Long)info.get(COMMIT_SIZE_KEY, 0L);
if (commitSize <= 0) {
commitSize = (Long)info.get(TOTAL_BYTES_SIZE_KEY);
}
// calculate estimated size as a side-effect
commitSize = estimatedSize(maxDoc, numDocs, commitSize);
info.getProperties().put(BYTES_SIZE_KEY, commitSize);
if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes ||
(Long)info.get(DOCS_SIZE_KEY) > aboveDocs) {
if (waitForElapsed(coreName, now, lastAboveEventMap)) {
List<Replica> infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
if (!infos.contains(info)) {
if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes) {
info.getProperties().put(VIOLATION_KEY, ABOVE_BYTES_PROP);
} else {
info.getProperties().put(VIOLATION_KEY, ABOVE_DOCS_PROP);
}
infos.add(info);
splittable.add(info.getName());
}
}
} else {
// no violation - clear waitForElapsed
lastAboveEventMap.remove(coreName);
}
});
// collection / list(info)
Map<String, List<Replica>> belowSize = new HashMap<>();
currentSizes.forEach((coreName, info) -> {
if (((Long)info.get(BYTES_SIZE_KEY) < belowBytes ||
(Long)info.get(DOCS_SIZE_KEY) < belowDocs) &&
// make sure we don't produce conflicting ops
!splittable.contains(info.getName())) {
if (waitForElapsed(coreName, now, lastBelowEventMap)) {
List<Replica> infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
if (!infos.contains(info)) {
if ((Long)info.get(BYTES_SIZE_KEY) < belowBytes) {
info.getProperties().put(VIOLATION_KEY, BELOW_BYTES_PROP);
} else {
info.getProperties().put(VIOLATION_KEY, BELOW_DOCS_PROP);
}
infos.add(info);
}
}
} else {
// no violation - clear waitForElapsed
lastBelowEventMap.remove(coreName);
}
});
if (aboveSize.isEmpty() && belowSize.isEmpty()) {
log.trace("NO VIOLATIONS: Now={}", now);
log.trace("lastAbove={}", lastAboveEventMap);
log.trace("lastBelow={}", lastBelowEventMap);
return;
}
// find the earliest time when a condition was exceeded
final AtomicLong eventTime = new AtomicLong(now);
// calculate ops
final List<TriggerEvent.Op> ops = new ArrayList<>();
aboveSize.forEach((coll, replicas) -> {
// sort by decreasing size to first split the largest ones
// XXX see the comment below about using DOCS_SIZE_PROP in lieu of BYTES_SIZE_PROP
replicas.sort((r1, r2) -> {
long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
if (delta > 0) {
return -1;
} else if (delta < 0) {
return 1;
} else {
return 0;
}
});
replicas.forEach(r -> {
if (ops.size() >= maxOps) {
return;
}
TriggerEvent.Op op = new TriggerEvent.Op(aboveOp);
op.addHint(Suggester.Hint.COLL_SHARD, new Pair<>(coll, r.getShard()));
Map<String, Object> params = new HashMap<>();
params.put(SPLIT_METHOD_PROP, splitMethod.toLower());
if (splitFuzz > 0) {
params.put(SPLIT_FUZZ_PROP, splitFuzz);
}
params.put(SPLIT_BY_PREFIX, splitByPrefix);
op.addHint(Suggester.Hint.PARAMS, params);
ops.add(op);
Long time = lastAboveEventMap.get(r.getCoreName());
if (time != null && eventTime.get() > time) {
eventTime.set(time);
}
});
});
belowSize.forEach((coll, replicas) -> {
if (replicas.size() < 2) {
return;
}
if (ops.size() >= maxOps) {
return;
}
// sort by increasing size
replicas.sort((r1, r2) -> {
// XXX this is not quite correct - if BYTES_SIZE_PROP decided that replica got here
// then we should be sorting by BYTES_SIZE_PROP. However, since DOCS and BYTES are
// loosely correlated it's simpler to sort just by docs (which better reflects the "too small"
// condition than index size, due to possibly existing deleted docs that still occupy space)
long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
if (delta > 0) {
return 1;
} else if (delta < 0) {
return -1;
} else {
return 0;
}
});
// TODO: MERGESHARDS is not implemented yet. For now take the top two smallest shards
// TODO: but in the future we probably need to get ones with adjacent ranges.
// TODO: generate as many MERGESHARDS as needed to consume all belowSize shards
TriggerEvent.Op op = new TriggerEvent.Op(belowOp);
op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(0).getShard()));
op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(1).getShard()));
ops.add(op);
Long time = lastBelowEventMap.get(replicas.get(0).getCoreName());
if (time != null && eventTime.get() > time) {
eventTime.set(time);
}
time = lastBelowEventMap.get(replicas.get(1).getCoreName());
if (time != null && eventTime.get() > time) {
eventTime.set(time);
}
});
if (ops.isEmpty()) {
return;
}
if (processor.process(new IndexSizeEvent(getName(), eventTime.get(), ops, aboveSize, belowSize))) {
// update last event times
aboveSize.forEach((coll, replicas) -> {
replicas.forEach(r -> lastAboveEventMap.put(r.getCoreName(), now));
});
belowSize.forEach((coll, replicas) -> {
if (replicas.size() < 2) {
return;
}
lastBelowEventMap.put(replicas.get(0).getCoreName(), now);
lastBelowEventMap.put(replicas.get(1).getCoreName(), now);
});
}
}
public static long estimatedSize(long maxDoc, long numDocs, long commitSize) {
if (maxDoc == 0) {
return 0;
}
if (maxDoc == numDocs) {
return commitSize;
}
return commitSize * numDocs / maxDoc;
}
private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
return false;
}
return true;
}
public static class IndexSizeEvent extends TriggerEvent {
public IndexSizeEvent(String source, long eventTime, List<Op> ops, Map<String, List<Replica>> aboveSize,
Map<String, List<Replica>> belowSize) {
super(TriggerEventType.INDEXSIZE, source, eventTime, null);
properties.put(TriggerEvent.REQUESTED_OPS, ops);
// avoid passing very large amounts of data here - just use replica names
TreeMap<String, String> above = new TreeMap<>();
aboveSize.forEach((coll, replicas) ->
replicas.forEach(r -> above.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
properties.put(ABOVE_SIZE_KEY, above);
TreeMap<String, String> below = new TreeMap<>();
belowSize.forEach((coll, replicas) ->
replicas.forEach(r -> below.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
properties.put(BELOW_SIZE_KEY, below);
}
}
}

View File

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

View File

@ -1,220 +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.autoscaling;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
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.Slice;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.Pair;
import org.apache.solr.core.SolrResourceLoader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.params.AutoScalingParams.ABOVE;
import static org.apache.solr.common.params.AutoScalingParams.BELOW;
import static org.apache.solr.common.params.AutoScalingParams.METRIC;
import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
public class MetricTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private String metric;
private Number above, below;
private String collection, shard, node, preferredOp;
private final Map<String, Long> lastNodeEvent = new ConcurrentHashMap<>();
public MetricTrigger(String name) {
super(TriggerEventType.METRIC, name);
TriggerUtils.requiredProperties(requiredProperties, validProperties, METRIC);
TriggerUtils.validProperties(validProperties, ABOVE, BELOW, PREFERRED_OP,
AutoScalingParams.COLLECTION,
AutoScalingParams.SHARD,
AutoScalingParams.NODE);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
this.metric = (String) properties.get(METRIC);
this.above = (Number) properties.get(ABOVE);
this.below = (Number) properties.get(BELOW);
this.collection = (String) properties.getOrDefault(AutoScalingParams.COLLECTION, Policy.ANY);
shard = (String) properties.getOrDefault(AutoScalingParams.SHARD, Policy.ANY);
if (collection.equals(Policy.ANY) && !shard.equals(Policy.ANY)) {
throw new TriggerValidationException("shard", "When 'shard' is other than #ANY then collection name must be also other than #ANY");
}
node = (String) properties.getOrDefault(AutoScalingParams.NODE, Policy.ANY);
preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
}
@Override
protected Map<String, Object> getState() {
return null;
}
@Override
protected void setState(Map<String, Object> state) {
lastNodeEvent.clear();
@SuppressWarnings({"unchecked"})
Map<String, Long> nodeTimes = (Map<String, Long>) state.get("lastNodeEvent");
if (nodeTimes != null) {
lastNodeEvent.putAll(nodeTimes);
}
}
@Override
public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed();
if (old instanceof MetricTrigger) {
MetricTrigger that = (MetricTrigger) old;
assert this.name.equals(that.name);
this.lastNodeEvent.clear();
this.lastNodeEvent.putAll(that.lastNodeEvent);
} else {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
"Unable to restore state from an unknown type of trigger");
}
}
@Override
public void run() {
AutoScaling.TriggerEventProcessor processor = processorRef.get();
if (processor == null) {
return;
}
Set<String> liveNodes = null;
if (node.equals(Policy.ANY)) {
if (collection.equals(Policy.ANY)) {
liveNodes = cloudManager.getClusterStateProvider().getLiveNodes();
} else {
final Set<String> nodes = new HashSet<>();
ClusterState.CollectionRef ref = cloudManager.getClusterStateProvider().getState(collection);
DocCollection docCollection;
if (ref == null || (docCollection = ref.get()) == null) {
log.warn("MetricTrigger could not find collection: {}", collection);
return;
}
if (shard.equals(Policy.ANY)) {
docCollection.getReplicas().forEach(replica -> {
nodes.add(replica.getNodeName());
});
} else {
Slice slice = docCollection.getSlice(shard);
if (slice == null) {
log.warn("MetricTrigger could not find collection: {} shard: {}", collection, shard);
return;
}
slice.getReplicas().forEach(replica -> nodes.add(replica.getNodeName()));
}
liveNodes = nodes;
}
} else {
liveNodes = Collections.singleton(node);
}
Map<String, Number> rates = new HashMap<>(liveNodes.size());
for (String node : liveNodes) {
Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, Collections.singletonList(metric));
values.forEach((tag, rate) -> rates.computeIfAbsent(node, s -> (Number) rate));
}
long now = cloudManager.getTimeSource().getTimeNs();
// check for exceeded rates and filter out those with less than waitFor from previous events
Map<String, Number> hotNodes = rates.entrySet().stream()
.filter(entry -> waitForElapsed(entry.getKey(), now, lastNodeEvent))
.filter(entry -> (below != null && Double.compare(entry.getValue().doubleValue(), below.doubleValue()) < 0) || (above != null && Double.compare(entry.getValue().doubleValue(), above.doubleValue()) > 0))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
if (hotNodes.isEmpty()) return;
final AtomicLong eventTime = new AtomicLong(now);
hotNodes.forEach((n, r) -> {
long time = lastNodeEvent.get(n);
if (eventTime.get() > time) {
eventTime.set(time);
}
});
if (processor.process(new MetricBreachedEvent(getName(), collection, shard, preferredOp, eventTime.get(), metric, hotNodes))) {
hotNodes.keySet().forEach(node -> lastNodeEvent.put(node, now));
}
}
private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
log.trace("name={}, lastTime={}, elapsed={}", name, lastTime, elapsed);
if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
return false;
}
return true;
}
public static class MetricBreachedEvent extends TriggerEvent {
public MetricBreachedEvent(String source, String collection, String shard, String preferredOp, long eventTime, String metric, Map<String, Number> hotNodes) {
super(TriggerEventType.METRIC, source, eventTime, null);
properties.put(METRIC, metric);
properties.put(AutoScalingParams.NODE, hotNodes);
if (!collection.equals(Policy.ANY)) {
properties.put(AutoScalingParams.COLLECTION, collection);
}
if (!shard.equals(Policy.ANY)) {
properties.put(AutoScalingParams.SHARD, shard);
}
properties.put(PREFERRED_OP, preferredOp);
// specify requested ops
List<Op> ops = new ArrayList<>(hotNodes.size());
for (String n : hotNodes.keySet()) {
Op op = new Op(CollectionParams.CollectionAction.get(preferredOp));
op.addHint(Suggester.Hint.SRC_NODE, n);
if (!collection.equals(Policy.ANY)) {
if (!shard.equals(Policy.ANY)) {
op.addHint(Suggester.Hint.COLL_SHARD, new Pair<>(collection, shard));
} else {
op.addHint(Suggester.Hint.COLL, collection);
}
}
ops.add(op);
}
properties.put(TriggerEvent.REQUESTED_OPS, ops);
}
}
}

View File

@ -1,250 +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.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
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.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_ACTIVE;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_INACTIVE;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_STATE;
import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
import static org.apache.solr.common.params.AutoScalingParams.REPLICA_TYPE;
/**
* Trigger for the {@link TriggerEventType#NODEADDED} event
*/
public class NodeAddedTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private Set<String> lastLiveNodes = new HashSet<>();
private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>();
private String preferredOp;
private Replica.Type replicaType = Replica.Type.NRT;
public NodeAddedTrigger(String name) {
super(TriggerEventType.NODEADDED, name);
TriggerUtils.validProperties(validProperties, PREFERRED_OP, REPLICA_TYPE);
}
@Override
public void init() throws Exception {
super.init();
lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
log.debug("NodeAddedTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
// pick up added nodes for which marker paths were created
try {
List<String> added = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
added.forEach(n -> {
String markerPath = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + n;
try {
Map<String, Object> markerData = Utils.getJson(stateManager, markerPath);
// skip inactive markers
if (markerData.getOrDefault(MARKER_STATE, MARKER_ACTIVE).equals(MARKER_INACTIVE)) {
return;
}
} catch (InterruptedException | IOException | KeeperException e) {
log.debug("-- ignoring marker {} state due to error{}", markerPath, e);
}
// don't add nodes that have since gone away
if (lastLiveNodes.contains(n) && !nodeNameVsTimeAdded.containsKey(n)) {
// since {@code #restoreState(AutoScaling.Trigger)} is called first, the timeAdded for a node may also be restored
log.debug("Adding node from marker path: {}", n);
nodeNameVsTimeAdded.put(n, cloudManager.getTimeSource().getTimeNs());
}
});
} catch (NoSuchElementException e) {
// ignore
} catch (Exception e) {
log.warn("Exception retrieving nodeLost markers", e);
}
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
preferredOp = preferredOp.toLowerCase(Locale.ROOT);
String replicaTypeStr = (String) properties.getOrDefault(REPLICA_TYPE, Replica.Type.NRT.name());
// verify
try {
replicaType = Replica.Type.valueOf(replicaTypeStr);
} catch (IllegalArgumentException | NullPointerException e) {
throw new TriggerValidationException("Unsupported replicaType=" + replicaTypeStr + " specified for node added trigger");
}
CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
switch (action) {
case ADDREPLICA:
case MOVEREPLICA:
case NONE:
break;
default:
throw new TriggerValidationException("Unsupported preferredOperation=" + preferredOp + " specified for node added trigger");
}
}
@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.clear();
this.lastLiveNodes.addAll(that.lastLiveNodes);
this.nodeNameVsTimeAdded.clear();
this.nodeNameVsTimeAdded.putAll(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();
@SuppressWarnings({"unchecked"})
Collection<String> lastLiveNodes = (Collection<String>)state.get("lastLiveNodes");
if (lastLiveNodes != null) {
this.lastLiveNodes.addAll(lastLiveNodes);
}
@SuppressWarnings({"unchecked"})
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);
Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
if (log.isDebugEnabled()) {
log.debug("Found livenodes: {}", newLiveNodes.size());
}
// have any nodes that we were tracking been removed from the cluster?
// if so, remove them from the tracking map
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 = cloudManager.getTimeSource().getTimeNs();
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 = cloudManager.getTimeSource().getTimeNs();
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) {
if (log.isDebugEnabled()) {
log.debug("NodeAddedTrigger {} firing registered processor for nodes: {} added at times {}, now={}", name,
nodeNames, times, cloudManager.getTimeSource().getTimeNs());
}
if (processor.process(new NodeAddedEvent(getEventType(), getName(), times, nodeNames, preferredOp, replicaType))) {
// remove from tracking set only if the fire was accepted
nodeNames.forEach(n -> {
log.debug("Removing new node from tracking: {}", n);
nodeNameVsTimeAdded.remove(n);
});
} else {
log.debug("Processor returned false for {}!", nodeNames);
}
} else {
nodeNames.forEach(n -> {
nodeNameVsTimeAdded.remove(n);
});
}
}
lastLiveNodes = new HashSet<>(newLiveNodes);
} catch (RuntimeException e) {
log.error("Unexpected exception in NodeAddedTrigger", e);
}
}
public static class NodeAddedEvent extends TriggerEvent {
public NodeAddedEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames, String preferredOp, Replica.Type replicaType) {
// 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);
properties.put(PREFERRED_OP, preferredOp);
properties.put(REPLICA_TYPE, replicaType);
}
}
}

View File

@ -1,240 +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.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
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.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.SolrException;
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.SolrResourceLoader;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_ACTIVE;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_INACTIVE;
import static org.apache.solr.cloud.autoscaling.OverseerTriggerThread.MARKER_STATE;
import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
/**
* Trigger for the {@link TriggerEventType#NODELOST} event
*/
public class NodeLostTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private Set<String> lastLiveNodes = new HashSet<>();
private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>();
private String preferredOp;
public NodeLostTrigger(String name) {
super(TriggerEventType.NODELOST, name);
TriggerUtils.validProperties(validProperties, PREFERRED_OP);
}
@Override
public void init() throws Exception {
super.init();
lastLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
log.debug("NodeLostTrigger {} - Initial livenodes: {}", name, lastLiveNodes);
// pick up lost nodes for which marker paths were created
try {
List<String> lost = stateManager.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
lost.forEach(n -> {
String markerPath = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + n;
try {
Map<String, Object> markerData = Utils.getJson(stateManager, markerPath);
// skip inactive markers
if (markerData.getOrDefault(MARKER_STATE, MARKER_ACTIVE).equals(MARKER_INACTIVE)) {
return;
}
} catch (InterruptedException | IOException | KeeperException e) {
log.debug("-- ignoring marker {} state due to error", markerPath, e);
}
// don't add nodes that have since came back
if (!lastLiveNodes.contains(n) && !nodeNameVsTimeRemoved.containsKey(n)) {
// since {@code #restoreState(AutoScaling.Trigger)} is called first, the timeRemoved for a node may also be restored
log.debug("Adding lost node from marker path: {}", n);
nodeNameVsTimeRemoved.put(n, cloudManager.getTimeSource().getTimeNs());
}
});
} catch (NoSuchElementException e) {
// ignore
} catch (Exception e) {
log.warn("Exception retrieving nodeLost markers", e);
}
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
preferredOp = preferredOp.toLowerCase(Locale.ROOT);
// verify
CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
switch (action) {
case MOVEREPLICA:
case DELETENODE:
case NONE:
break;
default:
throw new TriggerValidationException("Unsupported preferredOperation=" + preferredOp + " specified for node lost trigger");
}
}
@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.clear();
this.lastLiveNodes.addAll(that.lastLiveNodes);
this.nodeNameVsTimeRemoved.clear();
this.nodeNameVsTimeRemoved.putAll(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();
@SuppressWarnings({"unchecked"})
Collection<String> lastLiveNodes = (Collection<String>)state.get("lastLiveNodes");
if (lastLiveNodes != null) {
this.lastLiveNodes.addAll(lastLiveNodes);
}
@SuppressWarnings({"unchecked"})
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");
return;
}
}
Set<String> newLiveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
if (log.isDebugEnabled()) {
log.debug("Running NodeLostTrigger: {} with currently live nodes: {} and last live nodes: {}", name, newLiveNodes.size(), lastLiveNodes.size());
}
log.trace("Current Live Nodes for {}: {}", name, newLiveNodes);
log.trace("Last Live Nodes for {}: {}", name, lastLiveNodes);
// 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, cloudManager.getTimeSource().getTimeNs());
});
// 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 = cloudManager.getTimeSource().getTimeNs();
long te = TimeUnit.SECONDS.convert(now - timeRemoved, TimeUnit.NANOSECONDS);
if (te >= 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, preferredOp))) {
// remove from tracking set only if the fire was accepted
nodeNames.forEach(n -> {
nodeNameVsTimeRemoved.remove(n);
});
} else {
log.debug("NodeLostTrigger processor for lost nodes: {} is not ready, will try later", nodeNames);
}
} else {
log.debug("NodeLostTrigger firing, but no processor - so removing lost nodes: {}", nodeNames);
nodeNames.forEach(n -> {
nodeNameVsTimeRemoved.remove(n);
});
}
}
lastLiveNodes = new HashSet<>(newLiveNodes);
} catch (AlreadyClosedException e) {
} catch (RuntimeException e) {
log.error("Unexpected exception in NodeLostTrigger", e);
}
}
public static class NodeLostEvent extends TriggerEvent {
public NodeLostEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames, String preferredOp) {
// 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);
properties.put(PREFERRED_OP, preferredOp);
}
}
}

View File

@ -1,407 +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.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.ConnectException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.SolrCloseable;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
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, SolrCloseable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String MARKER_STATE = "state";
public static final String MARKER_ACTIVE = "active";
public static final String MARKER_INACTIVE = "inactive";
public static final int DEFAULT_AUTO_ADD_REPLICA_WAIT_FOR_SECONDS = 120;
private final SolrCloudManager cloudManager;
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 = 0;
private Map<String, AutoScaling.Trigger> activeTriggers = new HashMap<>();
private volatile int processedZnodeVersion = -1;
private volatile boolean isClosed = false;
private AutoScalingConfig autoScalingConfig;
public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager cloudManager) {
this.cloudManager = cloudManager;
scheduledTriggers = new ScheduledTriggers(loader, cloudManager);
triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
}
@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");
}
/**
* For tests.
* @lucene.internal
* @return current {@link ScheduledTriggers} instance
*/
public ScheduledTriggers getScheduledTriggers() {
return scheduledTriggers;
}
/**
* For tests, to ensure that all processing has been completed in response to an update of /autoscaling.json.
* @lucene.internal
* @return version of /autoscaling.json for which all configuration updates &amp; processing have been completed.
* Until then this value will always be smaller than the current znodeVersion of /autoscaling.json.
*/
public int getProcessedZnodeVersion() {
return processedZnodeVersion;
}
@Override
public boolean isClosed() {
return isClosed;
}
@Override
public void run() {
int lastZnodeVersion = znodeVersion;
// we automatically add a trigger for auto add replicas if it does not exists already
// we also automatically add a scheduled maintenance trigger
while (!isClosed) {
try {
if (Thread.currentThread().isInterrupted()) {
log.warn("Interrupted");
break;
}
AutoScalingConfig autoScalingConfig = cloudManager.getDistribStateManager().getAutoScalingConfig();
AutoScalingConfig updatedConfig = withDefaultPolicy(autoScalingConfig);
updatedConfig = withAutoAddReplicasTrigger(updatedConfig);
updatedConfig = withScheduledMaintenanceTrigger(updatedConfig);
if (updatedConfig.equals(autoScalingConfig)) break;
log.debug("Adding .auto_add_replicas and .scheduled_maintenance triggers");
cloudManager.getDistribStateManager().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(updatedConfig), updatedConfig.getZkVersion());
break;
} catch (AlreadyClosedException e) {
break;
} catch (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 (IOException | KeeperException e) {
if (e instanceof KeeperException.SessionExpiredException ||
(e.getCause()!=null && e.getCause() instanceof KeeperException.SessionExpiredException)) {
log.warn("Solr cannot talk to ZK, exiting {} main queue loop"
, getClass().getSimpleName(), e);
return;
} else {
log.error("A ZK error has occurred", e);
}
}
}
if (isClosed || Thread.currentThread().isInterrupted()) return;
try {
refreshAutoScalingConf(new AutoScalingWatcher());
} catch (ConnectException e) {
log.warn("ZooKeeper watch triggered for autoscaling conf, but Solr cannot talk to ZK: ", 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 {
updateLock.lockInterruptibly();
try {
// must check for close here before we await on the condition otherwise we can
// only be woken up on interruption
if (isClosed) {
log.info("OverseerTriggerThread has been closed, exiting.");
break;
}
log.debug("Current znodeVersion {}, lastZnodeVersion {}", znodeVersion, lastZnodeVersion);
if (znodeVersion == lastZnodeVersion) {
updated.await();
// are we closed?
if (isClosed) {
log.info("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);
} 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);
}
}
// nodeLost / nodeAdded markers are checked by triggers during their init() call
// which is invoked in scheduledTriggers.add(), so once this is done we can remove them
try {
// add new triggers and/or replace and close the replaced triggers
for (Map.Entry<String, AutoScaling.Trigger> entry : copy.entrySet()) {
try {
scheduledTriggers.add(entry.getValue());
} catch (AlreadyClosedException e) {
} catch (Exception e) {
log.warn("Exception initializing trigger {}, configuration ignored", entry.getKey(), e);
}
}
} 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);
}
}
log.debug("-- deactivating old nodeLost / nodeAdded markers");
deactivateMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
deactivateMarkers(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
processedZnodeVersion = znodeVersion;
}
}
private void deactivateMarkers(String path) {
DistribStateManager stateManager = cloudManager.getDistribStateManager();
try {
List<String> markers = stateManager.listData(path);
for (String marker : markers) {
String markerPath = path + "/" + marker;
try {
Map<String, Object> markerMap = new HashMap<>(Utils.getJson(stateManager, markerPath));
markerMap.put(MARKER_STATE, MARKER_INACTIVE);
stateManager.setData(markerPath, Utils.toJSON(markerMap), -1);
} catch (NoSuchElementException e) {
// ignore - already deleted
}
}
} catch (NoSuchElementException e) {
// ignore
} catch (Exception e) {
log.warn("Error deactivating old markers", 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 (ConnectException e) {
log.warn("ZooKeeper watch triggered for autoscaling conf, but we cannot talk to ZK: ", 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 InterruptedException, IOException {
updateLock.lock();
try {
if (isClosed) {
return;
}
AutoScalingConfig currentConfig = cloudManager.getDistribStateManager().getAutoScalingConfig(watcher);
if (log.isDebugEnabled()) {
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 withDefaultPolicy(AutoScalingConfig autoScalingConfig) {
Policy policy = autoScalingConfig.getPolicy();
if (policy.hasEmptyClusterPolicy()) {
policy = policy.withClusterPolicy(Policy.DEFAULT_CLUSTER_POLICY);
autoScalingConfig = autoScalingConfig.withPolicy(policy);
}
return autoScalingConfig;
}
private AutoScalingConfig withAutoAddReplicasTrigger(AutoScalingConfig autoScalingConfig) {
Map<String, Object> triggerProps = AutoScaling.AUTO_ADD_REPLICAS_TRIGGER_PROPS;
return withDefaultTrigger(triggerProps, autoScalingConfig);
}
private AutoScalingConfig withScheduledMaintenanceTrigger(AutoScalingConfig autoScalingConfig) {
Map<String, Object> triggerProps = AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_PROPS;
return withDefaultTrigger(triggerProps, autoScalingConfig);
}
private AutoScalingConfig withDefaultTrigger(Map<String, Object> triggerProps, AutoScalingConfig autoScalingConfig) {
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) (DEFAULT_AUTO_ADD_REPLICA_WAIT_FOR_SECONDS));
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();
try {
triggerMap.put(triggerName, triggerFactory.create(eventType, triggerName, cfg.properties));
} catch (TriggerValidationException e) {
log.warn("Error in trigger '{}' configuration, trigger config ignored: {}", triggerName, cfg, e);
}
}
return triggerMap;
}
}

View File

@ -1,227 +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.autoscaling;
import java.lang.invoke.MethodHandles;
import java.text.ParseException;
import java.time.Instant;
import java.time.format.DateTimeFormatter;
import java.time.format.DateTimeFormatterBuilder;
import java.time.temporal.ChronoField;
import java.util.Collections;
import java.util.Date;
import java.util.Locale;
import java.util.Map;
import java.util.TimeZone;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.DateMathParser;
import org.apache.solr.util.TimeZoneUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.params.AutoScalingParams.PREFERRED_OP;
/**
* A trigger which creates {@link TriggerEventType#SCHEDULED} events as per the configured schedule
*/
public class ScheduledTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String DEFAULT_GRACE_DURATION = "+15MINUTES";
private static final String LAST_RUN_AT = "lastRunAt";
static final String ACTUAL_EVENT_TIME = "actualEventTime";
private String everyStr;
private String graceDurationStr;
private String preferredOp;
private TimeZone timeZone;
private Instant lastRunAt;
public ScheduledTrigger(String name) {
super(TriggerEventType.SCHEDULED, name);
TriggerUtils.requiredProperties(requiredProperties, validProperties, "startTime", "every");
TriggerUtils.validProperties(validProperties, "timeZone", "graceDuration", AutoScalingParams.PREFERRED_OP);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
String timeZoneStr = (String) properties.get("timeZone");
this.timeZone = TimeZoneUtils.parseTimezone(timeZoneStr); // defaults to UTC
String startTimeStr = (String) properties.get("startTime");
this.everyStr = (String) properties.get("every");
this.graceDurationStr = (String) properties.getOrDefault("graceDuration", DEFAULT_GRACE_DURATION);
preferredOp = (String) properties.get(PREFERRED_OP);
if (preferredOp != null &&
CollectionParams.CollectionAction.get(preferredOp) == null) {
throw new TriggerValidationException(getName(), PREFERRED_OP, "unrecognized value of: '" + preferredOp + "'");
}
// attempt parsing to validate date math strings
// explicitly set NOW because it may be different for simulated time
Date now = new Date(TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getEpochTimeNs()));
Instant startTime = parseStartTime(now, startTimeStr, timeZoneStr);
DateMathParser.parseMath(now, startTime + everyStr, timeZone);
DateMathParser.parseMath(now, startTime + graceDurationStr, timeZone);
// We set lastRunAt to be the startTime (which could be a date math expression such as 'NOW')
// Ordinarily, NOW will always be evaluated in this constructor so it may seem that
// the trigger will always fire the first time.
// However, the lastRunAt is overwritten with the value from ZK
// during restoreState() operation (which is performed before run()) so the trigger works correctly
this.lastRunAt = startTime;
}
private Instant parseStartTime(Date now, String startTimeStr, String timeZoneStr) throws TriggerValidationException {
try {
// try parsing startTime as an ISO-8601 date time string
return DateMathParser.parseMath(now, startTimeStr).toInstant();
} catch (SolrException e) {
if (e.code() != SolrException.ErrorCode.BAD_REQUEST.code) {
throw new TriggerValidationException("startTime", "error parsing value '" + startTimeStr + "': " + e.toString());
}
}
if (timeZoneStr == null) {
throw new TriggerValidationException("timeZone",
"Either 'startTime' should be an ISO-8601 date time string or 'timeZone' must be not be null");
}
TimeZone timeZone = TimeZone.getTimeZone(timeZoneStr);
DateTimeFormatter dateTimeFormatter = new DateTimeFormatterBuilder()
.append(DateTimeFormatter.ISO_LOCAL_DATE).appendPattern("['T'[HH[:mm[:ss]]]]")
.parseDefaulting(ChronoField.HOUR_OF_DAY, 0)
.parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0)
.parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0)
.toFormatter(Locale.ROOT).withZone(timeZone.toZoneId());
try {
return Instant.from(dateTimeFormatter.parse(startTimeStr));
} catch (Exception e) {
throw new TriggerValidationException("startTime", "error parsing startTime '" + startTimeStr + "': " + e.toString());
}
}
@Override
protected Map<String, Object> getState() {
return Collections.singletonMap(LAST_RUN_AT, lastRunAt.toEpochMilli());
}
@Override
protected void setState(Map<String, Object> state) {
if (state.containsKey(LAST_RUN_AT)) {
this.lastRunAt = Instant.ofEpochMilli((Long) state.get(LAST_RUN_AT));
}
}
@Override
public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed();
if (old instanceof ScheduledTrigger) {
ScheduledTrigger scheduledTrigger = (ScheduledTrigger) old;
this.lastRunAt = scheduledTrigger.lastRunAt;
} else {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
"Unable to restore state from an unknown type of trigger");
}
}
@Override
public void run() {
synchronized (this) {
if (isClosed) {
log.debug("ScheduledTrigger ran but was already closed");
return;
}
}
TimeSource timeSource = cloudManager.getTimeSource();
DateMathParser dateMathParser = new DateMathParser(timeZone);
dateMathParser.setNow(new Date(lastRunAt.toEpochMilli()));
Instant nextRunTime, nextPlusGrace;
try {
Date next = dateMathParser.parseMath(everyStr);
dateMathParser.setNow(next);
nextPlusGrace = dateMathParser.parseMath(graceDurationStr).toInstant();
nextRunTime = next.toInstant();
} catch (ParseException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unable to calculate next run time. lastRan: " + lastRunAt.toString() + " and date math string: " + everyStr, e);
}
Instant now = Instant.ofEpochMilli(
TimeUnit.NANOSECONDS.toMillis(timeSource.getEpochTimeNs()));
AutoScaling.TriggerEventProcessor processor = processorRef.get();
if (now.isBefore(nextRunTime)) {
return; // it's not time yet
}
if (now.isAfter(nextPlusGrace)) {
// we are past time and we could not run per schedule so skip this event
if (log.isWarnEnabled()) {
log.warn("ScheduledTrigger was not able to run event at scheduled time: {}. Now: {}",
nextRunTime, now);
}
// Even though we are skipping the event, we need to notify any listeners of the IGNORED stage
// so we create a dummy event with the ignored=true flag and ScheduledTriggers will do the rest
if (processor != null && processor.process(new ScheduledEvent(getEventType(), getName(), timeSource.getTimeNs(),
preferredOp, now.toEpochMilli(), true))) {
lastRunAt = nextRunTime;
return;
}
}
if (processor != null) {
if (log.isDebugEnabled()) {
log.debug("ScheduledTrigger {} firing registered processor for scheduled time {}, now={}", name,
nextRunTime, now);
}
if (processor.process(new ScheduledEvent(getEventType(), getName(), timeSource.getTimeNs(),
preferredOp, now.toEpochMilli()))) {
lastRunAt = nextRunTime; // set to nextRunTime instead of now to avoid drift
}
} else {
lastRunAt = nextRunTime; // set to nextRunTime instead of now to avoid drift
}
}
public static class ScheduledEvent extends TriggerEvent {
public ScheduledEvent(TriggerEventType eventType, String source, long eventTime, String preferredOp, long actualEventTime) {
this(eventType, source, eventTime, preferredOp, actualEventTime, false);
}
public ScheduledEvent(TriggerEventType eventType, String source, long eventTime, String preferredOp, long actualEventTime, boolean ignored) {
super(eventType, source, eventTime, null, ignored);
if (preferredOp != null) {
properties.put(PREFERRED_OP, preferredOp);
}
properties.put(ACTUAL_EVENT_TIME, actualEventTime);
}
}
}

View File

@ -1,913 +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.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.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
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.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.RequestStatusResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.SolrException;
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.SolrResourceLoader;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.autoscaling.ExecutePlanAction.waitForTaskToFinish;
import static org.apache.solr.common.params.AutoScalingParams.ACTION_THROTTLE_PERIOD_SECONDS;
import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_COOLDOWN_PERIOD_SECONDS;
import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_CORE_POOL_SIZE;
import static org.apache.solr.common.params.AutoScalingParams.TRIGGER_SCHEDULE_DELAY_SECONDS;
import static org.apache.solr.common.util.ExecutorUtil.awaitTermination;
/**
* 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());
public static final int DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS = 1;
public static final int DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS = 5;
public static final int DEFAULT_COOLDOWN_PERIOD_SECONDS = 5;
public static final int DEFAULT_TRIGGER_CORE_POOL_SIZE = 4;
static final Map<String, Object> DEFAULT_PROPERTIES = new HashMap<>();
static {
DEFAULT_PROPERTIES.put(TRIGGER_SCHEDULE_DELAY_SECONDS, DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
DEFAULT_PROPERTIES.put(TRIGGER_COOLDOWN_PERIOD_SECONDS, DEFAULT_COOLDOWN_PERIOD_SECONDS);
DEFAULT_PROPERTIES.put(TRIGGER_CORE_POOL_SIZE, DEFAULT_TRIGGER_CORE_POOL_SIZE);
DEFAULT_PROPERTIES.put(ACTION_THROTTLE_PERIOD_SECONDS, DEFAULT_ACTION_THROTTLE_PERIOD_SECONDS);
}
protected static final Random RANDOM;
static {
// We try to make things reproducible in the context of our tests by initializing the random instance
// based on the current seed
String seed = System.getProperty("tests.seed");
if (seed == null) {
RANDOM = new Random();
} else {
RANDOM = new Random(seed.hashCode());
}
}
private final Map<String, TriggerWrapper> scheduledTriggerWrappers = 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 AtomicLong cooldownStart = new AtomicLong();
private final AtomicLong cooldownPeriod = new AtomicLong(TimeUnit.SECONDS.toNanos(DEFAULT_COOLDOWN_PERIOD_SECONDS));
private final AtomicLong triggerDelay = new AtomicLong(DEFAULT_SCHEDULED_TRIGGER_DELAY_SECONDS);
private final SolrCloudManager cloudManager;
private final DistribStateManager stateManager;
private final SolrResourceLoader loader;
private final Stats queueStats;
private final TriggerListeners listeners;
private final List<TriggerListener> additionalListeners = new ArrayList<>();
private AutoScalingConfig autoScalingConfig;
public ScheduledTriggers(SolrResourceLoader loader, SolrCloudManager cloudManager) {
scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool(DEFAULT_TRIGGER_CORE_POOL_SIZE,
new SolrNamedThreadFactory("ScheduledTrigger"));
scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("AutoscalingActionExecutor"));
this.cloudManager = cloudManager;
this.stateManager = cloudManager.getDistribStateManager();
this.loader = loader;
queueStats = new Stats();
listeners = new TriggerListeners();
// initialize cooldown timer
cooldownStart.set(cloudManager.getTimeSource().getTimeNs() - cooldownPeriod.get());
}
/**
* Set the current autoscaling config. This is invoked by {@link OverseerTriggerThread} when autoscaling.json is updated,
* and it re-initializes trigger listeners and other properties used by the framework
* @param autoScalingConfig current autoscaling.json
*/
public void setAutoScalingConfig(AutoScalingConfig autoScalingConfig) {
Map<String, Object> currentProps = new HashMap<>(DEFAULT_PROPERTIES);
if (this.autoScalingConfig != null) {
currentProps.putAll(this.autoScalingConfig.getProperties());
}
// reset listeners early in order to capture first execution of newly scheduled triggers
listeners.setAutoScalingConfig(autoScalingConfig);
for (Map.Entry<String, Object> entry : currentProps.entrySet()) {
Map<String, Object> newProps = autoScalingConfig.getProperties();
String key = entry.getKey();
if (newProps.containsKey(key) && !entry.getValue().equals(newProps.get(key))) {
if (log.isDebugEnabled()) {
log.debug("Changing value of autoscaling property: {} from: {} to: {}", key, entry.getValue(), newProps.get(key));
}
switch (key) {
case TRIGGER_SCHEDULE_DELAY_SECONDS:
triggerDelay.set(((Number) newProps.get(key)).intValue());
synchronized (this) {
scheduledTriggerWrappers.forEach((s, triggerWrapper) -> {
if (triggerWrapper.scheduledFuture.cancel(false)) {
triggerWrapper.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(
triggerWrapper, 0,
cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
TimeUnit.MILLISECONDS);
} else {
log.debug("Failed to cancel scheduled task: {}", s);
}
});
}
break;
case TRIGGER_COOLDOWN_PERIOD_SECONDS:
cooldownPeriod.set(TimeUnit.SECONDS.toNanos(((Number) newProps.get(key)).longValue()));
break;
case TRIGGER_CORE_POOL_SIZE:
this.scheduledThreadPoolExecutor.setCorePoolSize(((Number) newProps.get(key)).intValue());
break;
}
}
}
this.autoScalingConfig = autoScalingConfig;
// reset cooldown
cooldownStart.set(cloudManager.getTimeSource().getTimeNs() - cooldownPeriod.get());
}
/**
* 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) throws Exception {
if (isClosed) {
throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore");
}
TriggerWrapper st;
try {
st = new TriggerWrapper(newTrigger, cloudManager, queueStats);
} catch (Exception e) {
if (isClosed || e instanceof AlreadyClosedException) {
throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore");
}
if (cloudManager.isClosed()) {
log.error("Failed to add trigger {} - closing or disconnected from data provider", newTrigger.getName(), e);
} else {
log.error("Failed to add trigger {}", newTrigger.getName(), e);
}
return;
}
TriggerWrapper triggerWrapper = st;
TriggerWrapper old = scheduledTriggerWrappers.putIfAbsent(newTrigger.getName(), triggerWrapper);
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);
triggerWrapper.setReplay(false);
scheduledTriggerWrappers.replace(newTrigger.getName(), triggerWrapper);
}
newTrigger.setProcessor(event -> {
TriggerListeners triggerListeners = listeners.copy();
if (cloudManager.isClosed()) {
String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because Solr has been shutdown.", event.toString());
log.warn(msg);
triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
return false;
}
TriggerWrapper scheduledSource = scheduledTriggerWrappers.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());
triggerListeners.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 (scheduledSource.isClosed || source.isClosed()) {
String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s because the source trigger: %s has already been closed", event.toString(), source);
triggerListeners.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 (event.isIgnored()) {
log.debug("-------- Ignoring event: {}", event);
event.getProperties().put(TriggerEvent.IGNORED, true);
triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "Event was ignored.");
return true; // always return true for ignored events
}
// even though we pause all triggers during action execution there is a possibility that a trigger was already
// running at the time and would have already created an event so we reject such events during cooldown period
if (cooldownStart.get() + cooldownPeriod.get() > cloudManager.getTimeSource().getTimeNs()) {
log.debug("-------- Cooldown period - rejecting event: {}", event);
event.getProperties().put(TriggerEvent.COOLDOWN, true);
triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "In cooldown period.");
return false;
} else {
log.debug("++++++++ Cooldown inactive - processing event: {}", event);
// start cooldown here to immediately reject other events
cooldownStart.set(cloudManager.getTimeSource().getTimeNs());
}
if (hasPendingActions.compareAndSet(false, true)) {
// pause all triggers while we execute actions so triggers do not operate on a cluster in transition
pauseTriggers();
final boolean enqueued;
if (replaying) {
enqueued = false;
} else {
enqueued = triggerWrapper.enqueue(event);
}
// fire STARTED event listeners after enqueuing the event is successful
triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.STARTED);
List<TriggerAction> actions = source.getActions();
if (actions != null) {
if (actionExecutor.isShutdown()) {
String msg = String.format(Locale.ROOT, "Ignoring autoscaling event %s from trigger %s because the executor has already been closed", event.toString(), source);
triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.ABORTED, msg);
log.warn(msg);
hasPendingActions.set(false);
// we do not want to lose this event just because the trigger was closed, perhaps a replacement will need it
return false;
}
actionExecutor.submit(() -> {
assert hasPendingActions.get();
long eventProcessingStart = cloudManager.getTimeSource().getTimeNs();
TriggerListeners triggerListeners1 = triggerListeners.copy();
log.debug("-- processing actions for {}", event);
try {
// 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(cloudManager, newTrigger, new HashMap<>());
for (TriggerAction action : actions) {
@SuppressWarnings({"unchecked"})
List<String> beforeActions = (List<String>) actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.BEFORE_ACTION.toString(), k -> new ArrayList<String>());
beforeActions.add(action.getName());
triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.BEFORE_ACTION, action.getName(), actionContext);
try {
action.process(event, actionContext);
} catch (Exception e) {
triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.FAILED, action.getName(), actionContext, e, null);
throw new TriggerActionException(event.getSource(), action.getName(), "Error processing action for trigger event: " + event, e);
}
@SuppressWarnings({"unchecked"})
List<String> afterActions = (List<String>) actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.AFTER_ACTION.toString(), k -> new ArrayList<String>());
afterActions.add(action.getName());
triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.AFTER_ACTION, action.getName(), actionContext);
}
if (enqueued) {
TriggerEvent ev = triggerWrapper.dequeue();
assert ev.getId().equals(event.getId());
}
triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.SUCCEEDED);
} catch (TriggerActionException e) {
log.warn("Exception executing actions", e);
} catch (Exception e) {
triggerListeners1.fireListeners(event.getSource(), event, TriggerEventProcessorStage.FAILED);
log.warn("Unhandled exception executing actions", e);
} finally {
// update cooldown to the time when we actually finished processing the actions
cooldownStart.set(cloudManager.getTimeSource().getTimeNs());
hasPendingActions.set(false);
// resume triggers after cool down period
resumeTriggers(cloudManager.getTimeSource().convertDelay(TimeUnit.NANOSECONDS, cooldownPeriod.get(), TimeUnit.MILLISECONDS));
}
if (log.isDebugEnabled()) {
log.debug("-- processing took {} ms for event id={}",
TimeUnit.NANOSECONDS.toMillis(cloudManager.getTimeSource().getTimeNs() - eventProcessingStart), event.id);
}
});
} else {
if (enqueued) {
TriggerEvent ev = triggerWrapper.dequeue();
if (!ev.getId().equals(event.getId())) {
throw new RuntimeException("Wrong event dequeued, queue of " + triggerWrapper.trigger.getName()
+ " is broken! Expected event=" + event + " but got " + ev);
}
}
triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.SUCCEEDED);
hasPendingActions.set(false);
// resume triggers now
resumeTriggers(0);
}
return true;
} else {
log.debug("Ignoring event {}, already processing other actions.", event.id);
// there is an action in the queue and we don't want to enqueue another until it is complete
triggerListeners.fireListeners(event.getSource(), event, TriggerEventProcessorStage.IGNORED, "Already processing another event.");
return false;
}
});
newTrigger.init(); // mark as ready for scheduling
triggerWrapper.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(triggerWrapper, 0,
cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS),
TimeUnit.MILLISECONDS);
}
/**
* Pauses all scheduled trigger invocations without interrupting any that are in progress
* @lucene.internal
*/
public synchronized void pauseTriggers() {
if (log.isDebugEnabled()) {
log.debug("Pausing all triggers: {}", scheduledTriggerWrappers.keySet());
}
scheduledTriggerWrappers.forEach((s, triggerWrapper) -> triggerWrapper.scheduledFuture.cancel(false));
}
/**
* Resumes all previously cancelled triggers to be scheduled after the given initial delay
* @param afterDelayMillis the initial delay in milliseconds after which triggers should be resumed
* @lucene.internal
*/
public synchronized void resumeTriggers(long afterDelayMillis) {
List<Map.Entry<String, TriggerWrapper>> entries = new ArrayList<>(scheduledTriggerWrappers.entrySet());
Collections.shuffle(entries, RANDOM);
entries.forEach(e -> {
String key = e.getKey();
TriggerWrapper triggerWrapper = e.getValue();
if (triggerWrapper.scheduledFuture.isCancelled()) {
log.debug("Resuming trigger: {} after {}ms", key, afterDelayMillis);
triggerWrapper.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(triggerWrapper, afterDelayMillis,
cloudManager.getTimeSource().convertDelay(TimeUnit.SECONDS, triggerDelay.get(), TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
}
});
}
private void waitForPendingTasks(AutoScaling.Trigger newTrigger, List<TriggerAction> actions) throws AlreadyClosedException {
DistribStateManager stateManager = cloudManager.getDistribStateManager();
try {
for (TriggerAction action : actions) {
if (action instanceof ExecutePlanAction) {
String parentPath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + newTrigger.getName() + "/" + action.getName();
if (!stateManager.hasData(parentPath)) {
break;
}
List<String> children = stateManager.listData(parentPath);
if (children != null) {
for (String child : children) {
String path = parentPath + '/' + child;
VersionedData data = stateManager.getData(path, null);
if (data != null) {
@SuppressWarnings({"rawtypes"})
Map map = (Map) Utils.fromJSON(data.getData());
String requestid = (String) map.get("requestid");
try {
log.debug("Found pending task with requestid={}", requestid);
RequestStatusResponse statusResponse = waitForTaskToFinish(cloudManager, 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) {
stateManager.removeData(path, -1);
}
}
} catch (Exception e) {
if (cloudManager.isClosed()) {
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: {} to finish", requestid, e);
}
}
}
}
}
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Thread interrupted", e);
} catch (Exception e) {
if (cloudManager.isClosed()) {
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);
}
}
/**
* Remove and stop all triggers. Also cleans up any leftover
* state / events in ZK.
*/
public synchronized void removeAll() {
getScheduledTriggerNames().forEach(t -> {
log.info("-- removing trigger: {}", t);
remove(t);
});
}
/**
* 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) {
TriggerWrapper removed = scheduledTriggerWrappers.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 {
stateManager.removeRecursively(statePath, true, true);
} catch (Exception e) {
log.warn("Failed to remove state for removed trigger {}", statePath, e);
}
try {
stateManager.removeRecursively(eventsPath, true, true);
} catch (Exception e) {
log.warn("Failed to remove events for removed trigger {}", eventsPath, e);
}
}
/**
* @return an unmodifiable set of names of all triggers being managed by this class
*/
public synchronized Set<String> getScheduledTriggerNames() {
return Set.copyOf(scheduledTriggerWrappers.keySet()); // shallow copy
}
/**
* For use in white/grey box testing: The Trigger returned may be inspected,
* but should not be modified in any way.
*
* @param name the name of an existing trigger
* @return the current scheduled trigger with that name, or null if none exists
* @lucene.internal
*/
public synchronized AutoScaling.Trigger getTrigger(String name) {
TriggerWrapper w = scheduledTriggerWrappers.get(name);
return (null == w) ? null : w.trigger;
}
@Override
public void close() throws IOException {
synchronized (this) {
// mark that we are closed
isClosed = true;
for (TriggerWrapper triggerWrapper : scheduledTriggerWrappers.values()) {
IOUtils.closeQuietly(triggerWrapper);
}
scheduledTriggerWrappers.clear();
}
// shutdown and interrupt all running tasks because there's no longer any
// guarantee about cluster state
log.debug("Shutting down scheduled thread pool executor now");
scheduledThreadPoolExecutor.shutdownNow();
log.debug("Shutting down action executor now");
actionExecutor.shutdownNow();
listeners.close();
log.debug("Awaiting termination for action executor");
awaitTermination(actionExecutor);
log.debug("Awaiting termination for scheduled thread pool executor");
awaitTermination(scheduledThreadPoolExecutor);
log.debug("ScheduledTriggers closed completely");
}
/**
* Add a temporary listener for internal use (tests, simulation).
* @param listener listener instance
*/
public void addAdditionalListener(TriggerListener listener) {
listeners.addAdditionalListener(listener);
}
/**
* Remove a temporary listener for internal use (tests, simulation).
* @param listener listener instance
*/
public void removeAdditionalListener(TriggerListener listener) {
listeners.removeAdditionalListener(listener);
}
private class TriggerWrapper implements Runnable, Closeable {
AutoScaling.Trigger trigger;
ScheduledFuture<?> scheduledFuture;
TriggerEventQueue queue;
boolean replay;
volatile boolean isClosed;
TriggerWrapper(AutoScaling.Trigger trigger, SolrCloudManager cloudManager, Stats stats) throws IOException {
this.trigger = trigger;
this.queue = new TriggerEventQueue(cloudManager, 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()) {
// this synchronization is usually never under contention
// but the only reason to have it here is to ensure that when the set-properties API is used
// to change the schedule delay, we can safely cancel the old scheduled task
// and create another one with the new delay without worrying about concurrent
// execution of the same trigger instance
synchronized (TriggerWrapper.this) {
// replay accumulated events on first run, if any
try {
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;
}
} catch (AlreadyClosedException e) {
} catch (Exception e) {
log.error("Unexpected exception from trigger: {}", trigger.getName(), e);
}
try {
trigger.run();
} catch (AlreadyClosedException e) {
} 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<>();
List<TriggerListener> additionalListeners = new ArrayList<>();
ReentrantLock updateLock = new ReentrantLock();
public TriggerListeners() {
}
private TriggerListeners(Map<String, Map<TriggerEventProcessorStage, List<TriggerListener>>> listenersPerStage,
Map<String, TriggerListener> listenersPerName) {
this.listenersPerStage = new HashMap<>();
listenersPerStage.forEach((n, listeners) -> {
Map<TriggerEventProcessorStage, List<TriggerListener>> perStage = this.listenersPerStage.computeIfAbsent(n, name -> new HashMap<>());
listeners.forEach((s, lst) -> {
List<TriggerListener> newLst = perStage.computeIfAbsent(s, stage -> new ArrayList<>());
newLst.addAll(lst);
});
});
this.listenersPerName = new HashMap<>(listenersPerName);
}
public TriggerListeners copy() {
return new TriggerListeners(listenersPerStage, listenersPerName);
}
public void addAdditionalListener(TriggerListener listener) {
updateLock.lock();
try {
AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
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);
}
additionalListeners.add(listener);
} finally {
updateLock.unlock();
}
}
public void removeAdditionalListener(TriggerListener listener) {
updateLock.lock();
try {
listenersPerName.remove(listener.getConfig().name);
listenersPerStage.forEach((trigger, perStage) -> {
perStage.forEach((stage, listeners) -> {
listeners.remove(listener);
});
});
additionalListeners.remove(listener);
} finally {
updateLock.unlock();
}
}
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 = loader.newInstance(clazz, TriggerListener.class);
} catch (Exception e) {
log.warn("Invalid TriggerListener class name '{}', skipping...", clazz, e);
}
if (listener != null) {
try {
listener.configure(loader, cloudManager, config);
listener.init();
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);
}
}
// re-add additional listeners
List<TriggerListener> additional = new ArrayList<>(additionalListeners);
additionalListeners.clear();
for (TriggerListener listener : additional) {
addAdditionalListener(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 (!listener.isEnabled()) {
continue;
}
if (actionName != null) {
AutoScalingConfig.TriggerListenerConfig config = listener.getConfig();
if (stage == TriggerEventProcessorStage.BEFORE_ACTION) {
if (!config.beforeActions.contains(actionName)) {
continue;
}
} else if (stage == TriggerEventProcessorStage.AFTER_ACTION) {
if (!config.afterActions.contains(actionName)) {
continue;
}
}
}
try {
listener.onEvent(event, stage, actionName, context, error, message);
} catch (Exception e) {
log.warn("Exception running listener {}", listener.getConfig(), e);
}
}
} finally {
updateLock.unlock();
}
}
}
}

View File

@ -1,805 +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.autoscaling;
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.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.AtomicDouble;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.metrics.SolrCoreMetricManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Trigger for the {@link org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType#SEARCHRATE} event.
*/
public class SearchRateTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String COLLECTIONS_PROP = "collections";
public static final String METRIC_PROP = "metric";
public static final String MAX_OPS_PROP = "maxOps";
public static final String MIN_REPLICAS_PROP = "minReplicas";
public static final String ABOVE_RATE_PROP = "aboveRate";
public static final String BELOW_RATE_PROP = "belowRate";
public static final String ABOVE_NODE_RATE_PROP = "aboveNodeRate";
public static final String BELOW_NODE_RATE_PROP = "belowNodeRate";
public static final String ABOVE_OP_PROP = "aboveOp";
public static final String BELOW_OP_PROP = "belowOp";
public static final String ABOVE_NODE_OP_PROP = "aboveNodeOp";
public static final String BELOW_NODE_OP_PROP = "belowNodeOp";
// back-compat
public static final String BC_COLLECTION_PROP = "collection";
public static final String BC_RATE_PROP = "rate";
public static final String HOT_NODES = "hotNodes";
public static final String HOT_COLLECTIONS = "hotCollections";
public static final String HOT_SHARDS = "hotShards";
public static final String HOT_REPLICAS = "hotReplicas";
public static final String COLD_NODES = "coldNodes";
public static final String COLD_COLLECTIONS = "coldCollections";
public static final String COLD_SHARDS = "coldShards";
public static final String COLD_REPLICAS = "coldReplicas";
public static final String VIOLATION_PROP = "violationType";
public static final int DEFAULT_MAX_OPS = 3;
public static final String DEFAULT_METRIC = "QUERY./select.requestTimes:1minRate";
private String metric;
private int maxOps;
private Integer minReplicas = null;
private final Set<String> collections = new HashSet<>();
private String shard;
private String node;
private double aboveRate;
private double belowRate;
private double aboveNodeRate;
private double belowNodeRate;
private CollectionParams.CollectionAction aboveOp, belowOp, aboveNodeOp, belowNodeOp;
private final Map<String, Long> lastCollectionEvent = new ConcurrentHashMap<>();
private final Map<String, Long> lastNodeEvent = new ConcurrentHashMap<>();
private final Map<String, Long> lastShardEvent = new ConcurrentHashMap<>();
private final Map<String, Long> lastReplicaEvent = new ConcurrentHashMap<>();
private final Map<String, Object> state = new HashMap<>();
public SearchRateTrigger(String name) {
super(TriggerEventType.SEARCHRATE, name);
this.state.put("lastCollectionEvent", lastCollectionEvent);
this.state.put("lastNodeEvent", lastNodeEvent);
this.state.put("lastShardEvent", lastShardEvent);
this.state.put("lastReplicaEvent", lastReplicaEvent);
TriggerUtils.validProperties(validProperties,
COLLECTIONS_PROP, AutoScalingParams.SHARD, AutoScalingParams.NODE,
METRIC_PROP,
MAX_OPS_PROP,
MIN_REPLICAS_PROP,
ABOVE_OP_PROP,
BELOW_OP_PROP,
ABOVE_NODE_OP_PROP,
BELOW_NODE_OP_PROP,
ABOVE_RATE_PROP,
BELOW_RATE_PROP,
ABOVE_NODE_RATE_PROP,
BELOW_NODE_RATE_PROP,
// back-compat props
BC_COLLECTION_PROP,
BC_RATE_PROP);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
super.configure(loader, cloudManager, properties);
// parse config options
String collectionsStr = (String)properties.get(COLLECTIONS_PROP);
if (collectionsStr != null) {
collections.addAll(StrUtils.splitSmart(collectionsStr, ','));
}
// check back-compat collection prop
collectionsStr = (String)properties.get(BC_COLLECTION_PROP);
if (collectionsStr != null) {
if (!collectionsStr.equals(Policy.ANY)) {
collections.add(collectionsStr);
}
}
shard = (String)properties.getOrDefault(AutoScalingParams.SHARD, Policy.ANY);
if (!shard.equals(Policy.ANY) && (collections.isEmpty() || collections.size() > 1)) {
throw new TriggerValidationException(name, AutoScalingParams.SHARD, "When 'shard' is other than #ANY then exactly one collection name must be set");
}
node = (String)properties.getOrDefault(AutoScalingParams.NODE, Policy.ANY);
metric = (String)properties.getOrDefault(METRIC_PROP, DEFAULT_METRIC);
String maxOpsStr = String.valueOf(properties.getOrDefault(MAX_OPS_PROP, DEFAULT_MAX_OPS));
try {
maxOps = Integer.parseInt(maxOpsStr);
} catch (Exception e) {
throw new TriggerValidationException(name, MAX_OPS_PROP, "invalid value '" + maxOpsStr + "': " + e.toString());
}
Object o = properties.get(MIN_REPLICAS_PROP);
if (o != null) {
try {
minReplicas = Integer.parseInt(o.toString());
if (minReplicas < 1) {
throw new Exception("must be at least 1, or not set to use 'replicationFactor'");
}
} catch (Exception e) {
throw new TriggerValidationException(name, MIN_REPLICAS_PROP, "invalid value '" + o + "': " + e.toString());
}
}
Object above = properties.get(ABOVE_RATE_PROP);
Object below = properties.get(BELOW_RATE_PROP);
// back-compat rate prop
if (properties.containsKey(BC_RATE_PROP)) {
above = properties.get(BC_RATE_PROP);
}
if (above == null && below == null) {
throw new TriggerValidationException(name, ABOVE_RATE_PROP, "at least one of '" +
ABOVE_RATE_PROP + "' or '" + BELOW_RATE_PROP + "' must be set");
}
if (above != null) {
try {
aboveRate = Double.parseDouble(String.valueOf(above));
} catch (Exception e) {
throw new TriggerValidationException(name, ABOVE_RATE_PROP, "Invalid configuration value: '" + above + "': " + e.toString());
}
} else {
aboveRate = Double.MAX_VALUE;
}
if (below != null) {
try {
belowRate = Double.parseDouble(String.valueOf(below));
} catch (Exception e) {
throw new TriggerValidationException(name, BELOW_RATE_PROP, "Invalid configuration value: '" + below + "': " + e.toString());
}
} else {
belowRate = -1;
}
// node rates
above = properties.get(ABOVE_NODE_RATE_PROP);
below = properties.get(BELOW_NODE_RATE_PROP);
if (above != null) {
try {
aboveNodeRate = Double.parseDouble(String.valueOf(above));
} catch (Exception e) {
throw new TriggerValidationException(name, ABOVE_NODE_RATE_PROP, "Invalid configuration value: '" + above + "': " + e.toString());
}
} else {
aboveNodeRate = Double.MAX_VALUE;
}
if (below != null) {
try {
belowNodeRate = Double.parseDouble(String.valueOf(below));
} catch (Exception e) {
throw new TriggerValidationException(name, BELOW_NODE_RATE_PROP, "Invalid configuration value: '" + below + "': " + e.toString());
}
} else {
belowNodeRate = -1;
}
String aboveOpStr = String.valueOf(properties.getOrDefault(ABOVE_OP_PROP, CollectionParams.CollectionAction.ADDREPLICA.toLower()));
String belowOpStr = String.valueOf(properties.getOrDefault(BELOW_OP_PROP, CollectionParams.CollectionAction.DELETEREPLICA.toLower()));
aboveOp = CollectionParams.CollectionAction.get(aboveOpStr);
if (aboveOp == null) {
throw new TriggerValidationException(getName(), ABOVE_OP_PROP, "unrecognized value: '" + aboveOpStr + "'");
}
belowOp = CollectionParams.CollectionAction.get(belowOpStr);
if (belowOp == null) {
throw new TriggerValidationException(getName(), BELOW_OP_PROP, "unrecognized value: '" + belowOpStr + "'");
}
Object aboveNodeObj = properties.getOrDefault(ABOVE_NODE_OP_PROP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
// do NOT set the default to DELETENODE
Object belowNodeObj = properties.get(BELOW_NODE_OP_PROP);
try {
aboveNodeOp = CollectionParams.CollectionAction.get(String.valueOf(aboveNodeObj));
} catch (Exception e) {
throw new TriggerValidationException(getName(), ABOVE_NODE_OP_PROP, "unrecognized value: '" + aboveNodeObj + "'");
}
if (belowNodeObj != null) {
try {
belowNodeOp = CollectionParams.CollectionAction.get(String.valueOf(belowNodeObj));
} catch (Exception e) {
throw new TriggerValidationException(getName(), BELOW_NODE_OP_PROP, "unrecognized value: '" + belowNodeObj + "'");
}
}
}
@VisibleForTesting
Map<String, Object> getConfig() {
Map<String, Object> config = new HashMap<>();
config.put("name", name);
config.put(COLLECTIONS_PROP, collections);
config.put(AutoScalingParams.SHARD, shard);
config.put(AutoScalingParams.NODE, node);
config.put(METRIC_PROP, metric);
config.put(MAX_OPS_PROP, maxOps);
config.put(MIN_REPLICAS_PROP, minReplicas);
config.put(ABOVE_RATE_PROP, aboveRate);
config.put(BELOW_RATE_PROP, belowRate);
config.put(ABOVE_NODE_RATE_PROP, aboveNodeRate);
config.put(BELOW_NODE_RATE_PROP, belowNodeRate);
config.put(ABOVE_OP_PROP, aboveOp);
config.put(ABOVE_NODE_OP_PROP, aboveNodeOp);
config.put(BELOW_OP_PROP, belowOp);
config.put(BELOW_NODE_OP_PROP, belowNodeOp);
return config;
}
@Override
protected Map<String, Object> getState() {
return state;
}
@Override
protected void setState(Map<String, Object> state) {
lastCollectionEvent.clear();
lastNodeEvent.clear();
lastShardEvent.clear();
lastReplicaEvent.clear();
@SuppressWarnings({"unchecked"})
Map<String, Long> collTimes = (Map<String, Long>)state.get("lastCollectionEvent");
if (collTimes != null) {
lastCollectionEvent.putAll(collTimes);
}
@SuppressWarnings({"unchecked"})
Map<String, Long> nodeTimes = (Map<String, Long>)state.get("lastNodeEvent");
if (nodeTimes != null) {
lastNodeEvent.putAll(nodeTimes);
}
@SuppressWarnings({"unchecked"})
Map<String, Long> shardTimes = (Map<String, Long>)state.get("lastShardEvent");
if (shardTimes != null) {
lastShardEvent.putAll(shardTimes);
}
@SuppressWarnings({"unchecked"})
Map<String, Long> replicaTimes = (Map<String, Long>)state.get("lastReplicaEvent");
if (replicaTimes != null) {
lastReplicaEvent.putAll(replicaTimes);
}
}
@Override
public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed();
if (old instanceof SearchRateTrigger) {
SearchRateTrigger that = (SearchRateTrigger)old;
assert this.name.equals(that.name);
this.lastCollectionEvent.clear();
this.lastNodeEvent.clear();
this.lastShardEvent.clear();
this.lastReplicaEvent.clear();
this.lastCollectionEvent.putAll(that.lastCollectionEvent);
this.lastNodeEvent.putAll(that.lastNodeEvent);
this.lastShardEvent.putAll(that.lastShardEvent);
this.lastReplicaEvent.putAll(that.lastReplicaEvent);
} else {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
"Unable to restore state from an unknown type of trigger");
}
}
@Override
public void run() {
AutoScaling.TriggerEventProcessor processor = processorRef.get();
if (processor == null) {
return;
}
// collection, shard, list(replica + rate)
Map<String, Map<String, List<Replica>>> collectionRates = new HashMap<>();
// node, rate
Map<String, AtomicDouble> nodeRates = new HashMap<>();
// this replication factor only considers replica types that are searchable
// collection, shard, RF
Map<String, Map<String, AtomicInteger>> searchableReplicationFactors = new HashMap<>();
ClusterState clusterState = null;
try {
clusterState = cloudManager.getClusterStateProvider().getClusterState();
} catch (IOException e) {
log.warn("Error getting ClusterState", e);
return;
}
for (String node : cloudManager.getClusterStateProvider().getLiveNodes()) {
Map<String, Replica> metricTags = new HashMap<>();
// coll, shard, replica
Map<String, Map<String, List<Replica>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
infos.forEach((coll, shards) -> {
Map<String, AtomicInteger> replPerShard = searchableReplicationFactors.computeIfAbsent(coll, c -> new HashMap<>());
shards.forEach((sh, replicas) -> {
AtomicInteger repl = replPerShard.computeIfAbsent(sh, s -> new AtomicInteger());
replicas.forEach(replica -> {
// skip non-active replicas
if (replica.getState() != Replica.State.ACTIVE) {
return;
}
repl.incrementAndGet();
// we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
String replicaName = Utils.parseMetricsReplicaName(coll, replica.getCoreName());
if (replicaName == null) { // should never happen???
replicaName = replica.getName(); // which is actually coreNode name...
}
String registry = SolrCoreMetricManager.createRegistryName(true, coll, sh, replicaName, null);
String tag = "metrics:" + registry + ":" + metric;
metricTags.put(tag, replica);
});
});
});
if (metricTags.isEmpty()) {
continue;
}
Map<String, Object> rates = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
if (log.isDebugEnabled()) {
log.debug("### rates for node {}", node);
rates.forEach((tag, rate) -> log.debug("### " + tag + "\t" + rate)); // logOk
}
rates.forEach((tag, rate) -> {
Replica info = metricTags.get(tag);
if (info == null) {
log.warn("Missing replica info for response tag {}", tag);
} else {
Map<String, List<Replica>> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>());
List<Replica> perShard = perCollection.computeIfAbsent(info.getShard(), s -> new ArrayList<>());
info = (Replica)info.clone();
info.getProperties().put(AutoScalingParams.RATE, ((Number)rate).doubleValue());
perShard.add(info);
AtomicDouble perNode = nodeRates.computeIfAbsent(node, s -> new AtomicDouble());
perNode.addAndGet(((Number)rate).doubleValue());
}
});
}
if (log.isDebugEnabled()) {
collectionRates.forEach((coll, collRates) -> {
log.debug("## Collection: {}", coll);
collRates.forEach((s, replicas) -> {
log.debug("## - {}", s);
replicas.forEach(ri -> log.debug("## {} {}", ri.getCoreName(), ri.get(AutoScalingParams.RATE))); //logOk
});
});
}
long now = cloudManager.getTimeSource().getTimeNs();
Map<String, Double> hotNodes = new HashMap<>();
Map<String, Double> coldNodes = new HashMap<>();
// check for exceeded rates and filter out those with less than waitFor from previous events
nodeRates.entrySet().stream()
.filter(entry -> node.equals(Policy.ANY) || node.equals(entry.getKey()))
.forEach(entry -> {
if (entry.getValue().get() > aboveNodeRate) {
if (waitForElapsed(entry.getKey(), now, lastNodeEvent)) {
hotNodes.put(entry.getKey(), entry.getValue().get());
}
} else if (entry.getValue().get() < belowNodeRate) {
if (waitForElapsed(entry.getKey(), now, lastNodeEvent)) {
coldNodes.put(entry.getKey(), entry.getValue().get());
}
} else {
// no violation - clear waitForElapsed
// (violation is only valid if it persists throughout waitFor)
lastNodeEvent.remove(entry.getKey());
}
});
Map<String, Map<String, Double>> hotShards = new HashMap<>();
Map<String, Map<String, Double>> coldShards = new HashMap<>();
List<Replica> hotReplicas = new ArrayList<>();
List<Replica> coldReplicas = new ArrayList<>();
collectionRates.forEach((coll, shardRates) -> {
shardRates.forEach((sh, replicaRates) -> {
double totalShardRate = replicaRates.stream()
.map(r -> {
String elapsedKey = r.getCollection() + "." + r.getCoreName();
if ((Double)r.get(AutoScalingParams.RATE) > aboveRate) {
if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
hotReplicas.add(r);
}
} else if ((Double)r.get(AutoScalingParams.RATE) < belowRate) {
if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
coldReplicas.add(r);
}
} else {
// no violation - clear waitForElapsed
lastReplicaEvent.remove(elapsedKey);
}
return r;
})
.mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum();
// calculate average shard rate over all searchable replicas (see SOLR-12470)
double shardRate = totalShardRate / searchableReplicationFactors.get(coll).get(sh).doubleValue();
String elapsedKey = coll + "." + sh;
log.debug("-- {}: totalShardRate={}, shardRate={}", elapsedKey, totalShardRate, shardRate);
if ((collections.isEmpty() || collections.contains(coll)) &&
(shard.equals(Policy.ANY) || shard.equals(sh))) {
if (shardRate > aboveRate) {
if (waitForElapsed(elapsedKey, now, lastShardEvent)) {
hotShards.computeIfAbsent(coll, s -> new HashMap<>()).put(sh, shardRate);
}
} else if (shardRate < belowRate) {
if (waitForElapsed(elapsedKey, now, lastShardEvent)) {
coldShards.computeIfAbsent(coll, s -> new HashMap<>()).put(sh, shardRate);
log.debug("-- coldShard waitFor elapsed {}", elapsedKey);
} else {
if (log.isDebugEnabled()) {
Long lastTime = lastShardEvent.computeIfAbsent(elapsedKey, s -> now);
long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
if (log.isDebugEnabled()) {
log.debug("-- waitFor didn't elapse for {}, waitFor={}, elapsed={}", elapsedKey, getWaitForSecond(), elapsed);
}
}
}
} else {
// no violation - clear waitForElapsed
lastShardEvent.remove(elapsedKey);
}
}
});
});
Map<String, Double> hotCollections = new HashMap<>();
Map<String, Double> coldCollections = new HashMap<>();
collectionRates.forEach((coll, shardRates) -> {
double total = shardRates.entrySet().stream()
.mapToDouble(e -> e.getValue().stream()
.mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum()).sum();
if (collections.isEmpty() || collections.contains(coll)) {
if (total > aboveRate) {
if (waitForElapsed(coll, now, lastCollectionEvent)) {
hotCollections.put(coll, total);
}
} else if (total < belowRate) {
if (waitForElapsed(coll, now, lastCollectionEvent)) {
coldCollections.put(coll, total);
}
} else {
// no violation - clear waitForElapsed
lastCollectionEvent.remove(coll);
}
}
});
if (hotCollections.isEmpty() &&
hotShards.isEmpty() &&
hotReplicas.isEmpty() &&
hotNodes.isEmpty() &&
coldCollections.isEmpty() &&
coldShards.isEmpty() &&
coldReplicas.isEmpty() &&
coldNodes.isEmpty()) {
return;
}
// generate event
// find the earliest time when a condition was exceeded
final AtomicLong eventTime = new AtomicLong(now);
hotCollections.forEach((c, r) -> {
long time = lastCollectionEvent.get(c);
if (eventTime.get() > time) {
eventTime.set(time);
}
});
coldCollections.forEach((c, r) -> {
long time = lastCollectionEvent.get(c);
if (eventTime.get() > time) {
eventTime.set(time);
}
});
hotShards.forEach((c, shards) -> {
shards.forEach((s, r) -> {
long time = lastShardEvent.get(c + "." + s);
if (eventTime.get() > time) {
eventTime.set(time);
}
});
});
coldShards.forEach((c, shards) -> {
shards.forEach((s, r) -> {
long time = lastShardEvent.get(c + "." + s);
if (eventTime.get() > time) {
eventTime.set(time);
}
});
});
hotReplicas.forEach(r -> {
long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
if (eventTime.get() > time) {
eventTime.set(time);
}
});
coldReplicas.forEach(r -> {
long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
if (eventTime.get() > time) {
eventTime.set(time);
}
});
hotNodes.forEach((n, r) -> {
long time = lastNodeEvent.get(n);
if (eventTime.get() > time) {
eventTime.set(time);
}
});
coldNodes.forEach((n, r) -> {
long time = lastNodeEvent.get(n);
if (eventTime.get() > time) {
eventTime.set(time);
}
});
final List<TriggerEvent.Op> ops = new ArrayList<>();
final Set<String> violations = new HashSet<>();
calculateHotOps(ops, violations, searchableReplicationFactors, hotNodes, hotCollections, hotShards, hotReplicas);
calculateColdOps(ops, violations, clusterState, searchableReplicationFactors, coldNodes, coldCollections, coldShards, coldReplicas);
if (ops.isEmpty()) {
return;
}
if (processor.process(new SearchRateEvent(getName(), eventTime.get(), ops,
hotNodes, hotCollections, hotShards, hotReplicas,
coldNodes, coldCollections, coldShards, coldReplicas, violations))) {
// update lastEvent times
hotNodes.keySet().forEach(node -> lastNodeEvent.put(node, now));
coldNodes.keySet().forEach(node -> lastNodeEvent.put(node, now));
hotCollections.keySet().forEach(coll -> lastCollectionEvent.put(coll, now));
coldCollections.keySet().forEach(coll -> lastCollectionEvent.put(coll, now));
hotShards.entrySet().forEach(e -> e.getValue()
.forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
coldShards.entrySet().forEach(e -> e.getValue()
.forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
hotReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
coldReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
}
}
private void calculateHotOps(List<TriggerEvent.Op> ops,
Set<String> violations,
Map<String, Map<String, AtomicInteger>> searchableReplicationFactors,
Map<String, Double> hotNodes,
Map<String, Double> hotCollections,
Map<String, Map<String, Double>> hotShards,
List<Replica> hotReplicas) {
// calculate the number of replicas to add to each hot shard, based on how much the rate was
// exceeded - but within limits.
// first resolve a situation when only a node is hot but no collection / shard is hot
// TODO: eventually we may want to commission a new node
if (!hotNodes.isEmpty()) {
if (hotShards.isEmpty() && hotCollections.isEmpty()) {
// move replicas around
if (aboveNodeOp != null) {
hotNodes.forEach((n, r) -> {
ops.add(new TriggerEvent.Op(aboveNodeOp, Suggester.Hint.SRC_NODE, n));
violations.add(HOT_NODES);
});
}
} else {
// ignore - hot shards will result in changes that will change hot node status anyway
}
}
// add replicas
Map<String, Map<String, List<Pair<String, String>>>> hints = new HashMap<>();
// HOT COLLECTIONS
// currently we don't do anything for hot collections. Theoretically we could add
// 1 replica more to each shard, based on how close to the threshold each shard is
// but it's probably better to wait for a shard to become hot and be more precise.
// HOT SHARDS
hotShards.forEach((coll, shards) -> shards.forEach((s, r) -> {
List<Pair<String, String>> perShard = hints
.computeIfAbsent(coll, c -> new HashMap<>())
.computeIfAbsent(s, sh -> new ArrayList<>());
addReplicaHints(coll, s, r, searchableReplicationFactors.get(coll).get(s).get(), perShard);
violations.add(HOT_SHARDS);
}));
// HOT REPLICAS
// Hot replicas (while their shards are not hot) may be caused by
// dumb clients that use direct replica URLs - this is beyond our control
// so ignore them.
hints.values().forEach(m -> m.values().forEach(lst -> lst.forEach(p -> {
ops.add(new TriggerEvent.Op(aboveOp, Suggester.Hint.COLL_SHARD, p));
})));
}
/**
* This method implements a primitive form of proportional controller with a limiter.
*/
@SuppressWarnings({"unchecked", "rawtypes"})
private void addReplicaHints(String collection, String shard, double r, int replicationFactor, List<Pair<String, String>> hints) {
int numReplicas = (int)Math.round((r - aboveRate) / (double) replicationFactor);
// in one event add at least 1 replica
if (numReplicas < 1) {
numReplicas = 1;
}
// ... and at most maxOps replicas
if (numReplicas > maxOps) {
numReplicas = maxOps;
}
for (int i = 0; i < numReplicas; i++) {
hints.add(new Pair(collection, shard));
}
}
private void calculateColdOps(List<TriggerEvent.Op> ops,
Set<String> violations,
ClusterState clusterState,
Map<String, Map<String, AtomicInteger>> searchableReplicationFactors,
Map<String, Double> coldNodes,
Map<String, Double> coldCollections,
Map<String, Map<String, Double>> coldShards,
List<Replica> coldReplicas) {
// COLD COLLECTIONS
// Probably can't do anything reasonable about whole cold collections
// because they may be needed even if not used.
// COLD SHARDS & COLD REPLICAS:
// We remove cold replicas only from cold shards, otherwise we are susceptible to uneven
// replica routing (which is beyond our control).
// If we removed replicas from non-cold shards we could accidentally bring that shard into
// the hot range, which would result in adding replica, and that replica could again stay cold due to
// the same routing issue, which then would lead to removing that replica, etc, etc...
// Remove cold replicas but only when there's at least a minimum number of searchable
// replicas still available (additional non-searchable replicas may exist, too)
// NOTE: do this before adding ops for DELETENODE because we don't want to attempt
// deleting replicas that have been already moved elsewhere
Map<String, Map<String, List<Replica>>> byCollectionByShard = new HashMap<>();
coldReplicas.forEach(ri -> {
byCollectionByShard.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
.computeIfAbsent(ri.getShard(), s -> new ArrayList<>())
.add(ri);
});
coldShards.forEach((coll, perShard) -> {
perShard.forEach((shard, rate) -> {
List<Replica> replicas = byCollectionByShard
.getOrDefault(coll, Collections.emptyMap())
.getOrDefault(shard, Collections.emptyList());
if (replicas.isEmpty()) {
return;
}
// only delete if there's at least minRF searchable replicas left
int rf = searchableReplicationFactors.get(coll).get(shard).get();
// assume first that we only really need a leader and we may be
// allowed to remove other replicas
int minRF = 1;
// but check the official RF and don't go below that
Integer RF = clusterState.getCollection(coll).getReplicationFactor();
if (RF != null) {
minRF = RF;
}
// unless minReplicas is set explicitly
if (minReplicas != null) {
minRF = minReplicas;
}
if (minRF < 1) {
minRF = 1;
}
if (rf > minRF) {
// delete at most maxOps replicas at a time
AtomicInteger limit = new AtomicInteger(Math.min(maxOps, rf - minRF));
replicas.forEach(ri -> {
if (limit.get() == 0) {
return;
}
// don't delete a leader
if (ri.getBool(ZkStateReader.LEADER_PROP, false)) {
return;
}
TriggerEvent.Op op = new TriggerEvent.Op(belowOp,
Suggester.Hint.COLL_SHARD, new Pair<>(ri.getCollection(), ri.getShard()));
op.addHint(Suggester.Hint.REPLICA, ri.getName());
ops.add(op);
violations.add(COLD_SHARDS);
limit.decrementAndGet();
});
}
});
});
// COLD NODES:
// Unlike the case of hot nodes, if a node is cold then any monitored
// collections / shards / replicas located on that node are cold, too.
// HOWEVER, we check only replicas from selected collections / shards,
// so deleting a cold node is dangerous because it may interfere with these
// non-monitored resources - this is the reason the default belowNodeOp is null / ignored.
//
// Also, note that due to the way activity is measured only nodes that contain any
// monitored resources are considered - there may be cold nodes in the cluster that don't
// belong to the monitored collections and they will be ignored.
if (belowNodeOp != null) {
coldNodes.forEach((node, rate) -> {
ops.add(new TriggerEvent.Op(belowNodeOp, Suggester.Hint.SRC_NODE, node));
violations.add(COLD_NODES);
});
}
}
private boolean waitForElapsed(String name, long now, Map<String, Long> lastEventMap) {
Long lastTime = lastEventMap.computeIfAbsent(name, s -> now);
long elapsed = TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS);
if (log.isTraceEnabled()) {
log.trace("name={}, lastTime={}, elapsed={}, waitFor={}", name, lastTime, elapsed, getWaitForSecond());
}
if (TimeUnit.SECONDS.convert(now - lastTime, TimeUnit.NANOSECONDS) < getWaitForSecond()) {
return false;
}
return true;
}
public static class SearchRateEvent extends TriggerEvent {
public SearchRateEvent(String source, long eventTime, List<Op> ops,
Map<String, Double> hotNodes,
Map<String, Double> hotCollections,
Map<String, Map<String, Double>> hotShards,
List<Replica> hotReplicas,
Map<String, Double> coldNodes,
Map<String, Double> coldCollections,
Map<String, Map<String, Double>> coldShards,
List<Replica> coldReplicas,
Set<String> violations) {
super(TriggerEventType.SEARCHRATE, source, eventTime, null);
properties.put(TriggerEvent.REQUESTED_OPS, ops);
properties.put(HOT_NODES, hotNodes);
properties.put(HOT_COLLECTIONS, hotCollections);
properties.put(HOT_SHARDS, hotShards);
properties.put(HOT_REPLICAS, hotReplicas);
properties.put(COLD_NODES, coldNodes);
properties.put(COLD_COLLECTIONS, coldCollections);
properties.put(COLD_SHARDS, coldShards);
properties.put(COLD_REPLICAS, coldReplicas);
properties.put(VIOLATION_PROP, violations);
}
}
}

View File

@ -1,221 +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.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.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
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.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
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.SolrResourceLoader;
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;
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
super.configure(loader, cloudManager, config);
collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
Throwable error, String message) throws Exception {
try {
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
DocCollection coll = clusterState.getCollectionOrNull(collection);
if (coll == null) {
log.debug("Collection {} missing, skip sending event {}", collection, event);
return;
}
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);
req.setParam(CollectionAdminParams.COLLECTION, collection);
cloudManager.request(req);
} catch (Exception e) {
if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
// relatively benign but log this - collection still existed when we started
log.info("Collection {} missing, skip sending event {}", collection, event);
} else {
log.warn("Exception sending event. Collection: {}, event: {}, exception: {}", collection, event, 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));
}
});
}
@SuppressWarnings({"rawtypes"})
private void addOperations(SolrInputDocument doc, List<SolrRequest> operations) {
if (operations == null || operations.isEmpty()) {
return;
}
Set<String> collections = new HashSet<>();
for (SolrRequest req : operations) {
SolrParams params = req.getParams();
if (params == null) {
continue;
}
if (params.get(CollectionAdminParams.COLLECTION) != null) {
collections.add(params.get(CollectionAdminParams.COLLECTION));
}
// build a whitespace-separated param string
StringJoiner paramJoiner = new StringJoiner(" ");
paramJoiner.setEmptyValue("");
for (Iterator<String> it = params.getParameterNamesIterator(); it.hasNext(); ) {
final String name = it.next();
final String [] values = params.getParams(name);
for (String value : values) {
paramJoiner.add(name + "=" + value);
}
}
String paramString = paramJoiner.toString();
if (!paramString.isEmpty()) {
doc.addField("operations.params_ts", paramString);
}
}
if (!collections.isEmpty()) {
doc.addField(COLLECTIONS_FIELD, collections);
}
}
private void addResponses(SolrInputDocument doc, List<NamedList<Object>> responses) {
if (responses == null || responses.isEmpty()) {
return;
}
for (NamedList<Object> rsp : responses) {
Object o = rsp.get("success");
if (o != null) {
doc.addField("responses_ts", "success " + o);
} else {
o = rsp.get("failure");
if (o != null) {
doc.addField("responses_ts", "failure " + o);
} else { // something else
doc.addField("responses_ts", Utils.toJSONString(rsp));
}
}
}
}
private void addError(SolrInputDocument doc, Throwable error) {
if (error == null) {
return;
}
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
error.printStackTrace(pw);
pw.flush(); pw.close();
doc.addField("error.message_t", error.getMessage());
doc.addField("error.details_t", sw.toString());
}
}

View File

@ -1,51 +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.autoscaling;
import java.io.Closeable;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.core.SolrResourceLoader;
/**
* Interface for actions performed in response to a trigger being activated
*/
public interface TriggerAction extends Closeable {
/**
* Called when action is created but before it's initialized and used.
* This method should also verify that the configuration parameters are correct.
* It may be called multiple times.
* @param loader loader to use for instantiating sub-components
* @param cloudManager current instance of SolrCloudManager
* @param properties configuration properties
* @throws TriggerValidationException contains details of invalid configuration parameters.
*/
void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException;
/**
* Called before an action is first used. Any heavy object creation or initialization should
* be done in this method instead of the constructor or {@link #configure(SolrResourceLoader, SolrCloudManager, Map)} method.
*/
void init() throws Exception;
String getName();
void process(TriggerEvent event, ActionContext context) throws Exception;
}

View File

@ -1,87 +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.autoscaling;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.core.SolrResourceLoader;
/**
* Base class for {@link TriggerAction} implementations.
*/
public abstract class TriggerActionBase implements TriggerAction {
protected Map<String, Object> properties = new HashMap<>();
protected SolrResourceLoader loader;
protected SolrCloudManager cloudManager;
/**
* Set of valid property names. Subclasses may add to this set
* using {@link TriggerUtils#validProperties(Set, String...)}
*/
protected final Set<String> validProperties = new HashSet<>();
/**
* Set of required property names. Subclasses may add to this set
* using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
* (required properties are also valid properties).
*/
protected final Set<String> requiredProperties = new HashSet<>();
protected TriggerActionBase() {
// not strictly needed here because they are already checked during instantiation
TriggerUtils.validProperties(validProperties, "name", "class");
}
@Override
public String getName() {
String name = (String) properties.get("name");
if (name != null) {
return name;
} else {
return getClass().getSimpleName();
}
}
@Override
public void close() throws IOException {
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
this.loader = loader;
this.cloudManager = cloudManager;
if (properties != null) {
this.properties.putAll(properties);
}
// validate the config
Map<String, String> results = new HashMap<>();
TriggerUtils.checkProperties(this.properties, results, requiredProperties, validProperties);
if (!results.isEmpty()) {
throw new TriggerValidationException(getName(), results);
}
}
@Override
public void init() throws Exception {
}
}

View File

@ -1,33 +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.autoscaling;
/**
* Trigger action-specific exception.
*/
public class TriggerActionException extends Exception {
public final String triggerName;
public final String actionName;
public TriggerActionException(String triggerName, String actionName, String message, Throwable cause) {
super(message, cause);
this.triggerName = triggerName;
this.actionName = actionName;
}
}

View File

@ -1,300 +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.autoscaling;
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.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.util.IOUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
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 final String name;
protected SolrCloudManager cloudManager;
protected SolrResourceLoader loader;
protected DistribStateManager stateManager;
protected final Map<String, Object> properties = new HashMap<>();
/**
* Set of valid property names. Subclasses may add to this set
* using {@link TriggerUtils#validProperties(Set, String...)}
*/
protected final Set<String> validProperties = new HashSet<>();
/**
* Set of required property names. Subclasses may add to this set
* using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
* (required properties are also valid properties).
*/
protected final Set<String> requiredProperties = new HashSet<>();
protected final TriggerEventType eventType;
protected int waitForSecond;
protected Map<String,Object> lastState;
protected final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef = new AtomicReference<>();
protected List<TriggerAction> actions;
protected boolean enabled;
protected boolean isClosed;
protected TriggerBase(TriggerEventType eventType, String name) {
this.eventType = eventType;
this.name = name;
// subclasses may further modify this set to include other supported properties
TriggerUtils.validProperties(validProperties, "name", "class", "event", "enabled", "waitFor", "actions");
}
/**
* Return a set of valid property names supported by this trigger.
*/
public final Set<String> getValidProperties() {
return Collections.unmodifiableSet(this.validProperties);
}
/**
* Return a set of required property names supported by this trigger.
*/
public final Set<String> getRequiredProperties() {
return Collections.unmodifiableSet(this.requiredProperties);
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
this.cloudManager = cloudManager;
this.loader = loader;
this.stateManager = cloudManager.getDistribStateManager();
if (properties != null) {
this.properties.putAll(properties);
}
this.enabled = Boolean.parseBoolean(String.valueOf(this.properties.getOrDefault("enabled", "true")));
this.waitForSecond = ((Number) this.properties.getOrDefault("waitFor", -1L)).intValue();
@SuppressWarnings({"unchecked"})
List<Map<String, Object>> o = (List<Map<String, Object>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
actions = new ArrayList<>(3);
for (Map<String, Object> map : o) {
TriggerAction action = null;
try {
action = loader.newInstance((String)map.get("class"), TriggerAction.class);
} catch (Exception e) {
throw new TriggerValidationException("action", "exception creating action " + map + ": " + e.toString());
}
action.configure(loader, cloudManager, map);
actions.add(action);
}
} else {
actions = Collections.emptyList();
}
Map<String, String> results = new HashMap<>();
TriggerUtils.checkProperties(this.properties, results, requiredProperties, validProperties);
if (!results.isEmpty()) {
throw new TriggerValidationException(name, results);
}
}
@Override
public void init() throws Exception {
try {
if (!stateManager.hasData(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH)) {
stateManager.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
}
} catch (AlreadyExistsException e) {
// ignore
} catch (InterruptedException | KeeperException | IOException e) {
log.warn("Exception checking ZK path {}", ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, e);
throw e;
}
for (TriggerAction action : actions) {
action.init();
}
}
@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 isClosed() {
synchronized (this) {
return isClosed;
}
}
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
IOUtils.closeWhileHandlingException(actions);
}
}
@Override
public int hashCode() {
return Objects.hash(name, properties);
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj.getClass().equals(this.getClass())) {
TriggerBase that = (TriggerBase) obj;
return this.name.equals(that.name)
&& this.properties.equals(that.properties);
}
return false;
}
/**
* 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);
/**
* Returns an immutable deep copy of this trigger's state, suitible for saving.
* This method is public only for tests that wish to do grey-box introspection
*
* @see #getState
* @lucene.internal
*/
@SuppressWarnings({"unchecked"})
public Map<String,Object> deepCopyState() {
return Utils.getDeepCopy(getState(), 10, false, true);
}
@Override
public void saveState() {
Map<String,Object> state = deepCopyState();
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 (stateManager.hasData(path)) {
// update
stateManager.setData(path, data, -1);
} else {
// create
stateManager.createData(path, data, CreateMode.PERSISTENT);
}
lastState = state;
} catch (AlreadyExistsException e) {
} catch (InterruptedException | BadVersionException | IOException | KeeperException e) {
log.warn("Exception updating trigger state '{}'", path, e);
}
}
@Override
@SuppressWarnings({"unchecked"})
public void restoreState() {
byte[] data = null;
String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName();
try {
if (stateManager.hasData(path)) {
VersionedData versionedData = stateManager.getData(path);
data = versionedData.getData();
}
} catch (AlreadyClosedException e) {
} catch (Exception e) {
log.warn("Exception getting trigger state '{}'", path, e);
}
if (data != null) {
Map<String, Object> restoredState = (Map<String, Object>)Utils.fromJSON(data);
// make sure lastState is sorted
restoredState = Utils.getDeepCopy(restoredState, 10, false, true);
setState(restoredState);
lastState = restoredState;
}
}
}

View File

@ -1,315 +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.autoscaling;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.IdUtils;
/**
* Trigger event.
*/
public class TriggerEvent implements MapWriter {
public static final String IGNORED = "ignored";
public static final String COOLDOWN = "cooldown";
public static final String REPLAYING = "replaying";
public static final String NODE_NAMES = "nodeNames";
public static final String EVENT_TIMES = "eventTimes";
public static final String REQUESTED_OPS = "requestedOps";
public static final String UNSUPPORTED_OPS = "unsupportedOps";
public static final class Op implements MapWriter {
private final CollectionParams.CollectionAction action;
private final EnumMap<Suggester.Hint, Object> hints = new EnumMap<>(Suggester.Hint.class);
public Op(CollectionParams.CollectionAction action) {
this.action = action;
}
public Op(CollectionParams.CollectionAction action, Suggester.Hint hint, Object hintValue) {
this.action = action;
addHint(hint, hintValue);
}
@SuppressWarnings({"unchecked"})
public void addHint(Suggester.Hint hint, Object value) {
hint.validator.accept(value);
if (hint.multiValued) {
Collection<?> values = value instanceof Collection ? (Collection) value : Collections.singletonList(value);
((Set) hints.computeIfAbsent(hint, h -> new LinkedHashSet<>())).addAll(values);
} else if (value instanceof Map) {
hints.put(hint, value);
} else {
hints.put(hint, value == null ? null : String.valueOf(value));
}
}
public CollectionParams.CollectionAction getAction() {
return action;
}
public EnumMap<Suggester.Hint, Object> getHints() {
return hints;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put("action", action);
ew.put("hints", hints);
}
@SuppressWarnings({"unchecked", "rawtypes"})
public static Op fromMap(Map<String, Object> map) {
if (!map.containsKey("action")) {
return null;
}
CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(String.valueOf(map.get("action")));
if (action == null) {
return null;
}
Op op = new Op(action);
Map<Object, Object> hints = (Map<Object, Object>)map.get("hints");
if (hints != null && !hints.isEmpty()) {
hints.forEach((k, v) -> {
Suggester.Hint h = Suggester.Hint.get(k.toString());
if (h == null) {
return;
}
if (!(v instanceof Collection)) {
v = Collections.singletonList(v);
}
((Collection)v).forEach(vv -> {
if (vv instanceof Map) {
// maybe it's a Pair?
Map<String, Object> m = (Map<String, Object>)vv;
if (m.containsKey("first") && m.containsKey("second")) {
Pair p = Pair.parse(m);
if (p != null) {
op.addHint(h, p);
return;
}
}
}
op.addHint(h, vv);
});
});
}
return op;
}
@Override
public String toString() {
return "Op{" +
"action=" + action +
", hints=" + hints +
'}';
}
}
protected final String id;
protected final String source;
protected final long eventTime;
protected final TriggerEventType eventType;
protected final Map<String, Object> properties = new HashMap<>();
protected final boolean ignored;
public TriggerEvent(TriggerEventType eventType, String source, long eventTime,
Map<String, Object> properties) {
this(IdUtils.timeRandomId(eventTime), eventType, source, eventTime, properties, false);
}
public TriggerEvent(TriggerEventType eventType, String source, long eventTime,
Map<String, Object> properties, boolean ignored) {
this(IdUtils.timeRandomId(eventTime), eventType, source, eventTime, properties, ignored);
}
public TriggerEvent(String id, TriggerEventType eventType, String source, long eventTime,
Map<String, Object> properties) {
this(id, eventType, source, eventTime, properties, false);
}
public TriggerEvent(String id, TriggerEventType eventType, String source, long eventTime,
Map<String, Object> properties, boolean ignored) {
this.id = id;
this.eventType = eventType;
this.source = source;
this.eventTime = eventTime;
if (properties != null) {
this.properties.putAll(properties);
}
this.ignored = ignored;
}
/**
* 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);
}
/**
* Get a named event property or default value if missing.
*/
public Object getProperty(String name, Object defaultValue) {
Object v = properties.get(name);
if (v == null) {
return defaultValue;
} else {
return v;
}
}
/**
* Event type.
*/
public TriggerEventType getEventType() {
return eventType;
}
public boolean isIgnored() {
return ignored;
}
/**
* 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);
if (ignored) {
ew.put("ignored", true);
}
}
@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;
if (ignored != that.ignored) 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();
result = 31 * result + Boolean.hashCode(ignored);
return result;
}
@Override
public String toString() {
return Utils.toJSONString(this);
}
@SuppressWarnings({"unchecked"})
public static 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");
// properly deserialize some well-known complex properties
fixOps(TriggerEvent.REQUESTED_OPS, properties);
fixOps(TriggerEvent.UNSUPPORTED_OPS, properties);
TriggerEvent res = new TriggerEvent(id, eventType, source, eventTime, properties);
return res;
}
@SuppressWarnings({"unchecked"})
public static void fixOps(String type, Map<String, Object> properties) {
List<Object> ops = (List<Object>)properties.get(type);
if (ops != null && !ops.isEmpty()) {
for (int i = 0; i < ops.size(); i++) {
Object o = ops.get(i);
if (o instanceof Map) {
TriggerEvent.Op op = TriggerEvent.Op.fromMap((Map)o);
if (op != null) {
ops.set(i, op);
}
}
}
}
}
}

View File

@ -1,121 +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.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.charset.StandardCharsets;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.common.util.TimeSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
*
*/
public class TriggerEventQueue {
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;
private final DistributedQueue delegate;
public TriggerEventQueue(SolrCloudManager cloudManager, String triggerName, Stats stats) throws IOException {
// TODO: collect stats
this.delegate = cloudManager.getDistributedQueueFactory().makeQueue(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName);
this.triggerName = triggerName;
this.timeSource = cloudManager.getTimeSource();
}
public boolean offerEvent(TriggerEvent event) {
event.getProperties().put(ENQUEUE_TIME, timeSource.getTimeNs());
try {
byte[] data = Utils.toJSON(event);
delegate.offer(data);
return true;
} catch (Exception e) {
log.warn("Exception adding event {} to queue {}", event, triggerName, e);
return false;
}
}
public TriggerEvent peekEvent() {
byte[] data;
try {
while ((data = delegate.peek()) != null) {
if (data.length == 0) {
log.warn("ignoring empty data...");
continue;
}
try {
@SuppressWarnings({"unchecked"})
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, StandardCharsets.UTF_8));
continue;
}
}
}
catch (AlreadyClosedException e) {
}
catch (Exception e) {
log.warn("Exception peeking queue of trigger {}", triggerName, e);
}
return null;
}
public TriggerEvent pollEvent() {
byte[] data;
try {
while ((data = delegate.poll()) != null) {
if (data.length == 0) {
log.warn("ignoring empty data...");
continue;
}
try {
@SuppressWarnings({"unchecked"})
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, StandardCharsets.UTF_8));
continue;
}
}
} catch (Exception e) {
log.warn("Exception polling queue of trigger {}", triggerName, e);
}
return null;
}
private TriggerEvent fromMap(Map<String, Object> map) {
TriggerEvent res = TriggerEvent.fromMap(map);
res.getProperties().put(DEQUEUE_TIME, timeSource.getTimeNs());
return res;
}
}

View File

@ -1,65 +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.autoscaling;
import java.io.Closeable;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.core.SolrResourceLoader;
/**
* 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 {
/**
* Called when listener is created but before it's initialized and used.
* This method should also verify that the configuration parameters are correct.
* It may be called multiple times.
* @param loader loader to use for instantiating sub-components
* @param cloudManager current instance of SolrCloudManager
* @param config coniguration
* @throws TriggerValidationException contains details of invalid configuration parameters.
*/
void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException;
/**
* If this method returns false then the listener's {@link #onEvent(TriggerEvent, TriggerEventProcessorStage, String, ActionContext, Throwable, String)}
* method should not be called.
*/
boolean isEnabled();
void init() throws Exception;
AutoScalingConfig.TriggerListenerConfig getConfig();
/**
* This method is called when either a particular <code>stage</code> or
* <code>actionName</code> is reached during event processing.
* @param event current event being processed
* @param stage {@link TriggerEventProcessorStage} that this listener was registered for, or null
* @param actionName {@link TriggerAction} name that this listener was registered for, or null
* @param context optional {@link ActionContext} when the processing stage is related to an action, or null
* @param error optional {@link Throwable} error, or null
* @param message optional message
*/
void onEvent(TriggerEvent event, TriggerEventProcessorStage stage, String actionName, ActionContext context,
Throwable error, String message) throws Exception;
}

View File

@ -1,97 +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.autoscaling;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.core.SolrResourceLoader;
/**
* Base class for implementations of {@link TriggerListener}.
*/
public abstract class TriggerListenerBase implements TriggerListener {
protected AutoScalingConfig.TriggerListenerConfig config;
protected SolrCloudManager cloudManager;
protected SolrResourceLoader loader;
protected boolean enabled;
/**
* Set of valid property names. Subclasses may add to this set
* using {@link TriggerUtils#validProperties(Set, String...)}
*/
protected final Set<String> validProperties = new HashSet<>();
/**
* Set of required property names. Subclasses may add to this set
* using {@link TriggerUtils#requiredProperties(Set, Set, String...)}
* (required properties are also valid properties).
*/
protected final Set<String> requiredProperties = new HashSet<>();
/**
* Subclasses can add to this set if they want to allow arbitrary properties that
* start with one of valid prefixes.
*/
protected final Set<String> validPropertyPrefixes = new HashSet<>();
protected TriggerListenerBase() {
TriggerUtils.requiredProperties(requiredProperties, validProperties, "trigger");
TriggerUtils.validProperties(validProperties, "name", "class", "stage", "beforeAction", "afterAction", "enabled");
}
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, AutoScalingConfig.TriggerListenerConfig config) throws TriggerValidationException {
this.loader = loader;
this.cloudManager = cloudManager;
this.config = config;
this.enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
// validate the config
Map<String, String> results = new HashMap<>();
// prepare a copy to treat the prefix-based properties
Map<String, Object> propsToCheck = new HashMap<>(config.properties);
propsToCheck.keySet().removeIf(k ->
validPropertyPrefixes.stream().anyMatch(p -> k.startsWith(p)));
TriggerUtils.checkProperties(propsToCheck, results, requiredProperties, validProperties);
if (!results.isEmpty()) {
throw new TriggerValidationException(config.name, results);
}
}
@Override
public AutoScalingConfig.TriggerListenerConfig getConfig() {
return config;
}
@Override
public boolean isEnabled() {
return enabled;
}
@Override
public void init() throws Exception {
}
@Override
public void close() throws IOException {
}
}

View File

@ -1,88 +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.autoscaling;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
/**
*
*/
public class TriggerUtils {
// validation helper methods
public static void requiredProperties(Set<String> required, Set<String> valid, String... propertyNames) {
required.addAll(Arrays.asList(propertyNames));
valid.addAll(Arrays.asList(propertyNames));
}
public static void validProperties(Set<String> valid, String... propertyNames) {
valid.addAll(Arrays.asList(propertyNames));
}
public static void checkProperties(Map<String, Object> properties, Map<String, String> results, Set<String> required, Set<String> valid) {
checkValidPropertyNames(properties, results, valid);
checkRequiredPropertyNames(properties, results, required);
}
public static void checkValidPropertyNames(Map<String, Object> properties, Map<String, String> results, Set<String> valid) {
Set<String> currentNames = new HashSet<>(properties.keySet());
currentNames.removeAll(valid);
if (!currentNames.isEmpty()) {
for (String name : currentNames) {
results.put(name, "unknown property");
}
}
}
public static void checkRequiredPropertyNames(Map<String, Object> properties, Map<String, String> results, Set<String> required) {
Set<String> requiredNames = new HashSet<>(required);
requiredNames.removeAll(properties.keySet());
if (!requiredNames.isEmpty()) {
for (String name : requiredNames) {
results.put(name, "missing required property");
}
}
}
@SuppressWarnings({"unchecked", "rawtypes"})
public static void checkProperty(Map<String, Object> properties, Map<String, String> results, String name, boolean required, Class... acceptClasses) {
Object value = properties.get(name);
if (value == null) {
if (required) {
results.put(name, "missing required value");
} else {
return;
}
}
if (acceptClasses == null || acceptClasses.length == 0) {
return;
}
boolean accepted = false;
for (Class clz : acceptClasses) {
if (clz.isAssignableFrom(value.getClass())) {
accepted = true;
break;
}
}
if (!accepted) {
results.put(name, "value is not an expected type");
}
}
}

View File

@ -1,74 +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.autoscaling;
import java.util.HashMap;
import java.util.Map;
/**
* This class represents errors found when validating trigger configuration.
*/
public class TriggerValidationException extends Exception {
private final Map<String, String> details = new HashMap<>();
private final String name;
/**
* Create an exception.
* @param name name of the trigger / action / listener that caused the exception
* @param details details of invalid configuration - key is a property name,
* value is an error message.
*/
public TriggerValidationException(String name, Map<String, String> details) {
super();
this.name = name;
if (details != null) {
this.details.putAll(details);
}
}
/**
* Create an exception.
* @param name name of the trigger / action / listener that caused the exception
* @param keyValues zero or even number of arguments representing symbolic key
* (eg. property name) and the corresponding validation error message.
*/
public TriggerValidationException(String name, String... keyValues) {
super();
this.name = name;
if (keyValues == null || keyValues.length == 0) {
return;
}
if (keyValues.length % 2 != 0) {
throw new IllegalArgumentException("number of arguments representing key & value pairs must be even");
}
for (int i = 0; i < keyValues.length; i += 2) {
details.put(keyValues[i], keyValues[i + 1]);
}
}
public Map<String, String> getDetails() {
return details;
}
@Override
public String toString() {
return "TriggerValidationException{" +
"name=" + name +
", details='" + details + '\'' +
'}';
}
}

View File

@ -1,21 +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 for classes related to autoscaling
*/
package org.apache.solr.cloud.autoscaling;

View File

@ -1,24 +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.autoscaling.sim;
/**
* Interface that helps simulating action errors.
*/
public interface ActionError {
boolean shouldFail(String... args);
}

View File

@ -1,56 +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.autoscaling.sim;
import java.util.Iterator;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.SolrInputField;
/**
* Lightweight generator of fake documents
* NOTE: this iterator only ever returns the same document N times, which works ok
* for our "bulk index update" simulation. Obviously don't use this for real indexing.
*/
public class FakeDocIterator implements Iterator<SolrInputDocument> {
final SolrInputDocument doc = new SolrInputDocument();
final SolrInputField idField = new SolrInputField("id");
final long start, count;
long current, max;
FakeDocIterator(long start, long count) {
this.start = start;
this.count = count;
current = start;
max = start + count;
doc.put("id", idField);
idField.setValue("foo");
}
@Override
public boolean hasNext() {
return current < max;
}
@Override
public SolrInputDocument next() {
current++;
return doc;
}
}

View File

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

View File

@ -1,45 +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.autoscaling.sim;
import java.io.IOException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
/**
* Factory for {@link GenericDistributedQueue}.
*/
public class GenericDistributedQueueFactory implements DistributedQueueFactory {
private final DistribStateManager stateManager;
public GenericDistributedQueueFactory(DistribStateManager stateManager) {
this.stateManager = stateManager;
}
@Override
public DistributedQueue makeQueue(String path) throws IOException {
return new GenericDistributedQueue(stateManager, path);
}
@Override
public void removeQueue(String path) throws IOException {
}
}

View File

@ -1,113 +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.autoscaling.sim;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.solr.common.cloud.LiveNodesListener;
/**
* This class represents a set of live nodes and allows adding listeners to track their state.
*/
public class LiveNodesSet implements Iterable<String> {
private final Set<String> set = ConcurrentHashMap.newKeySet();
private final Set<LiveNodesListener> listeners = ConcurrentHashMap.newKeySet();
public Set<String> get() {
return Collections.unmodifiableSet(set);
}
public int size() {
return set.size();
}
public void registerLiveNodesListener(LiveNodesListener listener) {
listeners.add(listener);
}
public void removeLiveNodesListener(LiveNodesListener listener) {
listeners.remove(listener);
}
public void removeAllLiveNodesListeners() {
listeners.clear();
}
private void fireListeners(SortedSet<String> oldNodes, SortedSet<String> newNodes) {
for (LiveNodesListener listener : listeners) {
listener.onChange(oldNodes, newNodes);
}
}
public boolean isEmpty() {
return set.isEmpty();
}
public boolean contains(String id) {
return set.contains(id);
}
public synchronized boolean add(String id) {
if (set.contains(id)) {
return false;
}
TreeSet<String> oldNodes = new TreeSet<>(set);
set.add(id);
TreeSet<String> newNodes = new TreeSet<>(set);
fireListeners(oldNodes, newNodes);
return true;
}
public synchronized boolean addAll(Collection<String> nodes) {
TreeSet<String> oldNodes = new TreeSet<>(set);
boolean changed = set.addAll(nodes);
TreeSet<String> newNodes = new TreeSet<>(set);
if (changed) {
fireListeners(oldNodes, newNodes);
}
return changed;
}
public synchronized boolean remove(String id) {
if (!set.contains(id)) {
return false;
}
TreeSet<String> oldNodes = new TreeSet<>(set);
set.remove(id);
TreeSet<String> newNodes = new TreeSet<>(set);
fireListeners(oldNodes, newNodes);
return true;
}
public synchronized void clear() {
TreeSet<String> oldNodes = new TreeSet<>(set);
set.clear();
fireListeners(oldNodes, Collections.emptySortedSet());
}
@Override
public Iterator<String> iterator() {
return set.iterator();
}
}

View File

@ -1,94 +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.autoscaling.sim;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.function.Predicate;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.common.util.Pair;
/**
* A queue factory implementation that does nothing.
*/
public class NoopDistributedQueueFactory implements DistributedQueueFactory {
public static final DistributedQueueFactory INSTANCE = new NoopDistributedQueueFactory();
@Override
public DistributedQueue makeQueue(String path) throws IOException {
return NoopDistributedQueue.INSTANCE;
}
@Override
public void removeQueue(String path) throws IOException {
}
private static final class NoopDistributedQueue implements DistributedQueue {
static final DistributedQueue INSTANCE = new NoopDistributedQueue();
@Override
public byte[] peek() throws Exception {
return new byte[0];
}
@Override
public byte[] peek(boolean block) throws Exception {
return new byte[0];
}
@Override
public byte[] peek(long wait) throws Exception {
return new byte[0];
}
@Override
public byte[] poll() throws Exception {
return new byte[0];
}
@Override
public byte[] remove() throws Exception {
return new byte[0];
}
@Override
public byte[] take() throws Exception {
return new byte[0];
}
@Override
public void offer(byte[] data) throws Exception {
}
@Override
public Map<String, Object> getStats() {
return Collections.emptyMap();
}
@Override
public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
return Collections.emptyList();
}
}
}

View File

@ -1,649 +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.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.jute.Record;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.NotEmptyException;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.cloud.ActionThrottle;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.Utils;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.util.IdUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.CheckVersionRequest;
import org.apache.zookeeper.proto.CreateRequest;
import org.apache.zookeeper.proto.DeleteRequest;
import org.apache.zookeeper.proto.SetDataRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simulated {@link DistribStateManager} that keeps all data locally in a static structure. Instances of this
* class are identified by their id in order to simulate the deletion of ephemeral nodes when {@link #close()} is
* invoked.
*/
public class SimDistribStateManager implements DistribStateManager {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final class Node {
ReentrantLock dataLock = new ReentrantLock();
private int version = 0;
private int seq = 0;
private final CreateMode mode;
// copyFrom needs to modify this
private String owner;
private final String path;
private final String name;
private final Node parent;
private byte[] data = null;
private Map<String, Node> children = new ConcurrentHashMap<>();
Set<Watcher> dataWatches = ConcurrentHashMap.newKeySet();
Set<Watcher> childrenWatches = ConcurrentHashMap.newKeySet();
Node(Node parent, String name, String path, CreateMode mode, String owner) {
this.parent = parent;
this.name = name;
this.path = path;
this.mode = mode;
this.owner = owner;
}
Node(Node parent, String name, String path, byte[] data, CreateMode mode, String owner) {
this(parent, name, path, mode, owner);
this.data = data;
}
public void clear() {
dataLock.lock();
try {
children.clear();
version = 0;
seq = 0;
dataWatches.clear();
childrenWatches.clear();
data = null;
} finally {
dataLock.unlock();
}
}
public void setData(byte[] data, int version) throws BadVersionException, IOException {
Set<Watcher> currentWatchers = new HashSet<>(dataWatches);
dataLock.lock();
try {
if (version != -1 && version != this.version) {
throw new BadVersionException(version, path);
}
if (data != null) {
this.data = Arrays.copyOf(data, data.length);
} else {
this.data = null;
}
this.version++;
dataWatches.clear();
} finally {
dataLock.unlock();
}
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeDataChanged, Watcher.Event.KeeperState.SyncConnected, path));
}
}
public VersionedData getData(Watcher w) {
dataLock.lock();
try {
VersionedData res = new VersionedData(version, data, mode, owner);
if (w != null && !dataWatches.contains(w)) {
dataWatches.add(w);
}
return res;
} finally {
dataLock.unlock();
}
}
public void setChild(String name, Node child) {
assert child.name.equals(name);
Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
dataLock.lock();
try {
children.put(name, child);
childrenWatches.clear();
} finally {
dataLock.unlock();
}
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
}
}
public void removeChild(String name, int version) throws NoSuchElementException, BadVersionException, IOException {
Node n = children.get(name);
if (n == null) {
throw new NoSuchElementException(path + "/" + name);
}
if (version != -1 && version != n.version) {
throw new BadVersionException(version, path);
}
children.remove(name);
Set<Watcher> currentWatchers = new HashSet<>(childrenWatches);
childrenWatches.clear();
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeChildrenChanged, Watcher.Event.KeeperState.SyncConnected, path));
}
currentWatchers = new HashSet<>(n.dataWatches);
n.dataWatches.clear();
for (Watcher w : currentWatchers) {
w.process(new WatchedEvent(Watcher.Event.EventType.NodeDeleted, Watcher.Event.KeeperState.SyncConnected, n.path));
}
// TODO: not sure if it's correct to recurse and fire watches???
Set<String> kids = new HashSet<>(n.children.keySet());
for (String kid : kids) {
n.removeChild(kid, -1);
}
}
public void removeEphemeralChildren(String id) throws NoSuchElementException, BadVersionException, IOException {
Set<String> kids = new HashSet<>(children.keySet());
for (String kid : kids) {
Node n = children.get(kid);
if (n == null) {
continue;
}
if ((CreateMode.EPHEMERAL == n.mode || CreateMode.EPHEMERAL_SEQUENTIAL == n.mode) &&
id.equals(n.owner)) {
removeChild(n.name, -1);
} else {
n.removeEphemeralChildren(id);
}
}
}
}
private final ReentrantLock multiLock = new ReentrantLock();
public static Node createNewRootNode() {
return new Node(null, "", "/", CreateMode.PERSISTENT, "0");
}
private final ExecutorService watchersPool;
private final AtomicReference<ActionThrottle> throttleRef = new AtomicReference<>();
private final AtomicReference<ActionError> errorRef = new AtomicReference<>();
private final String id;
private final Node root;
private int juteMaxbuffer = 0xfffff;
public SimDistribStateManager() {
this(null);
}
/**
* Construct new state manager that uses provided root node for storing data.
* @param root if null then a new root node will be created.
*/
public SimDistribStateManager(Node root) {
this.id = IdUtils.timeRandomId();
this.root = root != null ? root : createNewRootNode();
watchersPool = ExecutorUtil.newMDCAwareFixedThreadPool(10, new SolrNamedThreadFactory("sim-watchers"));
String bufferSize = System.getProperty("jute.maxbuffer", Integer.toString(0xffffff));
juteMaxbuffer = Integer.parseInt(bufferSize);
}
/**
* Copy all content from another DistribStateManager.
* @param other another state manager.
* @param failOnExists abort copy when one or more paths already exist (the state of this manager remains unchanged).
*/
public void copyFrom(DistribStateManager other, boolean failOnExists) throws InterruptedException, IOException, KeeperException, AlreadyExistsException, BadVersionException {
List<String> tree = other.listTree("/");
if (log.isInfoEnabled()) {
log.info("- copying {} resources...", tree.size());
}
// check if any node exists
for (String path : tree) {
if (hasData(path) && failOnExists) {
throw new AlreadyExistsException(path);
}
}
for (String path : tree) {
VersionedData data = other.getData(path);
if (hasData(path)) {
setData(path, data.getData(), -1);
} else {
makePath(path, data.getData(), data.getMode(), failOnExists);
}
// hack: set the version and owner to be the same as the source
Node n = traverse(path, false, CreateMode.PERSISTENT);
n.version = data.getVersion();
n.owner = data.getOwner();
}
}
public SimDistribStateManager(ActionThrottle actionThrottle, ActionError actionError) {
this(null, actionThrottle, actionError);
}
public SimDistribStateManager(Node root, ActionThrottle actionThrottle, ActionError actionError) {
this(root);
this.throttleRef.set(actionThrottle);
this.errorRef.set(actionError);
}
private SimDistribStateManager(String id, ExecutorService watchersPool, Node root, ActionThrottle actionThrottle,
ActionError actionError) {
this.id = id;
this.watchersPool = watchersPool;
this.root = root;
this.throttleRef.set(actionThrottle);
this.errorRef.set(actionError);
}
/**
* Create a copy of this instance using a specified ephemeral owner id. This is useful when performing
* node operations that require using a specific id. Note: this instance should never be closed, it can
* be just discarded after use.
* @param id ephemeral owner id
*/
public SimDistribStateManager withEphemeralId(String id) {
return new SimDistribStateManager(id, watchersPool, root, throttleRef.get(), errorRef.get()) {
@Override
public void close() {
throw new UnsupportedOperationException("this instance should never be closed - instead close the parent instance.");
}
};
}
/**
* Get the root node of the tree used by this instance. It could be a static shared root node.
*/
public Node getRoot() {
return root;
}
/**
* Clear this instance. All nodes, watchers and data is deleted.
*/
public void clear() {
root.clear();
}
private void throttleOrError(String path) throws IOException {
ActionError err = errorRef.get();
if (err != null && err.shouldFail(path)) {
throw new IOException("Simulated error, path=" + path);
}
ActionThrottle throttle = throttleRef.get();
if (throttle != null) {
throttle.minimumWaitBetweenActions();
throttle.markAttemptingAction();
}
}
// this method should always be invoked under lock
private Node traverse(String path, boolean create, CreateMode mode) throws IOException {
if (path == null || path.isEmpty()) {
return null;
}
throttleOrError(path);
if (path.equals("/")) {
return root;
}
if (path.charAt(0) == '/') {
path = path.substring(1);
}
StringBuilder currentPath = new StringBuilder();
String[] elements = path.split("/");
Node parentNode = root;
Node n = null;
for (int i = 0; i < elements.length; i++) {
String currentName = elements[i];
currentPath.append('/');
n = parentNode.children != null ? parentNode.children.get(currentName) : null;
if (n == null) {
if (create) {
n = createNode(parentNode, mode, currentPath, currentName,null, true);
} else {
break;
}
} else {
currentPath.append(currentName);
}
parentNode = n;
}
return n;
}
private Node createNode(Node parentNode, CreateMode mode, StringBuilder fullChildPath, String baseChildName, byte[] data, boolean attachToParent) throws IOException {
String nodeName = baseChildName;
if ((parentNode.mode == CreateMode.EPHEMERAL || parentNode.mode == CreateMode.EPHEMERAL_SEQUENTIAL) &&
(mode == CreateMode.EPHEMERAL || mode == CreateMode.EPHEMERAL_SEQUENTIAL)) {
throw new IOException("NoChildrenEphemerals for " + parentNode.path);
}
if (CreateMode.PERSISTENT_SEQUENTIAL == mode || CreateMode.EPHEMERAL_SEQUENTIAL == mode) {
nodeName = nodeName + String.format(Locale.ROOT, "%010d", parentNode.seq);
parentNode.seq++;
}
fullChildPath.append(nodeName);
String owner = mode == CreateMode.EPHEMERAL || mode == CreateMode.EPHEMERAL_SEQUENTIAL ? id : "0";
Node child = new Node(parentNode, nodeName, fullChildPath.toString(), data, mode, owner);
if (attachToParent) {
parentNode.setChild(nodeName, child);
}
return child;
}
@Override
public void close() throws IOException {
multiLock.lock();
try {
// remove all my ephemeral nodes
root.removeEphemeralChildren(id);
} catch (BadVersionException e) {
// not happening
} finally {
multiLock.unlock();
}
}
@Override
public boolean hasData(String path) throws IOException {
multiLock.lock();
try {
return traverse(path, false, CreateMode.PERSISTENT) != null;
} finally {
multiLock.unlock();
}
}
@Override
public List<String> listData(String path) throws NoSuchElementException, IOException {
multiLock.lock();
try {
Node n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
List<String> res = new ArrayList<>(n.children.keySet());
Collections.sort(res);
return res;
} finally {
multiLock.unlock();
}
}
@Override
public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException {
Node n;
List<String> res;
multiLock.lock();
try {
n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
res = new ArrayList<>(n.children.keySet());
Collections.sort(res);
} finally {
multiLock.unlock();
}
if (watcher != null) {
n.dataWatches.add(watcher);
n.childrenWatches.add(watcher);
}
return res;
}
@Override
public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException {
Node n = null;
multiLock.lock();
try {
n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
} finally {
multiLock.unlock();
}
return n.getData(watcher);
}
@Override
public void makePath(String path) throws IOException {
multiLock.lock();
try {
traverse(path, true, CreateMode.PERSISTENT);
} finally {
multiLock.unlock();
}
}
@Override
public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
Node n = null;
multiLock.lock();
try {
if (failOnExists && hasData(path)) {
throw new AlreadyExistsException(path);
}
n = traverse(path, true, createMode);
} finally {
multiLock.unlock();
}
try {
n.setData(data, -1);
} catch (BadVersionException e) {
throw new IOException("should not happen!", e);
}
}
@Override
public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, NoSuchElementException, IOException {
if ((CreateMode.EPHEMERAL == mode || CreateMode.PERSISTENT == mode) && hasData(path)) {
throw new AlreadyExistsException(path);
}
String relPath = path.charAt(0) == '/' ? path.substring(1) : path;
if (relPath.length() == 0) { //Trying to create root-node, return null.
// TODO should trying to create a root node throw an exception since its always init'd in the ctor?
return null;
}
String[] elements = relPath.split("/");
StringBuilder parentStringBuilder = new StringBuilder();
Node parentNode = null;
if (elements.length == 1) { // Direct descendant of '/'.
parentNode = getRoot();
} else { // Indirect descendant of '/', lookup parent node
for (int i = 0; i < elements.length - 1; i++) {
parentStringBuilder.append('/');
parentStringBuilder.append(elements[i]);
}
if (!hasData(parentStringBuilder.toString())) {
throw new NoSuchElementException(parentStringBuilder.toString());
}
parentNode = traverse(parentStringBuilder.toString(), false, mode);
}
multiLock.lock();
try {
String nodeName = elements[elements.length-1];
Node childNode = createNode(parentNode, mode, parentStringBuilder.append("/"), nodeName, data,false);
parentNode.setChild(childNode.name, childNode);
return childNode.path;
} finally {
multiLock.unlock();
}
}
@Override
public void removeData(String path, int version) throws NoSuchElementException, NotEmptyException, BadVersionException, IOException {
multiLock.lock();
Node parent;
Node n;
try {
n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
parent = n.parent;
if (parent == null) {
throw new IOException("Cannot remove root node");
}
if (!n.children.isEmpty()) {
throw new NotEmptyException(path);
}
} finally {
multiLock.unlock();
}
// outside the lock to avoid deadlock with update lock
parent.removeChild(n.name, version);
}
@Override
public void setData(String path, byte[] data, int version) throws NoSuchElementException, BadVersionException, IOException {
if (data != null && data.length > juteMaxbuffer) {
throw new IOException("Len error " + data.length);
}
multiLock.lock();
Node n = null;
try {
n = traverse(path, false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(path);
}
} finally {
multiLock.unlock();
}
n.setData(data, version);
}
@Override
public List<OpResult> multi(Iterable<Op> ops) throws BadVersionException, NoSuchElementException, AlreadyExistsException, IOException, KeeperException, InterruptedException {
multiLock.lock();
List<OpResult> res = new ArrayList<>();
try {
for (Op op : ops) {
Record r = op.toRequestRecord();
try {
if (op instanceof Op.Check) {
CheckVersionRequest rr = (CheckVersionRequest)r;
Node n = traverse(rr.getPath(), false, CreateMode.PERSISTENT);
if (n == null) {
throw new NoSuchElementException(rr.getPath());
}
if (rr.getVersion() != -1 && n.version != rr.getVersion()) {
throw new Exception("version mismatch");
}
// everything ok
res.add(new OpResult.CheckResult());
} else if (op instanceof Op.Create) {
CreateRequest rr = (CreateRequest)r;
createData(rr.getPath(), rr.getData(), CreateMode.fromFlag(rr.getFlags()));
res.add(new OpResult.CreateResult(rr.getPath()));
} else if (op instanceof Op.Delete) {
DeleteRequest rr = (DeleteRequest)r;
removeData(rr.getPath(), rr.getVersion());
res.add(new OpResult.DeleteResult());
} else if (op instanceof Op.SetData) {
SetDataRequest rr = (SetDataRequest)r;
setData(rr.getPath(), rr.getData(), rr.getVersion());
VersionedData vd = getData(rr.getPath());
Stat s = new Stat();
s.setVersion(vd.getVersion());
res.add(new OpResult.SetDataResult(s));
} else {
throw new Exception("Unknown Op: " + op);
}
} catch (Exception e) {
res.add(new OpResult.ErrorResult(KeeperException.Code.APIERROR.intValue()));
}
}
} finally {
multiLock.unlock();
}
return res;
}
@Override
@SuppressWarnings({"unchecked"})
public AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws InterruptedException, IOException {
Map<String, Object> map = new HashMap<>();
int version = 0;
try {
VersionedData data = getData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, watcher);
if (data != null && data.getData() != null && data.getData().length > 0) {
map = (Map<String, Object>) Utils.fromJSON(data.getData());
version = data.getVersion();
}
} catch (NoSuchElementException e) {
// ignore
}
map.put(AutoScalingParams.ZK_VERSION, version);
return new AutoScalingConfig(map);
}
// ------------ simulator methods --------------
public void simSetAutoScalingConfig(AutoScalingConfig cfg) throws Exception {
try {
makePath(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
} catch (Exception e) {
// ignore
}
setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(cfg), -1);
}
}

View File

@ -1,287 +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.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Predicate;
import com.codahale.metrics.Timer;
import com.google.common.base.Preconditions;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.cloud.Stats;
import org.apache.solr.common.util.Pair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simulated {@link DistributedQueueFactory} that keeps all data in memory. Unlike
* the {@link GenericDistributedQueueFactory} this queue implementation data is not
* exposed anywhere.
*/
public class SimDistributedQueueFactory implements DistributedQueueFactory {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
Map<String, SimDistributedQueue> queues = new ConcurrentHashMap<>();
public SimDistributedQueueFactory() {
}
@Override
public DistributedQueue makeQueue(final String path) throws IOException {
return queues.computeIfAbsent(path, p -> new SimDistributedQueue(path));
}
@Override
public void removeQueue(String path) throws IOException {
queues.remove(path);
}
public static class SimDistributedQueue implements DistributedQueue {
private final Queue<Pair<String, byte[]>> queue = new ConcurrentLinkedQueue<>();
private final ReentrantLock updateLock = new ReentrantLock();
private final Condition changed = updateLock.newCondition();
private final Stats stats = new Stats();
private final String dir;
private int seq = 0;
public SimDistributedQueue(String dir) {
this.dir = dir;
}
@Override
public byte[] peek() throws Exception {
Timer.Context time = stats.time(dir + "_peek");
try {
Pair<String, byte[]> pair = queue.peek();
return pair != null ? pair.second() : null;
} finally {
time.stop();
}
}
@Override
public byte[] peek(boolean block) throws Exception {
return block ? peek(Long.MAX_VALUE) : peek();
}
@Override
public byte[] peek(long wait) throws Exception {
Timer.Context time;
if (wait == Long.MAX_VALUE) {
time = stats.time(dir + "_peek_wait_forever");
} else {
time = stats.time(dir + "_peek_wait" + wait);
}
try {
Pair<String, byte[]> pair = peekInternal(wait);
return pair != null ? pair.second() : null;
} finally {
time.stop();
}
}
private Pair<String, byte[]> peekInternal(long wait) throws Exception {
Preconditions.checkArgument(wait > 0);
long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
updateLock.lockInterruptibly();
try {
while (waitNanos > 0) {
Pair<String, byte[]> pair = queue.peek();
if (pair != null) {
return pair;
}
waitNanos = changed.awaitNanos(waitNanos);
if (waitNanos < 0) { // timed out
return null;
}
}
} finally {
updateLock.unlock();
}
return null;
}
@Override
public byte[] poll() throws Exception {
Timer.Context time = stats.time(dir + "_poll");
updateLock.lockInterruptibly();
try {
Pair<String, byte[]> pair = queue.poll();
if (pair != null) {
changed.signalAll();
return pair.second();
} else {
return null;
}
} finally {
updateLock.unlock();
time.stop();
}
}
@Override
public byte[] remove() throws Exception {
Timer.Context time = stats.time(dir + "_remove");
updateLock.lockInterruptibly();
try {
byte[] res = queue.remove().second();
changed.signalAll();
return res;
} finally {
updateLock.unlock();
time.stop();
}
}
@Override
public byte[] take() throws Exception {
Timer.Context timer = stats.time(dir + "_take");
updateLock.lockInterruptibly();
try {
while (true) {
byte[] result = poll();
if (result != null) {
return result;
}
changed.await();
}
} finally {
updateLock.unlock();
timer.stop();
}
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void offer(byte[] data) throws Exception {
Timer.Context time = stats.time(dir + "_offer");
updateLock.lockInterruptibly();
try {
queue.offer(new Pair(String.format(Locale.ROOT, "qn-%010d", seq), data));
seq++;
if (log.isTraceEnabled()) {
log.trace("=== offer {}", System.nanoTime());
}
changed.signalAll();
} finally {
updateLock.unlock();
time.stop();
}
}
@Override
public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception {
updateLock.lockInterruptibly();
try {
List<Pair<String, byte[]>> res = new LinkedList<>();
final int maximum = max < 0 ? Integer.MAX_VALUE : max;
final AtomicReference<Pair<String, byte[]>> pairRef = new AtomicReference<>();
queue.forEach(pair -> {
if (acceptFilter != null && !acceptFilter.test(pair.first())) {
return;
}
if (res.size() < maximum) {
pairRef.set(pair);
res.add(pair);
}
});
if (res.size() < maximum && waitMillis > 0) {
long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
waitNanos = changed.awaitNanos(waitNanos);
if (waitNanos < 0) {
return res;
}
AtomicBoolean seen = new AtomicBoolean(false);
queue.forEach(pair -> {
if (!seen.get()) {
if (pairRef.get() == null) {
seen.set(true);
} else {
if (pairRef.get().first().equals(pair.first())) {
seen.set(true);
return;
}
}
}
if (!seen.get()) {
return;
}
if (!acceptFilter.test(pair.first())) {
return;
}
if (res.size() < maximum) {
res.add(pair);
pairRef.set(pair);
} else {
return;
}
});
}
return res;
} finally {
updateLock.unlock();
}
}
public Stats getZkStats() {
return stats;
}
@Override
public Map<String, Object> getStats() {
if (stats == null) {
return Collections.emptyMap();
}
Map<String, Object> res = new HashMap<>();
res.put("queueLength", stats.getQueueLength());
final Map<String, Object> statsMap = new HashMap<>();
res.put("stats", statsMap);
stats.getStats().forEach((op, stat) -> {
final Map<String, Object> statMap = new HashMap<>();
statMap.put("success", stat.success.get());
statMap.put("errors", stat.errors.get());
final List<Map<String, Object>> failed = new ArrayList<>(stat.failureDetails.size());
statMap.put("failureDetails", failed);
stat.failureDetails.forEach(failedOp -> {
Map<String, Object> fo = new HashMap<>();
fo.put("req", failedOp.req);
fo.put("resp", failedOp.resp);
});
statsMap.put(op, statMap);
});
return res;
}
}
}

View File

@ -1,359 +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.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Simulated {@link NodeStateProvider}.
* Note: in order to setup node-level metrics use {@link #simSetNodeValues(String, Map)}. However, in order
* to setup core-level metrics use {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean, boolean)}.
*/
public class SimNodeStateProvider implements NodeStateProvider {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final Map<String, Map<String, Object>> nodeValues = new ConcurrentHashMap<>();
private final SimClusterStateProvider clusterStateProvider;
private final SimDistribStateManager stateManager;
private final LiveNodesSet liveNodesSet;
private final ReentrantLock lock = new ReentrantLock();
public SimNodeStateProvider(LiveNodesSet liveNodesSet, SimDistribStateManager stateManager,
SimClusterStateProvider clusterStateProvider,
Map<String, Map<String, Object>> nodeValues) {
this.liveNodesSet = liveNodesSet;
this.stateManager = stateManager;
this.clusterStateProvider = clusterStateProvider;
if (nodeValues != null) {
this.nodeValues.putAll(nodeValues);
}
}
// -------- simulator setup methods ------------
/**
* Get a node value
* @param node node id
* @param key property name
* @return property value or null if property or node doesn't exist.
*/
public Object simGetNodeValue(String node, String key) {
Map<String, Object> values = nodeValues.get(node);
if (values == null) {
return null;
}
return values.get(key);
}
/**
* Atomically update a node value.
* @param node node id
* @param key property name
* @param updater updater function
* @return previous property value or null if property or node didn't exist.
*/
public Object simUpdateNodeValue(String node, String key, Function<Object, Object> updater) throws InterruptedException {
lock.lockInterruptibly();
try {
Map<String, Object> values = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
return values.put(key, updater.apply(values.get(key)));
} finally {
lock.unlock();
}
}
/**
* Set node values.
* NOTE: if values contain 'nodeRole' key then /roles.json is updated.
* @param node node id
* @param values values.
*/
public void simSetNodeValues(String node, Map<String, Object> values) throws InterruptedException {
lock.lockInterruptibly();
try {
Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
existing.clear();
if (values != null) {
existing.putAll(values);
}
if (values == null || values.isEmpty() || values.containsKey("nodeRole")) {
saveRoles();
}
} finally {
lock.unlock();
}
}
/**
* Set a node value, replacing any previous value.
* NOTE: if key is 'nodeRole' then /roles.json is updated.
* @param node node id
* @param key property name
* @param value property value
*/
public void simSetNodeValue(String node, String key, Object value) throws InterruptedException {
lock.lockInterruptibly();
try {
Map<String, Object> existing = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
if (value == null) {
existing.remove(key);
} else {
existing.put(key, value);
}
if (key.equals("nodeRole")) {
saveRoles();
}
} finally {
lock.unlock();
}
}
/**
* Add a node value, creating a list of values if necessary.
* NOTE: if key is 'nodeRole' then /roles.json is updated.
* @param node node id
* @param key property name
* @param value property value.
*/
@SuppressWarnings({"unchecked"})
public void simAddNodeValue(String node, String key, Object value) throws InterruptedException {
lock.lockInterruptibly();
try {
Map<String, Object> values = nodeValues.computeIfAbsent(node, n -> new ConcurrentHashMap<>());
Object existing = values.get(key);
if (existing == null) {
values.put(key, value);
} else if (existing instanceof Set) {
((Set)existing).add(value);
} else {
Set<Object> vals = new HashSet<>();
vals.add(existing);
vals.add(value);
values.put(key, vals);
}
if (key.equals("nodeRole")) {
saveRoles();
}
} finally {
lock.unlock();
}
}
/**
* Remove node values. If values contained a 'nodeRole' key then
* /roles.json is updated.
* @param node node id
*/
public void simRemoveNodeValues(String node) throws InterruptedException {
log.debug("--removing value for {}", node);
lock.lockInterruptibly();
try {
Map<String, Object> values = nodeValues.remove(node);
if (values != null && values.containsKey("nodeRole")) {
saveRoles();
}
} finally {
lock.unlock();
}
}
/**
* Remove values that correspond to dead nodes. If values contained a 'nodeRole'
* key then /roles.json is updated.
*/
public void simRemoveDeadNodes() throws InterruptedException {
Set<String> myNodes = new HashSet<>(nodeValues.keySet());
myNodes.removeAll(liveNodesSet.get());
lock.lockInterruptibly();
try {
AtomicBoolean updateRoles = new AtomicBoolean(false);
myNodes.forEach(n -> {
log.debug("- removing dead node values: {}", n);
Map<String, Object> vals = nodeValues.remove(n);
if (vals.containsKey("nodeRole")) {
updateRoles.set(true);
}
});
if (updateRoles.get()) {
saveRoles();
}
} finally {
lock.unlock();
}
}
/**
* Return a set of nodes that are not live but their values are still present.
*/
public Set<String> simGetDeadNodes() {
Set<String> myNodes = new TreeSet<>(nodeValues.keySet());
myNodes.removeAll(liveNodesSet.get());
return myNodes;
}
/**
* Get all node values.
*/
public Map<String, Map<String, Object>> simGetAllNodeValues() {
return nodeValues;
}
/** Get all values for a selected node. */
public Map<String, Object> simGetNodeValues(String node) {
return nodeValues.getOrDefault(node, Collections.emptyMap());
}
private void saveRoles() {
final Map<String, Set<String>> roles = new HashMap<>();
nodeValues.forEach((n, values) -> {
String nodeRole = (String)values.get("nodeRole");
if (nodeRole != null) {
roles.computeIfAbsent(nodeRole, role -> new HashSet<>()).add(n);
}
});
try {
stateManager.setData(ZkStateReader.ROLES, Utils.toJSON(roles), -1);
} catch (Exception e) {
throw new RuntimeException("Unexpected exception saving roles " + roles, e);
}
}
private static final Pattern REGISTRY_PATTERN = Pattern.compile("^solr\\.core\\.([\\w.-_]+?)\\.(shard[\\d_]+?)\\.(replica.*)");
private static final Pattern METRIC_KEY_PATTERN = Pattern.compile("^metrics:([^:]+?):([^:]+?)(:([^:]+))?$");
/**
* Simulate getting replica metrics values. This uses per-replica properties set in
* {@link SimClusterStateProvider#simSetCollectionValue(String, String, Object, boolean, boolean)} and
* similar methods.
* @param node node id
* @param tags metrics names
* @return map of metrics names / values
*/
public Map<String, Object> getReplicaMetricsValues(String node, Collection<String> tags) {
if (!liveNodesSet.contains(node)) {
throw new RuntimeException("non-live node " + node);
}
Map<String, Object> values = new HashMap<>();
for (String tag : tags) {
Matcher m = METRIC_KEY_PATTERN.matcher(tag);
if (!m.matches() || m.groupCount() < 2) {
log.warn("Invalid metrics: tag: {}", tag);
continue;
}
String registryName = m.group(1);
String key = m.group(3) != null ? m.group(2) + m.group(3) : m.group(2);
if (!registryName.startsWith("solr.core.")) {
// skip - this is probably solr.node or solr.jvm metric
continue;
}
m = REGISTRY_PATTERN.matcher(registryName);
if (!m.matches()) {
log.warn("Invalid registry name: {}", registryName);
continue;
}
String collection = m.group(1);
String shard = m.group(2);
String replica = m.group(3);
List<Replica> replicas = clusterStateProvider.simGetReplicaInfos(collection, shard);
replicas.forEach(r -> {
if (r.getNodeName().equals(node) && r.getCoreName().endsWith(replica)) {
Object value = r.getProperties().get(key);
if (value != null) {
values.put(tag, value);
} else {
value = r.getProperties().get(tag);
if (value != null) {
values.put(tag, value);
}
}
}
});
}
return values;
}
// ---------- interface methods -------------
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
log.trace("-- requested values for {}: {}", node, tags);
if (!liveNodesSet.contains(node)) {
throw new RuntimeException("non-live node " + node);
}
if (tags.isEmpty()) {
return new HashMap<>();
}
Map<String, Object> metrics = getReplicaMetricsValues(node, tags.stream().filter(s -> s.startsWith("metrics:solr.core.")).collect(Collectors.toList()));
Map<String, Object> result = new HashMap<>(metrics);
Map<String, Object> values = nodeValues.get(node);
if (values == null) {
return result;
}
result.putAll(values.entrySet().stream()
.filter(e -> tags.contains(e.getKey()))
.filter(e -> e.getValue() != null)
.collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
return result;
}
@Override
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
List<Replica> replicas = clusterStateProvider.simGetReplicaInfos(node);
if (replicas == null || replicas.isEmpty()) {
return new HashMap<>();
}
Map<String, Map<String, List<Replica>>> res = new HashMap<>();
// TODO: probably needs special treatment for "metrics:solr.core..." tags
for (Replica r : replicas) {
@SuppressWarnings({"unchecked"})
Map<String, List<Replica>> perCollection = res.computeIfAbsent(r.getCollection(), Utils.NEW_HASHMAP_FUN);
@SuppressWarnings({"unchecked"})
List<Replica> perShard = perCollection.computeIfAbsent(r.getShard(), Utils.NEW_ARRAYLIST_FUN);
// XXX filter out some properties?
perShard.add(r);
}
return res;
}
@Override
public void close() throws IOException {
}
}

View File

@ -1,434 +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.autoscaling.sim;
import java.lang.invoke.MethodHandles;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
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.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.Row;
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
import org.apache.solr.client.solrj.request.CollectionApiMapping;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
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.Utils;
import org.apache.solr.util.RedactionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Various utility methods useful for autoscaling simulations and snapshots.
*/
public class SimUtils {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final Set<String> COMMON_REPLICA_TAGS = new HashSet<>(Arrays.asList(
Variable.Type.CORE_IDX.metricsAttribute,
Variable.Type.CORE_IDX.tagName,
"SEARCHER.searcher.numDocs",
"SEARCHER.searcher.maxDoc",
"SEARCHER.searcher.indexCommitSize",
"QUERY./select.requests",
"UPDATE./update.requests"
));
public static final Set<String> COMMON_NODE_TAGS = new HashSet<>(Arrays.asList(
Variable.Type.CORES.tagName,
Variable.Type.FREEDISK.tagName,
Variable.Type.NODE.tagName,
Variable.Type.NODE_ROLE.tagName,
Variable.Type.TOTALDISK.tagName,
Variable.Type.DISKTYPE.tagName,
Variable.Type.HEAPUSAGE.tagName,
Variable.Type.HOST.tagName,
Variable.Type.IP_1.tagName,
Variable.Type.IP_2.tagName,
Variable.Type.IP_3.tagName,
Variable.Type.IP_4.tagName,
Variable.Type.PORT.tagName,
Variable.Type.SYSLOADAVG.tagName,
"withCollection"
));
/**
* Check consistency of data in a {@link SolrCloudManager}. This may be needed when constructing a simulated
* instance from potentially inconsistent data (eg. partial snapshots taken at different points in time).
* @param solrCloudManager source manager
* @param config optional {@link AutoScalingConfig} instance used to determine what node and replica metrics to check.
*/
public static void checkConsistency(SolrCloudManager solrCloudManager, AutoScalingConfig config) throws Exception {
if (config == null) {
config = solrCloudManager.getDistribStateManager().getAutoScalingConfig();
}
Set<String> replicaTags = new HashSet<>(COMMON_REPLICA_TAGS);
replicaTags.addAll(config.getPolicy().getPerReplicaAttributes());
// verify replicas are consistent and data is available
Map<String, Map<String, Replica>> allReplicas = new HashMap<>();
solrCloudManager.getClusterStateProvider().getClusterState().forEachCollection(coll -> {
coll.getReplicas().forEach(r -> {
if (allReplicas.containsKey(r.getName())) {
throw new RuntimeException("duplicate core_node name in clusterState: " + allReplicas.get(r.getName()) + " versus " + r);
} else {
allReplicas.computeIfAbsent(coll.getName(), c -> new HashMap<>()).put(r.getName(), r);
}
});
});
Map<String, Map<String, Replica>> allReplicaInfos = new HashMap<>();
solrCloudManager.getClusterStateProvider().getLiveNodes().forEach(n -> {
Map<String, Map<String, List<Replica>>> infos = solrCloudManager.getNodeStateProvider().getReplicaInfo(n, replicaTags);
infos.forEach((coll, shards) -> shards.forEach((shard, replicas) -> replicas.forEach(r -> {
if (allReplicaInfos.containsKey(r.getName())) {
throw new RuntimeException("duplicate core_node name in NodeStateProvider: " + allReplicaInfos.get(r.getName()) + " versus " + r);
} else {
allReplicaInfos.computeIfAbsent(coll, c -> new HashMap<>()).put(r.getName(), r);
}
})));
});
if (!allReplicaInfos.keySet().equals(allReplicas.keySet())) {
Set<String> notInClusterState = allReplicaInfos.keySet().stream()
.filter(k -> !allReplicas.containsKey(k))
.collect(Collectors.toSet());
Set<String> notInNodeProvider = allReplicas.keySet().stream()
.filter(k -> !allReplicaInfos.containsKey(k))
.collect(Collectors.toSet());
throw new RuntimeException("Mismatched replica data between ClusterState and NodeStateProvider:\n\t" +
"collection not in ClusterState: " + notInClusterState + "\n\t" +
"collection not in NodeStateProvider: " + notInNodeProvider);
}
allReplicaInfos.keySet().forEach(collection -> {
Set<String> infosCores = allReplicaInfos.getOrDefault(collection, Collections.emptyMap()).keySet();
Map<String, Replica> replicas = allReplicas.getOrDefault(collection, Collections.emptyMap());
Set<String> csCores = replicas.keySet();
if (!infosCores.equals(csCores)) {
Set<String> notInClusterState = infosCores.stream()
.filter(k -> !csCores.contains(k))
.collect(Collectors.toSet());
Set<String> notInNodeProvider = csCores.stream()
.filter(k -> !infosCores.contains(k) && replicas.get(k).isActive(solrCloudManager.getClusterStateProvider().getLiveNodes()))
.collect(Collectors.toSet());
if (!notInClusterState.isEmpty() || !notInNodeProvider.isEmpty()) {
throw new RuntimeException("Mismatched replica data for collection " + collection + " between ClusterState and NodeStateProvider:\n\t" +
"replica in NodeStateProvider but not in ClusterState: " + notInClusterState + "\n\t" +
"replica in ClusterState but not in NodeStateProvider: " + notInNodeProvider);
}
}
});
// verify all replicas have size info
allReplicaInfos.forEach((coll, replicas) -> replicas.forEach((core, ri) -> {
Number size = (Number) ri.get(Variable.Type.CORE_IDX.metricsAttribute);
if (size == null) {
size = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
if (size == null) {
// for (String node : solrCloudManager.getClusterStateProvider().getLiveNodes()) {
// log.error("Check for missing values: {}: {}", node, solrCloudManager.getNodeStateProvider().getReplicaInfo(node, SnapshotNodeStateProvider.REPLICA_TAGS));
// }
throw new RuntimeException("missing replica size information: " + ri);
}
}
}
));
}
/**
* Calculate statistics of node / collection and replica layouts for the provided {@link SolrCloudManager}.
* @param cloudManager manager
* @param config autoscaling config, or null if the one from the provided manager should be used
* @param verbose if true then add more details about replicas.
* @return a map containing detailed statistics
*/
public static Map<String, Object> calculateStats(SolrCloudManager cloudManager, AutoScalingConfig config, boolean verbose) throws Exception {
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
Map<String, Map<String, Number>> collStats = new TreeMap<>();
Policy.Session session = config.getPolicy().createSession(cloudManager);
clusterState.forEachCollection(coll -> {
Map<String, Number> perColl = collStats.computeIfAbsent(coll.getName(), n -> new LinkedHashMap<>());
AtomicInteger numCores = new AtomicInteger();
HashMap<String, Map<String, AtomicInteger>> nodes = new HashMap<>();
coll.getSlices().forEach(s -> {
numCores.addAndGet(s.getReplicas().size());
s.getReplicas().forEach(r -> {
nodes.computeIfAbsent(r.getNodeName(), n -> new HashMap<>())
.computeIfAbsent(s.getName(), slice -> new AtomicInteger()).incrementAndGet();
});
});
int maxCoresPerNode = 0;
int minCoresPerNode = 0;
int maxActualShardsPerNode = 0;
int minActualShardsPerNode = 0;
int maxShardReplicasPerNode = 0;
int minShardReplicasPerNode = 0;
if (!nodes.isEmpty()) {
minCoresPerNode = Integer.MAX_VALUE;
minActualShardsPerNode = Integer.MAX_VALUE;
minShardReplicasPerNode = Integer.MAX_VALUE;
for (Map<String, AtomicInteger> counts : nodes.values()) {
int total = counts.values().stream().mapToInt(c -> c.get()).sum();
for (AtomicInteger count : counts.values()) {
if (count.get() > maxShardReplicasPerNode) {
maxShardReplicasPerNode = count.get();
}
if (count.get() < minShardReplicasPerNode) {
minShardReplicasPerNode = count.get();
}
}
if (total > maxCoresPerNode) {
maxCoresPerNode = total;
}
if (total < minCoresPerNode) {
minCoresPerNode = total;
}
if (counts.size() > maxActualShardsPerNode) {
maxActualShardsPerNode = counts.size();
}
if (counts.size() < minActualShardsPerNode) {
minActualShardsPerNode = counts.size();
}
}
}
perColl.put("activeShards", coll.getActiveSlices().size());
perColl.put("inactiveShards", coll.getSlices().size() - coll.getActiveSlices().size());
perColl.put("rf", coll.getReplicationFactor());
perColl.put("maxActualShardsPerNode", maxActualShardsPerNode);
perColl.put("minActualShardsPerNode", minActualShardsPerNode);
perColl.put("maxShardReplicasPerNode", maxShardReplicasPerNode);
perColl.put("minShardReplicasPerNode", minShardReplicasPerNode);
perColl.put("numCores", numCores.get());
perColl.put("numNodes", nodes.size());
perColl.put("maxCoresPerNode", maxCoresPerNode);
perColl.put("minCoresPerNode", minCoresPerNode);
});
Map<String, Map<String, Object>> nodeStats = new TreeMap<>();
Map<Integer, AtomicInteger> coreStats = new TreeMap<>();
List<Row> rows = session.getSortedNodes();
// check consistency
if (rows.size() != clusterState.getLiveNodes().size()) {
throw new Exception("Mismatch between autoscaling matrix size (" + rows.size() + ") and liveNodes size (" + clusterState.getLiveNodes().size() + ")");
}
for (Row row : rows) {
Map<String, Object> nodeStat = nodeStats.computeIfAbsent(row.node, n -> new LinkedHashMap<>());
nodeStat.put("isLive", row.isLive());
for (Cell cell : row.getCells()) {
nodeStat.put(cell.getName(), cell.getValue());
}
// nodeStat.put("freedisk", row.getVal("freedisk", 0));
// nodeStat.put("totaldisk", row.getVal("totaldisk", 0));
int cores = ((Number)row.getVal("cores", 0)).intValue();
// nodeStat.put("cores", cores);
coreStats.computeIfAbsent(cores, num -> new AtomicInteger()).incrementAndGet();
Map<String, Map<String, Map<String, Object>>> collReplicas = new TreeMap<>();
// check consistency
AtomicInteger rowCores = new AtomicInteger();
row.forEachReplica(ri -> rowCores.incrementAndGet());
if (cores != rowCores.get()) {
throw new Exception("Mismatch between autoscaling matrix row replicas (" + rowCores.get() + ") and number of cores (" + cores + ")");
}
row.forEachReplica(ri -> {
Map<String, Object> perReplica = collReplicas.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>())
.computeIfAbsent(ri.getCoreName().substring(ri.getCollection().length() + 1), core -> new LinkedHashMap<>());
// if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
// perReplica.put(Variable.Type.CORE_IDX.tagName, ri.getVariable(Variable.Type.CORE_IDX.tagName));
// }
if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) != null) {
perReplica.put(Variable.Type.CORE_IDX.metricsAttribute, ri.get(Variable.Type.CORE_IDX.metricsAttribute));
if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
perReplica.put(Variable.Type.CORE_IDX.tagName, ri.get(Variable.Type.CORE_IDX.tagName));
} else {
perReplica.put(Variable.Type.CORE_IDX.tagName,
Variable.Type.CORE_IDX.convertVal(ri.get(Variable.Type.CORE_IDX.metricsAttribute)));
}
}
perReplica.put("coreNode", ri.getName());
if (ri.isLeader() || ri.getBool("leader", false)) {
perReplica.put("leader", true);
Double totalSize = (Double)collStats.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
.computeIfAbsent("avgShardSize", size -> 0.0);
Number riSize = (Number)ri.get(Variable.Type.CORE_IDX.metricsAttribute);
if (riSize != null) {
totalSize += riSize.doubleValue();
collStats.get(ri.getCollection()).put("avgShardSize", totalSize);
Double max = (Double)collStats.get(ri.getCollection()).get("maxShardSize");
if (max == null) max = 0.0;
if (riSize.doubleValue() > max) {
collStats.get(ri.getCollection()).put("maxShardSize", riSize.doubleValue());
}
Double min = (Double)collStats.get(ri.getCollection()).get("minShardSize");
if (min == null) min = Double.MAX_VALUE;
if (riSize.doubleValue() < min) {
collStats.get(ri.getCollection()).put("minShardSize", riSize.doubleValue());
}
} else {
throw new RuntimeException("ReplicaInfo without size information: " + ri);
}
}
if (verbose) {
nodeStat.put("replicas", collReplicas);
}
});
}
// calculate average per shard and convert the units
for (Map<String, Number> perColl : collStats.values()) {
Number avg = perColl.get("avgShardSize");
if (avg != null) {
avg = avg.doubleValue() / perColl.get("activeShards").doubleValue();
perColl.put("avgShardSize", (Number)Variable.Type.CORE_IDX.convertVal(avg));
}
Number num = perColl.get("maxShardSize");
if (num != null) {
perColl.put("maxShardSize", (Number)Variable.Type.CORE_IDX.convertVal(num));
}
num = perColl.get("minShardSize");
if (num != null) {
perColl.put("minShardSize", (Number)Variable.Type.CORE_IDX.convertVal(num));
}
}
Map<String, Object> stats = new LinkedHashMap<>();
stats.put("coresPerNodes", coreStats);
stats.put("sortedNodeStats", nodeStats);
stats.put("collectionStats", collStats);
return stats;
}
private static final Map<String, String> v2v1Mapping = new HashMap<>();
static {
for (CollectionApiMapping.Meta meta : CollectionApiMapping.Meta.values()) {
if (meta.action != null) {
String key;
if (meta.commandName != null) {
key = meta.commandName;
} else {
key = meta.action.toLower();
}
v2v1Mapping.put(key, meta.action.toLower());
} else {
log.warn("V2 action {} has no equivalent V1 action", meta);
}
}
}
/**
* Convert a V2 {@link org.apache.solr.client.solrj.request.CollectionAdminRequest} to regular {@link org.apache.solr.common.params.SolrParams}
* @param req request
* @return request payload and parameters converted to V1 params
*/
@SuppressWarnings({"unchecked"})
public static ModifiableSolrParams v2AdminRequestToV1Params(V2Request req) {
Map<String, Object> reqMap = new HashMap<>();
req.toMap(reqMap);
String path = (String)reqMap.get("path");
if (!(path.startsWith("/c/") || path.startsWith("/collections/")) || path.length() < 4) {
throw new UnsupportedOperationException("Unsupported V2 request path: " + reqMap);
}
Map<String, Object> cmd;
Object cmdObj = reqMap.get("command");
if (cmdObj instanceof String) {
cmd = (Map<String, Object>)Utils.fromJSONString((String)cmdObj);
} else if (cmdObj instanceof Map) {
cmd = (Map<String, Object>)cmdObj;
} else {
throw new UnsupportedOperationException("Unsupported 'command': " + cmdObj + " (of type " + cmdObj.getClass() + ")");
}
if (cmd.size() != 1) {
throw new UnsupportedOperationException("Unsupported multi-command V2 request: " + reqMap);
}
String a = cmd.keySet().iterator().next();
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("path", "/admin/collections");
if (req.getParams() != null) {
params.add(req.getParams());
}
Map<String, Object> reqParams = (Map<String, Object>)cmd.get(a);
for (Map.Entry<String, Object> e : reqParams.entrySet()) {
params.add(e.getKey(), e.getValue().toString());
}
// trim the leading /
path = path.substring(1);
String[] pathEls = path.split("/");
if (pathEls.length < 2) {
throw new UnsupportedOperationException("Unsupported V2 request path: " + reqMap);
}
params.set(CollectionAdminParams.COLLECTION, pathEls[1]);
if (pathEls.length > 3) {
if (!pathEls[2].equals("shards")) {
throw new UnsupportedOperationException("Invalid V2 request path: expected 'shards' but was '" + pathEls[2] + "'");
}
if (!pathEls[3].isBlank()) {
params.set("shard", pathEls[3]);
}
}
if (pathEls.length > 4 && !pathEls[4].isBlank()) {
params.set("replica", pathEls[4]);
}
// re-map from v2 to v1 action
a = v2v1Mapping.get(a);
if (a == null) {
throw new UnsupportedOperationException("Unsupported V2 request: " + reqMap);
}
params.add(CoreAdminParams.ACTION, a);
return params;
}
/**
* Prepare collection and node / host names for redaction.
* @param clusterState cluster state
*/
public static RedactionUtils.RedactionContext getRedactionContext(ClusterState clusterState) {
RedactionUtils.RedactionContext ctx = new RedactionUtils.RedactionContext();
TreeSet<String> names = new TreeSet<>(clusterState.getLiveNodes());
for (String nodeName : names) {
String urlString = Utils.getBaseUrlForNodeName(nodeName, "http");
try {
URL u = new URL(urlString);
// protocol format
String hostPort = u.getHost() + ":" + u.getPort();
ctx.addName(u.getHost() + ":" + u.getPort(), RedactionUtils.NODE_REDACTION_PREFIX);
// node name format
ctx.addEquivalentName(hostPort, u.getHost() + "_" + u.getPort() + "_", RedactionUtils.NODE_REDACTION_PREFIX);
} catch (MalformedURLException e) {
log.warn("Invalid URL for node name {}, replacing including protocol and path", nodeName, e);
ctx.addName(urlString, RedactionUtils.NODE_REDACTION_PREFIX);
ctx.addEquivalentName(urlString, Utils.getBaseUrlForNodeName(nodeName, "https"), RedactionUtils.NODE_REDACTION_PREFIX);
}
}
names.clear();
names.addAll(clusterState.getCollectionStates().keySet());
names.forEach(n -> ctx.addName(n, RedactionUtils.COLL_REDACTION_PREFIX));
return ctx;
}
}

View File

@ -1,260 +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.autoscaling.sim;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import org.apache.commons.io.IOUtils;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ObjectCache;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.RedactionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Read-only snapshot of another {@link SolrCloudManager}.
*/
public class SnapshotCloudManager implements SolrCloudManager {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private ObjectCache objectCache = new ObjectCache();
private SnapshotClusterStateProvider clusterStateProvider;
private SnapshotNodeStateProvider nodeStateProvider;
private SnapshotDistribStateManager distribStateManager;
private TimeSource timeSource;
public static final String MANAGER_STATE_KEY = "managerState";
public static final String CLUSTER_STATE_KEY = "clusterState";
public static final String NODE_STATE_KEY = "nodeState";
public static final String DISTRIB_STATE_KEY = "distribState";
public static final String AUTOSCALING_STATE_KEY = "autoscalingState";
public static final String STATISTICS_STATE_KEY = "statistics";
public static final String AUTOSCALING_JSON_KEY = "autoscaling";
public static final List<String> REQUIRED_KEYS = Arrays.asList(
MANAGER_STATE_KEY,
CLUSTER_STATE_KEY,
NODE_STATE_KEY,
DISTRIB_STATE_KEY
);
public SnapshotCloudManager(SolrCloudManager other, AutoScalingConfig config) throws Exception {
this.timeSource = other.getTimeSource();
this.clusterStateProvider = new SnapshotClusterStateProvider(other.getClusterStateProvider());
this.nodeStateProvider = new SnapshotNodeStateProvider(other, config);
this.distribStateManager = new SnapshotDistribStateManager(other.getDistribStateManager(), config);
SimUtils.checkConsistency(this, config);
}
@SuppressWarnings({"unchecked"})
public SnapshotCloudManager(Map<String, Object> snapshot) throws Exception {
Objects.requireNonNull(snapshot);
init(
(Map<String, Object>)snapshot.getOrDefault(MANAGER_STATE_KEY, Collections.emptyMap()),
(Map<String, Object>)snapshot.getOrDefault(CLUSTER_STATE_KEY, Collections.emptyMap()),
(Map<String, Object>)snapshot.getOrDefault(NODE_STATE_KEY, Collections.emptyMap()),
(Map<String, Object>)snapshot.getOrDefault(DISTRIB_STATE_KEY, Collections.emptyMap()),
(Map<String, Object>)snapshot.getOrDefault(AUTOSCALING_JSON_KEY, Collections.emptyMap())
);
}
public void saveSnapshot(File targetDir, boolean withAutoscaling, boolean redact) throws Exception {
Map<String, Object> snapshot = getSnapshot(withAutoscaling, redact);
ClusterState clusterState = getClusterStateProvider().getClusterState();
RedactionUtils.RedactionContext ctx = SimUtils.getRedactionContext(clusterState);
targetDir.mkdirs();
for (Map.Entry<String, Object> e : snapshot.entrySet()) {
FileOutputStream out = new FileOutputStream(new File(targetDir, e.getKey() + ".json"));
if (redact) {
String data = Utils.toJSONString(e.getValue());
data = RedactionUtils.redactNames(ctx.getRedactions(), data);
IOUtils.write(data.getBytes("UTF-8"), out);
} else {
IOUtils.write(Utils.toJSON(e.getValue()), out);
}
out.flush();
out.close();
}
}
@SuppressWarnings({"unchecked"})
public static SnapshotCloudManager readSnapshot(File sourceDir) throws Exception {
if (!sourceDir.exists()) {
throw new Exception("Source path doesn't exist: " + sourceDir);
}
if (!sourceDir.isDirectory()) {
throw new Exception("Source path is not a directory: " + sourceDir);
}
Map<String, Object> snapshot = new HashMap<>();
List<String> allKeys = new ArrayList<>(REQUIRED_KEYS);
allKeys.add(AUTOSCALING_JSON_KEY);
int validData = 0;
for (String key : allKeys) {
File src = new File(sourceDir, key + ".json");
if (src.exists()) {
InputStream is = new FileInputStream(src);
Map<String, Object> data = (Map<String, Object>)Utils.fromJSON(is);
is.close();
snapshot.put(key, data);
if (REQUIRED_KEYS.contains(key)) {
validData++;
}
}
}
if (validData < REQUIRED_KEYS.size()) {
throw new Exception("Some data is missing - expected: " + REQUIRED_KEYS + ", found: " + snapshot.keySet());
}
return new SnapshotCloudManager(snapshot);
}
private void init(Map<String, Object> managerState, Map<String, Object> clusterState, Map<String, Object> nodeState,
Map<String, Object> distribState, Map<String, Object> autoscalingJson) throws Exception {
Objects.requireNonNull(managerState);
Objects.requireNonNull(clusterState);
Objects.requireNonNull(nodeState);
Objects.requireNonNull(distribState);
this.timeSource = TimeSource.get((String)managerState.getOrDefault("timeSource", "simTime:50"));
this.clusterStateProvider = new SnapshotClusterStateProvider(clusterState);
this.nodeStateProvider = new SnapshotNodeStateProvider(nodeState);
if (autoscalingJson == null || autoscalingJson.isEmpty()) {
this.distribStateManager = new SnapshotDistribStateManager(distribState);
} else {
this.distribStateManager = new SnapshotDistribStateManager(distribState, new AutoScalingConfig(autoscalingJson));
}
SimUtils.checkConsistency(this, null);
}
public Map<String, Object> getSnapshot(boolean withAutoscaling, boolean redact) throws Exception {
Map<String, Object> snapshot = new LinkedHashMap<>(4);
Map<String, Object> managerState = new HashMap<>();
managerState.put("timeSource", timeSource.toString());
snapshot.put(MANAGER_STATE_KEY, managerState);
RedactionUtils.RedactionContext ctx = redact ? SimUtils.getRedactionContext(clusterStateProvider.getClusterState()) : null;
snapshot.put(CLUSTER_STATE_KEY, clusterStateProvider.getSnapshot());
snapshot.put(NODE_STATE_KEY, nodeStateProvider.getSnapshot());
snapshot.put(DISTRIB_STATE_KEY, distribStateManager.getSnapshot(ctx));
if (withAutoscaling) {
AutoScalingConfig config = distribStateManager.getAutoScalingConfig();
Policy.Session session = config.getPolicy().createSession(this);
List<Suggester.SuggestionInfo> suggestions = PolicyHelper.getSuggestions(config, this);
Map<String, Object> diagnostics = new LinkedHashMap<>();
PolicyHelper.getDiagnostics(session).toMap(diagnostics);
List<Map<String, Object>> suggestionDetails = new ArrayList<>(suggestions.size());
suggestions.forEach(s -> {
Map<String, Object> map = new LinkedHashMap<>();
map.put("suggestion", s);
if (s.getOperation() != null) {
SolrParams params = s.getOperation().getParams();
if (s.getOperation() instanceof V2Request) {
params = SimUtils.v2AdminRequestToV1Params((V2Request)s.getOperation());
}
Replica info = nodeStateProvider.getReplicaInfo(
params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
if (info == null) {
log.warn("Can't find ReplicaInfo for suggested operation: {}", s);
} else {
map.put("replica", info);
}
}
suggestionDetails.add(map);
});
Map<String, Object> autoscaling = new LinkedHashMap<>();
autoscaling.put("suggestions", suggestionDetails);
autoscaling.put("diagnostics", diagnostics);
snapshot.put(AUTOSCALING_STATE_KEY, autoscaling);
}
snapshot.put(STATISTICS_STATE_KEY, SimUtils.calculateStats(this, distribStateManager.getAutoScalingConfig(), true));
return snapshot;
}
@Override
public ClusterStateProvider getClusterStateProvider() {
return clusterStateProvider;
}
@Override
public NodeStateProvider getNodeStateProvider() {
return nodeStateProvider;
}
@Override
public DistribStateManager getDistribStateManager() {
return distribStateManager;
}
@Override
public DistributedQueueFactory getDistributedQueueFactory() {
return NoopDistributedQueueFactory.INSTANCE;
}
@Override
public ObjectCache getObjectCache() {
return objectCache;
}
@Override
public TimeSource getTimeSource() {
return timeSource;
}
@Override
@SuppressWarnings({"rawtypes"})
public SolrResponse request(SolrRequest req) throws IOException {
throw new UnsupportedOperationException("request");
}
@Override
public byte[] httpRequest(String url, SolrRequest.METHOD method, Map<String, String> headers, String payload, int timeout, boolean followRedirects) throws IOException {
throw new UnsupportedOperationException("httpRequest");
}
@Override
public void close() throws IOException {
}
}

View File

@ -1,152 +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.autoscaling.sim;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.util.Utils;
import org.noggit.CharArr;
import org.noggit.JSONWriter;
/**
* Read-only snapshot of another {@link ClusterStateProvider}.
*/
public class SnapshotClusterStateProvider implements ClusterStateProvider {
final Set<String> liveNodes;
final ClusterState clusterState;
final Map<String, Object> clusterProperties;
public SnapshotClusterStateProvider(ClusterStateProvider other) throws Exception {
liveNodes = Set.copyOf(other.getLiveNodes());
ClusterState otherState = other.getClusterState();
clusterState = new ClusterState(liveNodes, otherState.getCollectionsMap());
clusterProperties = new HashMap<>(other.getClusterProperties());
}
@SuppressWarnings({"unchecked"})
public SnapshotClusterStateProvider(Map<String, Object> snapshot) {
Objects.requireNonNull(snapshot);
liveNodes = Set.copyOf((Collection<String>)snapshot.getOrDefault("liveNodes", Collections.emptySet()));
clusterProperties = (Map<String, Object>)snapshot.getOrDefault("clusterProperties", Collections.emptyMap());
Map<String, Object> stateMap = new HashMap<>((Map<String, Object>)snapshot.getOrDefault("clusterState", Collections.emptyMap()));
Map<String, DocCollection> collectionStates = new HashMap<>();
// back-compat with format = 1
Integer stateVersion = Integer.valueOf(String.valueOf(stateMap.getOrDefault("version", 0)));
stateMap.remove("version");
stateMap.forEach((name, state) -> {
Map<String, Object> mutableState = (Map<String, Object>)state;
Map<String, Object> collMap = (Map<String, Object>) mutableState.get(name);
if (collMap == null) {
// snapshot in format 1
collMap = mutableState;
mutableState = Collections.singletonMap(name, state);
}
int version = Integer.parseInt(String.valueOf(collMap.getOrDefault("zNodeVersion", stateVersion)));
collMap.remove("zNodeVersion");
byte[] data = Utils.toJSON(mutableState);
ClusterState collState = ClusterState.createFromJson(version, data, Collections.emptySet());
collectionStates.put(name, collState.getCollection(name));
});
clusterState = new ClusterState(liveNodes, collectionStates);
}
public Map<String, Object> getSnapshot() {
Map<String, Object> snapshot = new HashMap<>();
snapshot.put("liveNodes", liveNodes);
if (clusterProperties != null) {
snapshot.put("clusterProperties", clusterProperties);
}
Map<String, Object> stateMap = new HashMap<>();
snapshot.put("clusterState", stateMap);
clusterState.forEachCollection(coll -> {
CharArr out = new CharArr();
JSONWriter writer = new JSONWriter(out, 2);
coll.write(writer);
String json = out.toString();
try {
@SuppressWarnings({"unchecked"})
Map<String, Object> collMap = new LinkedHashMap<>((Map<String, Object>)Utils.fromJSON(json.getBytes("UTF-8")));
collMap.put("zNodeVersion", coll.getZNodeVersion());
// format compatible with the real /state.json, which uses a mini-ClusterState
// consisting of a single collection
stateMap.put(coll.getName(), Collections.singletonMap(coll.getName(), collMap));
} catch (UnsupportedEncodingException e) {
throw new RuntimeException("should not happen!", e);
}
});
return snapshot;
}
@Override
public ClusterState.CollectionRef getState(String collection) {
return clusterState.getCollectionRef(collection);
}
@Override
public Set<String> getLiveNodes() {
return liveNodes;
}
@Override
public List<String> resolveAlias(String alias) {
throw new UnsupportedOperationException("resolveAlias");
}
@Override
public Map<String, String> getAliasProperties(String alias) {
throw new UnsupportedOperationException("getAliasProperties");
}
@Override
public ClusterState getClusterState() throws IOException {
return clusterState;
}
@Override
public Map<String, Object> getClusterProperties() {
return clusterProperties;
}
@Override
public String getPolicyNameByCollection(String coll) {
DocCollection collection = clusterState.getCollectionOrNull(coll);
return collection == null ? null : (String)collection.getProperties().get("policy");
}
@Override
public void connect() {
}
@Override
public void close() throws IOException {
}
}

View File

@ -1,228 +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.autoscaling.sim;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
import org.apache.solr.client.solrj.cloud.autoscaling.NotEmptyException;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.util.Base64;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.RedactionUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.Watcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Read-only snapshot of another {@link DistribStateManager}
*/
public class SnapshotDistribStateManager implements DistribStateManager {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
LinkedHashMap<String, VersionedData> dataMap = new LinkedHashMap<>();
/**
* Populate this instance from another {@link DistribStateManager} instance.
* @param other another instance
* @param config optional {@link AutoScalingConfig}, which will overwrite any existing config.
*/
public SnapshotDistribStateManager(DistribStateManager other, AutoScalingConfig config) throws Exception {
List<String> tree = other.listTree("/");
if (log.isDebugEnabled()) {
log.debug("- copying {} resources from {}", tree.size(), other.getClass().getSimpleName());
}
for (String path : tree) {
dataMap.put(path, other.getData(path));
}
if (config != null) { // overwrite existing
VersionedData vd = new VersionedData(config.getZkVersion(), Utils.toJSON(config), CreateMode.PERSISTENT, "0");
dataMap.put(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, vd);
}
}
/**
* Populate this instance from a previously generated snapshot.
* @param snapshot previous snapshot created using this class.
*/
public SnapshotDistribStateManager(Map<String, Object> snapshot) {
this(snapshot, null);
}
/**
* Populate this instance from a previously generated snapshot.
* @param snapshot previous snapshot created using this class.
* @param config optional config to override the one from snapshot, may be null
*/
public SnapshotDistribStateManager(Map<String, Object> snapshot, AutoScalingConfig config) {
snapshot.forEach((path, value) -> {
@SuppressWarnings({"unchecked"})
Map<String, Object> map = (Map<String, Object>)value;
Number version = (Number)map.getOrDefault("version", 0);
String owner = (String)map.get("owner");
String modeStr = (String)map.getOrDefault("mode", CreateMode.PERSISTENT.toString());
CreateMode mode = CreateMode.valueOf(modeStr);
byte[] bytes = null;
if (map.containsKey("data")) {
bytes = Base64.base64ToByteArray((String)map.get("data"));
}
dataMap.put(path, new VersionedData(version.intValue(), bytes, mode, owner));
});
if (config != null) { // overwrite existing
VersionedData vd = new VersionedData(config.getZkVersion(), Utils.toJSON(config), CreateMode.PERSISTENT, "0");
dataMap.put(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, vd);
}
if (log.isDebugEnabled()) {
log.debug("- loaded snapshot of {} resources", dataMap.size());
}
}
// content of these nodes is a UTF-8 String and it needs to be redacted
private static final Set<Pattern> REDACTED = new HashSet<>() {{
add(Pattern.compile("/aliases\\.json"));
add(Pattern.compile("/autoscaling\\.json"));
add(Pattern.compile("/clusterstate\\.json"));
add(Pattern.compile("/collections/.*?/state\\.json"));
add(Pattern.compile("/collections/.*?/leaders/shard.*?/leader"));
add(Pattern.compile("/overseer_elect/leader"));
}};
/**
* Create a snapshot of all content in this instance.
*/
public Map<String, Object> getSnapshot(RedactionUtils.RedactionContext ctx) {
Map<String, Object> snapshot = new LinkedHashMap<>();
dataMap.forEach((path, vd) -> {
Map<String, Object> data = new HashMap<>();
if (vd.getData() != null && ctx != null && REDACTED.stream().anyMatch(p -> p.matcher(path).matches())) {
String str = new String(vd.getData(), Charset.forName("UTF-8"));
str = RedactionUtils.redactNames(ctx.getRedactions(), str);
vd = new VersionedData(vd.getVersion(), str.getBytes(Charset.forName("UTF-8")), vd.getMode(), vd.getOwner());
}
vd.toMap(data);
snapshot.put(path, data);
});
return snapshot;
}
@Override
public boolean hasData(String path) throws IOException, KeeperException, InterruptedException {
return dataMap.containsKey(path);
}
@Override
public List<String> listData(String path) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
return listData(path, null);
}
@Override
public List<String> listTree(String path) {
return dataMap.keySet().stream()
.filter(p -> p.startsWith(path))
.collect(Collectors.toList());
}
@Override
public List<String> listData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
final String prefix = path + "/";
return dataMap.entrySet().stream()
.filter(e -> e.getKey().startsWith(prefix))
.map(e -> {
String suffix = e.getKey().substring(prefix.length());
int idx = suffix.indexOf('/');
if (idx == -1) {
return suffix;
} else {
return suffix.substring(0, idx);
}
})
.collect(Collectors.toList());
}
@Override
public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException, KeeperException, InterruptedException {
if (!dataMap.containsKey(path)) {
throw new NoSuchElementException(path);
}
return dataMap.get(path);
}
@Override
public void makePath(String path) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
throw new UnsupportedOperationException("makePath");
}
@Override
public void makePath(String path, byte[] data, CreateMode createMode, boolean failOnExists) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
throw new UnsupportedOperationException("makePath");
}
@Override
public String createData(String path, byte[] data, CreateMode mode) throws AlreadyExistsException, IOException, KeeperException, InterruptedException {
throw new UnsupportedOperationException("createData");
}
@Override
public void removeData(String path, int version) throws NoSuchElementException, IOException, NotEmptyException, KeeperException, InterruptedException, BadVersionException {
throw new UnsupportedOperationException("removeData");
}
@Override
public void setData(String path, byte[] data, int version) throws BadVersionException, NoSuchElementException, IOException, KeeperException, InterruptedException {
throw new UnsupportedOperationException("setData");
}
@Override
public List<OpResult> multi(Iterable<Op> ops) throws BadVersionException, NoSuchElementException, AlreadyExistsException, IOException, KeeperException, InterruptedException {
throw new UnsupportedOperationException("multi");
}
@Override
@SuppressWarnings({"unchecked"})
public AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws InterruptedException, IOException {
VersionedData vd = dataMap.get(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH);
Map<String, Object> map = new HashMap<>();
if (vd != null && vd.getData() != null && vd.getData().length > 0) {
map = (Map<String, Object>) Utils.fromJSON(vd.getData());
map.put(AutoScalingParams.ZK_VERSION, vd.getVersion());
}
return new AutoScalingConfig(map);
}
@Override
public void close() throws IOException {
}
}

View File

@ -1,203 +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.autoscaling.sim;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
import org.apache.solr.common.cloud.Replica;
/**
* Read-only snapshot of another {@link NodeStateProvider}.
*/
public class SnapshotNodeStateProvider implements NodeStateProvider {
private Map<String, Map<String, Object>> nodeValues = new LinkedHashMap<>();
private Map<String, Map<String, Map<String, List<Replica>>>> replicaInfos = new LinkedHashMap<>();
private static double GB = 1024.0d * 1024.0d * 1024.0d;
/**
* Populate this instance from another instance of {@link SolrCloudManager}.
* @param other another instance
* @param config optional {@link AutoScalingConfig}, which will be used to determine what node and
* replica tags to retrieve. If this is null then the other instance's config will be used.
*/
@SuppressWarnings({"unchecked"})
public SnapshotNodeStateProvider(SolrCloudManager other, AutoScalingConfig config) throws Exception {
if (config == null) {
config = other.getDistribStateManager().getAutoScalingConfig();
}
Set<String> nodeTags = new HashSet<>(SimUtils.COMMON_NODE_TAGS);
nodeTags.addAll(config.getPolicy().getParamNames());
Set<String> replicaTags = new HashSet<>(SimUtils.COMMON_REPLICA_TAGS);
replicaTags.addAll(config.getPolicy().getPerReplicaAttributes());
for (String node : other.getClusterStateProvider().getLiveNodes()) {
nodeValues.put(node, new LinkedHashMap<>(other.getNodeStateProvider().getNodeValues(node, nodeTags)));
Map<String, Map<String, List<Replica>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
infos.forEach((collection, shards) -> {
shards.forEach((shard, replicas) -> {
replicas.forEach(r -> {
List<Replica> myReplicas = replicaInfos
.computeIfAbsent(node, n -> new LinkedHashMap<>())
.computeIfAbsent(collection, c -> new LinkedHashMap<>())
.computeIfAbsent(shard, s -> new ArrayList<>());
Map<String, Object> rMap = new LinkedHashMap<>();
r.toMap(rMap);
if (r.isLeader()) { // ReplicaInfo.toMap doesn't write this!!!
((Map<String, Object>)rMap.values().iterator().next()).put("leader", "true");
}
Replica ri = new Replica(rMap);
// put in "leader" again if present
if (r.isLeader()) {
ri.getProperties().put("leader", "true");
}
// externally produced snapshots may not include the right units
if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) == null) {
if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
Number indexSizeGB = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
ri.getProperties().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
} else {
throw new RuntimeException("Missing size information for replica: " + ri);
}
}
myReplicas.add(ri);
});
});
});
}
}
/**
* Populate this instance from a previously generated snapshot.
* @param snapshot previous snapshot created using this class.
*/
@SuppressWarnings({"unchecked"})
public SnapshotNodeStateProvider(Map<String, Object> snapshot) {
Objects.requireNonNull(snapshot);
nodeValues = (Map<String, Map<String, Object>>)snapshot.getOrDefault("nodeValues", Collections.emptyMap());
((Map<String, Object>)snapshot.getOrDefault("replicaInfos", Collections.emptyMap())).forEach((node, v) -> {
Map<String, Map<String, List<Replica>>> perNode = replicaInfos.computeIfAbsent(node, n -> new LinkedHashMap<>());
((Map<String, Object>)v).forEach((collection, shards) -> {
Map<String, List<Replica>> perColl = perNode.computeIfAbsent(collection, c -> new LinkedHashMap<>());
((Map<String, Object>)shards).forEach((shard, replicas) -> {
List<Replica> infos = perColl.computeIfAbsent(shard, s -> new ArrayList<>());
((List<Map<String, Object>>)replicas).forEach(replicaMap -> {
Replica ri = new Replica(new LinkedHashMap<>(replicaMap)); // constructor modifies this map
if (ri.isLeader()) {
ri.getProperties().put("leader", "true");
}
// externally produced snapshots may not include the right units
if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) == null) {
if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
Number indexSizeGB = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
ri.getProperties().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
} else {
throw new RuntimeException("Missing size information for replica: " + ri);
}
}
infos.add(ri);
});
});
});
});
}
/**
* Create a snapshot of all node and replica tag values available from the original source, per the original
* autoscaling configuration. Note:
*/
@SuppressWarnings({"unchecked"})
public Map<String, Object> getSnapshot() {
Map<String, Object> snapshot = new LinkedHashMap<>();
snapshot.put("nodeValues", nodeValues);
Map<String, Map<String, Map<String, List<Map<String, Object>>>>> replicaInfosMap = new LinkedHashMap<>();
snapshot.put("replicaInfos", replicaInfosMap);
replicaInfos.forEach((node, perNode) -> {
perNode.forEach((collection, shards) -> {
shards.forEach((shard, replicas) -> {
replicas.forEach(r -> {
List<Map<String, Object>> myReplicas = replicaInfosMap
.computeIfAbsent(node, n -> new LinkedHashMap<>())
.computeIfAbsent(collection, c -> new LinkedHashMap<>())
.computeIfAbsent(shard, s -> new ArrayList<>());
Map<String, Object> rMap = new LinkedHashMap<>();
r.toMap(rMap);
if (r.isLeader()) { // ReplicaInfo.toMap doesn't write this!!!
((Map<String, Object>)rMap.values().iterator().next()).put("leader", "true");
}
myReplicas.add(rMap);
});
});
});
});
return snapshot;
}
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return new LinkedHashMap<>(nodeValues.getOrDefault(node, Collections.emptyMap()));
}
@Override
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
Map<String, Map<String, List<Replica>>> result = new LinkedHashMap<>();
Map<String, Map<String, List<Replica>>> infos = replicaInfos.getOrDefault(node, Collections.emptyMap());
// deep copy
infos.forEach((coll, shards) -> {
shards.forEach((shard, replicas) -> {
replicas.forEach(ri -> {
List<Replica> myReplicas = result
.computeIfAbsent(coll, c -> new LinkedHashMap<>())
.computeIfAbsent(shard, s -> new ArrayList<>());
Replica myReplica = (Replica)ri.clone();
myReplicas.add(myReplica);
});
});
});
return result;
}
public Replica getReplicaInfo(String collection, String coreNode) {
for (Map<String, Map<String, List<Replica>>> perNode : replicaInfos.values()) {
for (List<Replica> perShard : perNode.getOrDefault(collection, Collections.emptyMap()).values()) {
for (Replica ri : perShard) {
if (ri.getName().equals(coreNode)) {
return (Replica)ri.clone();
}
}
}
}
return null;
}
@Override
public void close() throws IOException {
}
}

View File

@ -1,94 +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.
*/
/**
* Simulated environment for autoscaling.
*
* <h2>Goals</h2>
* <ul>
* <li>Use the actual unchanged autoscaling code for cluster state monitoring and autoscaling plan execution.</li>
* <li>Support testing large clusters (&gt; 100 nodes).</li>
* <li>Support fast testing using accelerated time (eg. 100x faster).</li>
* <li>Support enough of other Solr functionality for the test results to be meaningful.</li>
* </ul>
*
* <h2>Simulated SolrCloudManager - {@link org.apache.solr.cloud.autoscaling.sim.SimCloudManager}</h2>
* This implementation of {@link org.apache.solr.client.solrj.cloud.SolrCloudManager}
* uses the following simulated components:
* <ul>
* <li>{@link org.apache.solr.cloud.autoscaling.sim.SimDistribStateManager} - in-memory ZK look-alike, with support for Watcher-s, ephemeral and sequential nodes.</li>
* <li>{@link org.apache.solr.cloud.autoscaling.sim.SimClusterStateProvider} - manages collection, replica infos, states and replica metrics.</li>
* <li>{@link org.apache.solr.cloud.autoscaling.sim.SimNodeStateProvider} - manages node metrics.</li>
* <li>{@link org.apache.solr.cloud.autoscaling.sim.GenericDistributedQueue} - DistributedQueue that uses SimDistribStateManager.</li>
* </ul>
* SimCloudManager also maintains an up-to-date /live_nodes in SimDistribStateManager, provides a SolrClient instance for use in tests,
* and provides several convenience methods for setting up simulated clusters, populating node and replica metrics, collecting
* autoscaling-related event history, collecting autoscaling event statistics, etc.
*
* SimCloudManager runs actual {@link org.apache.solr.cloud.autoscaling.OverseerTriggerThread} so that it
* uses real trigger and trigger action implementations, as well as real event scheduling and processing code.
* It also provides methods for simulating Overseer leader change.
*
* An important part of the SimCloudManager is also a request handler that processes common autoscaling
* and collection admin requests. Autoscaling requests are processes by an instance of
* {@link org.apache.solr.cloud.autoscaling.AutoScalingHandler} (and result in changes in respective
* data stored in {@link org.apache.solr.cloud.autoscaling.sim.SimDistribStateManager}). Collection
* admin commands are simulated, ie. they don't use actual {@link org.apache.solr.handler.admin.CollectionsHandler}
* due to the complex dependencies on real components.
*
* <h2>{@link org.apache.solr.cloud.autoscaling.sim.SimClusterStateProvider}</h2>
* This components maintains collection and replica states:
* <ul>
* <li>Simulates delays between request and the actual cluster state changes</li>
* <li>Marks replicas as down when a node goes down (optionally preserving the replica metrics in order to simulate a node coming back), and keeps track of per-node cores and disk space.</li>
* <li>Runs a shard leader election look-alike on collection state updates.</li>
* <li>Maintains up-to-date /clusterstate.json and /clusterprops.json in SimDistribStateManager (which in turn notifies Watcher-s about collection updates).
* Currently for simplicity it uses the old single /clusterstate.json format for representing ClusterState.</li>
* </ul>
*
* <h2>{@link org.apache.solr.cloud.autoscaling.sim.SimNodeStateProvider}</h2>
* This component maintains node metrics. When a simulated cluster is set up using eg.
* {@link org.apache.solr.cloud.autoscaling.sim.SimCloudManager#createCluster(int, org.apache.solr.common.util.TimeSource)}
* method, each simulated node is initialized with some basic metrics that are expected by the autoscaling
* framework, such as node name, fake system load average, heap usage and disk usage.
*
* The number of cores and disk space metrics may be used in autoscaling calculations, so they are
* tracked and adjusted by {@link org.apache.solr.cloud.autoscaling.sim.SimClusterStateProvider} according
* to the currently active replicas located on each node.
*
* <h2>Limitations of the simulation framework</h2>
* Currently the simulation framework is limited to testing the core autoscaling API in a single JVM.
* Using it for other purposes would require extensive modifications in Solr and in the framework code.
*
* Specifically, the framework supports testing the following autoscaling components:
* <ul>
* <li>OverseerTriggerThread and components that it uses.</li>
* <li>Autoscaling config, triggers, trigger listeners, ScheduledTriggers, trigger event queues, ComputePlanAction / ExecutePlanAction, etc.</li>
* </ul>
* Overseer and CollectionsHandler Cmd implementations are NOT used, so cannot be properly tested - some of their functionality is simulated.
* Other SolrCloud components make too many direct references to ZkStateReader, or direct HTTP requests, or rely on too many other components and require much more complex functionality - they may be refactored later but the effort may be too high.
*
* Simulation framework definitely does not support the following functionality:
* <ul>
* <li>Solr searching and indexing</li>
* <li>Any component that uses ZkController (eg. CoreContainer)</li>
* <li>Any component that uses ShardHandler (eg. CollectionsHandler Cmd-s)</li>
* </ul>
*
*/
package org.apache.solr.cloud.autoscaling.sim;

View File

@ -30,7 +30,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.StringUtils;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.cloud.CloudUtil;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.Assign;

View File

@ -70,7 +70,6 @@ import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.OverseerTaskQueue;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
import org.apache.solr.common.AlreadyClosedException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@ -90,7 +89,6 @@ import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
import org.apache.solr.filestore.PackageStoreAPI;
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.ContainerPluginsApi;
@ -137,7 +135,6 @@ import org.slf4j.LoggerFactory;
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;
@ -242,8 +239,6 @@ public class CoreContainer {
protected volatile MetricsCollectorHandler metricsCollectorHandler;
protected volatile AutoscalingHistoryHandler autoscalingHistoryHandler;
private volatile SolrClientCache solrClientCache;
private final ObjectCache objectCache = new ObjectCache();
@ -259,8 +254,6 @@ public class CoreContainer {
public final static long INITIAL_CORE_LOAD_COMPLETE = 0x4L;
private volatile long status = 0L;
protected volatile AutoScalingHandler autoScalingHandler;
private ExecutorService coreContainerAsyncTaskExecutor = ExecutorUtil.newMDCAwareCachedThreadPool("Core Container Async Task");
private enum CoreInitFailedAction {fromleader, none}
@ -737,7 +730,6 @@ public class CoreContainer {
createMetricsHistoryHandler();
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);
@ -889,10 +881,6 @@ public class CoreContainer {
containerHandlers.getApiBag().registerObject(containerPluginsApi.readAPI);
containerHandlers.getApiBag().registerObject(containerPluginsApi.editAPI);
zkSys.getZkController().checkOverseerDesignate();
// initialize this handler here when SolrCloudManager is ready
autoScalingHandler = new AutoScalingHandler(getZkController().getSolrCloudManager(), loader);
containerHandlers.put(AutoScalingHandler.HANDLER_PATH, autoScalingHandler);
autoScalingHandler.initializeMetrics(solrMetricsContext, AutoScalingHandler.HANDLER_PATH);
}
// This is a bit redundant but these are two distinct concepts for all they're accomplished at the same time.
status |= LOAD_COMPLETE | INITIAL_CORE_LOAD_COMPLETE;

View File

@ -67,8 +67,7 @@ public class SolrResourceLoader implements ResourceLoader, Closeable, SolrClassL
private static final String[] packages = {
"", "analysis.", "schema.", "handler.", "handler.tagger.", "search.", "update.", "core.", "response.", "request.",
"update.processor.", "util.", "spelling.", "handler.component.", "handler.dataimport.",
"spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.", "handler.admin.",
"cloud.autoscaling."
"spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.", "handler.admin."
};
private static final Charset UTF_8 = StandardCharsets.UTF_8;

View File

@ -1,166 +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.handler.admin;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import org.apache.solr.api.Api;
import org.apache.solr.api.ApiBag;
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
@SuppressWarnings({"unchecked"})
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(Locale.ROOT, e.getValue(), value));
}
}
}
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder(Collections.singletonList(coreContainer.getZkController().getZkServerAddress()), Optional.empty()).withSocketTimeout(30000).withConnectionTimeout(15000)
.withHttpClient(coreContainer.getUpdateShardHandler().getDefaultHttpClient())
.build()) {
QueryResponse qr = cloudSolrClient.query(collection, params);
rsp.setAllValues(qr.getResponse());
} catch (Exception e) {
if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
// relatively benign
String msg = "Collection " + collection + " does not exist.";
log.info(msg);
rsp.getValues().add("error", msg);
} else {
throw e;
}
}
}
@Override
public String getDescription() {
return "A handler to return autoscaling event history";
}
@Override
public Category getCategory() {
return Category.ADMIN;
}
@Override
public Boolean registerV2() {
return Boolean.TRUE;
}
@Override
public Collection<Api> getApis() {
return ApiBag.wrapRequestHandlers(this, "autoscaling.history");
}
}

View File

@ -56,7 +56,6 @@ import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
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.params.CollectionParams;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
@ -103,7 +102,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.POLICY;
import static org.apache.solr.client.solrj.response.RequestStatusState.COMPLETED;
import static org.apache.solr.client.solrj.response.RequestStatusState.FAILED;
import static org.apache.solr.client.solrj.response.RequestStatusState.NOT_FOUND;
@ -125,7 +123,6 @@ import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;
import static org.apache.solr.common.cloud.DocCollection.RULE;
import static org.apache.solr.common.cloud.DocCollection.SNITCH;
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.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
@ -462,13 +459,11 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
CREATE_NODE_SET,
CREATE_NODE_SET_SHUFFLE,
SHARDS_PROP,
AUTO_ADD_REPLICAS,
RULE,
SNITCH,
PULL_REPLICAS,
TLOG_REPLICAS,
NRT_REPLICAS,
POLICY,
WAIT_FOR_FINAL_STATE,
WITH_COLLECTION,
ALIAS);
@ -558,10 +553,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
PULL_REPLICAS,
TLOG_REPLICAS,
REPLICATION_FACTOR,
POLICY,
CREATE_NODE_SET,
CREATE_NODE_SET_SHUFFLE,
AUTO_ADD_REPLICAS,
"shards",
CommonParams.ROWS,
CommonParams.Q,
@ -974,9 +967,6 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
new ZkNodeProps(all)).getClusterStatus(rsp.getValues());
return null;
}),
UTILIZENODE_OP(UTILIZENODE, (req, rsp, h) -> {
return copy(req.getParams().required(), null, AutoScalingParams.NODE);
}),
ADDREPLICAPROP_OP(ADDREPLICAPROP, (req, rsp, h) -> {
Map<String, Object> map = copy(req.getParams().required(), null,
COLLECTION_PROP,
@ -1169,7 +1159,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
}
// from CREATE_OP:
copy(req.getParams(), params, COLL_CONF, REPLICATION_FACTOR, NRT_REPLICAS, TLOG_REPLICAS,
PULL_REPLICAS, AUTO_ADD_REPLICAS, CREATE_NODE_SET, CREATE_NODE_SET_SHUFFLE);
PULL_REPLICAS, CREATE_NODE_SET, CREATE_NODE_SET_SHUFFLE);
copyPropertiesWithPrefix(req.getParams(), params, COLL_PROP_PREFIX);
return params;
}),

View File

@ -60,9 +60,9 @@ import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.client.solrj.impl.SolrClientNodeStateProvider.Variable;
import org.apache.solr.cloud.LeaderElector;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.common.SolrException;
@ -136,7 +136,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
DEFAULT_NODE_GAUGES.add("CONTAINER.fs.coreRoot.usableSpace");
DEFAULT_CORE_GAUGES.add(Variable.Type.CORE_IDX.metricsAttribute);
DEFAULT_CORE_GAUGES.add(Variable.CORE_IDX.metricsAttribute);
DEFAULT_CORE_COUNTERS.add("QUERY./select.requests");
DEFAULT_CORE_COUNTERS.add("UPDATE./update.requests");

View File

@ -48,9 +48,6 @@ public interface PermissionNameProvider {
SECURITY_EDIT_PERM("security-edit", null),
SECURITY_READ_PERM("security-read", null),
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),
METRICS_HISTORY_READ_PERM("metrics-history-read", null),
FILESTORE_READ_PERM("filestore-read", null),
FILESTORE_WRITE_PERM("filestore-write", null),

View File

@ -19,7 +19,6 @@ package org.apache.solr.util;
import javax.net.ssl.SSLPeerUnverifiedException;
import java.io.Console;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
@ -44,7 +43,6 @@ import java.util.Collection;
import java.util.Collections;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
@ -74,7 +72,6 @@ import org.apache.commons.exec.Executor;
import org.apache.commons.exec.OS;
import org.apache.commons.exec.environment.EnvironmentUtils;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.BooleanUtils;
import org.apache.commons.lang3.SystemUtils;
import org.apache.http.HttpEntity;
@ -96,28 +93,14 @@ import org.apache.solr.client.solrj.SolrClient;
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.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.autoscaling.sim.NoopDistributedQueueFactory;
import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
import org.apache.solr.cloud.autoscaling.sim.SimScenario;
import org.apache.solr.cloud.autoscaling.sim.SimUtils;
import org.apache.solr.cloud.autoscaling.sim.SnapshotCloudManager;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
@ -130,12 +113,9 @@ 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.params.SolrParams;
import org.apache.solr.common.util.ContentStreamBase;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.security.Sha256AuthenticationProvider;
import org.apache.solr.util.configuration.SSLConfigurationsFactory;
import org.noggit.CharArr;
@ -412,8 +392,6 @@ public class SolrCLI implements CLIO {
return new UtilsTool();
else if ("auth".equals(toolType))
return new AuthTool();
else if ("autoscaling".equals(toolType))
return new AutoscalingTool();
else if ("export".equals(toolType))
return new ExportTool();
else if ("package".equals(toolType))
@ -436,7 +414,6 @@ public class SolrCLI implements CLIO {
formatter.printHelp("healthcheck", getToolOptions(new HealthcheckTool()));
formatter.printHelp("status", getToolOptions(new StatusTool()));
formatter.printHelp("api", getToolOptions(new ApiTool()));
formatter.printHelp("autoscaling", getToolOptions(new AutoscalingTool()));
formatter.printHelp("create_collection", getToolOptions(new CreateCollectionTool()));
formatter.printHelp("create_core", getToolOptions(new CreateCoreTool()));
formatter.printHelp("create", getToolOptions(new CreateTool()));
@ -859,373 +836,6 @@ public class SolrCLI implements CLIO {
return result;
}
public static class AutoscalingTool extends ToolBase {
public AutoscalingTool() {
this(CLIO.getOutStream());
}
public AutoscalingTool(PrintStream stdout) {
super(stdout);
}
@Override
public Option[] getOptions() {
return new Option[] {
Option.builder("zkHost")
.argName("HOST")
.hasArg()
.required(false)
.desc("Address of the Zookeeper ensemble; defaults to: "+ZK_HOST)
.build(),
Option.builder("a")
.argName("CONFIG")
.hasArg()
.required(false)
.desc("Autoscaling config file, defaults to the one deployed in the cluster.")
.longOpt("config")
.build(),
Option.builder("s")
.desc("Show calculated suggestions")
.longOpt("suggestions")
.build(),
Option.builder("c")
.desc("Show ClusterState (collections layout)")
.longOpt("clusterState")
.build(),
Option.builder("d")
.desc("Show calculated diagnostics")
.longOpt("diagnostics")
.build(),
Option.builder("n")
.desc("Show sorted nodes with diagnostics")
.longOpt("sortedNodes")
.build(),
Option.builder("r")
.desc("Redact node and collection names (original names will be consistently randomized)")
.longOpt("redact")
.build(),
Option.builder("stats")
.desc("Show summarized collection & node statistics.")
.build(),
Option.builder("save")
.desc("Store autoscaling snapshot of the current cluster.")
.argName("DIR")
.hasArg()
.build(),
Option.builder("load")
.desc("Load autoscaling snapshot of the cluster instead of using the real one.")
.argName("DIR")
.hasArg()
.build(),
Option.builder("simulate")
.desc("Simulate execution of all suggestions.")
.build(),
Option.builder("i")
.desc("Max number of simulation iterations.")
.argName("NUMBER")
.hasArg()
.longOpt("iterations")
.build(),
Option.builder("ss")
.desc("Save autoscaling snapshots at each step of simulated execution.")
.argName("DIR")
.longOpt("saveSimulated")
.hasArg()
.build(),
Option.builder("scenario")
.desc("Execute a scenario from a file (and ignore all other options).")
.argName("FILE")
.hasArg()
.build(),
Option.builder("all")
.desc("Turn on all options to get all available information.")
.build()
};
}
@Override
public String getName() {
return "autoscaling";
}
protected void runImpl(CommandLine cli) throws Exception {
raiseLogLevelUnlessVerbose(cli);
if (cli.hasOption("scenario")) {
String data = IOUtils.toString(new FileInputStream(cli.getOptionValue("scenario")), "UTF-8");
try (SimScenario scenario = SimScenario.load(data)) {
scenario.verbose = verbose;
scenario.console = CLIO.getOutStream();
scenario.run();
}
return;
}
SnapshotCloudManager cloudManager;
AutoScalingConfig config = null;
String configFile = cli.getOptionValue("a");
if (configFile != null) {
CLIO.err("- reading autoscaling config from " + configFile);
config = new AutoScalingConfig(IOUtils.toByteArray(new FileInputStream(configFile)));
}
if (cli.hasOption("load")) {
File sourceDir = new File(cli.getOptionValue("load"));
CLIO.err("- loading autoscaling snapshot from " + sourceDir.getAbsolutePath());
cloudManager = SnapshotCloudManager.readSnapshot(sourceDir);
if (config == null) {
CLIO.err("- reading autoscaling config from the snapshot.");
config = cloudManager.getDistribStateManager().getAutoScalingConfig();
}
} else {
String zkHost = cli.getOptionValue("zkHost", ZK_HOST);
log.debug("Connecting to Solr cluster: {}", zkHost);
try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder(Collections.singletonList(zkHost), Optional.empty()).build()) {
String collection = cli.getOptionValue("collection");
if (collection != null)
cloudSolrClient.setDefaultCollection(collection);
cloudSolrClient.connect();
try (SolrClientCloudManager realCloudManager = new SolrClientCloudManager(NoopDistributedQueueFactory.INSTANCE, cloudSolrClient)) {
if (config == null) {
CLIO.err("- reading autoscaling config from the cluster.");
config = realCloudManager.getDistribStateManager().getAutoScalingConfig();
}
cloudManager = new SnapshotCloudManager(realCloudManager, config);
}
}
}
boolean redact = cli.hasOption("r");
if (cli.hasOption("save")) {
File targetDir = new File(cli.getOptionValue("save"));
cloudManager.saveSnapshot(targetDir, true, redact);
CLIO.err("- saved autoscaling snapshot to " + targetDir.getAbsolutePath());
}
HashSet<String> liveNodes = new HashSet<>(cloudManager.getClusterStateProvider().getLiveNodes());
boolean withSuggestions = cli.hasOption("s");
boolean withDiagnostics = cli.hasOption("d") || cli.hasOption("n");
boolean withSortedNodes = cli.hasOption("n");
boolean withClusterState = cli.hasOption("c");
boolean withStats = cli.hasOption("stats");
if (cli.hasOption("all")) {
withSuggestions = true;
withDiagnostics = true;
withSortedNodes = true;
withClusterState = true;
withStats = true;
}
// prepare to redact also host names / IPs in base_url and other properties
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
RedactionUtils.RedactionContext ctx = null;
if (redact) {
ctx = SimUtils.getRedactionContext(clusterState);
}
if (!withSuggestions && !withDiagnostics) {
withSuggestions = true;
}
Map<String, Object> results = prepareResults(cloudManager, config, withClusterState,
withStats, withSuggestions, withSortedNodes, withDiagnostics);
if (cli.hasOption("simulate")) {
String iterStr = cli.getOptionValue("i", "10");
String saveSimulated = cli.getOptionValue("saveSimulated");
int iterations;
try {
iterations = Integer.parseInt(iterStr);
} catch (Exception e) {
log.warn("Invalid option 'i' value, using default 10:", e);
iterations = 10;
}
Map<String, Object> simulationResults = new HashMap<>();
simulate(cloudManager, config, simulationResults, saveSimulated, withClusterState,
withStats, withSuggestions, withSortedNodes, withDiagnostics, iterations, redact);
results.put("simulation", simulationResults);
}
String data = Utils.toJSONString(results);
if (redact) {
data = RedactionUtils.redactNames(ctx.getRedactions(), data);
}
stdout.println(data);
}
private Map<String, Object> prepareResults(SolrCloudManager clientCloudManager,
AutoScalingConfig config,
boolean withClusterState,
boolean withStats,
boolean withSuggestions,
boolean withSortedNodes,
boolean withDiagnostics) throws Exception {
Policy.Session session = config.getPolicy().createSession(clientCloudManager);
ClusterState clusterState = clientCloudManager.getClusterStateProvider().getClusterState();
List<Suggester.SuggestionInfo> suggestions = Collections.emptyList();
long start, end;
if (withSuggestions) {
CLIO.err("- calculating suggestions...");
start = TimeSource.NANO_TIME.getTimeNs();
suggestions = PolicyHelper.getSuggestions(config, clientCloudManager);
end = TimeSource.NANO_TIME.getTimeNs();
CLIO.err(" (took " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms)");
}
Map<String, Object> diagnostics = Collections.emptyMap();
if (withDiagnostics) {
CLIO.err("- calculating diagnostics...");
start = TimeSource.NANO_TIME.getTimeNs();
MapWriter mw = PolicyHelper.getDiagnostics(session);
diagnostics = new LinkedHashMap<>();
mw.toMap(diagnostics);
end = TimeSource.NANO_TIME.getTimeNs();
CLIO.err(" (took " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms)");
}
Map<String, Object> results = new LinkedHashMap<>();
if (withClusterState) {
Map<String, Object> map = new LinkedHashMap<>();
map.put("liveNodes", new TreeSet<>(clusterState.getLiveNodes()));
map.put("collections", clusterState.getCollectionsMap());
results.put("CLUSTERSTATE", map);
}
if (withStats) {
results.put("STATISTICS", SimUtils.calculateStats(clientCloudManager, config, verbose));
}
if (withSuggestions) {
results.put("SUGGESTIONS", suggestions);
}
if (!withSortedNodes) {
diagnostics.remove("sortedNodes");
}
if (withDiagnostics) {
results.put("DIAGNOSTICS", diagnostics);
}
return results;
}
private void simulate(SolrCloudManager cloudManager,
AutoScalingConfig config,
Map<String, Object> results,
String saveSimulated,
boolean withClusterState,
boolean withStats,
boolean withSuggestions,
boolean withSortedNodes,
boolean withDiagnostics, int iterations, boolean redact) throws Exception {
File saveDir = null;
if (saveSimulated != null) {
saveDir = new File(saveSimulated);
if (!saveDir.exists()) {
if (!saveDir.mkdirs()) {
throw new Exception("Unable to create 'saveSimulated' directory: " + saveDir.getAbsolutePath());
}
} else if (!saveDir.isDirectory()) {
throw new Exception("'saveSimulated' path exists and is not a directory! " + saveDir.getAbsolutePath());
}
}
int SPEED = 50;
SimCloudManager simCloudManager = SimCloudManager.createCluster(cloudManager, config, TimeSource.get("simTime:" + SPEED));
int loop = 0;
List<Suggester.SuggestionInfo> suggestions = Collections.emptyList();
Map<String, Object> intermediate = new LinkedHashMap<>();
results.put("intermediate", intermediate);
while (loop < iterations) {
LinkedHashMap<String, Object> perStep = new LinkedHashMap<>();
long start = TimeSource.NANO_TIME.getTimeNs();
suggestions = PolicyHelper.getSuggestions(config, simCloudManager);
CLIO.err("-- step " + loop + ", " + suggestions.size() + " suggestions.");
long end = TimeSource.NANO_TIME.getTimeNs();
CLIO.err(" - calculated in " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms (real time ≈ simulated time)");
if (suggestions.isEmpty()) {
break;
}
SnapshotCloudManager snapshotCloudManager = new SnapshotCloudManager(simCloudManager, config);
if (saveDir != null) {
File target = new File(saveDir, "step" + loop + "_start");
snapshotCloudManager.saveSnapshot(target, true, redact);
}
if (verbose) {
Map<String, Object> snapshot = snapshotCloudManager.getSnapshot(false, redact);
snapshot.remove(SnapshotCloudManager.DISTRIB_STATE_KEY);
snapshot.remove(SnapshotCloudManager.MANAGER_STATE_KEY);
perStep.put("snapshotStart", snapshot);
}
intermediate.put("step" + loop, perStep);
int unresolvedCount = 0;
start = TimeSource.NANO_TIME.getTimeNs();
List<Map<String, Object>> perStepOps = new ArrayList<>(suggestions.size());
if (withSuggestions) {
perStep.put("suggestions", suggestions);
perStep.put("opDetails", perStepOps);
}
for (Suggester.SuggestionInfo suggestion : suggestions) {
SolrRequest<?> operation = suggestion.getOperation();
if (operation == null) {
unresolvedCount++;
if (suggestion.getViolation() == null) {
CLIO.err(" - ignoring suggestion without violation and without operation: " + suggestion);
}
continue;
}
SolrParams params = operation.getParams();
if (operation instanceof V2Request) {
params = SimUtils.v2AdminRequestToV1Params((V2Request)operation);
}
Map<String, Object> paramsMap = new LinkedHashMap<>();
params.toMap(paramsMap);
Replica info = simCloudManager.getSimClusterStateProvider().simGetReplicaInfo(
params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
if (info == null) {
CLIO.err("Could not find ReplicaInfo for params: " + params);
} else if (verbose) {
paramsMap.put("replicaInfo", info);
} else if (info.get(Variable.Type.CORE_IDX.tagName) != null) {
paramsMap.put(Variable.Type.CORE_IDX.tagName, info.get(Variable.Type.CORE_IDX.tagName));
}
if (withSuggestions) {
perStepOps.add(paramsMap);
}
try {
simCloudManager.request(operation);
} catch (Exception e) {
CLIO.err("Aborting - error executing suggestion " + suggestion + ": " + e);
Map<String, Object> error = new HashMap<>();
error.put("suggestion", suggestion);
error.put("replicaInfo", info);
error.put("exception", e);
perStep.put("error", error);
break;
}
}
end = TimeSource.NANO_TIME.getTimeNs();
long realTime = TimeUnit.NANOSECONDS.toMillis(end - start);
long simTime = realTime * SPEED;
CLIO.err(" - executed in " + realTime + " ms (real time), " + simTime + " ms (simulated time)");
if (unresolvedCount == suggestions.size()) {
CLIO.err("--- aborting simulation, only unresolved violations remain");
break;
}
if (withStats) {
perStep.put("statsExecutionStop", SimUtils.calculateStats(simCloudManager, config, verbose));
}
snapshotCloudManager = new SnapshotCloudManager(simCloudManager, config);
if (saveDir != null) {
File target = new File(saveDir, "step" + loop + "_stop");
snapshotCloudManager.saveSnapshot(target, true, redact);
}
if (verbose) {
Map<String, Object> snapshot = snapshotCloudManager.getSnapshot(false, redact);
snapshot.remove(SnapshotCloudManager.DISTRIB_STATE_KEY);
snapshot.remove(SnapshotCloudManager.MANAGER_STATE_KEY);
perStep.put("snapshotStop", snapshot);
}
loop++;
}
if (loop == iterations && !suggestions.isEmpty()) {
CLIO.err("### Failed to apply all suggestions in " + iterations + " steps. Remaining suggestions: " + suggestions + "\n");
}
results.put("finalState", prepareResults(simCloudManager, config, withClusterState, withStats,
withSuggestions, withSortedNodes, withDiagnostics));
}
}
/**
* Get the status of a Solr server.
*/

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -1,206 +0,0 @@
{
"/clusterstate.json":{
"owner":"0",
"mode":"PERSISTENT",
"version":0},
"/live_nodes":{
"owner":"0",
"mode":"PERSISTENT",
"version":0},
"/live_nodes/N_11_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_65p_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_8_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_74_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_1i_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_4g_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_2_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_a_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_1c_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_16_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_6c_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_v_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_3a_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_1d_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_1_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_u_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_7_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_t_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_6i_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_d4_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_17_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_1f_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_do_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_3_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_303_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_29_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_9o_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_7e_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_1m_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_4_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_m_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_dj_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_e_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_13_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_g_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_2w_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_z_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_cs_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_3a7_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_aw_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_0_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_3to_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_4f_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_1h_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_2u_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_b9_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_6_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/live_nodes/N_5_solr":{
"owner":"0",
"mode":"EPHEMERAL",
"version":0},
"/autoscaling.json":{
"owner":"0",
"mode":"PERSISTENT",
"data":"ewogICJjbHVzdGVyLXByZWZlcmVuY2VzIjpbCiAgICB7CiAgICAgICJtaW5pbWl6ZSI6ImNvcmVzIiwKICAgICAgInByZWNpc2lvbiI6MX0sCiAgICB7CiAgICAgICJtYXhpbWl6ZSI6ImZyZWVkaXNrIiwKICAgICAgInByZWNpc2lvbiI6MTB9XSwKICAiY2x1c3Rlci1wb2xpY3kiOlsKICAgIHsKICAgICAgInJlcGxpY2EiOiIjQUxMIiwKICAgICAgImNvbGxlY3Rpb24iOiJDT0xMXzIiLAogICAgICAic3lzcHJvcC5wb29sIjoicG9vbC0wMSJ9LAogICAgewogICAgICAicmVwbGljYSI6IiNBTEwiLAogICAgICAiY29sbGVjdGlvbiI6IkNPTExfMSIsCiAgICAgICJzeXNwcm9wLnBvb2wiOiJwb29sLTAyIn0sCiAgICB7CiAgICAgICJyZXBsaWNhIjoiI0FMTCIsCiAgICAgICJjb2xsZWN0aW9uIjoiQ09MTF8wIiwKICAgICAgInN5c3Byb3AucG9vbCI6InBvb2wtMDIifSwKICAgIHsKICAgICAgInJlcGxpY2EiOiI8MiIsCiAgICAgICJzaGFyZCI6IiNFQUNIIiwKICAgICAgIm5vZGUiOiIjQU5ZIn0sCiAgICB7CiAgICAgICJyZXBsaWNhIjoiI0VRVUFMIiwKICAgICAgInNoYXJkIjoiI0VBQ0giLAogICAgICAic3lzcHJvcC5heiI6IiNFQUNIIn1dLAogICJ0cmlnZ2VycyI6e30sCiAgImxpc3RlbmVycyI6e30sCiAgInByb3BlcnRpZXMiOnt9fQ==",
"version":0}}

View File

@ -1 +0,0 @@
{"timeSource":"SimTimeSource:50.0"}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -1,195 +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.IOException;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.lucene.util.LuceneTestCase;
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.SolrCloudManager;
import org.apache.solr.client.solrj.request.RequestWriter;
import org.apache.solr.client.solrj.request.RequestWriter.StringPayloadContentWriter;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.client.solrj.response.SolrResponseBase;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.util.TimeOut;
import org.junit.Assert;
import static org.apache.solr.common.params.CommonParams.JSON_MIME;
/**
* Some useful methods for SolrCloud tests.
*/
public class CloudTestUtils {
public static final int DEFAULT_TIMEOUT = 90;
/**
* Wait for a particular named trigger to be scheduled.
* <p>
* This is a convenience method that polls the autoscaling API looking for a trigger with the
* specified name using the {@link #DEFAULT_TIMEOUT}. It is particularly useful for tests
* that want to know when the Overseer has finished scheduling the automatic triggers on startup.
* </p>
*
* @param cloudManager current instance of {@link SolrCloudManager}
* @param triggerName the name of the trigger we need to see sheduled in order to return successfully
* @see #suspendTrigger
*/
public static long waitForTriggerToBeScheduled(final SolrCloudManager cloudManager,
final String triggerName)
throws InterruptedException, TimeoutException, IOException {
TimeOut timeout = new TimeOut(DEFAULT_TIMEOUT, TimeUnit.SECONDS, cloudManager.getTimeSource());
while (!timeout.hasTimedOut()) {
final SolrResponse response = cloudManager.request(AutoScalingRequest.create(SolrRequest.METHOD.GET, null));
@SuppressWarnings({"unchecked"})
final Map<String,?> triggers = (Map<String,?>) response.getResponse().get("triggers");
Assert.assertNotNull("null triggers in response from autoscaling request", triggers);
if ( triggers.containsKey(triggerName) ) {
return timeout.timeElapsed(TimeUnit.MILLISECONDS);
}
timeout.sleep(100);
}
throw new TimeoutException("Never saw trigger with name: " + triggerName);
}
/**
* Suspends the trigger with the specified name
* <p>
* This is a convenience method that sends a <code>suspend-trigger</code> command to the autoscaling
* API for the specified trigger. It is particularly useful for tests that may need to disable automatic
* triggers such as <code>.scheduled_maintenance</code> in order to test their own
* triggers.
* </p>
*
* @param cloudManager current instance of {@link SolrCloudManager}
* @param triggerName the name of the trigger to suspend. This must already be scheduled.
* @see #assertAutoScalingRequest
* @see #waitForTriggerToBeScheduled
*/
public static void suspendTrigger(final SolrCloudManager cloudManager,
final String triggerName) throws IOException {
assertAutoScalingRequest(cloudManager, "{'suspend-trigger' : {'name' : '"+triggerName+"'} }");
}
/**
* Creates &amp; executes an autoscaling request against the current cluster, asserting that
* the result is a success.
*
* @param cloudManager current instance of {@link SolrCloudManager}
* @param json The request to POST to the AutoScaling Handler
* @see AutoScalingRequest#create
*/
public static void assertAutoScalingRequest(final SolrCloudManager cloudManager,
final String json) throws IOException {
// TODO: a lot of code that directly uses AutoScalingRequest.create should use this method
@SuppressWarnings({"rawtypes"})
final SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, json);
final SolrResponse rsp = cloudManager.request(req);
final String result = rsp.getResponse().get("result").toString();
Assert.assertEquals("Unexpected result from auto-scaling command: " + json + " -> " + rsp,
"success", result);
}
/**
* Helper class for sending (JSON) autoscaling requests that can randomize between V1 and V2 requests
*/
@SuppressWarnings({"rawtypes"})
public static class AutoScalingRequest extends SolrRequest {
private SolrParams params = null;
/**
* Creates a request using a randomized root path (V1 vs V2)
*
* @param m HTTP Method to use
* @aram message JSON payload, may be null
*/
@SuppressWarnings({"rawtypes"})
public static SolrRequest create(SolrRequest.METHOD m, String message) {
return create(m, null, message);
}
/**
* Creates a request using a randomized root path (V1 vs V2)
*
* @param m HTTP Method to use
* @param subPath optional sub-path under <code>"$ROOT/autoscaling"</code>. may be null,
* otherwise must start with "/"
* @param message JSON payload, may be null
*/
@SuppressWarnings({"rawtypes"})
public static SolrRequest create(SolrRequest.METHOD m, String subPath, String message) {
return create(m,subPath,message,null);
}
@SuppressWarnings({"rawtypes"})
public static SolrRequest create(SolrRequest.METHOD m, String subPath, String message, SolrParams params) {
final boolean useV1 = LuceneTestCase.random().nextBoolean();
String path = useV1 ? "/admin/autoscaling" : "/cluster/autoscaling";
if (null != subPath) {
assert subPath.startsWith("/");
path += subPath;
}
return useV1
? new AutoScalingRequest(m, path, message).withParams(params)
: new V2Request.Builder(path).withMethod(m).withParams(params).withPayload(message).build();
}
protected final String message;
/**
* Simple request
* @param m HTTP Method to use
* @param path path to send request to
* @param message JSON payload, may be null
*/
private AutoScalingRequest(METHOD m, String path, String message) {
super(m, path);
this.message = message;
}
AutoScalingRequest withParams(SolrParams params){
this.params = params;
return this;
}
@Override
public SolrParams getParams() {
return params;
}
@Override
public RequestWriter.ContentWriter getContentWriter(String expectedType) {
return message == null ? null : new StringPayloadContentWriter(message, JSON_MIME);
}
@Override
protected SolrResponse createResponse(SolrClient client) {
return new SolrResponseBase();
}
}
}

View File

@ -20,7 +20,6 @@ import static java.util.Arrays.asList;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_DEF;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.NUM_SHARDS_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
import static org.apache.solr.common.params.CollectionAdminParams.DEFAULTS;
@ -63,7 +62,6 @@ import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.CoreAdminParams;
@ -90,8 +88,6 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
.addConfig("conf2", configset("cloud-dynamic"))
.configure();
// clear any persisted auto scaling configuration
zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
}
@After
@ -550,22 +546,20 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
// sanity check our expected default
final ClusterProperties props = new ClusterProperties(zkClient());
assertEquals("Expecting prop to default to unset, test needs upated",
props.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, null), null);
CollectionAdminResponse response = CollectionAdminRequest.setClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, "true")
CollectionAdminResponse response = CollectionAdminRequest.setClusterProperty(ZkStateReader.MAX_CORES_PER_NODE, "42")
.process(cluster.getSolrClient());
assertEquals(0, response.getStatus());
assertEquals("Cluster property was not set", props.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, null), "true");
assertEquals("Cluster property was not set", props.getClusterProperty(ZkStateReader.MAX_CORES_PER_NODE, null), "42");
// Unset ClusterProp that we set.
CollectionAdminRequest.setClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, null).process(cluster.getSolrClient());
assertEquals("Cluster property was not unset", props.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, null), null);
CollectionAdminRequest.setClusterProperty(ZkStateReader.MAX_CORES_PER_NODE, null).process(cluster.getSolrClient());
assertEquals("Cluster property was not unset", props.getClusterProperty(ZkStateReader.MAX_CORES_PER_NODE, null), null);
response = CollectionAdminRequest.setClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, "false")
response = CollectionAdminRequest.setClusterProperty(ZkStateReader.MAX_CORES_PER_NODE, "1")
.process(cluster.getSolrClient());
assertEquals(0, response.getStatus());
assertEquals("Cluster property was not set", props.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, null), "false");
assertEquals("Cluster property was not set", props.getClusterProperty(ZkStateReader.MAX_CORES_PER_NODE, null), "1");
}
@Test

View File

@ -30,7 +30,6 @@ import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.GenericSolrRequest;
import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.SolrParams;
@ -49,6 +48,7 @@ import org.slf4j.LoggerFactory;
*
*/
@LuceneTestCase.Slow
@LuceneTestCase.Nightly
@LogLevel("org.apache.solr.handler.admin=DEBUG")
public class MetricsHistoryIntegrationTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@ -59,18 +59,11 @@ public class MetricsHistoryIntegrationTest extends SolrCloudTestCase {
@BeforeClass
public static void setupCluster() throws Exception {
boolean simulated = TEST_NIGHTLY ? random().nextBoolean() : true;
if (simulated) {
cloudManager = SimCloudManager.createCluster(1, TimeSource.get("simTime:50"));
solrClient = ((SimCloudManager)cloudManager).simGetSolrClient();
}
configureCluster(1)
.addConfig("conf", configset("cloud-minimal"))
.configure();
if (!simulated) {
cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
solrClient = cluster.getSolrClient();
}
cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
solrClient = cluster.getSolrClient();
timeSource = cloudManager.getTimeSource();
// create .system
CollectionAdminRequest.createCollection(CollectionAdminParams.SYSTEM_COLL, null, 1, 1)
@ -78,19 +71,12 @@ public class MetricsHistoryIntegrationTest extends SolrCloudTestCase {
CloudUtil.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL,
30, TimeUnit.SECONDS, CloudUtil.clusterShape(1, 1));
solrClient.query(CollectionAdminParams.SYSTEM_COLL, params(CommonParams.Q, "*:*"));
// sleep a little to allow the handler to collect some metrics
if (simulated) {
timeSource.sleep(90000);
} else {
timeSource.sleep(100000);
}
// sleep until next generation of kids grow up to allow the handler to collect some metrics
timeSource.sleep(100000);
}
@AfterClass
public static void teardown() throws Exception {
if (cloudManager instanceof SimCloudManager) {
cloudManager.close();
}
solrClient = null;
cloudManager = null;
}

View File

@ -105,7 +105,6 @@ public class MoveReplicaTest extends SolrCloudTestCase {
// random create tlog or pull type replicas with nrt
boolean isTlog = random().nextBoolean();
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, 1, isTlog ? 1 : 0, !isTlog ? 1 : 0);
create.setAutoAddReplicas(false);
cloudClient.request(create);
addDocs(coll, 100);
@ -249,7 +248,6 @@ public class MoveReplicaTest extends SolrCloudTestCase {
// random create tlog or pull type replicas with nrt
boolean isTlog = random().nextBoolean();
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, 1, isTlog ? 1 : 0, !isTlog ? 1 : 0);
create.setAutoAddReplicas(false);
cloudClient.request(create);
addDocs(coll, 100);

View File

@ -37,8 +37,7 @@ import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
import org.apache.solr.client.solrj.cloud.VersionedData;
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
import org.apache.solr.cloud.Overseer.LeaderStatus;
import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent;
@ -81,9 +80,6 @@ import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.params.CollectionAdminParams.CLUSTER;
import static org.apache.solr.common.params.CollectionAdminParams.DEFAULTS;
import static org.apache.solr.common.params.CollectionAdminParams.USE_LEGACY_REPLICA_ASSIGNMENT;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyBoolean;
import static org.mockito.Mockito.anyInt;
@ -127,7 +123,6 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
private static HttpClient httpClientMock;
private static ObjectCache objectCache;
private static AutoScalingConfig autoScalingConfig = new AutoScalingConfig(Collections.emptyMap());
private Map<String, byte[]> zkClientData = new HashMap<>();
private final Map<String, ClusterState.CollectionRef> collectionsSet = new HashMap<>();
private final List<ZkNodeProps> replicas = new ArrayList<>();
@ -294,7 +289,6 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
when(zkStateReaderMock.getZkClient()).thenReturn(solrZkClientMock);
when(zkStateReaderMock.getClusterState()).thenReturn(clusterStateMock);
when(zkStateReaderMock.getAutoScalingConfig()).thenReturn(autoScalingConfig);
when(zkStateReaderMock.getAliases()).thenReturn(Aliases.EMPTY);
when(clusterStateMock.getCollection(anyString())).thenAnswer(invocation -> {
@ -373,10 +367,8 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
when(cloudDataProviderMock.getClusterStateProvider()).thenReturn(clusterStateProviderMock);
when(clusterStateProviderMock.getClusterState()).thenReturn(clusterStateMock);
when(clusterStateProviderMock.getLiveNodes()).thenReturn(liveNodes);
when(clusterStateProviderMock.getClusterProperties()).thenReturn(Utils.makeMap(DEFAULTS, Utils.makeMap(CLUSTER, Utils.makeMap(USE_LEGACY_REPLICA_ASSIGNMENT, true))));
when(cloudDataProviderMock.getDistribStateManager()).thenReturn(stateManagerMock);
when(cloudManagerMock.getDistribStateManager()).thenReturn(distribStateManagerMock);
when(distribStateManagerMock.getAutoScalingConfig()).thenReturn(new AutoScalingConfig(Collections.emptyMap()));
Mockito.doAnswer(
new Answer<Void>() {
@ -456,7 +448,6 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
when(zkControllerMock.getZkClient()).thenReturn(solrZkClientMock);
when(cloudManagerMock.getDistribStateManager()).thenReturn(distribStateManagerMock);
when(distribStateManagerMock.getAutoScalingConfig()).thenReturn(new AutoScalingConfig(Collections.emptyMap()));
Mockito.doAnswer(
new Answer<Void>() {

View File

@ -1,127 +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.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Set;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrRequest;
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.request.CoreAdminRequest;
import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.cloud.ReplaceNodeTest.createReplaceNodeRequest;
@LuceneTestCase.AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11067")
public class ReplaceNodeNoTargetTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(6)
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-dynamic").resolve("conf"))
.configure();
}
protected String getSolrXml() {
return "solr.xml";
}
@Test
@LuceneTestCase.AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11067")
public void test() throws Exception {
String coll = "replacenodetest_coll_notarget";
if (log.isInfoEnabled()) {
log.info("total_jettys: {}", cluster.getJettySolrRunners().size());
}
CloudSolrClient cloudClient = cluster.getSolrClient();
Set<String> liveNodes = cloudClient.getZkStateReader().getClusterState().getLiveNodes();
ArrayList<String> l = new ArrayList<>(liveNodes);
Collections.shuffle(l, random());
String node2bdecommissioned = l.get(0);
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'replica':'<5', 'shard': '#EACH', 'node': '#ANY'}]}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
log.info("Creating collection...");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 5, 2, 0, 0);
cloudClient.request(create);
cluster.waitForActiveCollection(coll, 5, 10);
if (log.isInfoEnabled()) {
log.info("Current core status list for node we plan to decommision: {} => {}",
node2bdecommissioned,
getCoreStatusForNamedNode(cloudClient, node2bdecommissioned).getCoreStatus());
log.info("Decommisioning node: {}", node2bdecommissioned);
}
createReplaceNodeRequest(node2bdecommissioned, null, null).processAsync("001", cloudClient);
CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus("001");
boolean success = false;
CollectionAdminRequest.RequestStatusResponse rsp = null;
for (int i = 0; i < 300; i++) {
rsp = requestStatus.process(cloudClient);
if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
success = true;
break;
}
assertFalse("async replace node request aparently failed: " + rsp.toString(),
rsp.getRequestStatus() == RequestStatusState.FAILED);
Thread.sleep(50);
}
assertTrue("async replace node request should have finished successfully by now, last status: " + rsp,
success);
CoreAdminResponse status = getCoreStatusForNamedNode(cloudClient, node2bdecommissioned);
assertEquals("Expected no cores for decommisioned node: "
+ status.getCoreStatus().toString(),
0, status.getCoreStatus().size());
}
/**
* Given a cloud client and a nodename, build an HTTP client for that node, and ask it for it's core status
*/
private CoreAdminResponse getCoreStatusForNamedNode(final CloudSolrClient cloudClient,
final String nodeName) throws Exception {
try (HttpSolrClient coreclient = getHttpSolrClient
(cloudClient.getZkStateReader().getBaseUrlForNodeName(nodeName))) {
return CoreAdminRequest.getStatus(null, coreclient);
}
}
}

View File

@ -1,242 +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.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.SolrClientCloudManager;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.util.CommonTestInjection;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.handler.component.TrackingShardHandlerFactory;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.rule.ImplicitSnitch.SYSPROP;
public class RoutingToNodesWithPropertiesTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String PROP_NAME = SYSPROP + "zone";
final static String COLLECTION = "coll";
private final List<String> zone1Nodes = new ArrayList<>();
private final List<String> zone2Nodes = new ArrayList<>();
private final LinkedList<TrackingShardHandlerFactory.ShardRequestAndParams> zone1Queue = new LinkedList<>();
private final LinkedList<TrackingShardHandlerFactory.ShardRequestAndParams> zone2Queue = new LinkedList<>();
@Before
public void setupCluster() throws Exception {
CommonTestInjection.setAdditionalProps(ImmutableMap.of("zone", "us-west1"));
configureCluster(2)
.withSolrXml(TEST_PATH().resolve("solr-trackingshardhandler.xml"))
.addConfig("config", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
.configure();
zone1Nodes.addAll(cluster.getJettySolrRunners().stream().map(JettySolrRunner::getNodeName).collect(Collectors.toSet()));
CommonTestInjection.setAdditionalProps(ImmutableMap.of("zone", "us-west2"));
zone2Nodes.add(cluster.startJettySolrRunner().getNodeName());
zone2Nodes.add(cluster.startJettySolrRunner().getNodeName());
String commands = "{set-cluster-policy :[{" +
" 'replica':'#EQUAL'," +
" 'shard':'#EACH'," +
" 'sysprop.zone':'#EACH'}]}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, commands);
cluster.getSolrClient().request(req);
CollectionAdminRequest.createCollection(COLLECTION, 2, 2)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 4);
// Checking putting replicas
for (Slice slice : getCollectionState(COLLECTION).getSlices()) {
int numReplicaInZone1 = 0;
int numReplicaInZone2 = 0;
for (Replica replica : slice.getReplicas()) {
if (zone1Nodes.contains(replica.getNodeName()))
numReplicaInZone1++;
if (zone2Nodes.contains(replica.getNodeName()))
numReplicaInZone2++;
}
assertEquals(1, numReplicaInZone1);
assertEquals(1, numReplicaInZone2);
}
// check inject props
try (SolrCloudManager cloudManager = new SolrClientCloudManager(new ZkDistributedQueueFactory(cluster.getZkClient()),
cluster.getSolrClient())) {
for (String zone1Node: zone1Nodes) {
NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
Map<String, Object> map = nodeStateProvider.getNodeValues(zone1Node, Collections.singletonList(PROP_NAME));
assertEquals("us-west1", map.get(PROP_NAME));
}
for (String zone2Node: zone2Nodes) {
NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
Map<String, Object> map = nodeStateProvider.getNodeValues(zone2Node, Collections.singletonList(PROP_NAME));
assertEquals("us-west2", map.get(PROP_NAME));
}
for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
if (zone1Nodes.contains(jetty.getNodeName())) {
((TrackingShardHandlerFactory)jetty.getCoreContainer().getShardHandlerFactory()).setTrackingQueue(zone1Queue);
} else {
((TrackingShardHandlerFactory)jetty.getCoreContainer().getShardHandlerFactory()).setTrackingQueue(zone2Queue);
}
}
for (int i = 0; i < 20; i++) {
new UpdateRequest()
.add("id", String.valueOf(i))
.process(cluster.getSolrClient(), COLLECTION);
}
new UpdateRequest()
.commit(cluster.getSolrClient(), COLLECTION);
}
}
@After
public void after() {
TestInjection.reset();
}
@Test
public void test() throws Exception {
final int NUM_TRY = 10;
CollectionAdminRequest
.setClusterProperty(ZkStateReader.DEFAULT_SHARD_PREFERENCES, ShardParams.SHARDS_PREFERENCE_NODE_WITH_SAME_SYSPROP +":"+PROP_NAME)
.process(cluster.getSolrClient());
{
TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS, TimeSource.NANO_TIME);
timeOut.waitFor("Timeout waiting for sysprops are cached in all nodes", () -> {
int total = 0;
for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
total += runner.getCoreContainer().getZkController().getSysPropsCacher().getCacheSize();
}
return total == cluster.getJettySolrRunners().size() * cluster.getJettySolrRunners().size();
});
}
for (int i = 0; i < NUM_TRY; i++) {
SolrQuery qRequest = new SolrQuery("*:*");
ModifiableSolrParams qParams = new ModifiableSolrParams();
qParams.add(ShardParams.SHARDS_INFO, "true");
qRequest.add(qParams);
QueryResponse qResponse = cluster.getSolrClient().query(COLLECTION, qRequest);
Object shardsInfo = qResponse.getResponse().get(ShardParams.SHARDS_INFO);
assertNotNull("Unable to obtain "+ShardParams.SHARDS_INFO, shardsInfo);
SimpleOrderedMap<?> shardsInfoMap = (SimpleOrderedMap<?>)shardsInfo;
String firstReplicaAddr = ((SimpleOrderedMap) shardsInfoMap.getVal(0)).get("shardAddress").toString();
String secondReplicaAddr = ((SimpleOrderedMap) shardsInfoMap.getVal(1)).get("shardAddress").toString();
boolean firstReplicaInZone1 = false;
boolean secondReplicaInZone1 = false;
for (String zone1Node : zone1Nodes) {
zone1Node = zone1Node.replace("_solr", "");
firstReplicaInZone1 = firstReplicaInZone1 || firstReplicaAddr.contains(zone1Node);
secondReplicaInZone1 = secondReplicaInZone1 || secondReplicaAddr.contains(zone1Node);
}
assertEquals(firstReplicaInZone1, secondReplicaInZone1);
}
// intense asserting using TrackingShardHandlerFactory
assertRoutingToSameZone();
// Cachers should be stop running
CollectionAdminRequest
.setClusterProperty(ZkStateReader.DEFAULT_SHARD_PREFERENCES, ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE+":PULL")
.process(cluster.getSolrClient());
{
TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS, TimeSource.NANO_TIME);
timeOut.waitFor("Timeout waiting for sysPropsCache stop", () -> {
int numNodeStillRunningCache = 0;
for (JettySolrRunner runner: cluster.getJettySolrRunners()) {
if (runner.getCoreContainer().getZkController().getSysPropsCacher().isRunning()) {
numNodeStillRunningCache++;
}
}
return numNodeStillRunningCache == 0;
});
}
// Testing disable default shard preferences
CollectionAdminRequest
.setClusterProperty(ZkStateReader.DEFAULT_SHARD_PREFERENCES, null)
.process(cluster.getSolrClient());
{
TimeOut timeOut = new TimeOut(20, TimeUnit.SECONDS, TimeSource.NANO_TIME);
timeOut.waitFor("Timeout waiting cluster properties get updated", () -> {
int numNodeGetUpdatedPref = 0;
int numNodeStillRunningCache = 0;
for (JettySolrRunner runner: cluster.getJettySolrRunners()) {
if (runner.getCoreContainer().getZkController()
.getZkStateReader().getClusterProperties().containsKey(ZkStateReader.DEFAULT_SHARD_PREFERENCES)) {
numNodeGetUpdatedPref++;
}
if (runner.getCoreContainer().getZkController().getSysPropsCacher().isRunning()) {
numNodeStillRunningCache++;
}
}
return numNodeGetUpdatedPref == 0 && numNodeStillRunningCache == 0;
});
}
}
private void assertRoutingToSameZone() {
for (TrackingShardHandlerFactory.ShardRequestAndParams sreq: zone1Queue) {
String firstNode = sreq.shard.split("\\|")[0];
assertTrue(zone1Nodes.stream().anyMatch(s -> firstNode.contains(s.replace('_','/'))));
}
for (TrackingShardHandlerFactory.ShardRequestAndParams sreq: zone2Queue) {
String firstNode = sreq.shard.split("\\|")[0];
assertTrue(zone2Nodes.stream().anyMatch(s -> firstNode.contains(s.replace('_','/'))));
}
}
}

View File

@ -82,7 +82,7 @@ import org.slf4j.LoggerFactory;
QuickPatchThreadsFilter.class,
BadHdfsThreadsFilter.class // hdfs currently leaks thread(s)
})
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.*=DEBUG")
@LogLevel("org.apache.solr.cloud.*=DEBUG")
@LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 20-Jul-2018
public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@ -161,8 +161,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
private void testBasics() throws Exception {
String collection1 = "solrj_collection";
Create createCollectionRequest = CollectionAdminRequest.createCollection(collection1,"conf1",2,2)
.setRouterField("myOwnField")
.setAutoAddReplicas(true);
.setRouterField("myOwnField");
CollectionAdminResponse response = createCollectionRequest.process(cloudClient);
assertEquals(0, response.getStatus());
@ -171,8 +170,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
String collection2 = "solrj_collection2";
createCollectionRequest = CollectionAdminRequest.createCollection(collection2,"conf1",2,2)
.setRouterField("myOwnField")
.setAutoAddReplicas(false);
.setRouterField("myOwnField");
CollectionAdminResponse response2 = createCollectionRequest.process(getCommonCloudSolrClient());
assertEquals(0, response2.getStatus());
@ -182,8 +180,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
String collection3 = "solrj_collection3";
createCollectionRequest = CollectionAdminRequest.createCollection(collection3,"conf1",5,1)
.setRouterField("myOwnField")
.setAutoAddReplicas(true);
.setRouterField("myOwnField");
CollectionAdminResponse response3 = createCollectionRequest.process(getCommonCloudSolrClient());
assertEquals(0, response3.getStatus());
@ -194,8 +191,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
// a collection has only 1 replica per a shard
String collection4 = "solrj_collection4";
createCollectionRequest = CollectionAdminRequest.createCollection(collection4,"conf1",5,1)
.setRouterField("text")
.setAutoAddReplicas(true);
.setRouterField("text");
CollectionAdminResponse response4 = createCollectionRequest.process(getCommonCloudSolrClient());
assertEquals(0, response4.getStatus());

View File

@ -20,7 +20,6 @@ package org.apache.solr.cloud;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterProperties;
import org.apache.solr.common.cloud.ZkStateReader;
import org.junit.BeforeClass;
import org.junit.Test;
@ -38,17 +37,6 @@ public class TestClusterProperties extends SolrCloudTestCase {
super.setUp();
props = new ClusterProperties(zkClient());
}
@Test
public void testClusterProperties() throws Exception {
assertEquals("false", props.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, "false"));
CollectionAdminRequest.setClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, "true").process(cluster.getSolrClient());
assertEquals("true", props.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, "false"));
CollectionAdminRequest.setClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, "false").process(cluster.getSolrClient());
assertEquals("false", props.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, "true"));
}
@Test
public void testSetPluginClusterProperty() throws Exception {

View File

@ -1,184 +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.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
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.cloud.CloudTestUtils.AutoScalingRequest;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
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;
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;org.apache.solr.client.solrj.impl.SolrClientDataProvider=DEBUG;org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper=TRACE")
public class TestUtilizeNode extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(3)
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-dynamic").resolve("conf"))
.configure();
NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
JettySolrRunner overseerJetty = null;
String overseerLeader = (String) overSeerStatus.get("leader");
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner jetty = cluster.getJettySolrRunner(i);
if (jetty.getNodeName().equals(overseerLeader)) {
overseerJetty = jetty;
break;
}
}
if (overseerJetty == null) {
fail("no overseer leader!");
}
}
protected String getSolrXml() {
return "solr.xml";
}
@Before
public void beforeTest() throws Exception {
cluster.deleteAllCollections();
}
@Test
public void test() throws Exception {
cluster.waitForAllNodes(5);
String coll = "utilizenodecoll";
CloudSolrClient cloudClient = cluster.getSolrClient();
log.info("Creating Collection...");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, 2);
cloudClient.request(create);
log.info("Spinning up additional jettyX...");
JettySolrRunner jettyX = cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
assertNoReplicas("jettyX should not yet be utilized: ", coll, jettyX);
if (log.isInfoEnabled()) {
log.info("Sending UTILIZE command for jettyX ({})", jettyX.getNodeName());
}
cloudClient.request(new CollectionAdminRequest.UtilizeNode(jettyX.getNodeName()));
// TODO: aparently we can't assert this? ...
//
// assertSomeReplicas("jettyX should now be utilized: ", coll, jettyX);
//
// ... it appears from the docs that unless there are policy violations,
// this can be ignored unless jettyX has less "load" then other jetty instances?
//
// if the above is true, that means that this test is incredibly weak...
// unless we know jettyX has at least one replica, then all the subsequent testing of the
// port blacklist & additional UTILIZE command for jettyY are a waste of time.
//
// should we skip spinning up a *new* jettyX, and instead just pick an existing jetty?
if (log.isInfoEnabled()) {
log.info("jettyX replicas prior to being blacklisted: {}", getReplicaList(coll, jettyX));
}
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'port':" + jettyX.getLocalPort() +
" , 'replica':0}" +
" ]" +
"}";
if (log.isInfoEnabled()) {
log.info("Setting new policy to blacklist jettyX ({}) port={}",
jettyX.getNodeName(), jettyX.getLocalPort());
}
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
NamedList<Object> response = cloudClient.request(req);
assertEquals(req + " => " + response,
"success", response.get("result").toString());
log.info("Spinning up additional jettyY...");
JettySolrRunner jettyY = cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
assertNoReplicas("jettyY should not yet be utilized: ", coll, jettyY);
if (log.isInfoEnabled()) {
log.info("jettyX replicas prior to utilizing jettyY: {}", getReplicaList(coll, jettyX));
log.info("Sending UTILIZE command for jettyY ({})", jettyY.getNodeName()); // logOk
}
cloudClient.request(new CollectionAdminRequest.UtilizeNode(jettyY.getNodeName()));
assertSomeReplicas("jettyY should now be utilized: ", coll, jettyY);
}
/**
* Gets the list of replicas for the specified collection hosted on the specified node
* and then asserts that it has no replicas
*/
private void assertNoReplicas(String prefix, String collectionName, JettySolrRunner jettyNode) throws IOException {
final List<Replica> replicas = getReplicaList(collectionName, jettyNode);
assertEquals(prefix + " " + jettyNode.getNodeName() + " => " + replicas,
0, replicas.size());
}
/**
* Gets the list of replicas for the specified collection hosted on the specified node
* and then asserts that it there is at least one
*/
private void assertSomeReplicas(String prefix, String collectionName, JettySolrRunner jettyNode) throws IOException {
final List<Replica> replicas = getReplicaList(collectionName, jettyNode);
assertTrue(prefix + " " + jettyNode.getNodeName() + " => " + replicas,
0 < replicas.size());
}
/**
* Returns a list of all Replicas for the specified collection hosted on the specified node using
* an <em>uncached</em> ClusterState call (so it should be authoritative from ZK).
*/
private List<Replica> getReplicaList(String collectionName, JettySolrRunner jettyNode) throws IOException {
DocCollection collection = cluster.getSolrClient().getClusterStateProvider()
// we do *NOT* want to trust the cache, because anytime we call this method we have just
// done a lot of mucking with the cluster
.getClusterState().getCollectionOrNull(collectionName, false);
List<Replica> results = new ArrayList<>(3);
if (collection != null) {
collection.forEachReplica((s, replica) -> {
if (replica.getNodeName().equals(jettyNode.getNodeName())) {
results.add(replica);
}
});
}
return results;
}
}

View File

@ -1,611 +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.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
import org.apache.solr.cloud.autoscaling.ActionContext;
import org.apache.solr.cloud.autoscaling.ComputePlanAction;
import org.apache.solr.cloud.autoscaling.ExecutePlanAction;
import org.apache.solr.cloud.autoscaling.TriggerActionBase;
import org.apache.solr.cloud.autoscaling.TriggerEvent;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TimeOut;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.params.CollectionAdminParams.WITH_COLLECTION;
/**
* Tests for co-locating a collection with another collection such that any Collection API
* always ensures that the co-location is never broken.
*
* See SOLR-11990 for more details.
*/
@LogLevel("org.apache.solr.cloud.autoscaling=TRACE;org.apache.solr.client.solrj.cloud.autoscaling=DEBUG;org.apache.solr.cloud.overseer=DEBUG")
public class TestWithCollection extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static SolrCloudManager cloudManager;
private static final int NUM_JETTIES = 2;
@Before
public void setupCluster() throws Exception {
configureCluster(NUM_JETTIES)
.addConfig("conf", configset("cloud-minimal"))
.configure();
if (zkClient().exists(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, true)) {
zkClient().setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, "{}".getBytes(StandardCharsets.UTF_8), true);
}
cluster.getSolrClient().setDefaultCollection(null);
cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
LATCH = new CountDownLatch(1);
}
@After
public void teardownCluster() throws Exception {
shutdownCluster();
}
@AfterClass
public static void cleanUpAfterClass() throws Exception {
cloudManager = null;
}
private void deleteChildrenRecursively(String path) throws Exception {
cloudManager.getDistribStateManager().removeRecursively(path, true, false);
}
@Test
public void testCreateCollectionNoWithCollection() throws IOException, SolrServerException {
String prefix = "testCreateCollectionNoWithCollection";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
try {
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc).process(solrClient);
} catch (BaseHttpSolrClient.RemoteSolrException e) {
assertTrue(e.getMessage().contains("The 'withCollection' does not exist"));
}
CollectionAdminRequest.createCollection(abc, 2, 1)
.process(solrClient);
try {
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc).process(solrClient);
} catch (BaseHttpSolrClient.RemoteSolrException e) {
assertTrue(e.getMessage().contains("The `withCollection` must have only one shard, found: 2"));
}
}
public void testCreateCollection() throws Exception {
String prefix = "testCreateCollection";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
String chosenNode = cluster.getRandomJetty(random()).getNodeName();
CollectionAdminRequest.createCollection(abc, 1, 1)
.setCreateNodeSet(chosenNode) // randomize to avoid choosing the first node always
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
DocCollection c1 = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(xyz);
assertNotNull(c1);
assertEquals(abc, c1.getStr(WITH_COLLECTION));
Replica replica = c1.getReplicas().get(0);
String nodeName = replica.getNodeName();
assertEquals(chosenNode, nodeName);
}
@Test
//Commented 14-Oct-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 23-Aug-2018
public void testDeleteWithCollection() throws IOException, SolrServerException, InterruptedException {
String prefix = "testDeleteWithCollection";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
CollectionAdminRequest.createCollection(abc, 1, 1)
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
try {
CollectionAdminResponse response = CollectionAdminRequest.deleteCollection(abc).process(solrClient);
fail("Deleting collection: " + abc + " should have failed with an exception. Instead response was: " + response.getResponse());
} catch (BaseHttpSolrClient.RemoteSolrException e) {
assertTrue(e.getMessage().contains("is co-located with collection"));
}
// delete the co-located collection first
CollectionAdminRequest.deleteCollection(xyz).process(solrClient);
// deleting the with collection should succeed now
CollectionAdminRequest.deleteCollection(abc).process(solrClient);
xyz = xyz + "_2";
abc = abc + "_2";
CollectionAdminRequest.createCollection(abc, 1, 1)
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
// sanity check
try {
CollectionAdminResponse response = CollectionAdminRequest.deleteCollection(abc).process(solrClient);
fail("Deleting collection: " + abc + " should have failed with an exception. Instead response was: " + response.getResponse());
} catch (BaseHttpSolrClient.RemoteSolrException e) {
assertTrue(e.getMessage().contains("is co-located with collection"));
}
CollectionAdminRequest.modifyCollection(xyz, null)
.unsetAttribute("withCollection")
.process(solrClient);
TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
DocCollection c1 = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(xyz);
if (c1.getStr("withCollection") == null) break;
Thread.sleep(200);
}
DocCollection c1 = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(xyz);
assertNull(c1.getStr("withCollection"));
CollectionAdminRequest.deleteCollection(abc).process(solrClient);
}
@Test
public void testAddReplicaSimple() throws Exception {
String prefix = "testAddReplica";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
String chosenNode = cluster.getRandomJetty(random()).getNodeName();
log.info("Chosen node {} for collection {}", chosenNode, abc);
CollectionAdminRequest.createCollection(abc, 1, 1)
.setCreateNodeSet(chosenNode) // randomize to avoid choosing the first node always
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
String otherNode = null;
for (JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
if (!chosenNode.equals(jettySolrRunner.getNodeName())) {
otherNode = jettySolrRunner.getNodeName();
}
}
CollectionAdminRequest.addReplicaToShard(xyz, "shard1")
.setNode(otherNode)
.process(solrClient);
DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
assertTrue(collection.getReplicas().stream().noneMatch(replica -> withCollection.getReplicas(replica.getNodeName()).isEmpty()));
}
@Test
// commented out on: 17-Feb-2019 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 20-Sep-2018
public void testAddReplicaWithPolicy() throws Exception {
String prefix = "testAddReplicaWithPolicy";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'node':'#ANY'}," +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
String chosenNode = cluster.getRandomJetty(random()).getNodeName();
log.info("Chosen node {} for collection {}", chosenNode, abc);
CollectionAdminRequest.createCollection(abc, 1, 1)
.setCreateNodeSet(chosenNode) // randomize to avoid choosing the first node always
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
assertEquals(chosenNode, collection.getReplicas().iterator().next().getNodeName());
// zkClient().printLayoutToStdOut();
CollectionAdminRequest.addReplicaToShard(xyz, "shard1")
.process(solrClient);
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
assertTrue(collection.getReplicas().stream().noneMatch(
replica -> withCollection.getReplicas(replica.getNodeName()) == null
|| withCollection.getReplicas(replica.getNodeName()).isEmpty()));
}
@Test
public void testMoveReplicaMainCollection() throws Exception {
String prefix = "testMoveReplicaMainCollection";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'node':'#ANY'}," +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
String chosenNode = cluster.getRandomJetty(random()).getNodeName();
log.info("Chosen node {} for collection {}", chosenNode, abc);
CollectionAdminRequest.createCollection(abc, 1, 1)
.setCreateNodeSet(chosenNode) // randomize to avoid choosing the first node always
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
String otherNode = null;
for (JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
if (!chosenNode.equals(jettySolrRunner.getNodeName())) {
otherNode = jettySolrRunner.getNodeName();
}
}
DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
assertNull(collection.getReplicas(otherNode)); // sanity check
assertNull(withCollection.getReplicas(otherNode)); // sanity check
CollectionAdminRequest.MoveReplica moveReplica = new CollectionAdminRequest.MoveReplica(xyz, collection.getReplicas().iterator().next().getName(), otherNode);
moveReplica.setWaitForFinalState(true);
moveReplica.process(solrClient);
// zkClient().printLayoutToStdOut();
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz); // refresh
DocCollection withCollectionRefreshed = solrClient.getZkStateReader().getClusterState().getCollection(abc); // refresh
assertTrue(collection.getReplicas().stream().noneMatch(
replica -> withCollectionRefreshed.getReplicas(replica.getNodeName()) == null
|| withCollectionRefreshed.getReplicas(replica.getNodeName()).isEmpty()));
}
@Test
//Commented 14-Oct-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 23-Aug-2018
public void testMoveReplicaWithCollection() throws Exception {
String prefix = "testMoveReplicaWithCollection";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'node':'#ANY'}," +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
String chosenNode = cluster.getRandomJetty(random()).getNodeName();
log.info("Chosen node {} for collection {}", chosenNode, abc);
CollectionAdminRequest.createCollection(abc, 1, 1)
.setCreateNodeSet(chosenNode) // randomize to avoid choosing the first node always
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
assertEquals(chosenNode, collection.getReplicas().iterator().next().getNodeName());
String otherNode = null;
for (JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
if (!chosenNode.equals(jettySolrRunner.getNodeName())) {
otherNode = jettySolrRunner.getNodeName();
}
}
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
assertNull(collection.getReplicas(otherNode)); // sanity check
assertNull(withCollection.getReplicas(otherNode)); // sanity check
try {
new CollectionAdminRequest.MoveReplica(abc, collection.getReplicas().iterator().next().getName(), otherNode)
.process(solrClient);
fail("Expected moving a replica of 'withCollection': " + abc + " to fail");
} catch (BaseHttpSolrClient.RemoteSolrException e) {
assertTrue(e.getMessage().contains("Collection: testMoveReplicaWithCollection_abc is co-located with collection: testMoveReplicaWithCollection_xyz"));
}
// zkClient().printLayoutToStdOut();
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz); // refresh
DocCollection withCollectionRefreshed = solrClient.getZkStateReader().getClusterState().getCollection(abc); // refresh
// sanity check that the failed move operation didn't actually change our co-location guarantees
assertTrue(collection.getReplicas().stream().noneMatch(
replica -> withCollectionRefreshed.getReplicas(replica.getNodeName()) == null
|| withCollectionRefreshed.getReplicas(replica.getNodeName()).isEmpty()));
}
/**
* Tests that when a new node is added to the cluster and autoscaling framework
* moves replicas to the new node, we maintain all co-locating guarantees
*/
// commented out on: 01-Apr-2019 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 15-Sep-2018
public void testNodeAdded() throws Exception {
String prefix = "testNodeAdded";
String xyz = prefix + "_xyz";
String abc = prefix + "_abc";
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'node':'#ANY'}," +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
String chosenNode = cluster.getRandomJetty(random()).getNodeName();
log.info("Chosen node {} for collection {}", chosenNode, abc);
CollectionAdminRequest.createCollection(abc, 1, 1)
.setCreateNodeSet(chosenNode) // randomize to avoid choosing the first node always
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
assertEquals(chosenNode, collection.getReplicas().iterator().next().getNodeName());
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger1'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '0s'," +
"'actions' : [" +
"{'name' : 'compute', 'class' : '" + ComputePlanAction.class.getName() + "'}" +
"{'name' : 'execute', 'class' : '" + ExecutePlanAction.class.getName() + "'}" +
"{'name' : 'compute', 'class' : '" + CapturingAction.class.getName() + "'}" +
"]" +
"}}";
req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
solrClient.request(req);
Optional<JettySolrRunner> other = cluster.getJettySolrRunners()
.stream().filter(j -> !chosenNode.equals(j.getNodeName())).findAny();
String otherNode = other.orElseThrow(AssertionError::new).getNodeName();
// add an extra replica of abc collection on a different node
CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest.addReplicaToShard(abc, "shard1")
.setNode(otherNode);
addReplica.setWaitForFinalState(true);
addReplica.process(solrClient);
// refresh
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
// sanity check
assertColocated(collection, otherNode, withCollection);
assertEquals(1, collection.getReplicas().size());
Replica xyzReplica = collection.getReplicas().get(0);
// start a new node
JettySolrRunner newNode = cluster.startJettySolrRunner();
assertTrue("Action was not fired till 30 seconds", LATCH.await(30, TimeUnit.SECONDS));
// refresh
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
// sanity check
assertColocated(collection, otherNode, withCollection);
// assert that the replica of xyz collection was not moved
assertNotNull(collection.getReplica(xyzReplica.getName()));
assertEquals(chosenNode, collection.getReplicas().get(0).getNodeName());
// add an extra replica of xyz collection -- this should be placed on the 'otherNode'
addReplica = CollectionAdminRequest.addReplicaToShard(xyz, "shard1");
addReplica.setWaitForFinalState(true);
addReplica.process(solrClient);
// refresh
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
List<Replica> replicas = collection.getReplicas(otherNode);
assertNotNull(replicas);
assertEquals(1, replicas.size());
replicas = withCollection.getReplicas(otherNode);
assertNotNull(replicas);
assertEquals(1, replicas.size());
// add an extra replica of xyz collection -- this should be placed on the 'newNode'
addReplica = CollectionAdminRequest.addReplicaToShard(xyz, "shard1");
addReplica.setWaitForFinalState(true);
addReplica.process(solrClient);
// refresh
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
assertNotNull(collection.getReplicas(newNode.getNodeName()));
replicas = collection.getReplicas(newNode.getNodeName());
assertNotNull(replicas);
assertEquals(1, replicas.size());
replicas = withCollection.getReplicas(newNode.getNodeName());
assertNotNull(replicas);
assertEquals(1, replicas.size());
}
public void testMultipleWithCollections() throws Exception {
String prefix = "testMultipleWithCollections";
String xyz = prefix + "_xyz";
String xyz2 = prefix + "_xyz2";
String abc = prefix + "_abc";
String abc2 = prefix + "_abc2";
// start 2 more nodes so we have 4 in total
cluster.startJettySolrRunner();
cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'node':'#ANY'}," +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
solrClient.request(req);
String chosenNode = cluster.getJettySolrRunner(0).getNodeName();
log.info("Chosen node {} for collection {}", chosenNode, abc);
CollectionAdminRequest.createCollection(abc, 1, 1)
.setCreateNodeSet(chosenNode)
.process(solrClient);
CollectionAdminRequest.createCollection(xyz, 1, 1)
.setWithCollection(abc)
.process(solrClient);
String chosenNode2 = cluster.getJettySolrRunner(1).getNodeName();
log.info("Chosen node {} for collection {}", chosenNode2, abc2);
CollectionAdminRequest.createCollection(abc2, 1, 1)
.setCreateNodeSet(chosenNode2)
.process(solrClient);
CollectionAdminRequest.createCollection(xyz2, 1, 1)
.setWithCollection(abc2)
.process(solrClient);
// refresh
DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
DocCollection collection2 = solrClient.getZkStateReader().getClusterState().getCollection(xyz2);
DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
DocCollection withCollection2 = solrClient.getZkStateReader().getClusterState().getCollection(abc2);
// sanity check
assertColocated(collection, chosenNode2, withCollection); // no replica should be on chosenNode2
assertColocated(collection2, chosenNode, withCollection2); // no replica should be on chosenNode
String chosenNode3 = cluster.getJettySolrRunner(2).getNodeName();
CollectionAdminRequest.addReplicaToShard(xyz, "shard1")
.setNode(chosenNode3)
.process(solrClient);
String chosenNode4 = cluster.getJettySolrRunner(2).getNodeName();
CollectionAdminRequest.addReplicaToShard(xyz2, "shard1")
.setNode(chosenNode4)
.process(solrClient);
collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
collection2 = solrClient.getZkStateReader().getClusterState().getCollection(xyz2);
withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
withCollection2 = solrClient.getZkStateReader().getClusterState().getCollection(abc2);
// sanity check
assertColocated(collection, null, withCollection);
assertColocated(collection2, null, withCollection2);
}
/**
* Asserts that all replicas of collection are colocated with at least one
* replica of the withCollection and none of them should be on the given 'noneOnNode'.
*/
private void assertColocated(DocCollection collection, String noneOnNode, DocCollection withCollection) {
// sanity check
assertTrue(collection.getReplicas().stream().noneMatch(
replica -> withCollection.getReplicas(replica.getNodeName()) == null
|| withCollection.getReplicas(replica.getNodeName()).isEmpty()));
if (noneOnNode != null) {
assertTrue(collection.getReplicas().stream().noneMatch(
replica -> noneOnNode.equals(replica.getNodeName())));
}
}
private static CountDownLatch LATCH = new CountDownLatch(1);
public static class CapturingAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
LATCH.countDown();
}
}
}

View File

@ -121,9 +121,6 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
CollectionAdminRequest.createCollectionWithImplicitRouter(getCollectionName(), "conf1", "shard1,shard2", replFactor, numTlogReplicas, numPullReplicas) :
CollectionAdminRequest.createCollection(getCollectionName(), "conf1", NUM_SHARDS, replFactor, numTlogReplicas, numPullReplicas);
if (random().nextBoolean()) {
create.setAutoAddReplicas(true);//just to assert it survives the restoration
}
Properties coreProps = new Properties();
coreProps.put("customKey", "customValue");//just to assert it survives the restoration
create.setProperties(coreProps);
@ -366,7 +363,6 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
assertEquals(origShardToDocCount, getShardToDocCountMap(client, restoreCollection));
}
assertEquals(backupCollection.getAutoAddReplicas(), restoreCollection.getAutoAddReplicas());
assertEquals(sameConfig ? "conf1" : "customConfigName",
cluster.getSolrClient().getZkStateReader().readConfigName(restoreCollectionName));

View File

@ -1,204 +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.api.collections;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.stream.Collectors;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.cloud.DistribStateManager;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
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.ClusterStateProvider;
import org.apache.solr.client.solrj.impl.ZkDistribStateManager;
import org.apache.solr.cloud.ZkTestServer;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
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.util.ExecutorUtil;
import org.apache.solr.common.util.Utils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class AssignTest extends SolrTestCaseJ4 {
@Override
@Before
public void setUp() throws Exception {
super.setUp();
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
}
@Test
public void testAssignNode() throws Exception {
assumeWorkingMockito();
SolrZkClient zkClient = mock(SolrZkClient.class);
Map<String, byte[]> zkClientData = new HashMap<>();
when(zkClient.setData(anyString(), any(), anyInt(), anyBoolean())).then(invocation -> {
zkClientData.put(invocation.getArgument(0), invocation.getArgument(1));
return null;
}
);
when(zkClient.getData(anyString(), any(), any(), anyBoolean())).then(invocation ->
zkClientData.get(invocation.getArgument(0)));
// TODO: fix this to be independent of ZK
ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
String nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node1", nodeName);
nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node1", nodeName);
nodeName = Assign.assignCoreNodeName(stateManager, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node2", nodeName);
}
@Test
public void testIdIsUnique() throws Exception {
Path zkDir = createTempDir("zkData");
ZkTestServer server = new ZkTestServer(zkDir);
Object fixedValue = new Object();
String[] collections = new String[]{"c1","c2","c3","c4","c5","c6","c7","c8","c9"};
Map<String, ConcurrentHashMap<Integer, Object>> collectionUniqueIds = new HashMap<>();
for (String c : collections) {
collectionUniqueIds.put(c, new ConcurrentHashMap<>());
}
ExecutorService executor = ExecutorUtil.newMDCAwareCachedThreadPool("threadpool");
try {
server.run();
try (SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), 10000)) {
assertTrue(zkClient.isConnected());
for (String c : collections) {
zkClient.makePath("/collections/" + c, true);
}
// TODO: fix this to be independent of ZK
ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
List<Future<?>> futures = new ArrayList<>();
for (int i = 0; i < 73; i++) {
futures.add(executor.submit(() -> {
String collection = collections[random().nextInt(collections.length)];
int id = Assign.incAndGetId(stateManager, collection, 0);
Object val = collectionUniqueIds.get(collection).put(id, fixedValue);
if (val != null) {
fail("ZkController do not generate unique id for " + collection);
}
}));
}
for (Future<?> future : futures) {
future.get();
}
}
assertEquals(73, (long) collectionUniqueIds.values().stream()
.map(ConcurrentHashMap::size)
.reduce((m1, m2) -> m1 + m2).get());
} finally {
server.shutdown();
ExecutorUtil.shutdownAndAwaitTermination(executor);
}
}
@Test
public void testBuildCoreName() throws Exception {
Path zkDir = createTempDir("zkData");
ZkTestServer server = new ZkTestServer(zkDir);
server.run();
try (SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), 10000)) {
// TODO: fix this to be independent of ZK
ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
Map<String, Slice> slices = new HashMap<>();
slices.put("shard1", new Slice("shard1", new HashMap<>(), null,"collection1"));
slices.put("shard2", new Slice("shard2", new HashMap<>(), null,"collection1"));
DocCollection docCollection = new DocCollection("collection1", slices, null, DocRouter.DEFAULT);
assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildSolrCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));
assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildSolrCoreName(stateManager, docCollection, "shard2", Replica.Type.PULL));
} finally {
server.shutdown();
}
}
@Test
public void testUseLegacyByDefault() throws Exception {
assumeWorkingMockito();
SolrCloudManager solrCloudManager = mock(SolrCloudManager.class);
ClusterStateProvider clusterStateProvider = mock(ClusterStateProvider.class);
when(solrCloudManager.getClusterStateProvider()).thenReturn(clusterStateProvider);
DistribStateManager distribStateManager = mock(DistribStateManager.class);
when(solrCloudManager.getDistribStateManager()).thenReturn(distribStateManager);
when(distribStateManager.getAutoScalingConfig()).thenReturn(new AutoScalingConfig(Collections.emptyMap()));
// first we don't set any cluster property and assert that legacy assignment is used
when(clusterStateProvider.getClusterProperties()).thenReturn(Collections.emptyMap());
// verify
boolean usePolicyFramework = Assign.usePolicyFramework(solrCloudManager);
assertFalse(usePolicyFramework);
// another sanity check
when(clusterStateProvider.getClusterProperties()).thenReturn(Utils.makeMap("defaults", Collections.emptyMap()));
// verify
usePolicyFramework = Assign.usePolicyFramework(solrCloudManager);
assertFalse(usePolicyFramework);
// first we set useLegacyReplicaAssignment=false, so autoscaling should always be used
when(clusterStateProvider.getClusterProperties()).thenReturn(Utils.makeMap("defaults", Utils.makeMap("cluster", Utils.makeMap("useLegacyReplicaAssignment", false))));
// verify
usePolicyFramework = Assign.usePolicyFramework(solrCloudManager);
assertTrue(usePolicyFramework);
// now we set useLegacyReplicaAssignment=true, so autoscaling can only be used if an explicit policy or preference exists
when(clusterStateProvider.getClusterProperties()).thenReturn(Utils.makeMap("defaults", Utils.makeMap("cluster", Utils.makeMap("useLegacyReplicaAssignment", true))));
when(distribStateManager.getAutoScalingConfig()).thenReturn(new AutoScalingConfig(Collections.emptyMap()));
assertFalse(Assign.usePolicyFramework(solrCloudManager));
// lets provide a custom preference and assert that autoscaling is used even if useLegacyReplicaAssignment=false
// our custom preferences are exactly the same as the default ones
// but because we are providing them explicitly, they must cause autoscaling to turn on
@SuppressWarnings({"rawtypes"})
List<Map> customPreferences = Policy.DEFAULT_PREFERENCES
.stream().map(preference -> preference.getOriginal()).collect(Collectors.toList());
when(distribStateManager.getAutoScalingConfig()).thenReturn(new AutoScalingConfig(Utils.makeMap("cluster-preferences", customPreferences)));
assertTrue(Assign.usePolicyFramework(solrCloudManager));
}
}

View File

@ -21,10 +21,8 @@ import java.util.Map;
import java.util.stream.Collectors;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.CloudTestUtils;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
@ -32,6 +30,7 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.zookeeper.KeeperException;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
@Slow
@ -50,6 +49,7 @@ public class CollectionTooManyReplicasTest extends SolrCloudTestCase {
}
@Test
@Ignore // Since maxShardsPerNode was removed in SOLR-12847 and autoscaling framework was removed in SOLR-14656, this test is broken
public void testAddTooManyReplicas() throws Exception {
final String collectionName = "TooManyReplicasInSeveralFlavors";
@ -70,8 +70,8 @@ public class CollectionTooManyReplicasTest extends SolrCloudTestCase {
.process(cluster.getSolrClient());
// equivalent to maxShardsPerNode=1
String commands = "{ set-cluster-policy: [ {replica: '<2', shard: '#ANY', node: '#ANY', strict: true} ] }";
cluster.getSolrClient().request(CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
// String commands = "{ set-cluster-policy: [ {replica: '<2', shard: '#ANY', node: '#ANY', strict: true} ] }";
// cluster.getSolrClient().request(CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
// this should fail because the policy prevents it
Exception e = expectThrows(Exception.class, () -> {
@ -116,10 +116,11 @@ public class CollectionTooManyReplicasTest extends SolrCloudTestCase {
}
@Test
@Ignore // Since maxShardsPerNode was removed in SOLR-12847 and autoscaling framework was removed in SOLR-14656, this test is broken
public void testAddShard() throws Exception {
// equivalent to maxShardsPerNode=2
String commands = "{ set-cluster-policy: [ {replica: '<3', shard: '#ANY', node: '#ANY', strict: true} ] }";
cluster.getSolrClient().request(CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
// String commands = "{ set-cluster-policy: [ {replica: '<3', shard: '#ANY', node: '#ANY', strict: true} ] }";
// cluster.getSolrClient().request(CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
String collectionName = "TooManyReplicasWhenAddingShards";
CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "shardstart", 2)

View File

@ -1,292 +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.api.collections;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
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.cloud.CloudTestUtils;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ConcurrentCreateCollectionTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
static int NODES = 2;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(NODES)
.addConfig("conf", configset("cloud-minimal"))
//.addConfig("conf", configset("_default"))
.configure();
}
@Before
@Override
public void setUp() throws Exception {
super.setUp();
}
@After
@Override
public void tearDown() throws Exception {
super.tearDown();
cluster.deleteAllCollections();
}
private CollectionAdminRequest.Create createCollectionRequest(String cname, int numShards, int numReplicas) throws Exception {
CollectionAdminRequest.Create creq = CollectionAdminRequest
// .createCollection(cname, "conf", NODES - 1, NODES - 1)
.createCollection(cname, "conf", numShards, numReplicas);
creq.setWaitForFinalState(true);
creq.setAutoAddReplicas(true);
return creq;
}
public void testConcurrentCreatePlacement() throws Exception {
final int nThreads = 2;
final int createsPerThread = 1;
final int nShards = 1;
final int repFactor = 2;
final boolean useClusterPolicy = false;
final boolean useCollectionPolicy = true;
final boolean startUnbalanced = true; // can help make a smaller test that can still reproduce an issue.
final int unbalancedSize = 1; // the number of replicas to create first
final boolean stopNode = false; // only applicable when startUnbalanced==true... stops a node during first collection creation, then restarts
final CloudSolrClient client = cluster.getSolrClient();
if (startUnbalanced) {
/*** This produces a failure (multiple replicas of single shard on same node) when run with NODES=4 and
final int nThreads = 2;
final int createsPerThread = 1;
final int nShards = 2;
final int repFactor = 2;
final boolean useClusterPolicy = false;
final boolean useCollectionPolicy = true;
final boolean startUnbalanced = true;
// NOTE: useClusterPolicy=true seems to fix it! So does putting both creates in a single thread!
// NOTE: even creating a single replica to start with causes failure later on.
Also reproduced with smaller cluster: NODES=2 and
final int nThreads = 2;
final int createsPerThread = 1;
final int nShards = 1;
final int repFactor = 2;
final boolean useClusterPolicy = false;
final boolean useCollectionPolicy = true;
final boolean startUnbalanced = true;
Also, with NODES=3:
final int nThreads = 2;
final int createsPerThread = 1;
final int nShards = 1;
final int repFactor = 2;
final boolean useClusterPolicy = false;
final boolean useCollectionPolicy = true;
final boolean startUnbalanced = false;
// Also succeeded in replicating a bug where all 5 replicas were on a single node: CORES=5, nThreads=5, repFactor=5,
// unbalancedSize = 16 (4 replicas on each of the up nodes), stopNode=true
***/
JettySolrRunner downJetty = cluster.getJettySolrRunners().get(0);
if (stopNode) {
cluster.stopJettySolrRunner(downJetty);
}
String cname = "STARTCOLLECTION";
CollectionAdminRequest.Create creq = CollectionAdminRequest
// .createCollection(cname, "conf", NODES - 1, NODES - 1)
.createCollection(cname, "conf", unbalancedSize, 1);
creq.setWaitForFinalState(true);
// creq.setAutoAddReplicas(true);
if (useCollectionPolicy) { creq.setPolicy("policy1"); }
creq.process(client);
if (stopNode) {
// this will start it with a new port.... does it matter?
cluster.startJettySolrRunner(downJetty);
}
}
if (useClusterPolicy) {
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
// " {'cores':'<100', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
// " {'replica':'<2', 'node': '#ANY'}," +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
client.request(req);
}
if (useCollectionPolicy) {
// NOTE: the mere act of setting this named policy prevents LegacyAssignStrategy from being used, even if the policy is
// not used during collection creation.
String commands = "{set-policy : {" +
" policy1 : [{replica:'<2' , node:'#ANY'}]" +
",policy2 : [{replica:'<2' , shard:'#EACH', node:'#ANY'}]" +
"}}";
client.request(CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, commands));
/*** take defaults for cluster preferences
String cmd = "{" +
" 'set-cluster-preferences': [" +
// " {'cores':'<100', 'node':'#ANY'}," +
" {minimize:cores}" +
" ]" +
"}";
SolrRequest req = CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.POST, cmd);
client.request(req);
***/
}
/***
SolrRequest req = CloudTestUtils.AutoScalingRequest.create(SolrRequest.METHOD.GET, null);
SolrResponse response = req.process(client);
log.info("######### AUTOSCALE {}", response);
***/
byte[] data = client.getZkStateReader().getZkClient().getData("/autoscaling.json", null, null, true);
if (log.isInfoEnabled()) {
log.info("AUTOSCALE DATA: {}", new String(data, "UTF-8"));
}
final AtomicInteger collectionNum = new AtomicInteger();
Thread[] indexThreads = new Thread[nThreads];
for (int i=0; i<nThreads; i++) {
indexThreads[i] = new Thread(() -> {
try {
for (int j=0; j<createsPerThread; j++) {
int num = collectionNum.incrementAndGet();
// Thread.sleep(num*1000);
String collectionName = "collection" + num;
CollectionAdminRequest.Create createReq = CollectionAdminRequest
.createCollection(collectionName, "conf", nShards, repFactor)
// .setMaxShardsPerNode(1) // should be default
;
createReq.setWaitForFinalState(false);
if (useCollectionPolicy) {
createReq.setPolicy("policy1");
}
createReq.setAutoAddReplicas(true);
createReq.process(client);
// cluster.waitForActiveCollection(collectionName, 1, repFactor);
// Thread.sleep(10000);
}
} catch (Exception e) {
fail(e.getMessage());
}
});
}
for (Thread thread : indexThreads) {
thread.start();
}
for (Thread thread : indexThreads) {
thread.join();
}
int expectedTotalReplicas = unbalancedSize + nThreads * createsPerThread * nShards * repFactor;
int expectedPerNode = expectedTotalReplicas / NODES;
boolean expectBalanced = (expectedPerNode * NODES == expectedTotalReplicas);
Map<String,List<Replica>> replicaMap = new HashMap<>();
ClusterState cstate = client.getZkStateReader().getClusterState();
for (DocCollection collection : cstate.getCollectionsMap().values()) {
for (Replica replica : collection.getReplicas()) {
String url = replica.getBaseUrl();
List<Replica> replicas = replicaMap.get(url);
if (replicas == null) {
replicas = new ArrayList<>();
replicaMap.put(url, replicas);
}
replicas.add(replica);
}
}
// check if nodes are balanced
boolean failed = false;
for (List<Replica> replicas : replicaMap.values()) {
if (replicas.size() != expectedPerNode ) {
if (expectBalanced) {
failed = true;
}
log.error("UNBALANCED CLUSTER: expected replicas per node {} but got {}", expectedPerNode, replicas.size());
}
}
// check if there were multiple replicas of the same shard placed on the same node
for (DocCollection collection : cstate.getCollectionsMap().values()) {
for (Slice slice : collection.getSlices()) {
Map<String, Replica> nodeToReplica = new HashMap<>();
for (Replica replica : slice.getReplicas()) {
Replica prev = nodeToReplica.put(replica.getBaseUrl(), replica);
if (prev != null) {
failed = true;
// NOTE: with a replication factor > 2, this will print multiple times per bad slice.
log.error("MULTIPLE REPLICAS OF SINGLE SHARD ON SAME NODE: r1={} r2={}", prev, replica);
}
}
}
}
if (failed) {
log.error("Cluster state {}", cstate.getCollectionsMap());
}
assertEquals(replicaMap.size(), NODES); // make sure something was created
assertTrue(!failed);
}
}

View File

@ -1,472 +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.autoscaling;
import static org.apache.solr.common.util.Utils.makeMap;
import java.lang.invoke.MethodHandles;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
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.QueryRequest;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.cloud.MiniSolrCloudCluster;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.CollectionStatePredicate;
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.common.util.TimeSource;
import org.apache.solr.util.TimeOut;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@org.apache.solr.util.LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.autoscaling.NodeLostTrigger=TRACE;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG")
public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected String getConfigSet() {
return "cloud-minimal";
}
@Before
public void setupCluster() throws Exception {
configureCluster(3)
.addConfig("conf", configset(getConfigSet()))
.withSolrXml(TEST_PATH().resolve("solr.xml"))
.configure();
new V2Request.Builder("/cluster")
.withMethod(SolrRequest.METHOD.POST)
.withPayload("{set-obj-property:{defaults : {cluster: {useLegacyReplicaAssignment:true}}}}")
.build()
.process(cluster.getSolrClient());
new V2Request.Builder("/cluster/autoscaling")
.withMethod(SolrRequest.METHOD.POST)
.withPayload("{'set-trigger':{'name':'.auto_add_replicas','event':'nodeLost','waitFor':'5s','enabled':'true','actions':[{'name':'auto_add_replicas_plan','class':'solr.AutoAddReplicasPlanAction'},{'name':'execute_plan','class':'solr.ExecutePlanAction'}]}}")
.build()
.process(cluster.getSolrClient());
}
@After
public void tearDown() throws Exception {
try {
shutdownCluster();
} finally {
super.tearDown();
}
}
/**
* Test that basic autoAddReplicaLogic kicks in when a node is lost
*/
@Test
public void testSimple() throws Exception {
final String COLLECTION = "test_simple";
final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(true)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 4);
// start the tests
JettySolrRunner lostJetty = random().nextBoolean() ? jetty1 : jetty2;
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
waitForNodeLeave(lostNodeName);
waitForState(COLLECTION + "=(2,4) w/o down replicas",
COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION);
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
assertTrue("Timeout waiting for all live and active",
ClusterStateUtil.waitForAllActiveAndLiveReplicas(zkStateReader, 90000));
}
/**
* Test that basic autoAddReplicaLogic logic is <b>not</b> used if the cluster prop for it is disabled
* (even if sys prop is set after collection is created)
*/
@Test
public void testClusterPropOverridesCollecitonProp() throws Exception {
final String COLLECTION = "test_clusterprop";
final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(true)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 4);
// check cluster property is considered
disableAutoAddReplicasInCluster();
JettySolrRunner lostJetty = random().nextBoolean() ? jetty1 : jetty2;
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
waitForNodeLeave(lostNodeName);
waitForState(COLLECTION + "=(2,2)", COLLECTION,
clusterShape(2, 2), 90, TimeUnit.SECONDS);
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
assertTrue("Timeout waiting for all live and active",
ClusterStateUtil.waitForAllActiveAndLiveReplicas(zkStateReader, 90000));
waitForState(COLLECTION + "=(2,4) w/o down replicas",
COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
}
/**
* Test that we can modify a collection after creation to add autoAddReplicas.
*/
@Test
public void testAddCollectionPropAfterCreation() throws Exception {
final String COLLECTION = "test_addprop";
final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(false) // NOTE: false
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 4);
log.info("Modifying {} to use autoAddReplicas", COLLECTION);
new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.MODIFYCOLLECTION) {
@Override
public SolrParams getParams() {
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
params.set("collection", COLLECTION);
params.set("autoAddReplicas", true);
return params;
}
}.process(cluster.getSolrClient());
JettySolrRunner lostJetty = random().nextBoolean() ? jetty1 : jetty2;
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
waitForNodeLeave(lostNodeName);
waitForState(COLLECTION + "=(2,4) w/o down replicas",
COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION);
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
assertTrue("Timeout waiting for all live and active",
ClusterStateUtil.waitForAllActiveAndLiveReplicas(zkStateReader, 90000));
}
/**
* Test a specific sequence of problematic events:
* <ul>
* <li>create a collection with autoAddReplicas=<b>false</b></li>
* <li>stop a nodeX in use by the collection</li>
* <li>re-start nodeX</li>
* <li>set autoAddReplicas=<b>true</b></li>
* <li>re-stop nodeX</li>
* </ul>
*/
@Test
@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13811")
public void testRapidStopStartStopWithPropChange() throws Exception {
// This is the collection we'll be focused on in our testing...
final String COLLECTION = "test_stoptwice";
// This is a collection we'll use as a "marker" to ensure we "wait" for the
// autoAddReplicas logic (via NodeLostTrigger) to kick in at least once before proceeding...
final String ALT_COLLECTION = "test_dummy";
final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
final JettySolrRunner jetty1 = cluster.getJettySolrRunner(1);
final JettySolrRunner jetty2 = cluster.getJettySolrRunner(2);
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(false) // NOTE: false
.process(cluster.getSolrClient());
if (log.isInfoEnabled()) {
log.info("Creating {} using jetty1:{}/{} and jetty2:{}/{}", ALT_COLLECTION,
jetty1.getNodeName(), jetty1.getLocalPort(),
jetty2.getNodeName(), jetty2.getLocalPort());
}
CollectionAdminRequest.createCollection(ALT_COLLECTION, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(true) // NOTE: true
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(COLLECTION, 2, 4);
cluster.waitForActiveCollection(ALT_COLLECTION, 2, 4);
JettySolrRunner lostJetty = random().nextBoolean() ? jetty1 : jetty2;
String lostNodeName = lostJetty.getNodeName();
List<Replica> replacedHdfsReplicas = getReplacedSharedFsReplicas(COLLECTION, zkStateReader, lostNodeName);
if (log.isInfoEnabled()) {
log.info("Stopping random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
waitForNodeLeave(lostNodeName);
// ensure that our marker collection indicates that the autoAddReplicas logic
// has detected the down node and done some processing
waitForState(ALT_COLLECTION + "=(2,4) w/o down replicas",
ALT_COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
waitForState(COLLECTION + "=(2,2)", COLLECTION, clusterShape(2, 2));
if (log.isInfoEnabled()) {
log.info("Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
// save time, don't bother waiting for lostJetty to start until after updating collection prop...
log.info("Modifying {} to use autoAddReplicas", COLLECTION);
new CollectionAdminRequest.AsyncCollectionAdminRequest(CollectionParams.CollectionAction.MODIFYCOLLECTION) {
@Override
public SolrParams getParams() {
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
params.set("collection", COLLECTION);
params.set("autoAddReplicas", true);
return params;
}
}.process(cluster.getSolrClient());
// make sure lostJetty is fully up before stopping again...
waitForNodeLive(lostJetty);
if (log.isInfoEnabled()) {
log.info("Re-Stopping (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
waitForNodeLeave(lostNodeName);
// TODO: this is the problematic situation...
// wether or not NodeLostTrigger noticed that lostJetty was re-started and shutdown *again*
// and that the new auoAddReplicas=true since the last time lostJetty was shutdown is respected
waitForState(COLLECTION + "=(2,4) w/o down replicas",
COLLECTION, clusterShapeNoDownReplicas(2,4), 90, TimeUnit.SECONDS);
checkSharedFsReplicasMovedCorrectly(replacedHdfsReplicas, zkStateReader, COLLECTION);
if (log.isInfoEnabled()) {
log.info("Re-Re-starting (same) random node: {} / {}", lostNodeName, lostJetty.getLocalPort());
}
lostJetty.start();
waitForNodeLive(lostJetty);
assertTrue("Timeout waiting for all live and active",
ClusterStateUtil.waitForAllActiveAndLiveReplicas(zkStateReader, 90000));
}
private void disableAutoAddReplicasInCluster() throws SolrServerException, IOException {
@SuppressWarnings({"rawtypes"})
Map m = makeMap(
"action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
"name", ZkStateReader.AUTO_ADD_REPLICAS,
"val", "false");
@SuppressWarnings({"unchecked"})
QueryRequest request = new QueryRequest(new MapSolrParams(m));
request.setPath("/admin/collections");
cluster.getSolrClient().request(request);
}
private void enableAutoAddReplicasInCluster() throws SolrServerException, IOException {
@SuppressWarnings({"rawtypes"})
Map m = makeMap(
"action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
"name", ZkStateReader.AUTO_ADD_REPLICAS);
@SuppressWarnings({"unchecked"})
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;
}
/**
* {@link MiniSolrCloudCluster#waitForNode} Doesn't check isRunning first, and we don't want to
* use {@link MiniSolrCloudCluster#waitForAllNodes} because we don't want to waste cycles checking
* nodes we aren't messing with
*/
private void waitForNodeLive(final JettySolrRunner jetty)
throws InterruptedException, TimeoutException, IOException {
if (log.isInfoEnabled()) {
log.info("waitForNodeLive: {}/{}", jetty.getNodeName(), jetty.getLocalPort());
}
TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while(!timeout.hasTimedOut()) {
if (jetty.isRunning()) {
break;
}
try {
Thread.sleep(100);
} catch (InterruptedException e) {
// ignore
}
}
if (timeout.hasTimedOut()) {
throw new TimeoutException("Waiting for Jetty to stop timed out");
}
cluster.waitForNode(jetty, 30);
}
private void waitForNodeLeave(String lostNodeName) throws InterruptedException, TimeoutException {
log.info("waitForNodeLeave: {}", lostNodeName);
ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
reader.waitForLiveNodes(30, TimeUnit.SECONDS, (o, n) -> !n.contains(lostNodeName));
}
private static CollectionStatePredicate clusterShapeNoDownReplicas(final int expectedShards,
final int expectedReplicas) {
return (liveNodes, collectionState)
-> (clusterShape(expectedShards, expectedReplicas).matches(liveNodes, collectionState)
&& collectionState.getReplicas().size() == expectedReplicas);
}
}

View File

@ -1,263 +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.autoscaling;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
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.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.V2Request;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.SolrException;
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.common.util.SuppressForbidden;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
@BeforeClass
public static void setupCluster() throws Exception {
System.setProperty("solr.httpclient.retries", "4");
System.setProperty("solr.retries.on.forward", "1");
System.setProperty("solr.retries.to.followers", "1");
}
@Before
public void beforeTest() throws Exception {
configureCluster(3)
.addConfig("conf", configset("cloud-minimal"))
.configure();
new V2Request.Builder("/cluster")
.withMethod(SolrRequest.METHOD.POST)
.withPayload("{set-obj-property:{defaults : {cluster: {useLegacyReplicaAssignment:true}}}}")
.build()
.process(cluster.getSolrClient());
}
@After
public void afterTest() throws Exception {
shutdownCluster();
}
@Test
//Commented out 11-Dec-2018 @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13028")
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";
String collection3 = "testSimple3";
CollectionAdminRequest.createCollection(collection1, "conf", 2, 2)
.setCreateNodeSet(jetty1.getNodeName()+","+jetty2.getNodeName())
.setAutoAddReplicas(true)
.process(cluster.getSolrClient());
CollectionAdminRequest.createCollection(collection2, "conf", 1, 2)
.setCreateNodeSet(jetty2.getNodeName()+","+jetty3.getNodeName())
.setAutoAddReplicas(false)
.process(cluster.getSolrClient());
// the number of cores in jetty1 (6) will be larger than jetty3 (1)
CollectionAdminRequest.createCollection(collection3, "conf", 3, 1)
.setCreateNodeSet(jetty1.getNodeName())
.setAutoAddReplicas(false)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(collection1, 2, 4);
cluster.waitForActiveCollection(collection2, 1, 2);
cluster.waitForActiveCollection(collection3, 3, 3);
// we remove the implicit created trigger, so the replicas won't be moved
String removeTriggerCommand = "{" +
"'remove-trigger' : {" +
"'name' : '.auto_add_replicas'," +
"'removeListeners': true" +
"}" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, removeTriggerCommand);
@SuppressWarnings({"rawtypes"})
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());
ZkStateReader reader = cluster.getSolrClient().getZkStateReader();
lostJetty.stop();
cluster.waitForJettyToStop(lostJetty);
reader.waitForLiveNodes(30, TimeUnit.SECONDS, missingLiveNode(lostNodeName));
@SuppressWarnings({"rawtypes"})
List<SolrRequest> operations = getOperations(jetty3, lostNodeName);
assertOperations(collection1, operations, lostNodeName, cloudDescriptors, null);
lostJetty.start();
cluster.waitForAllNodes(30);
cluster.waitForActiveCollection(collection1, 2, 4);
cluster.waitForActiveCollection(collection2, 1, 2);
cluster.waitForActiveCollection(collection3, 3, 3);
assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), 30000));
String setClusterPreferencesCommand = "{" +
"'set-cluster-preferences': [" +
"{'minimize': 'cores','precision': 0}]" +
"}";
req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPreferencesCommand);
// you can hit a stale connection from pool when restarting jetty
try (CloudSolrClient cloudClient = new CloudSolrClient.Builder(Collections.singletonList(cluster.getZkServer().getZkAddress()),
Optional.empty())
.withSocketTimeout(45000).withConnectionTimeout(15000).build()) {
response = cloudClient.request(req);
}
assertEquals(response.get("result").toString(), "success");
lostJetty = random().nextBoolean()? jetty1 : jetty2;
String lostNodeName2 = lostJetty.getNodeName();
cloudDescriptors = lostJetty.getCoreContainer().getCores().stream()
.map(solrCore -> solrCore.getCoreDescriptor().getCloudDescriptor())
.collect(Collectors.toList());
lostJetty.stop();
reader.waitForLiveNodes(30, TimeUnit.SECONDS, missingLiveNode(lostNodeName2));
try {
operations = getOperations(jetty3, lostNodeName2);
} catch (SolrException e) {
// we might get a stale connection from the pool after jetty restarts
operations = getOperations(jetty3, lostNodeName2);
}
assertOperations(collection1, operations, lostNodeName2, cloudDescriptors, jetty3);
lostJetty.start();
cluster.waitForAllNodes(30);
cluster.waitForActiveCollection(collection1, 2, 4);
cluster.waitForActiveCollection(collection2, 1, 2);
cluster.waitForActiveCollection(collection3, 3, 3);
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;
String lostNodeName3 = lostJetty.getNodeName();
lostJetty.stop();
reader.waitForLiveNodes(30, TimeUnit.SECONDS, missingLiveNode(lostNodeName3));
operations = getOperations(jetty3, lostNodeName3);
assertNull(operations);
}
@SuppressForbidden(reason = "Needs currentTimeMillis to create unique id")
@SuppressWarnings({"rawtypes"})
private List<SolrRequest> getOperations(JettySolrRunner actionJetty, String lostNodeName) throws Exception {
try (AutoAddReplicasPlanAction action = new AutoAddReplicasPlanAction()) {
action.configure(actionJetty.getCoreContainer().getResourceLoader(), actionJetty.getCoreContainer().getZkController().getSolrCloudManager(), new HashMap<>());
TriggerEvent lostNode = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST, ".auto_add_replicas", Collections.singletonList(System.currentTimeMillis()), Collections.singletonList(lostNodeName), CollectionParams.CollectionAction.MOVEREPLICA.toLower());
ActionContext context = new ActionContext(actionJetty.getCoreContainer().getZkController().getSolrCloudManager(), null, new HashMap<>());
action.process(lostNode, context);
@SuppressWarnings({"unchecked", "rawtypes"})
List<SolrRequest> operations = (List) context.getProperty("operations");
return operations;
}
}
private void assertOperations(String collection,
@SuppressWarnings({"rawtypes"})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 (@SuppressWarnings({"rawtypes"})SolrRequest solrRequest : operations) {
assertTrue(solrRequest instanceof CollectionAdminRequest.MoveReplica);
SolrParams params = solrRequest.getParams();
assertEquals(params.get("collection"), collection);
String replica = params.get("replica");
boolean found = false;
Iterator<CloudDescriptor> it = cloudDescriptors.iterator();
while (it.hasNext()) {
CloudDescriptor cd = it.next();
if (cd.getCollectionName().equals(collection) && cd.getCoreNodeName().equals(replica)) {
found = true;
it.remove();
break;
}
}
assertTrue("Can not find "+replica+ " in node " + lostNodeName, found);
String targetNode = params.get("targetNode");
assertFalse("Target node match the lost one " + lostNodeName, lostNodeName.equals(targetNode));
if (destJetty != null) {
assertEquals("Target node is not as expectation", destJetty.getNodeName(), targetNode);
}
}
for (CloudDescriptor cd : cloudDescriptors) {
if (cd.getCollectionName().equals(collection)) {
fail("Exist replica which is not moved " + cd);
}
}
}
}

View File

@ -1,65 +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.autoscaling;
import java.util.HashMap;
import java.util.Map;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
/**
*
*/
public class CapturedEvent {
public final AutoScalingConfig.TriggerListenerConfig config;
public final TriggerEventProcessorStage stage;
public final String actionName;
public final TriggerEvent event;
public final String message;
public final Map<String, Object> context = new HashMap<>();
public final long timestamp;
public CapturedEvent(long timestamp, ActionContext context, AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName,
TriggerEvent event, String message) {
if (context != null) {
context._forEachEntry((o, o2) -> CapturedEvent.this.context.put((String) o, o2));
TriggerEvent.fixOps("properties." + TriggerEvent.REQUESTED_OPS, this.context);
TriggerEvent.fixOps("properties." + TriggerEvent.UNSUPPORTED_OPS, this.context);
}
this.config = config;
this.stage = stage;
this.actionName = actionName;
this.event = event;
this.message = message;
this.timestamp = timestamp;
}
@Override
public String toString() {
return "CapturedEvent{" +
"timestamp=" + timestamp +
", stage=" + stage +
", actionName='" + actionName + '\'' +
", event=" + event +
", context=" + context +
", config=" + config +
", message='" + message + '\'' +
'}';
}
}

View File

@ -1,794 +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.autoscaling;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientNodeStateProvider;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
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.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.params.AutoScalingParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.LogLevel;
import org.junit.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
/**
* Test for {@link ComputePlanAction}
*/
@LogLevel("org.apache.solr.cloud.autoscaling=DEBUG;org.apache.solr.cloud.Overseer=DEBUG;org.apache.solr.cloud.overseer=DEBUG;org.apache.solr.client.solrj.impl.SolrClientDataProvider=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);
@SuppressWarnings({"rawtypes"})
private static final AtomicReference<Map> actionContextPropsRef = new AtomicReference<>();
private static final AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
private static SolrCloudManager cloudManager;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(NODE_COUNT)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Before
public void setUp() throws Exception {
super.setUp();
// remove everything from autoscaling.json in ZK
zkClient().setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, "{}".getBytes(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) {
JettySolrRunner j = cluster.stopJettySolrRunner(i1);
cluster.waitForJettyToStop(j);
break;
}
}
}
}
cluster.deleteAllCollections();
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(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 = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPreferencesCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH);
deleteChildrenRecursively(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH);
reset();
}
private void reset() {
fired.set(false);
triggerFiredLatch = new CountDownLatch(1);
actionContextPropsRef.set(null);
eventRef.set(null);
AssertingTriggerAction.expectedNode = null;
}
private void deleteChildrenRecursively(String path) throws Exception {
cloudManager.getDistribStateManager().removeRecursively(path, true, false);
}
@After
public void printState() throws Exception {
log.debug("-------------_ FINAL STATE --------------");
SolrCloudManager cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
for (String node: cloudManager.getClusterStateProvider().getLiveNodes()) {
Map<String, Object> values = cloudManager.getNodeStateProvider().getNodeValues(node, ImplicitSnitch.tags);
if (log.isDebugEnabled()) {
log.debug("* Node values: {}\n{}", node, Utils.toJSONString(values));
}
}
if (log.isDebugEnabled()) {
log.debug("* Live nodes: {}", cloudManager.getClusterStateProvider().getLiveNodes());
}
ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
if (log.isDebugEnabled()) {
state.forEachCollection(coll -> log.debug("* Collection {} state: {}", coll.getName(), coll));
}
}
@AfterClass
public static void cleanUpAfterClass() throws Exception {
cloudManager = null;
}
@Test
@LuceneTestCase.AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
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();
AssertingTriggerAction.expectedNode = node;
CloudSolrClient solrClient = cluster.getSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '7s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + ComputePlanActionTest.AssertingTriggerAction.class.getName() + "'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testNodeLost",
"conf",1, 2);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
"testNodeLost", clusterShape(1, 2));
ClusterState clusterState = cluster.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);
assertTrue(node2.getNodeName() + "is not live yet", cluster.getSolrClient().getZkStateReader().getClusterState().liveNodesContain(node2.getNodeName()) );
// 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;
}
}
log.info("Stopped_node : {}", node);
cluster.waitForAllNodes(30);
assertTrue("Trigger was not fired even after 10 seconds", triggerFiredLatch.await(10, TimeUnit.SECONDS));
assertTrue(fired.get());
@SuppressWarnings({"rawtypes"})
Map context = actionContextPropsRef.get();
assertNotNull(context);
@SuppressWarnings({"unchecked", "rawtypes"})
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null , "+ getNodeStateProviderState() + eventRef.get(), operations);
assertEquals("ComputePlanAction should have computed exactly 1 operation", 1, operations.size());
@SuppressWarnings({"rawtypes"})
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) {
JettySolrRunner j = cluster.stopJettySolrRunner(i);
cluster.waitForJettyToStop(j);
break;
}
}
}
static String getNodeStateProviderState() {
String result = "SolrClientNodeStateProvider.DEBUG";
if(SolrClientNodeStateProvider.INST != null) {
result+= Utils.toJSONString(SolrClientNodeStateProvider.INST);
}
return result;
}
// commented out on: 24-Dec-2018 @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2-Aug-2018
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() + "'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(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.process(solrClient);
cluster.waitForActiveCollection("testNodeWithMultipleReplicasLost", 2, 6);
waitForState("Timed out waiting for replicas of new collection to be active",
"testNodeWithMultipleReplicasLost", clusterShape(2, 6));
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;
JettySolrRunner j = cluster.stopJettySolrRunner(i);
cluster.waitForJettyToStop(j);
break;
}
}
assertNotNull(stoppedNodeName);
assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(15, TimeUnit.SECONDS));
assertTrue(fired.get());
TriggerEvent triggerEvent = eventRef.get();
assertNotNull(triggerEvent);
assertEquals(TriggerEventType.NODELOST, triggerEvent.getEventType());
// TODO assertEquals(stoppedNodeName, triggerEvent.getProperty(TriggerEvent.NODE_NAME));
@SuppressWarnings({"rawtypes"})
Map context = actionContextPropsRef.get();
assertNotNull(context);
@SuppressWarnings({"unchecked", "rawtypes"})
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null "+ getNodeStateProviderState() + actionContextPropsRef.get(), operations);
if (log.isInfoEnabled()) {
operations.forEach(solrRequest -> log.info(solrRequest.getParams().toString()));
}
assertEquals("ComputePlanAction should have computed exactly 2 operation", 2, operations.size());
for (@SuppressWarnings({"rawtypes"})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
// commented out on: 17-Feb-2019 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
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() + "'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(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 = AutoScalingRequest.create(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 = AutoScalingRequest.create(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());
@SuppressWarnings({"rawtypes"})
Map context = actionContextPropsRef.get();
assertNotNull(context);
@SuppressWarnings({"unchecked", "rawtypes"})
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null" + getNodeStateProviderState() + context, operations);
assertEquals("ComputePlanAction should have computed exactly 1 operation", 1, operations.size());
@SuppressWarnings({"rawtypes"})
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 {
static volatile String expectedNode;
@Override
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
}
@Override
public void init() {
}
@Override
public String getName() {
return null;
}
@Override
public void process(TriggerEvent event, ActionContext context) {
if (expectedNode != null) {
@SuppressWarnings({"rawtypes"})
Collection nodes = (Collection) event.getProperty(TriggerEvent.NODE_NAMES);
if (nodes == null || !nodes.contains(expectedNode)) return;//this is not the event we are looking for
}
if (fired.compareAndSet(false, true)) {
eventRef.set(event);
actionContextPropsRef.set(context.getProperties());
triggerFiredLatch.countDown();
}
}
@Override
public void close() throws IOException {
}
}
@Test
//2018-06-18 (commented) @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
public void testSelectedCollectionsByName() throws Exception {
String collectionsFilter = "'testSelected1,testSelected2'";
testCollectionsPredicate(collectionsFilter, Collections.emptyMap());
}
@Test
//2018-06-18 (commented) @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 09-Apr-2018
public void testSelectedCollectionsByPolicy() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String setSearchPolicyCommand = "{" +
" 'set-policy': {" +
" 'search': [" +
" {'replica':'<5', 'shard': '#EACH', 'node': '#ANY'}," +
" ]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setSearchPolicyCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String collectionsFilter = "{'policy': 'search'}";
Map<String, String> createCollectionParameters = new HashMap<>();
createCollectionParameters.put("testSelected1", "search");
createCollectionParameters.put("testSelected2", "search");
testCollectionsPredicate(collectionsFilter, createCollectionParameters);
}
private void testCollectionsPredicate(String collectionsFilter, Map<String, String> createCollectionParameters) throws Exception {
if (log.isInfoEnabled()) {
log.info("Found number of jetties: {}", cluster.getJettySolrRunners().size());
}
// 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', 'collections' : " + collectionsFilter + "}," +
"{'name':'test','class':'" + ComputePlanActionTest.AssertingTriggerAction.class.getName() + "'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection("testSelected1",
"conf", 2, 2);
if (createCollectionParameters.get("testSelected1") != null) {
create.setPolicy(createCollectionParameters.get("testSelected1"));
}
create.process(solrClient);
create = CollectionAdminRequest.createCollection("testSelected2",
"conf", 2, 2);
if (createCollectionParameters.get("testSelected2") != null) {
create.setPolicy(createCollectionParameters.get("testSelected2"));
}
create.process(solrClient);
create = CollectionAdminRequest.createCollection("testSelected3",
"conf", 2, 2);
if (createCollectionParameters.get("testSelected3") != null) {
create.setPolicy(createCollectionParameters.get("testSelected3"));
}
create.process(solrClient);
cluster.waitForActiveCollection("testSelected1", 2, 4);
cluster.waitForActiveCollection("testSelected2", 2, 4);
cluster.waitForActiveCollection("testSelected3", 2, 4);
waitForState("Timed out waiting for replicas of new collection to be active",
"testSelected1", clusterShape(2, 4));
waitForState("Timed out waiting for replicas of new collection to be active",
"testSelected2", clusterShape(2, 4));
waitForState("Timed out waiting for replicas of new collection to be active",
"testSelected3", clusterShape(2, 4));
// find a node that has replicas from all collections
SolrCloudManager cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
NodeStateProvider stateProvider = cloudManager.getNodeStateProvider();
List<String> nodes = new ArrayList<>();
cloudManager.getClusterStateProvider().getLiveNodes().forEach(n -> {
Map<String, Map<String, List<Replica>>> map = stateProvider.getReplicaInfo(n, ImplicitSnitch.tags);
if (map.containsKey("testSelected3") && map.containsKey("testSelected2") && map.containsKey("testSelected1")) {
nodes.add(n);
}
});
assertTrue(nodes.size() > 0);
// kill first such node
String node = nodes.get(0);
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
if (cluster.getJettySolrRunner(i).getNodeName().equals(node)) {
JettySolrRunner j = cluster.stopJettySolrRunner(i);
cluster.waitForJettyToStop(j);
break;
}
}
assertTrue("Trigger was not fired even after 5 seconds", triggerFiredLatch.await(5, TimeUnit.SECONDS));
assertTrue(fired.get());
@SuppressWarnings({"rawtypes"})
Map context = actionContextPropsRef.get();
assertNotNull(context);
@SuppressWarnings({"unchecked", "rawtypes"})
List<SolrRequest> operations = (List<SolrRequest>) context.get("operations");
assertNotNull("The operations computed by ComputePlanAction should not be null. " + getNodeStateProviderState() + context, operations);
assertEquals("ComputePlanAction should have computed exactly 2 operations", 2, operations.size());
@SuppressWarnings({"rawtypes"})
SolrRequest request = operations.get(0);
SolrParams params = request.getParams();
assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
assertFalse("not expected testSelected3", "testSelected3".equals(params.get("collection")));
request = operations.get(1);
params = request.getParams();
assertEquals("Expected MOVEREPLICA action after adding node", MOVEREPLICA, CollectionParams.CollectionAction.get(params.get("action")));
assertFalse("not expected testSelected3", "testSelected3".equals(params.get("collection")));
}
@Test
public void testNodeAddedTriggerWithAddReplicaPreferredOp_1Shard() throws Exception {
String collectionNamePrefix = "testNodeAddedTriggerWithAddReplicaPreferredOp_1Shard";
int numShards = 1;
int numCollections = 5;
nodeAddedTriggerWithAddReplicaPreferredOp(collectionNamePrefix, numShards, numCollections);
}
@Test
public void testNodeAddedTriggerWithAddReplicaPreferredOpReplicaType_1Shard() throws Exception {
String collectionNamePrefix = "testNodeAddedTriggerWithAddReplicaPreferredOpReplicaType_1Shard";
int numShards = 1;
int numCollections = 5;
nodeAddedTriggerWithAddReplicaPreferredOpReplicaType(collectionNamePrefix, numShards, numCollections);
}
@Test
// commented out on: 24-Dec-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
public void testNodeAddedTriggerWithAddReplicaPreferredOp_2Shard() throws Exception {
String collectionNamePrefix = "testNodeAddedTriggerWithAddReplicaPreferredOp_2Shard";
int numShards = 2;
int numCollections = 5;
nodeAddedTriggerWithAddReplicaPreferredOp(collectionNamePrefix, numShards, numCollections);
}
private void nodeAddedTriggerWithAddReplicaPreferredOp(String collectionNamePrefix, int numShards, int numCollections) throws Exception {
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'" + AutoScalingParams.PREFERRED_OP + "':'addreplica'," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + AssertingTriggerAction.class.getName() + "'}]" +
"}}";
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<" + (1 + numCollections * numShards) + "', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
nodeAddedTriggerWithAddReplicaPreferredOp(collectionNamePrefix, numShards, numCollections, setTriggerCommand, setClusterPolicyCommand);
}
private void nodeAddedTriggerWithAddReplicaPreferredOpReplicaType(String collectionNamePrefix, int numShards, int numCollections) throws Exception {
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_added_trigger'," +
"'event' : 'nodeAdded'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'" + AutoScalingParams.PREFERRED_OP + "':'addreplica'," +
"'" + AutoScalingParams.REPLICA_TYPE + "':'" + Replica.Type.PULL + "'," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'test','class':'" + AssertingTriggerAction.class.getName() + "'}]" +
"}}";
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<" + (1 + numCollections * numShards) + "', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
nodeAddedTriggerWithAddReplicaPreferredOp(collectionNamePrefix, numShards, numCollections, setTriggerCommand, setClusterPolicyCommand, 0, 1, 0);
}
private void nodeAddedTriggerWithAddReplicaPreferredOp(String collectionNamePrefix, int numShards, int numCollections, String setTriggerCommand, String setClusterPolicyCommand) throws Exception {
nodeAddedTriggerWithAddReplicaPreferredOp(collectionNamePrefix, numShards, numCollections, setTriggerCommand, setClusterPolicyCommand, 1, null, null);
}
private void nodeAddedTriggerWithAddReplicaPreferredOp(String collectionNamePrefix, int numShards, int numCollections, String setTriggerCommand, String setClusterPolicyCommand, Integer nNrtReplicas, Integer nTlogReplicas, Integer nPullReplicas) throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionNamePrefix + "_0",
"conf", numShards, nNrtReplicas, nTlogReplicas, nPullReplicas);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
collectionNamePrefix + "_0", (liveNodes, collectionState) ->
collectionState.getReplicas().stream().allMatch(replica -> replica.isActive(liveNodes)));
JettySolrRunner newNode = cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
assertTrue(triggerFiredLatch.await(30, TimeUnit.SECONDS));
assertTrue(fired.get());
@SuppressWarnings({"rawtypes"})
Map actionContext = actionContextPropsRef.get();
@SuppressWarnings({"rawtypes"})
List operations = (List) actionContext.get("operations");
assertNotNull(operations);
assertEquals(numShards, operations.size());
Set<String> affectedShards = new HashSet<>(2);
for (Object operation : operations) {
assertTrue(operation instanceof CollectionAdminRequest.AddReplica);
CollectionAdminRequest.AddReplica addReplica = (CollectionAdminRequest.AddReplica) operation;
assertEquals(newNode.getNodeName(), addReplica.getNode());
assertEquals(collectionNamePrefix + "_0", addReplica.getCollection());
affectedShards.add(addReplica.getShard());
}
assertEquals(numShards, affectedShards.size());
for (int i = 1; i < numCollections; i++) {
create = CollectionAdminRequest.createCollection(collectionNamePrefix + "_" + i,
"conf", numShards, 2);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
collectionNamePrefix + "_" + i, (liveNodes, collectionState) ->
collectionState.getReplicas().stream().allMatch(replica -> replica.isActive(liveNodes)));
}
reset();
newNode = cluster.startJettySolrRunner();
assertTrue(triggerFiredLatch.await(30, TimeUnit.SECONDS));
assertTrue(fired.get());
actionContext = actionContextPropsRef.get();
operations = (List) actionContext.get("operations");
assertNotNull(operations);
assertEquals(numCollections * numShards, operations.size());
Set<String> affectedCollections = new HashSet<>(numCollections);
affectedShards = new HashSet<>(numShards);
Set<Pair<String, String>> affectedCollShards = new HashSet<>(numCollections * numShards);
for (Object operation : operations) {
assertTrue(operation instanceof CollectionAdminRequest.AddReplica);
CollectionAdminRequest.AddReplica addReplica = (CollectionAdminRequest.AddReplica) operation;
assertEquals(newNode.getNodeName(), addReplica.getNode());
affectedCollections.add(addReplica.getCollection());
affectedShards.add(addReplica.getShard());
affectedCollShards.add(new Pair<>(addReplica.getCollection(), addReplica.getShard()));
}
assertEquals(numCollections, affectedCollections.size());
assertEquals(numShards, affectedShards.size());
assertEquals(numCollections * numShards, affectedCollShards.size());
}
@Test
// commented out on: 17-Feb-2019 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 14-Oct-2018
public void testNodeLostTriggerWithDeleteNodePreferredOp() throws Exception {
String collectionNamePrefix = "testNodeLostTriggerWithDeleteNodePreferredOp";
int numCollections = 1 + random().nextInt(3), numShards = 1 + random().nextInt(3);
CloudSolrClient solrClient = cluster.getSolrClient();
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : 'node_lost_trigger'," +
"'event' : 'nodeLost'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'" + AutoScalingParams.PREFERRED_OP + "':'deletenode'," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'execute_plan','class':'solr.ExecutePlanAction'}" +
"{'name':'test','class':'" + AssertingTriggerAction.class.getName() + "'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<" + (1 + numCollections * numShards) + "', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
JettySolrRunner newNode = cluster.startJettySolrRunner();
// cache the node name because it won't be available once the node is shutdown
String newNodeName = newNode.getNodeName();
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionNamePrefix + "_0",
"conf", numShards, 2);
create.process(solrClient);
waitForState("Timed out waiting for replicas of new collection to be active",
collectionNamePrefix + "_0", (liveNodes, collectionState) ->
collectionState.getReplicas().stream().allMatch(replica -> replica.isActive(liveNodes)));
cluster.stopJettySolrRunner(newNode);
assertTrue(triggerFiredLatch.await(30, TimeUnit.SECONDS));
assertTrue(fired.get());
@SuppressWarnings({"rawtypes"})
Map actionContext = actionContextPropsRef.get();
@SuppressWarnings({"rawtypes"})
List operations = (List) actionContext.get("operations");
assertNotNull(operations);
assertEquals(1, operations.size());
for (Object operation : operations) {
assertTrue(operation instanceof CollectionAdminRequest.DeleteNode);
CollectionAdminRequest.DeleteNode deleteNode = (CollectionAdminRequest.DeleteNode) operation;
SolrParams deleteNodeParams = deleteNode.getParams();
assertEquals(newNodeName, deleteNodeParams.get("node"));
}
waitForState("Timed out waiting for all shards to have only 1 replica",
collectionNamePrefix + "_0", clusterShape(numShards, numShards));
}
}

View File

@ -1,381 +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.autoscaling;
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.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import com.google.common.collect.Lists;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.CloudTestUtils.AutoScalingRequest;
import org.apache.solr.cloud.CloudUtil;
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.core.SolrResourceLoader;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TestInjection;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.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;
private SolrResourceLoader loader;
private SolrCloudManager cloudManager;
public static class StartAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
startedProcessing.countDown();
}
}
private static CountDownLatch startedProcessing = new CountDownLatch(1);
public static class FinishAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) throws Exception {
finishedProcessing.countDown();
}
}
private static CountDownLatch finishedProcessing = new CountDownLatch(1);
@BeforeClass
public static void setupCluster() throws Exception {
}
@Before
public void setUp() throws Exception {
super.setUp();
configureCluster(NODE_COUNT)
.addConfig("conf", configset("cloud-minimal"))
.configure();
// clear any persisted auto scaling configuration
Stat stat = zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
cloudManager = cluster.getJettySolrRunner(0).getCoreContainer().getZkController().getSolrCloudManager();
finishedProcessing = new CountDownLatch(1);
startedProcessing = new CountDownLatch(1);
}
@After
public void tearDown() throws Exception {
shutdownCluster();
super.tearDown();
TestInjection.reset();
}
@Test
public void testExecute() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String collectionName = "testExecute";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
"conf", 1, 2);
create.process(solrClient);
cluster.waitForActiveCollection(collectionName, 1, 2);
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.configure(loader, cloudManager, 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.OVERSEERSTATUS) {
@Override
public void setAsyncId(String asyncId) {
super.setAsyncId(asyncId);
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);
@SuppressWarnings({"rawtypes"})
Map m = (Map) Utils.fromJSON(data);
if (m.containsKey("requestid")) {
znodeCreated.set(m.get("requestid").equals(asyncId));
}
}
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
};
List<CollectionAdminRequest.AsyncCollectionAdminRequest> operations = Lists.asList(moveReplica, new CollectionAdminRequest.AsyncCollectionAdminRequest[]{mockRequest});
NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent
(TriggerEventType.NODELOST, "mock_trigger_name",
Collections.singletonList(cloudManager.getTimeSource().getTimeNs()),
Collections.singletonList(sourceNodeName),
CollectionParams.CollectionAction.MOVEREPLICA.toLower());
ActionContext actionContext = new ActionContext(survivor.getCoreContainer().getZkController().getSolrCloudManager(), null,
new HashMap<>(Collections.singletonMap("operations", operations)));
action.process(nodeLostEvent, actionContext);
// assertTrue("ExecutePlanAction should have stored the requestid in ZK before executing the request", znodeCreated.get());
@SuppressWarnings({"unchecked"})
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'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(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.process(solrClient);
cluster.waitForActiveCollection(collectionName, 1, 2);
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) {
JettySolrRunner j = cluster.stopJettySolrRunner(i);
cluster.waitForJettyToStop(j);
}
}
Thread.sleep(1000);
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(docCollection.toString(), 2, replicasOnSurvivor.size());
}
@Test
public void testTaskTimeout() throws Exception {
int DELAY = 2000;
boolean taskTimeoutFail = random().nextBoolean();
TestInjection.delayInExecutePlanAction = DELAY;
CloudSolrClient solrClient = cluster.getSolrClient();
String triggerName = "node_lost_trigger2";
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : '" + triggerName + "'," +
"'event' : 'nodeLost'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name':'execute_plan','class':'solr.ExecutePlanAction', 'taskTimeoutSeconds' : '1','taskTimeoutFail':'" + taskTimeoutFail + "'}," +
"{'name':'finish','class':'" + FinishAction.class.getName() + "'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String collectionName = "testTaskTimeout";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
"conf", 1, 2);
create.process(solrClient);
cluster.waitForActiveCollection(collectionName, 1, 2);
waitForState("Timed out waiting for replicas of new collection to be active",
collectionName, clusterShape(1, 2));
JettySolrRunner sourceNode = cluster.getRandomJetty(random());
for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
JettySolrRunner runner = cluster.getJettySolrRunner(i);
if (runner == sourceNode) {
JettySolrRunner j = cluster.stopJettySolrRunner(i);
cluster.waitForJettyToStop(j);
}
}
boolean await = finishedProcessing.await(DELAY * 5, TimeUnit.MILLISECONDS);
if (taskTimeoutFail) {
assertFalse("finished processing event but should fail", await);
} else {
assertTrue("did not finish processing event in time", await);
}
String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + triggerName + "/execute_plan";
assertTrue(path + " does not exist", zkClient().exists(path, true));
List<String> requests = zkClient().getChildren(path, null, true);
assertFalse("some requests should be still present", requests.isEmpty());
// in either case the task will complete and move the replica as needed
waitForState("Timed out waiting for replicas of collection to be 2 again",
collectionName, clusterShape(1, 2));
}
@Test
public void testTaskFail() throws Exception {
TestInjection.failInExecutePlanAction = true;
CloudSolrClient solrClient = cluster.getSolrClient();
String triggerName = "node_lost_trigger3";
String setTriggerCommand = "{" +
"'set-trigger' : {" +
"'name' : '" + triggerName + "'," +
"'event' : 'nodeLost'," +
"'waitFor' : '1s'," +
"'enabled' : true," +
"'actions' : [{'name':'start', 'class' : '" + StartAction.class.getName() + "'}," +
"{'name':'compute_plan','class':'solr.ComputePlanAction'}," +
"{'name':'execute_plan','class':'solr.ExecutePlanAction'}," +
"{'name':'finish','class':'" + FinishAction.class.getName() + "'}]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setTriggerCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String collectionName = "testTaskFail";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName,
"conf", 1, 2);
create.process(solrClient);
cluster.waitForActiveCollection(collectionName, 1, 2);
waitForState("Timed out waiting for replicas of new collection to be active",
collectionName, clusterShape(1, 2));
// don't stop the jetty that runs our SolrCloudManager
JettySolrRunner runner = cluster.stopJettySolrRunner(1);
cluster.waitForJettyToStop(runner);
boolean await = startedProcessing.await(10, TimeUnit.SECONDS);
assertTrue("did not start processing event in time", await);
await = finishedProcessing.await(2, TimeUnit.SECONDS);
assertFalse("finished processing event but should fail", await);
String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + triggerName + "/execute_plan";
assertTrue(path + " does not exist", zkClient().exists(path, true));
List<String> requests = zkClient().getChildren(path, null, true);
assertTrue("there should be no requests pending but got " + requests, requests.isEmpty());
// the task never completed - we actually lost a replica
try {
CloudUtil.waitForState(cloudManager, collectionName, 5, TimeUnit.SECONDS,
CloudUtil.clusterShape(1, 2));
fail("completed a task that should have failed");
} catch (TimeoutException te) {
// expected
}
}
}

View File

@ -1,61 +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.autoscaling;
import com.carrotsearch.randomizedtesting.annotations.Nightly;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.lucene.util.QuickPatchThreadsFilter;
import org.apache.lucene.util.TimeUnits;
import org.apache.solr.SolrIgnoredThreadsFilter;
import org.apache.solr.cloud.hdfs.HdfsTestUtil;
import org.apache.solr.util.BadHdfsThreadsFilter;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@Slow
@Nightly
@ThreadLeakFilters(defaultFilters = true, filters = {
SolrIgnoredThreadsFilter.class,
QuickPatchThreadsFilter.class,
BadHdfsThreadsFilter.class // hdfs currently leaks thread(s)
})
@TimeoutSuite(millis = TimeUnits.HOUR)
public class HdfsAutoAddReplicasIntegrationTest extends AutoAddReplicasIntegrationTest {
private static MiniDFSCluster dfsCluster;
@BeforeClass
public static void setupClass() throws Exception {
dfsCluster = HdfsTestUtil.setupClass(createTempDir().toFile().getAbsolutePath());
}
@AfterClass
public static void teardownClass() throws Exception {
try {
HdfsTestUtil.teardownClass(dfsCluster);
} finally {
dfsCluster = null;
}
}
@Override
protected String getConfigSet() {
return "cloud-hdfs";
}
}

View File

@ -1,208 +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.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.nio.charset.StandardCharsets;
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.CloudTestUtils.AutoScalingRequest;
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;
/**
*
*/
@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() + "'}" +
"]" +
"}}";
@SuppressWarnings({"rawtypes"})
SolrRequest req = AutoScalingRequest.create(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.beforeAction:invalid}/${stage}'," +
"'payload': 'actionName=${actionName}, source=${event.source}, type=${event.eventType}'," +
"'header.X-Foo' : '${config.name:invalid}'" +
"}" +
"}";
req = AutoScalingRequest.create(SolrRequest.METHOD.POST, setListenerCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
assertEquals(mockService.requests.toString(), 0, mockService.requests.size());
cluster.startJettySolrRunner();
cluster.waitForAllNodes(30);
boolean await = triggerFiredLatch.await(20, TimeUnit.SECONDS);
assertTrue("The trigger did not fire at all", await);
Thread.sleep(5000);
assertEquals(mockService.requests.toString(), 4, mockService.requests.size());
mockService.requests.forEach(s -> assertTrue(s.contains("Content-Type: application/json")));
mockService.requests.forEach(s -> assertTrue(s.contains("X-Foo: foo")));
mockService.requests.forEach(s -> assertTrue(s.contains("source=node_added_trigger")));
mockService.requests.forEach(s -> assertTrue(s.contains("type=NODEADDED")));
String request = mockService.requests.get(0);
assertTrue(request, request.startsWith("/foo/test/STARTED"));
assertTrue(request, request.contains("actionName=,")); // empty actionName
request = mockService.requests.get(1);
assertTrue(request, request.startsWith("/foo/test/BEFORE_ACTION"));
assertTrue(request, request.contains("actionName=test,")); // actionName
request = mockService.requests.get(2);
assertTrue(request, request.startsWith("/foo/test/AFTER_ACTION"));
assertTrue(request, request.contains("actionName=test,")); // actionName
request = mockService.requests.get(3);
assertTrue(request, request.startsWith("/foo/test/SUCCEEDED"));
assertTrue(request, request.contains("actionName=,")); // empty actionName
}
public static class TestDummyAction extends TriggerActionBase {
@Override
public void process(TriggerEvent event, ActionContext context) {
triggerFiredLatch.countDown();
}
}
private static class MockService extends Thread {
public final List<String> requests = new ArrayList<>();
private 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, StandardCharsets.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();
}
}
}
}

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