Merge branch 'feature/autoscaling_solr7'

# Conflicts:
#	solr/CHANGES.txt
This commit is contained in:
Shalin Shekhar Mangar 2017-06-06 09:26:56 +05:30
commit d633c957cb
53 changed files with 4000 additions and 194 deletions

View File

@ -125,6 +125,12 @@ New Features
participate in elections.
(Tomás Fernández Löbbe)
* SOLR-10373: Implement read API for autoscaling configuration at /admin/autoscaling or
/cluster/autoscaling paths. (shalin)
* SOLR-10677: Expose a diagnostics API to return nodes sorted by load in descending order and
any policy violations. (shalin)
Bug Fixes
----------------------
* SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.
@ -238,6 +244,12 @@ Other Changes
* SOLR-10713: Ignore .pid and .out files in solr working directory (Jason Gerlowski via Mike Drob)
* SOLR-10764: AutoScalingHandler should validate policy and preferences before updating zookeeper. (shalin)
* SOLR-10782: Improve error handling and tests for Snitch and subclasses and general cleanups. (Noble Paul, shalin)
* SOLR-10419: All collection APIs should use the new Policy framework for replica placement. (Noble Paul, shalin)
================== 6.7.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -16,6 +16,7 @@
*/
package org.apache.solr.cloud;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@ -28,6 +29,11 @@ import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
import org.apache.solr.cloud.autoscaling.Policy;
import org.apache.solr.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.rule.ReplicaAssigner;
import org.apache.solr.cloud.rule.Rule;
import org.apache.solr.common.SolrException;
@ -35,11 +41,17 @@ import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.zookeeper.KeeperException;
import static java.util.Collections.singletonMap;
import static org.apache.solr.cloud.autoscaling.Policy.POLICY;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
public class Assign {
@ -150,7 +162,7 @@ public class Assign {
// could be created on live nodes given maxShardsPerNode, Replication factor (if from createShard) etc.
public static List<ReplicaCount> getNodesForNewReplicas(ClusterState clusterState, String collectionName,
String shard, int numberOfNodes,
Object createNodeSet, CoreContainer cc) {
Object createNodeSet, CoreContainer cc) throws KeeperException, InterruptedException {
DocCollection coll = clusterState.getCollection(collectionName);
Integer maxShardsPerNode = coll.getInt(MAX_SHARDS_PER_NODE, 1);
List<String> createNodeList = null;
@ -179,8 +191,23 @@ public class Assign {
}
List l = (List) coll.get(DocCollection.RULE);
Map<ReplicaAssigner.Position, String> positions = null;
if (l != null) {
return getNodesViaRules(clusterState, shard, numberOfNodes, cc, coll, createNodeList, l);
positions = getNodesViaRules(clusterState, shard, numberOfNodes, cc, coll, createNodeList, l);
}
String policyName = coll.getStr(POLICY);
Map autoScalingJson = Utils.getJson(cc.getZkController().getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
if (policyName != null || autoScalingJson.get(Policy.CLUSTER_POLICY) != null) {
positions= Assign.getPositionsUsingPolicy(collectionName, Collections.singletonList(shard), numberOfNodes,
policyName, cc.getZkController().getZkStateReader(), createNodeList);
}
if(positions != null){
List<ReplicaCount> repCounts = new ArrayList<>();
for (String s : positions.values()) {
repCounts.add(new ReplicaCount(s));
}
return repCounts;
}
ArrayList<ReplicaCount> sortedNodeList = new ArrayList<>(nodeNameVsShardCount.values());
@ -188,8 +215,31 @@ public class Assign {
return sortedNodeList;
}
public static Map<ReplicaAssigner.Position, String> getPositionsUsingPolicy(String collName, List<String> shardNames, int numReplicas,
String policyName, ZkStateReader zkStateReader,
List<String> nodesList) throws KeeperException, InterruptedException {
try (CloudSolrClient csc = new CloudSolrClient.Builder()
.withClusterStateProvider(new ZkClientClusterStateProvider(zkStateReader))
.build()) {
SolrClientDataProvider clientDataProvider = new SolrClientDataProvider(csc);
Map<String, Object> autoScalingJson = Utils.getJson(zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
Map<String, List<String>> locations = PolicyHelper.getReplicaLocations(collName,
autoScalingJson,
clientDataProvider, singletonMap(collName, policyName), shardNames, numReplicas, nodesList);
Map<ReplicaAssigner.Position, String> result = new HashMap<>();
for (Map.Entry<String, List<String>> e : locations.entrySet()) {
List<String> value = e.getValue();
for (int i = 0; i < value.size(); i++) {
result.put(new ReplicaAssigner.Position(e.getKey(), i, Replica.Type.NRT), value.get(i));
}
}
return result;
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error closing CloudSolrClient",e);
}
}
private static List<ReplicaCount> getNodesViaRules(ClusterState clusterState, String shard, int numberOfNodes,
private static Map<ReplicaAssigner.Position, String> getNodesViaRules(ClusterState clusterState, String shard, int numberOfNodes,
CoreContainer cc, DocCollection coll, List<String> createNodeList, List l) {
ArrayList<Rule> rules = new ArrayList<>();
for (Object o : l) rules.add(new Rule((Map) o));
@ -214,11 +264,7 @@ public class Assign {
shardVsNodes,
nodesList, cc, clusterState).getNodeMappings();
List<ReplicaCount> repCounts = new ArrayList<>();
for (String s : positions.values()) {
repCounts.add(new ReplicaCount(s));
}
return repCounts;
return positions;// getReplicaCounts(positions);
}
private static HashMap<String, ReplicaCount> getNodeNameVsShardCount(String collectionName,

View File

@ -162,7 +162,7 @@ public class CreateCollectionCmd implements Cmd {
+ " shards to be created (higher than the allowed number)");
}
positionVsNodes = ocmh.identifyNodes(clusterState, nodeList, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
positionVsNodes = ocmh.identifyNodes(clusterState, nodeList, collectionName, message, shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
}
ZkStateReader zkStateReader = ocmh.zkStateReader;

View File

@ -16,51 +16,6 @@
*/
package org.apache.solr.cloud;
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;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BACKUP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATEALIAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESNAPSHOT;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEALIAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETENODE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICAPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESNAPSHOT;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MIGRATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MIGRATESTATEFORMAT;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOCK_COLL_TASK;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOCK_REPLICA_TASK;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOCK_SHARD_TASK;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MODIFYCOLLECTION;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.OVERSEERSTATUS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REBALANCELEADERS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.RELOAD;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.REPLACENODE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.RESTORE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.SPLITSHARD;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.util.Utils.makeMap;
import java.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
@ -78,6 +33,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import com.google.common.collect.ImmutableMap;
import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
@ -86,6 +42,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.cloud.autoscaling.Policy;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.cloud.rule.ReplicaAssigner;
import org.apache.solr.cloud.rule.ReplicaAssigner.Position;
@ -124,7 +81,23 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.ImmutableMap;
import static org.apache.solr.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;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
import static org.apache.solr.common.util.Utils.makeMap;
/**
* A {@link OverseerMessageHandler} that handles Collections API related
@ -166,6 +139,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
ZkStateReader.MAX_SHARDS_PER_NODE, "1",
ZkStateReader.AUTO_ADD_REPLICAS, "false",
DocCollection.RULE, null,
POLICY, null,
SNITCH, null));
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@ -732,13 +706,17 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
Map<Position, String> identifyNodes(ClusterState clusterState,
List<String> nodeList,
String collectionName,
ZkNodeProps message,
List<String> shardNames,
int numNrtReplicas,
int numTlogReplicas,
int numPullReplicas) throws IOException {
int numPullReplicas) throws KeeperException, InterruptedException {
List<Map> rulesMap = (List) message.get("rule");
if (rulesMap == null) {
String policyName = message.getStr(POLICY);
Map autoScalingJson = Utils.getJson(zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
if (rulesMap == null && policyName == null) {
int i = 0;
Map<Position, String> result = new HashMap<>();
for (String aShard : shardNames) {
@ -759,10 +737,15 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
} else {
if (numTlogReplicas + numPullReplicas != 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
Replica.Type.TLOG + " or " + Replica.Type.PULL + " replica types not supported with placement rules");
Replica.Type.TLOG + " or " + Replica.Type.PULL + " replica types not supported with placement rules or cluster policies");
}
}
if (policyName != null || autoScalingJson.get(Policy.CLUSTER_POLICY) != null) {
return Assign.getPositionsUsingPolicy(collectionName,
shardNames, numNrtReplicas, policyName, zkStateReader, nodeList);
} else {
List<Rule> rules = new ArrayList<>();
for (Object map : rulesMap) rules.add(new Rule((Map) map));
@ -779,6 +762,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
return replicaAssigner.getNodeMappings();
}
}
Map<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreNames) throws InterruptedException {
Map<String, Replica> result = new HashMap<>();

View File

@ -214,7 +214,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
restoreCollection.getSlices().forEach(x -> sliceNames.add(x.getName()));
Map<ReplicaAssigner.Position, String> positionVsNodes = ocmh.identifyNodes(clusterState, nodeList,
message, sliceNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
restoreCollectionName, message, sliceNames, numNrtReplicas, numTlogReplicas, numPullReplicas);
//Create one replica per shard and copy backed up data to it
for (Slice slice : restoreCollection.getSlices()) {

View File

@ -381,9 +381,9 @@ public class SplitShardCmd implements Cmd {
// TODO: change this to handle sharding a slice into > 2 sub-shards.
Map<ReplicaAssigner.Position, String> nodeMap = ocmh.identifyNodes(clusterState,
new ArrayList<>(clusterState.getLiveNodes()),
collectionName,
new ZkNodeProps(collection.getProperties()),
subSlices, repFactor - 1, 0, 0);

View File

@ -668,6 +668,7 @@ public class ZkController {
byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
cmdExecutor.ensureExists(ZkStateReader.CLUSTER_STATE, emptyJson, CreateMode.PERSISTENT, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_SECURITY_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
cmdExecutor.ensureExists(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, emptyJson, CreateMode.PERSISTENT, zkClient);
}
private void init(CurrentCoreDescriptorProvider registerOnReconnect) {

View File

@ -0,0 +1,319 @@
/*
* 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.List;
import java.util.Map;
import com.google.common.collect.ImmutableSet;
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.impl.SolrClientDataProvider;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.CommandOperation;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.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.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
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());
private static ImmutableSet<String> singletonCommands = ImmutableSet.of("set-cluster-preferences", "set-cluster-policy");
protected final CoreContainer container;
private final List<Map<String, String>> DEFAULT_ACTIONS = new ArrayList<>(3);
public AutoScalingHandler(CoreContainer container) {
this.container = container;
}
@Override
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, '/');
if (parts.get(0).isEmpty()) parts.remove(0);
if (parts.size() < 2 || parts.size() > 3) {
// invalid
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown path: " + path);
}
Map<String, Object> map = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
if (parts.size() == 2) {
rsp.getValues().addAll(map);
} else if (parts.size() == 3 && "diagnostics".equals(parts.get(2))) {
handleDiagnostics(rsp, map);
}
} else {
if (req.getContentStreams() == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No commands specified for autoscaling");
}
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;
}
for (CommandOperation op : ops) {
switch (op.name) {
case "set-policy":
handleSetPolicies(req, rsp, op);
break;
case "remove-policy":
handleRemovePolicy(req, rsp, op);
break;
case "set-cluster-preferences":
handleSetClusterPreferences(req, rsp, op);
break;
case "set-cluster-policy":
handleSetClusterPolicy(req, rsp, op);
break;
default:
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown command: " + op.name);
}
}
}
} catch (Exception e) {
rsp.getValues().add("result", "failure");
throw e;
} finally {
RequestHandlerUtils.addExperimentalFormatWarning(rsp);
}
}
private void handleDiagnostics(SolrQueryResponse rsp, Map<String, Object> autoScalingConf) throws IOException {
Policy policy = new Policy(autoScalingConf);
try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) {
Policy.Session session = policy.createSession(new SolrClientDataProvider(build));
List<Row> sorted = session.getSorted();
List<Clause.Violation> violations = session.getViolations();
List<Preference> clusterPreferences = policy.getClusterPreferences();
List<Map<String, Object>> sortedNodes = new ArrayList<>(sorted.size());
for (Row row : sorted) {
Map<String, Object> map = Utils.makeMap("node", row.node);
for (Cell cell : row.cells) {
for (Preference clusterPreference : clusterPreferences) {
Policy.SortParam name = clusterPreference.name;
if (cell.name.equalsIgnoreCase(name.name())) {
map.put(name.name(), cell.val);
break;
}
}
}
sortedNodes.add(map);
}
Map<String, Object> map = new HashMap<>(2);
map.put("sortedNodes", sortedNodes);
map.put("violations", violations);
rsp.getValues().add("diagnostics", map);
}
}
private void handleSetClusterPolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
List clusterPolicy = (List) op.getCommandData();
if (clusterPolicy == null || !(clusterPolicy instanceof List)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "A list of cluster policies was not found");
}
zkSetClusterPolicy(container.getZkController().getZkStateReader(), clusterPolicy);
rsp.getValues().add("result", "success");
}
private void handleSetClusterPreferences(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
List preferences = (List) op.getCommandData();
if (preferences == null || !(preferences instanceof List)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "A list of cluster preferences not found");
}
zkSetPreferences(container.getZkController().getZkStateReader(), preferences);
rsp.getValues().add("result", "success");
}
private void handleRemovePolicy(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
String policyName = (String) op.getCommandData();
if (policyName.trim().length() == 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The policy name cannot be empty");
}
Map<String, Object> autoScalingConf = zkReadAutoScalingConf(container.getZkController().getZkStateReader());
Map<String, Object> policies = (Map<String, Object>) autoScalingConf.get("policies");
if (policies == null || !policies.containsKey(policyName)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No policy exists with name: " + policyName);
}
zkSetPolicies(container.getZkController().getZkStateReader(), policyName, null);
rsp.getValues().add("result", "success");
}
private void handleSetPolicies(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation op) throws KeeperException, InterruptedException, IOException {
Map<String, Object> policies = op.getDataMap();
for (Map.Entry<String, Object> policy : policies.entrySet()) {
String policyName = policy.getKey();
if (policyName == null || policyName.trim().length() == 0) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The policy name cannot be null or empty");
}
}
zkSetPolicies(container.getZkController().getZkStateReader(), null, policies);
rsp.getValues().add("result", "success");
}
private void zkSetPolicies(ZkStateReader reader, String policyBeRemoved, Map<String, Object> newPolicies) throws KeeperException, InterruptedException, IOException {
while (true) {
Stat stat = new Stat();
ZkNodeProps loaded = null;
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
loaded = ZkNodeProps.load(data);
Map<String, Object> policies = (Map<String, Object>) loaded.get("policies");
if (policies == null) policies = new HashMap<>(1);
if (newPolicies != null) {
policies.putAll(newPolicies);
} else {
policies.remove(policyBeRemoved);
}
loaded = loaded.plus("policies", policies);
verifyAutoScalingConf(loaded.getProperties());
try {
reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
} catch (KeeperException.BadVersionException bve) {
// somebody else has changed the configuration so we must retry
continue;
}
break;
}
}
private void zkSetPreferences(ZkStateReader reader, List preferences) throws KeeperException, InterruptedException, IOException {
while (true) {
Stat stat = new Stat();
ZkNodeProps loaded = null;
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
loaded = ZkNodeProps.load(data);
loaded = loaded.plus("cluster-preferences", preferences);
verifyAutoScalingConf(loaded.getProperties());
try {
reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
} catch (KeeperException.BadVersionException bve) {
// somebody else has changed the configuration so we must retry
continue;
}
break;
}
}
private void zkSetClusterPolicy(ZkStateReader reader, List clusterPolicy) throws KeeperException, InterruptedException, IOException {
while (true) {
Stat stat = new Stat();
ZkNodeProps loaded = null;
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, stat, true);
loaded = ZkNodeProps.load(data);
loaded = loaded.plus("cluster-policy", clusterPolicy);
verifyAutoScalingConf(loaded.getProperties());
try {
reader.getZkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(loaded), stat.getVersion(), true);
} catch (KeeperException.BadVersionException bve) {
// somebody else has changed the configuration so we must retry
continue;
}
break;
}
}
private void verifyAutoScalingConf(Map<String, Object> autoScalingConf) throws IOException {
try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) {
Policy policy = new Policy(autoScalingConf);
Policy.Session session = policy.createSession(new SolrClientDataProvider(build));
log.debug("Verified autoscaling configuration");
}
}
private Map<String, Object> zkReadAutoScalingConf(ZkStateReader reader) throws KeeperException, InterruptedException {
byte[] data = reader.getZkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
ZkNodeProps loaded = ZkNodeProps.load(data);
return loaded.getProperties();
}
@Override
public String getDescription() {
return "A handler for autoscaling configuration";
}
@Override
public Name getPermissionName(AuthorizationContext request) {
switch (request.getHttpMethod()) {
case "GET":
return Name.AUTOSCALING_READ_PERM;
case "POST":
return 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")) return this;
return null;
}
}

View File

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

View File

@ -19,6 +19,7 @@ package org.apache.solr.cloud.rule;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Collections;
import java.util.Map;
import org.apache.solr.client.solrj.SolrRequest;
@ -36,7 +37,7 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -55,32 +56,22 @@ public class ServerSnitchContext extends SnitchContext {
}
public Map getZkJson(String path) {
public Map getZkJson(String path) throws KeeperException, InterruptedException {
if (coreContainer.isZooKeeperAware()) {
try {
byte[] data = coreContainer.getZkController().getZkClient().getData(path, null, new Stat(), true);
if (data == null) return null;
return (Map) Utils.fromJSON(data);
} catch (Exception e) {
log.warn("Unable to read from ZK path : " + path, e);
return null;
}
return Utils.getJson(coreContainer.getZkController().getZkClient(), path, true);
} else {
return null;
return Collections.emptyMap();
}
}
public void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) {
if (callback == null) callback = this;
String url = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(node);
params.add("class", klas);
params.add(ACTION, INVOKE.toString());
//todo batch all requests to the same server
try {
SimpleSolrResponse rsp = invoke(coreContainer.getUpdateShardHandler(), url, CommonParams.CORES_HANDLER_PATH, params);
SimpleSolrResponse rsp = invoke(node, CommonParams.CORES_HANDLER_PATH, params);
Map<String, Object> returnedVal = (Map<String, Object>) rsp.getResponse().get(klas);
if(exception == null){
// log this
@ -94,8 +85,10 @@ public class ServerSnitchContext extends SnitchContext {
}
}
public SimpleSolrResponse invoke(UpdateShardHandler shardHandler, final String url, String path, SolrParams params)
public SimpleSolrResponse invoke(String solrNode, String path, SolrParams params)
throws IOException, SolrServerException {
String url = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(solrNode);
UpdateShardHandler shardHandler = coreContainer.getUpdateShardHandler();
GenericSolrRequest request = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
try (HttpSolrClient client = new HttpSolrClient.Builder(url).withHttpClient(shardHandler.getHttpClient())
.withResponseParser(new BinaryResponseParser()).build()) {

View File

@ -60,6 +60,7 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder;
import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.AuthSchemeRegistryProvider;
import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.CredentialsProviderProvider;
import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController;
@ -193,6 +194,8 @@ public class CoreContainer {
public final static long INITIAL_CORE_LOAD_COMPLETE = 0x4L;
private volatile long status = 0L;
protected AutoScalingHandler autoScalingHandler;
private enum CoreInitFailedAction { fromleader, none }
/**
@ -528,6 +531,9 @@ public class CoreContainer {
metricsCollectorHandler = createHandler(MetricsCollectorHandler.HANDLER_PATH, MetricsCollectorHandler.class.getName(), MetricsCollectorHandler.class);
// may want to add some configuration here in the future
metricsCollectorHandler.init(null);
autoScalingHandler = createHandler(AutoScalingHandler.HANDLER_PATH, AutoScalingHandler.class.getName(), AutoScalingHandler.class);
containerHandlers.put(AUTHZ_PATH, securityConfHandler);
securityConfHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AUTHZ_PATH);
containerHandlers.put(AUTHC_PATH, securityConfHandler);

View File

@ -90,7 +90,8 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
static final String[] packages = {
"", "analysis.", "schema.", "handler.", "search.", "update.", "core.", "response.", "request.",
"update.processor.", "util.", "spelling.", "handler.component.", "handler.dataimport.",
"spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.","handler.admin."
"spelling.suggest.", "spelling.suggest.fst.", "rest.schema.analysis.", "security.","handler.admin.",
"cloud.autoscaling."
};
private static final java.lang.String SOLR_CORE_NAME = "solr.core.name";
private static Set<String> loggedOnce = new ConcurrentSkipListSet<>();

View File

@ -82,7 +82,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial
public void handleRequestBody(final SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
String httpMethod = req.getHttpMethod();
String path = (String) req.getContext().get("path");
SolrConfigHandler.setWt(req, JSON);
RequestHandlerUtils.setWt(req, JSON);
List<String> pieces = StrUtils.splitSmart(path, '/');
String blobName = null;

View File

@ -20,6 +20,7 @@ import java.io.IOException;
import java.util.*;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.MapSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.UpdateParams;
@ -119,4 +120,16 @@ public class RequestHandlerUtils
}
return false;
}
/**
* @since 6.7
*/
public static void setWt(SolrQueryRequest req, String wt) {
SolrParams params = req.getParams();
if (params.get(CommonParams.WT) != null) return;//wt is set by user
Map<String, String> map = new HashMap<>(1);
map.put(CommonParams.WT, wt);
map.put("indent", "true");
req.setParams(SolrParams.wrapDefaults(params, new MapSolrParams(map)));
}
}

View File

@ -76,7 +76,7 @@ public class SchemaHandler extends RequestHandlerBase implements SolrCoreAware,
@Override
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
SolrConfigHandler.setWt(req, JSON);
RequestHandlerUtils.setWt(req, JSON);
String httpMethod = (String) req.getContext().get("httpMethod");
if ("POST".equals(httpMethod)) {
if (isImmutableConfigSet) {

View File

@ -124,7 +124,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
@Override
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
setWt(req, CommonParams.JSON);
RequestHandlerUtils.setWt(req, CommonParams.JSON);
String httpMethod = (String) req.getContext().get("httpMethod");
Command command = new Command(req, rsp, httpMethod);
if ("POST".equals(httpMethod)) {
@ -673,15 +673,6 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
return null;
}
public static void setWt(SolrQueryRequest req, String wt) {
SolrParams params = req.getParams();
if (params.get(CommonParams.WT) != null) return;//wt is set by user
Map<String, String> map = new HashMap<>(1);
map.put(CommonParams.WT, wt);
map.put("indent", "true");
req.setParams(SolrParams.wrapDefaults(params, new MapSolrParams(map)));
}
@Override
public SolrRequestHandler getSubHandler(String path) {
if (subPaths.contains(path)) return this;

View File

@ -109,6 +109,7 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.ONLY_IF_DOW
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.REQUESTID;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARD_UNIQUE;
import static org.apache.solr.cloud.autoscaling.Policy.POLICY;
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;
@ -402,7 +403,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
SNITCH,
PULL_REPLICAS,
TLOG_REPLICAS,
NRT_REPLICAS);
NRT_REPLICAS,
POLICY);
if (props.get(STATE_FORMAT) == null) {
props.put(STATE_FORMAT, "2");

View File

@ -34,7 +34,7 @@ import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.handler.SolrConfigHandler;
import org.apache.solr.handler.RequestHandlerUtils;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.security.AuthenticationPlugin;
@ -74,7 +74,7 @@ public abstract class SecurityConfHandler extends RequestHandlerBase implements
@Override
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
SolrConfigHandler.setWt(req, CommonParams.JSON);
RequestHandlerUtils.setWt(req, CommonParams.JSON);
String httpMethod = (String) req.getContext().get("httpMethod");
String path = (String) req.getContext().get("path");
String key = path.substring(path.lastIndexOf('/')+1);

View File

@ -26,6 +26,9 @@ import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexableField;
@ -144,6 +147,8 @@ public abstract class TextResponseWriter implements PushWriter {
writeNumber(name, (Number) val);
} else if (val instanceof Boolean) {
writeBool(name, (Boolean) val);
} else if (val instanceof AtomicBoolean) {
writeBool(name, ((AtomicBoolean) val).get());
} else if (val instanceof Date) {
writeDate(name, (Date) val);
} else if (val instanceof Document) {
@ -221,13 +226,17 @@ public abstract class TextResponseWriter implements PushWriter {
} else if (val instanceof Float) {
// we pass the float instead of using toString() because
// it may need special formatting. same for double.
writeFloat(name, ((Float)val).floatValue());
writeFloat(name, val.floatValue());
} else if (val instanceof Double) {
writeDouble(name, ((Double) val).doubleValue());
writeDouble(name, val.doubleValue());
} else if (val instanceof Short) {
writeInt(name, val.toString());
} else if (val instanceof Byte) {
writeInt(name, val.toString());
} else if (val instanceof AtomicInteger) {
writeInt(name, ((AtomicInteger) val).get());
} else if (val instanceof AtomicLong) {
writeLong(name, ((AtomicLong) val).get());
} else {
// default... for debugging only
writeStr(name, val.getClass().getName() + ':' + val.toString(), true);

View File

@ -47,6 +47,8 @@ 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),
ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
;
final String name;

View File

@ -0,0 +1,47 @@
{
"documentation": "TODO NOCOMMIT",
"description": "The Scaling API provides API for adding cluster level scaling rules, triggers and event listeners",
"methods": [
"GET",
"POST"
],
"url": {
"paths": [
"/cluster/autoscaling",
"/cluster/autoscaling/diagnostics"
]
},
"commands": {
"set-policy" : {
"type":"object",
"description": "The set-policy command allows you to add and update policies that apply to collections",
/* "patternProperties": {
"^.+$": {
"type": "array"
}
},*/
"additionalProperties": true
},
"set-cluster-policy" : {
"type" : "array",
"description" : "The set-cluster-policy command allows you to add and update cluster-level policy that acts as the base for all collection level policies, if any"
},
"set-cluster-preferences" : {
"type" : "array",
"description" : "The set-cluster-preferences command allows you to add and update cluster-level preferences that are used to sort nodes for selection in cluster operations"
},
"remove-policy": {
"description": "Remove a policy",
"type": "object",
"properties": {
"name": {
"type": "string",
"description": "The name of the policy to be removed"
}
},
"required": [
"name"
]
}
}
}

View File

@ -0,0 +1,338 @@
/*
* 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.List;
import java.util.Map;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.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.V2Request;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.ContentStreamBase;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
/**
* Test for AutoScalingHandler
*/
public class AutoScalingHandlerTest extends SolrCloudTestCase {
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(2)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Before
public void beforeTest() throws Exception {
// clear any persisted auto scaling configuration
zkClient().setData(SOLR_AUTOSCALING_CONF_PATH, Utils.toJSON(new ZkNodeProps()), true);
}
@Test
public void testPolicyAndPreferences() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
// add multiple policies
String setPolicyCommand = "{'set-policy': {" +
" 'xyz':[" +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'!overseer', 'replica':0}" +
" ]," +
" 'policy1':[" +
" {'cores':'<2', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
" ]" +
"}}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
NamedList<Object> response = null;
try {
response = solrClient.request(req);
fail("Adding a policy with 'cores' attribute should not have succeeded.");
} catch (SolrServerException e) {
// todo one of these catch blocks should not be needed after SOLR-10768
if (e.getRootCause() instanceof HttpSolrClient.RemoteSolrException) {
HttpSolrClient.RemoteSolrException rootCause = (HttpSolrClient.RemoteSolrException) e.getRootCause();
// expected
assertTrue(rootCause.getMessage().contains("cores is only allowed in 'cluster-policy'"));
} else {
throw e;
}
} catch (HttpSolrClient.RemoteSolrException e) {
// expected
assertTrue(e.getMessage().contains("cores is only allowed in 'cluster-policy'"));
} catch (Exception e) {
throw e;
}
setPolicyCommand = "{'set-policy': {" +
" 'xyz':[" +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'!overseer', 'replica':0}" +
" ]," +
" 'policy1':[" +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
" ]" +
"}}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
byte[] data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
ZkNodeProps loaded = ZkNodeProps.load(data);
Map<String, Object> policies = (Map<String, Object>) loaded.get("policies");
assertNotNull(policies);
assertNotNull(policies.get("xyz"));
assertNotNull(policies.get("policy1"));
// update default policy
setPolicyCommand = "{'set-policy': {" +
" 'xyz':[" +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
" ]" +
"}}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
policies = (Map<String, Object>) loaded.get("policies");
List conditions = (List) policies.get("xyz");
assertEquals(1, conditions.size());
// remove policy
String removePolicyCommand = "{remove-policy : policy1}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, removePolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
policies = (Map<String, Object>) loaded.get("policies");
assertNull(policies.get("policy1"));
// set preferences
String setPreferencesCommand = "{" +
" 'set-cluster-preferences': [" +
" {'minimize': 'cores', 'precision': 3}," +
" {'maximize': 'freedisk','precision': 100}," +
" {'minimize': 'sysLoadAvg','precision': 10}]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPreferencesCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
List preferences = (List) loaded.get("cluster-preferences");
assertEquals(3, preferences.size());
// set preferences
setPreferencesCommand = "{" +
" 'set-cluster-preferences': [" +
" {'minimize': 'sysLoadAvg','precision': 10}]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPreferencesCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
preferences = (List) loaded.get("cluster-preferences");
assertEquals(1, preferences.size());
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'!overseer', 'replica':0}" +
" ]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
data = zkClient().getData(SOLR_AUTOSCALING_CONF_PATH, null, null, true);
loaded = ZkNodeProps.load(data);
List clusterPolicy = (List) loaded.get("cluster-policy");
assertNotNull(clusterPolicy);
assertEquals(3, clusterPolicy.size());
}
@Test
public void testReadApi() throws Exception {
CloudSolrClient solrClient = cluster.getSolrClient();
String setClusterPolicyCommand = "{" +
" 'set-cluster-policy': [" +
" {'cores':'<10', 'node':'#ANY'}," +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]" +
"}";
SolrRequest req = createAutoScalingRequest(SolrRequest.METHOD.POST, setClusterPolicyCommand);
NamedList<Object> response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setPreferencesCommand = "{" +
" 'set-cluster-preferences': [" +
" {'minimize': 'cores', 'precision': 3}," +
" {'maximize': 'freedisk','precision': 100}," +
" {'minimize': 'sysLoadAvg','precision': 10}," +
" {'minimize': 'heapUsage','precision': 10}]" +
"}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPreferencesCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
String setPolicyCommand = "{'set-policy': {" +
" 'xyz':[" +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}," +
" {'nodeRole':'overseer', 'replica':0}" +
" ]," +
" 'policy1':[" +
" {'replica':'<2', 'shard': '#EACH', 'node': '#ANY'}" +
" ]" +
"}}";
req = createAutoScalingRequest(SolrRequest.METHOD.POST, setPolicyCommand);
response = solrClient.request(req);
assertEquals(response.get("result").toString(), "success");
req = createAutoScalingRequest(SolrRequest.METHOD.GET, null);
response = solrClient.request(req);
List<Map> clusterPrefs = (List<Map>) response.get("cluster-preferences");
assertNotNull(clusterPrefs);
assertEquals(4, clusterPrefs.size());
List<Map> clusterPolicy = (List<Map>) response.get("cluster-policy");
assertNotNull(clusterPolicy);
assertEquals(3, clusterPolicy.size());
Map policies = (Map) response.get("policies");
assertNotNull(policies);
assertEquals(2, policies.size());
assertNotNull(policies.get("xyz"));
assertNotNull(policies.get("policy1"));
req = createAutoScalingRequest(SolrRequest.METHOD.GET, "/diagnostics", null);
response = solrClient.request(req);
Map<String, Object> diagnostics = (Map<String, Object>) response.get("diagnostics");
List sortedNodes = (List) diagnostics.get("sortedNodes");
assertNotNull(sortedNodes);
assertEquals(2, sortedNodes.size());
for (int i = 0; i < 2; i++) {
Map node = (Map) sortedNodes.get(i);
assertNotNull(node);
assertEquals(5, node.size());
assertNotNull(node.get("node"));
assertNotNull(node.get("cores"));
assertEquals(0L, node.get("cores"));
assertNotNull(node.get("freedisk"));
assertTrue(node.get("freedisk") instanceof Double);
assertNotNull(node.get("sysLoadAvg"));
assertTrue(node.get("sysLoadAvg") instanceof Double);
assertNotNull(node.get("heapUsage"));
assertTrue(node.get("heapUsage") instanceof Double);
}
List<Map<String, Object>> violations = (List<Map<String, Object>>) diagnostics.get("violations");
assertNotNull(violations);
assertEquals(0, violations.size());
violations = (List<Map<String, Object>>) diagnostics.get("violations");
assertNotNull(violations);
assertEquals(0, violations.size());
// lets create a collection which violates the rule replicas < 2
CollectionAdminRequest.Create create = CollectionAdminRequest.Create.createCollection("readApiTestViolations", 1, 6);
create.setMaxShardsPerNode(10);
CollectionAdminResponse adminResponse = create.process(solrClient);
assertTrue(adminResponse.isSuccess());
// get the diagnostics output again
req = createAutoScalingRequest(SolrRequest.METHOD.GET, "/diagnostics", null);
response = solrClient.request(req);
diagnostics = (Map<String, Object>) response.get("diagnostics");
sortedNodes = (List) diagnostics.get("sortedNodes");
assertNotNull(sortedNodes);
violations = (List<Map<String, Object>>) diagnostics.get("violations");
assertNotNull(violations);
assertEquals(2, violations.size());
for (Map<String, Object> violation : violations) {
assertEquals("readApiTestViolations", violation.get("collection"));
assertEquals("shard1", violation.get("shard"));
assertEquals(Utils.makeMap("replica", "3", "delta", -1), violation.get("violation"));
assertNotNull(violation.get("clause"));
}
}
public static SolrRequest createAutoScalingRequest(SolrRequest.METHOD m, String message) {
return createAutoScalingRequest(m, null, message);
}
static SolrRequest createAutoScalingRequest(SolrRequest.METHOD m, String subPath, String message) {
boolean useV1 = random().nextBoolean();
String path = useV1 ? "/admin/autoscaling" : "/cluster/autoscaling";
path += subPath != null ? subPath : "";
return useV1
? new AutoScalingRequest(m, path, message)
: new V2Request.Builder(path).withMethod(m).withPayload(message).build();
}
static class AutoScalingRequest extends SolrRequest {
protected final String message;
AutoScalingRequest(METHOD m, String path, String message) {
super(m, path);
this.message = message;
}
@Override
public SolrParams getParams() {
return null;
}
@Override
public Collection<ContentStream> getContentStreams() throws IOException {
return message != null ? Collections.singletonList(new ContentStreamBase.StringStream(message)) : null;
}
@Override
protected SolrResponse createResponse(SolrClient client) {
return null;
}
}
}

View File

@ -0,0 +1,195 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.OverseerTaskProcessor;
import org.apache.solr.cloud.SolrCloudTestCase;
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.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.BeforeClass;
import org.junit.rules.ExpectedException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@LuceneTestCase.Slow
public class TestPolicyCloud extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@org.junit.Rule
public ExpectedException expectedException = ExpectedException.none();
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(5)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@After
public void after() throws Exception {
cluster.deleteAllCollections();
cluster.getSolrClient().getZkStateReader().getZkClient().setData(ZkStateReader.SOLR_AUTOSCALING_CONF_PATH,
"{}".getBytes(StandardCharsets.UTF_8), true);
}
public void testCreateCollectionAddReplica() throws Exception {
JettySolrRunner jetty = cluster.getRandomJetty(random());
int port = jetty.getLocalPort();
String commands = "{set-policy :{c1 : [{replica:2 , shard:'#EACH', port: '" + port + "'}]}}";
cluster.getSolrClient().request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
String collectionName = "testCreateCollectionAddReplica";
CollectionAdminRequest.createCollection(collectionName, 1, 1)
.setPolicy("c1")
.process(cluster.getSolrClient());
getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
CollectionAdminRequest.addReplicaToShard(collectionName, "shard1").process(cluster.getSolrClient());
waitForState("Timed out waiting to see 2 replicas for collection: " + collectionName,
collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 2);
getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
}
public void testCreateCollectionSplitShard() throws Exception {
JettySolrRunner firstNode = cluster.getRandomJetty(random());
int firstNodePort = firstNode.getLocalPort();
JettySolrRunner secondNode = null;
while (true) {
secondNode = cluster.getRandomJetty(random());
if (secondNode.getLocalPort() != firstNodePort) break;
}
int secondNodePort = secondNode.getLocalPort();
String commands = "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + firstNodePort + "'}, {replica:1, shard:'#EACH', port:'" + secondNodePort + "'}]}}";
NamedList<Object> response = cluster.getSolrClient().request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
assertEquals("success", response.get("result"));
String collectionName = "testCreateCollectionSplitShard";
CollectionAdminRequest.createCollection(collectionName, 1, 2)
.setPolicy("c1")
.setMaxShardsPerNode(10)
.process(cluster.getSolrClient());
DocCollection docCollection = getCollectionState(collectionName);
List<Replica> list = docCollection.getReplicas(firstNode.getNodeName());
int replicasOnNode1 = list != null ? list.size() : 0;
list = docCollection.getReplicas(secondNode.getNodeName());
int replicasOnNode2 = list != null ? list.size() : 0;
assertEquals("Expected exactly one replica of collection on node with port: " + firstNodePort, 1, replicasOnNode1);
assertEquals("Expected exactly one replica of collection on node with port: " + secondNodePort, 1, replicasOnNode2);
CollectionAdminRequest.splitShard(collectionName).setShardName("shard1").process(cluster.getSolrClient());
waitForState("Timed out waiting to see 6 replicas for collection: " + collectionName,
collectionName, (liveNodes, collectionState) -> collectionState.getReplicas().size() == 6);
docCollection = getCollectionState(collectionName);
list = docCollection.getReplicas(firstNode.getNodeName());
replicasOnNode1 = list != null ? list.size() : 0;
list = docCollection.getReplicas(secondNode.getNodeName());
replicasOnNode2 = list != null ? list.size() : 0;
assertEquals("Expected exactly three replica of collection on node with port: " + firstNodePort, 3, replicasOnNode1);
assertEquals("Expected exactly three replica of collection on node with port: " + secondNodePort, 3, replicasOnNode2);
}
public void testCreateCollectionAddShardUsingPolicy() throws Exception {
JettySolrRunner jetty = cluster.getRandomJetty(random());
int port = jetty.getLocalPort();
String commands = "{set-policy :{c1 : [{replica:1 , shard:'#EACH', port: '" + port + "'}]}}";
cluster.getSolrClient().request(AutoScalingHandlerTest.createAutoScalingRequest(SolrRequest.METHOD.POST, commands));
Map<String, Object> json = Utils.getJson(cluster.getZkClient(), ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, true);
assertEquals("full json:"+ Utils.toJSONString(json) , "#EACH",
Utils.getObjectByPath(json, true, "/policies/c1[0]/shard"));
CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", null, "s1,s2", 1)
.setPolicy("c1")
.process(cluster.getSolrClient());
DocCollection coll = getCollectionState("policiesTest");
assertEquals("c1", coll.getPolicyName());
assertEquals(2,coll.getReplicas().size());
coll.forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
CollectionAdminRequest.createShard("policiesTest", "s3").process(cluster.getSolrClient());
coll = getCollectionState("policiesTest");
assertEquals(1, coll.getSlice("s3").getReplicas().size());
coll.getSlice("s3").forEach(replica -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
}
public void testDataProvider() throws IOException, SolrServerException, KeeperException, InterruptedException {
CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2)
.process(cluster.getSolrClient());
DocCollection rulesCollection = getCollectionState("policiesTest");
SolrClientDataProvider provider = new SolrClientDataProvider(cluster.getSolrClient());
Map<String, Object> val = provider.getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
"freedisk",
"cores",
"heapUsage",
"sysLoadAvg"));
assertNotNull(val.get("freedisk"));
assertNotNull(val.get("heapUsage"));
assertNotNull(val.get("sysLoadAvg"));
assertTrue(((Number) val.get("cores")).intValue() > 0);
assertTrue("freedisk value is " + ((Number) val.get("freedisk")).doubleValue(), Double.compare(((Number) val.get("freedisk")).doubleValue(), 0.0d) > 0);
assertTrue("heapUsage value is " + ((Number) val.get("heapUsage")).doubleValue(), Double.compare(((Number) val.get("heapUsage")).doubleValue(), 0.0d) > 0);
assertTrue("sysLoadAvg value is " + ((Number) val.get("sysLoadAvg")).doubleValue(), Double.compare(((Number) val.get("sysLoadAvg")).doubleValue(), 0.0d) > 0);
String overseerNode = OverseerTaskProcessor.getLeaderNode(cluster.getZkClient());
cluster.getSolrClient().request(CollectionAdminRequest.addRole(overseerNode, "overseer"));
for (int i = 0; i < 10; i++) {
Map<String, Object> data = Utils.getJson(cluster.getZkClient(), ZkStateReader.ROLES, true);
if (i >= 9 && data.isEmpty()) {
throw new RuntimeException("NO overseer node created");
}
Thread.sleep(100);
}
val = provider.getNodeValues(overseerNode, Arrays.asList(
"nodeRole",
"ip_1", "ip_2", "ip_3", "ip_4",
"sysprop.java.version",
"sysprop.java.vendor"));
assertEquals("overseer", val.get("nodeRole"));
assertNotNull(val.get("ip_1"));
assertNotNull(val.get("ip_2"));
assertNotNull(val.get("ip_3"));
assertNotNull(val.get("ip_4"));
assertNotNull(val.get("sysprop.java.version"));
assertNotNull(val.get("sysprop.java.vendor"));
}
}

View File

@ -17,24 +17,28 @@
package org.apache.solr.cloud.rule;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import com.google.common.collect.Sets;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.cloud.rule.RemoteCallback;
import org.apache.solr.common.cloud.rule.SnitchContext;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.zookeeper.KeeperException;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.when;
public class ImplicitSnitchTest {
public class ImplicitSnitchTest extends LuceneTestCase {
private ImplicitSnitch snitch;
private SnitchContext context;
@ -186,4 +190,42 @@ public class ImplicitSnitchTest {
assertFalse(snitch.isKnownTag("ip_5"));
}
@Test
public void testExceptions() throws Exception {
ImplicitSnitch implicitSnitch = new ImplicitSnitch();
ServerSnitchContext noNodeExceptionSnitch = new ServerSnitchContext(null, null, new HashMap<>(), null) {
@Override
public Map getZkJson(String path) throws KeeperException, InterruptedException {
throw new KeeperException.NoNodeException();
}
};
implicitSnitch.getTags("", Collections.singleton(ImplicitSnitch.ROLE), noNodeExceptionSnitch);
Map map = (Map) noNodeExceptionSnitch.retrieve(ZkStateReader.ROLES); // todo it the key really supposed to /roles.json?
assertNotNull(map);
assertEquals(0, map.size());
implicitSnitch.getTags("", Collections.singleton(ImplicitSnitch.NODEROLE), noNodeExceptionSnitch);
map = (Map) noNodeExceptionSnitch.retrieve(ZkStateReader.ROLES); // todo it the key really supposed to /roles.json?
assertNotNull(map);
assertEquals(0, map.size());
ServerSnitchContext keeperExceptionSnitch = new ServerSnitchContext(null, null, new HashMap<>(), null) {
@Override
public Map getZkJson(String path) throws KeeperException, InterruptedException {
throw new KeeperException.ConnectionLossException();
}
};
expectThrows(SolrException.class, KeeperException.ConnectionLossException.class, () -> implicitSnitch.getTags("", Collections.singleton(ImplicitSnitch.ROLE), keeperExceptionSnitch));
expectThrows(SolrException.class, KeeperException.ConnectionLossException.class, () -> implicitSnitch.getTags("", Collections.singleton(ImplicitSnitch.NODEROLE), keeperExceptionSnitch));
ServerSnitchContext remoteExceptionSnitch = new ServerSnitchContext(null, null, new HashMap<>(), null) {
@Override
public void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) {
throw new RuntimeException();
}
};
expectThrows(SolrException.class, RuntimeException.class, () -> implicitSnitch.getTags("", Collections.singleton(ImplicitSnitch.CORES), remoteExceptionSnitch));
expectThrows(SolrException.class, RuntimeException.class, () -> implicitSnitch.getTags("", Collections.singleton(ImplicitSnitch.DISK), remoteExceptionSnitch));
expectThrows(SolrException.class, RuntimeException.class, () -> implicitSnitch.getTags("", Collections.singleton(ImplicitSnitch.SYSPROP + "xyz"), remoteExceptionSnitch));
}
}

View File

@ -58,7 +58,7 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.ToleratedUpdateError;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ClusterState.CollectionRef;
import org.apache.solr.common.cloud.CollectionStatePredicate;
import org.apache.solr.common.cloud.CollectionStateWatcher;
import org.apache.solr.common.cloud.DocCollection;
@ -83,8 +83,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;
import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
import static org.apache.solr.common.params.CommonParams.ID;
import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
/**
* SolrJ client class to communicate with SolrCloud.
@ -347,6 +347,7 @@ public class CloudSolrClient extends SolrClient {
public ZkStateReader getZkStateReader() {
if (stateProvider instanceof ZkClientClusterStateProvider) {
ZkClientClusterStateProvider provider = (ZkClientClusterStateProvider) stateProvider;
stateProvider.connect();
return provider.zkStateReader;
}
throw new IllegalStateException("This has no Zk stateReader");
@ -1210,7 +1211,7 @@ public class CloudSolrClient extends SolrClient {
&& !cacheEntry.shoulRetry()) return col;
}
ClusterState.CollectionRef ref = getCollectionRef(collection);
CollectionRef ref = getCollectionRef(collection);
if (ref == null) {
//no such collection exists
return null;
@ -1245,7 +1246,7 @@ public class CloudSolrClient extends SolrClient {
}
}
ClusterState.CollectionRef getCollectionRef(String collection) {
CollectionRef getCollectionRef(String collection) {
return stateProvider.getState(collection);
}

View File

@ -0,0 +1,256 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.client.solrj.impl;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.request.GenericSolrRequest;
import org.apache.solr.client.solrj.response.SimpleSolrResponse;
import org.apache.solr.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.cloud.autoscaling.Policy.ReplicaInfo;
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;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.cloud.rule.RemoteCallback;
import org.apache.solr.common.cloud.rule.SnitchContext;
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.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Class that implements {@link ClusterStateProvider} accepting a SolrClient
*/
public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
private final CloudSolrClient solrClient;
private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> data = new HashMap<>();
private Set<String> liveNodes;
private Map<String, Object> snitchSession = new HashMap<>();
private Map<String, Map> nodeVsTags = new HashMap<>();
public SolrClientDataProvider(CloudSolrClient solrClient) {
this.solrClient = solrClient;
ZkStateReader zkStateReader = solrClient.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
this.liveNodes = clusterState.getLiveNodes();
Map<String, ClusterState.CollectionRef> all = clusterState.getCollectionStates();
all.forEach((collName, ref) -> {
DocCollection coll = ref.get();
if (coll == null) return;
coll.forEachReplica((shard, replica) -> {
Map<String, Map<String, List<ReplicaInfo>>> nodeData = data.get(replica.getNodeName());
if (nodeData == null) data.put(replica.getNodeName(), nodeData = new HashMap<>());
Map<String, List<ReplicaInfo>> collData = nodeData.get(collName);
if (collData == null) nodeData.put(collName, collData = new HashMap<>());
List<ReplicaInfo> replicas = collData.get(shard);
if (replicas == null) collData.put(shard, replicas = new ArrayList<>());
replicas.add(new ReplicaInfo(replica.getName(), collName, shard, new HashMap<>()));
});
});
}
@Override
public String getPolicyNameByCollection(String coll) {
ClusterState.CollectionRef state = solrClient.getClusterStateProvider().getState(coll);
return state == null || state.get() == null ? null : (String) state.get().getProperties().get("policy");
}
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
AutoScalingSnitch snitch = new AutoScalingSnitch();
ClientSnitchCtx ctx = new ClientSnitchCtx(null, node, snitchSession, solrClient);
snitch.getTags(node, new HashSet<>(tags), ctx);
nodeVsTags.put(node, ctx.getTags());
return ctx.getTags();
}
@Override
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return data.getOrDefault(node, Collections.emptyMap());//todo fill other details
}
@Override
public Collection<String> getNodes() {
return liveNodes;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put("liveNodes", liveNodes);
ew.put("replicaInfo", Utils.getDeepCopy(data, 5));
ew.put("nodeValues", nodeVsTags);
}
static class ClientSnitchCtx
extends SnitchContext {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
ZkClientClusterStateProvider zkClientClusterStateProvider;
CloudSolrClient solrClient;
public ClientSnitchCtx(SnitchInfo perSnitch,
String node, Map<String, Object> session,
CloudSolrClient solrClient) {
super(perSnitch, node, session);
this.solrClient = solrClient;
this.zkClientClusterStateProvider = (ZkClientClusterStateProvider) solrClient.getClusterStateProvider();
}
@Override
public Map getZkJson(String path) throws KeeperException, InterruptedException {
return Utils.getJson(zkClientClusterStateProvider.getZkStateReader().getZkClient(), path, true);
}
public void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) {
}
public SimpleSolrResponse invoke(String solrNode, String path, SolrParams params)
throws IOException, SolrServerException {
String url = zkClientClusterStateProvider.getZkStateReader().getBaseUrlForNodeName(solrNode);
GenericSolrRequest request = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
try (HttpSolrClient client = new HttpSolrClient.Builder()
.withHttpClient(solrClient.getHttpClient())
.withBaseSolrUrl(url)
.withResponseParser(new BinaryResponseParser())
.build()) {
NamedList<Object> rsp = client.request(request);
request.response.nl = rsp;
return request.response;
}
}
}
//uses metrics API to get node information
static class AutoScalingSnitch extends ImplicitSnitch {
@Override
protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
ClientSnitchCtx snitchContext = (ClientSnitchCtx) ctx;
readSysProps(solrNode, requestedTags, snitchContext);
Set<String> groups = new HashSet<>();
List<String> prefixes = new ArrayList<>();
if (requestedTags.contains(DISK)) {
groups.add("solr.node");
prefixes.add("CONTAINER.fs.usableSpace");
}
if (requestedTags.contains(CORES)) {
groups.add("solr.core");
prefixes.add("CORE.coreName");
}
if (requestedTags.contains(SYSLOADAVG)) {
groups.add("solr.jvm");
prefixes.add("os.systemLoadAverage");
}
if (requestedTags.contains(HEAPUSAGE)) {
groups.add("solr.jvm");
prefixes.add("memory.heap.usage");
}
if (groups.isEmpty() || prefixes.isEmpty()) return;
ModifiableSolrParams params = new ModifiableSolrParams();
params.add("group", StrUtils.join(groups, ','));
params.add("prefix", StrUtils.join(prefixes, ','));
try {
SimpleSolrResponse rsp = snitchContext.invoke(solrNode, CommonParams.METRICS_PATH, params);
Map m = rsp.nl.asMap(4);
if (requestedTags.contains(DISK)) {
Number n = (Number) Utils.getObjectByPath(m, true, "metrics/solr.node/CONTAINER.fs.usableSpace");
if (n != null) ctx.getTags().put(DISK, n.doubleValue() / 1024.0d / 1024.0d / 1024.0d);
}
if (requestedTags.contains(CORES)) {
int count = 0;
Map cores = (Map) m.get("metrics");
for (Object o : cores.keySet()) {
if (o.toString().startsWith("solr.core.")) count++;
}
ctx.getTags().put(CORES, count);
}
if (requestedTags.contains(SYSLOADAVG)) {
Number n = (Number) Utils.getObjectByPath(m, true, "metrics/solr.jvm/os.systemLoadAverage");
if (n != null) ctx.getTags().put(SYSLOADAVG, n.doubleValue() * 100.0d);
}
if (requestedTags.contains(HEAPUSAGE)) {
Number n = (Number) Utils.getObjectByPath(m, true, "metrics/solr.jvm/memory.heap.usage");
if (n != null) ctx.getTags().put(HEAPUSAGE, n.doubleValue() * 100.0d);
}
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
private void readSysProps(String solrNode, Set<String> requestedTags, ClientSnitchCtx snitchContext) {
List<String> prefixes = null;
ModifiableSolrParams params;
List<String> sysProp = null;
for (String tag : requestedTags) {
if (!tag.startsWith(SYSPROP)) continue;
if (sysProp == null) {
prefixes = new ArrayList<>();
sysProp = new ArrayList<>();
prefixes.add("system.properties");
}
sysProp.add(tag.substring(SYSPROP.length()));
}
if (sysProp == null) return;
params = new ModifiableSolrParams();
params.add("prefix", StrUtils.join(prefixes, ','));
for (String s : sysProp) params.add("property", s);
try {
SimpleSolrResponse rsp = snitchContext.invoke(solrNode, CommonParams.METRICS_PATH, params);
Map m = rsp.nl.asMap(6);
for (String s : sysProp) {
Object v = Utils.getObjectByPath(m, true,
Arrays.asList("metrics", "solr.jvm", "system.properties", s));
if (v != null) snitchContext.getTags().put("sysprop." + s, v);
}
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
}
}
}
}

View File

@ -39,10 +39,16 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
ZkStateReader zkStateReader;
private boolean closeZkStateReader = true;
String zkHost;
int zkConnectTimeout = 10000;
int zkClientTimeout = 10000;
public ZkClientClusterStateProvider(ZkStateReader zkStateReader) {
this.zkStateReader = zkStateReader;
this.closeZkStateReader = false;
}
public ZkClientClusterStateProvider(Collection<String> zkHosts, String chroot) {
zkHost = buildZkHostString(zkHosts,chroot);
}
@ -55,6 +61,9 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
public ClusterState.CollectionRef getState(String collection) {
return zkStateReader.getClusterState().getCollectionRef(collection);
}
public ZkStateReader getZkStateReader(){
return zkStateReader;
}
@Override
public Set<String> liveNodes() {
@ -151,7 +160,7 @@ public class ZkClientClusterStateProvider implements ClusterStateProvider {
@Override
public void close() throws IOException {
if (zkStateReader != null) {
if (zkStateReader != null && closeZkStateReader) {
synchronized (this) {
if (zkStateReader != null)
zkStateReader.close();

View File

@ -16,8 +16,6 @@
*/
package org.apache.solr.client.solrj.request;
import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
@ -47,6 +45,8 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ContentStream;
import org.apache.solr.common.util.NamedList;
import static org.apache.solr.cloud.autoscaling.Policy.POLICY;
import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_PARAM;
import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM;
@ -344,6 +344,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
protected String configName = null;
protected String createNodeSet = null;
protected String routerName;
protected String policy;
protected String shards;
protected String routerField;
protected Integer numShards;
@ -492,9 +493,14 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
}
if(rule != null) params.set("rule", rule);
if(snitch != null) params.set("snitch", snitch);
params.setNonNull(POLICY, policy);
return params;
}
public Create setPolicy(String policy) {
this.policy = policy;
return this;
}
}
/**

View File

@ -102,7 +102,9 @@ public class V2Request extends SolrRequest {
* @return builder object
*/
public Builder withPayload(String payload) {
if (payload != null) {
this.payload = new ByteArrayInputStream(payload.getBytes(StandardCharsets.UTF_8));
}
return this;
}

View File

@ -0,0 +1,69 @@
/*
* 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.List;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.autoscaling.Policy.Suggester;
class AddReplicaSuggester extends Suggester {
SolrRequest init() {
SolrRequest operation = tryEachNode(true);
if (operation == null) operation = tryEachNode(false);
return operation;
}
SolrRequest tryEachNode(boolean strict) {
String coll = (String) hints.get(Hint.COLL);
String shard = (String) hints.get(Hint.SHARD);
if (coll == null || shard == null)
throw new RuntimeException("add-replica requires 'collection' and 'shard'");
//iterate through elements and identify the least loaded
List<Clause.Violation> leastSeriousViolation = null;
Integer targetNodeIndex = null;
for (int i = getMatrix().size() - 1; i >= 0; i--) {
Row row = getMatrix().get(i);
if (!isAllowed(row.node, Hint.TARGET_NODE)) continue;
Row tmpRow = row.addReplica(coll, shard);
tmpRow.violations.clear();
List<Clause.Violation> errs = testChangedMatrix(strict, getModifiedMatrix(getMatrix(), tmpRow, i));
if(!containsNewErrors(errs)) {
if(isLessSerious(errs, leastSeriousViolation)){
leastSeriousViolation = errs;
targetNodeIndex = i;
}
}
}
if (targetNodeIndex != null) {// there are no rule violations
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(coll, shard));
return CollectionAdminRequest
.addReplicaToShard(coll, shard)
.setNode(getMatrix().get(targetNodeIndex).node);
}
return null;
}
}

View File

@ -0,0 +1,57 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.util.HashMap;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.util.Utils;
class Cell implements MapWriter {
final int index;
final String name;
Object val, approxVal;
Cell(int index, String name, Object val) {
this.index = index;
this.name = name;
this.val = val;
}
Cell(int index, String name, Object val, Object approxVal) {
this.index = index;
this.name = name;
this.val = val;
this.approxVal = approxVal;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put(name, val);
}
@Override
public String toString() {
return Utils.toJSONString(this.toMap(new HashMap<>()));
}
public Cell copy() {
return new Cell(index, name, val, approxVal);
}
}

View File

@ -0,0 +1,460 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
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.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.cloud.autoscaling.Policy.ReplicaInfo;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import static java.util.Collections.singletonMap;
import static org.apache.solr.cloud.autoscaling.Clause.TestStatus.PASS;
import static org.apache.solr.cloud.autoscaling.Operand.EQUAL;
import static org.apache.solr.cloud.autoscaling.Operand.GREATER_THAN;
import static org.apache.solr.cloud.autoscaling.Operand.LESS_THAN;
import static org.apache.solr.cloud.autoscaling.Operand.NOT_EQUAL;
import static org.apache.solr.cloud.autoscaling.Operand.WILDCARD;
import static org.apache.solr.cloud.autoscaling.Policy.ANY;
import static org.apache.solr.common.params.CoreAdminParams.COLLECTION;
import static org.apache.solr.common.params.CoreAdminParams.REPLICA;
import static org.apache.solr.common.params.CoreAdminParams.SHARD;
// a set of conditions in a policy
public class Clause implements MapWriter, Comparable<Clause> {
Map<String, Object> original;
Condition collection, shard, replica, tag, globalTag;
boolean strict = true;
Clause(Map<String, Object> m) {
this.original = m;
strict = Boolean.parseBoolean(String.valueOf(m.getOrDefault("strict", "true")));
Optional<String> globalTagName = m.keySet().stream().filter(Policy.GLOBAL_ONLY_TAGS::contains).findFirst();
if (globalTagName.isPresent()) {
globalTag = parse(globalTagName.get(), m);
if (m.size() > 2) {
throw new RuntimeException("Only one extra tag supported for the tag " + globalTagName.get() + " in " + Utils.toJSONString(m));
}
tag = parse(m.keySet().stream()
.filter(s -> (!globalTagName.get().equals(s) && !IGNORE_TAGS.contains(s)))
.findFirst().get(), m);
} else {
collection = parse(COLLECTION, m);
shard = parse(SHARD, m);
if(m.get(REPLICA) == null){
throw new RuntimeException(StrUtils.formatString("'replica' is required in {0}", Utils.toJSONString(m)));
}
this.replica = parse(REPLICA, m);
if (replica.op == WILDCARD) throw new RuntimeException("replica val cannot be null" + Utils.toJSONString(m));
m.forEach((s, o) -> parseCondition(s, o));
}
if (tag == null)
throw new RuntimeException("Invalid op, must have one and only one tag other than collection, shard,replica " + Utils.toJSONString(m));
}
public boolean doesOverride(Clause that) {
return (collection.equals(that.collection) &&
tag.name.equals(that.tag.name));
}
public boolean isPerCollectiontag() {
return globalTag == null;
}
void parseCondition(String s, Object o) {
if (IGNORE_TAGS.contains(s)) return;
if (tag != null) {
throw new IllegalArgumentException("Only one tag other than collection, shard, replica is possible");
}
tag = parse(s, singletonMap(s, o));
}
@Override
public int compareTo(Clause that) {
try {
int v = Integer.compare(this.tag.op.priority, that.tag.op.priority);
if (v != 0) return v;
if (this.isPerCollectiontag() && that.isPerCollectiontag()) {
v = Integer.compare(this.replica.op.priority, that.replica.op.priority);
if (v == 0) {
v = Long.compare((Long) this.replica.val, (Long) that.replica.val);
v = this.replica.op == LESS_THAN ? v : v * -1;
}
return v;
} else {
return 0;
}
} catch (NullPointerException e) {
throw e;
}
}
void addTags(List<String> params) {
if (globalTag != null && !params.contains(globalTag.name)) params.add(globalTag.name);
if (tag != null && !params.contains(tag.name)) params.add(tag.name);
}
static class Condition {
final String name;
final Object val;
final Operand op;
Condition(String name, Object val, Operand op) {
this.name = name;
this.val = val;
this.op = op;
}
TestStatus match(Row row) {
return op.match(val, row.getVal(name));
}
TestStatus match(Object testVal) {
return op.match(this.val, testVal);
}
boolean isPass(Object inputVal) {
return op.match(val, validate(name, inputVal, false)) == PASS;
}
boolean isPass(Row row) {
return op.match(val, row.getVal(name)) == PASS;
}
@Override
public boolean equals(Object that) {
if (that instanceof Condition) {
Condition c = (Condition) that;
return Objects.equals(c.name, name) && Objects.equals(c.val, val) && c.op == op;
}
return false;
}
public Integer delta(Object val) {
return op.delta(this.val, val);
}
}
static Condition parse(String s, Map m) {
Object expectedVal = null;
Object val = m.get(s);
try {
String conditionName = s.trim();
Operand operand = null;
if (val == null) {
operand = WILDCARD;
expectedVal = Policy.ANY;
} else if (val instanceof String) {
String strVal = ((String) val).trim();
if (Policy.ANY.equals(strVal) || Policy.EACH.equals(strVal)) operand = WILDCARD;
else if (strVal.startsWith(NOT_EQUAL.operand)) operand = NOT_EQUAL;
else if (strVal.startsWith(GREATER_THAN.operand)) operand = GREATER_THAN;
else if (strVal.startsWith(LESS_THAN.operand)) operand = LESS_THAN;
else operand = EQUAL;
expectedVal = validate(s, strVal.substring(EQUAL == operand || WILDCARD == operand ? 0 : 1), true);
} else if (val instanceof Number) {
operand = EQUAL;
expectedVal = validate(s, val, true);
}
return new Condition(conditionName, expectedVal, operand);
} catch (Exception e) {
throw new IllegalArgumentException("Invalid tag : " + s + ":" + val, e);
}
}
public class Violation implements MapWriter {
final String shard, coll, node;
final Object actualVal;
final Integer delta;//how far is the actual value from the expected value
final Object tagKey;
private final int hash;
private Violation(String coll, String shard, String node, Object actualVal, Integer delta, Object tagKey) {
this.shard = shard;
this.coll = coll;
this.node = node;
this.delta = delta;
this.actualVal = actualVal;
this.tagKey = tagKey;
hash = ("" + coll + " " + shard + " " + node + " " + String.valueOf(tagKey) + " " + Utils.toJSONString(getClause().toMap(new HashMap<>()))).hashCode();
}
public Clause getClause() {
return Clause.this;
}
@Override
public int hashCode() {
return hash;
}
//if the delta is lower , this violation is less serious
public boolean isLessSerious(Violation that) {
return that.delta != null && delta != null &&
Math.abs(delta) < Math.abs(that.delta);
}
@Override
public boolean equals(Object that) {
if (that instanceof Violation) {
Violation v = (Violation) that;
return Objects.equals(this.shard, v.shard) &&
Objects.equals(this.coll, v.coll) &&
Objects.equals(this.node, v.node) &&
Objects.equals(this.tagKey, v.tagKey)
;
}
return false;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.putIfNotNull("collection", coll);
ew.putIfNotNull("shard", shard);
ew.putIfNotNull("node", node);
ew.putIfNotNull("tagKey", String.valueOf(tagKey));
ew.putIfNotNull("violation", (MapWriter) ew1 -> {
ew1.put(getClause().isPerCollectiontag() ? "replica" : tag.name,
String.valueOf(actualVal));
ew1.putIfNotNull("delta", delta);
});
ew.put("clause", getClause());
}
}
public List<Violation> test(List<Row> allRows) {
List<Violation> violations = new ArrayList<>();
if (isPerCollectiontag()) {
Map<String, Map<String, Map<String, AtomicInteger>>> replicaCount = computeReplicaCounts(allRows);
for (Map.Entry<String, Map<String, Map<String, AtomicInteger>>> e : replicaCount.entrySet()) {
if (!collection.isPass(e.getKey())) continue;
for (Map.Entry<String, Map<String, AtomicInteger>> shardVsCount : e.getValue().entrySet()) {
if (!shard.isPass(shardVsCount.getKey())) continue;
for (Map.Entry<String, AtomicInteger> counts : shardVsCount.getValue().entrySet()) {
if (!replica.isPass(counts.getValue())) {
violations.add(new Violation(
e.getKey(),
shardVsCount.getKey(),
tag.name.equals("node") ? counts.getKey() : null,
counts.getValue(),
replica.delta(counts.getValue()),
counts.getKey()
));
}
}
}
}
} else {
for (Row r : allRows) {
if (!tag.isPass(r)) {
violations.add(new Violation(null, null, r.node, r.getVal(tag.name), tag.delta(r.getVal(tag.name)), null));
}
}
}
return violations;
}
private Map<String, Map<String, Map<String, AtomicInteger>>> computeReplicaCounts(List<Row> allRows) {
Map<String, Map<String, Map<String, AtomicInteger>>> collVsShardVsTagVsCount = new HashMap<>();
for (Row row : allRows)
for (Map.Entry<String, Map<String, List<ReplicaInfo>>> colls : row.collectionVsShardVsReplicas.entrySet()) {
String collectionName = colls.getKey();
if (!collection.isPass(collectionName)) continue;
collVsShardVsTagVsCount.putIfAbsent(collectionName, new HashMap<>());
Map<String, Map<String, AtomicInteger>> collMap = collVsShardVsTagVsCount.get(collectionName);
for (Map.Entry<String, List<ReplicaInfo>> shards : colls.getValue().entrySet()) {
String shardName = shards.getKey();
if (ANY.equals(shard.val)) shardName = ANY;
if (!shard.isPass(shardName)) break;
collMap.putIfAbsent(shardName, new HashMap<>());
Map<String, AtomicInteger> tagVsCount = collMap.get(shardName);
Object tagVal = row.getVal(tag.name);
tagVsCount.putIfAbsent(tag.isPass(tagVal) ? String.valueOf(tagVal) : "", new AtomicInteger());
if (tag.isPass(tagVal)) {
tagVsCount.get(String.valueOf(tagVal)).addAndGet(shards.getValue().size());
}
}
}
return collVsShardVsTagVsCount;
}
public boolean isStrict() {
return strict;
}
@Override
public String toString() {
return Utils.toJSONString(original);
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
for (Map.Entry<String, Object> e : original.entrySet()) ew.put(e.getKey(), e.getValue());
}
enum TestStatus {
NOT_APPLICABLE, FAIL, PASS
}
private static final Set<String> IGNORE_TAGS = new HashSet<>(Arrays.asList(REPLICA, COLLECTION, SHARD, "strict"));
static class ValidateInfo {
final Class type;
final Set<String> vals;
final Number min;
final Number max;
ValidateInfo(Class type, Set<String> vals, Number min, Number max) {
this.type = type;
this.vals = vals;
this.min = min;
if(min != null && !type.isInstance(min)) throw new RuntimeException("wrong min value type, expected: " + type.getName() + " actual: " + min.getClass().getName());
this.max = max;
if(max != null && !type.isInstance(max)) throw new RuntimeException("wrong max value type, expected: " + type.getName() + " actual: " + max.getClass().getName());
}
}
/**
*
* @param name name of the condition
* @param val value of the condition
* @param isRuleVal is this provided in the rule
* @return actual validated value
*/
public static Object validate(String name, Object val, boolean isRuleVal) {
if (val == null) return null;
ValidateInfo info = validatetypes.get(name);
if (info == null && name.startsWith(ImplicitSnitch.SYSPROP)) info = validatetypes.get("STRING");
if (info == null) throw new RuntimeException("Unknown type :" + name);
if (info.type == Double.class) {
Double num = parseDouble(name, val);
if (isRuleVal) {
if (info.min != null)
if (Double.compare(num, (Double) info.min) == -1)
throw new RuntimeException(name + ": " + val + " must be greater than " + info.min);
if (info.max != null)
if (Double.compare(num, (Double) info.max) == 1)
throw new RuntimeException(name + ": " + val + " must be less than " + info.max);
}
return num;
} else if (info.type == Long.class) {
Long num = parseLong(name, val);
if (isRuleVal) {
if (info.min != null)
if (num < info.min.longValue())
throw new RuntimeException(name + ": " + val + " must be greater than " + info.min);
if (info.max != null)
if (num > info.max.longValue())
throw new RuntimeException(name + ": " + val + " must be less than " + info.max);
}
return num;
} else if (info.type == String.class) {
if (isRuleVal && info.vals != null && !info.vals.contains(val))
throw new RuntimeException(name + ": " + val + " must be one of " + StrUtils.join(info.vals, ','));
return val;
} else {
throw new RuntimeException("Invalid type ");
}
}
public static Long parseLong(String name, Object val) {
if (val == null) return null;
if (val instanceof Long) return (Long) val;
Number num = null;
if (val instanceof String) {
try {
num = Long.parseLong(((String) val).trim());
} catch (NumberFormatException e) {
try {
num = Double.parseDouble((String) val);
} catch (NumberFormatException e1) {
throw new RuntimeException(name + ": " + val + "not a valid number", e);
}
}
} else if (val instanceof Number) {
num = (Number) val;
}
if (num != null) {
return num.longValue();
}
throw new RuntimeException(name + ": " + val + "not a valid number");
}
public static Double parseDouble(String name, Object val) {
if (val == null) return null;
if (val instanceof Double) return (Double) val;
Number num = null;
if (val instanceof String) {
try {
num = Double.parseDouble((String) val);
} catch (NumberFormatException e) {
throw new RuntimeException(name + ": " + val + "not a valid number", e);
}
} else if (val instanceof Number) {
num = (Number) val;
}
if (num != null) {
return num.doubleValue();
}
throw new RuntimeException(name + ": " + val + "not a valid number");
}
private static final Map<String, ValidateInfo> validatetypes = new HashMap<>();
static {
validatetypes.put("collection", new ValidateInfo(String.class, null, null, null));
validatetypes.put("shard", new ValidateInfo(String.class, null, null, null));
validatetypes.put("replica", new ValidateInfo(Long.class, null, 0L, null));
validatetypes.put(ImplicitSnitch.PORT, new ValidateInfo(Long.class, null, 1L, 65535L));
validatetypes.put(ImplicitSnitch.DISK, new ValidateInfo(Double.class, null, 0d, Double.MAX_VALUE));
validatetypes.put(ImplicitSnitch.NODEROLE, new ValidateInfo(String.class, Collections.singleton("overseer"), null, null));
validatetypes.put(ImplicitSnitch.CORES, new ValidateInfo(Long.class, null, 0L, Long.MAX_VALUE));
validatetypes.put(ImplicitSnitch.SYSLOADAVG, new ValidateInfo(Double.class, null, 0d, 100d));
validatetypes.put(ImplicitSnitch.HEAPUSAGE, new ValidateInfo(Double.class, null, 0d, null));
validatetypes.put("NUMBER", new ValidateInfo(Long.class, null, 0L, Long.MAX_VALUE));//generic number validation
validatetypes.put("STRING", new ValidateInfo(String.class, null, null, null));//generic string validation
validatetypes.put("node", new ValidateInfo(String.class, null, null, null));
for (String ip : ImplicitSnitch.IP_SNITCHES) validatetypes.put(ip, new ValidateInfo(Long.class, null, 0L, 255L));
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.Collection;
import java.util.List;
import java.util.Map;
public interface ClusterDataProvider extends Closeable {
/**Get the value of each tag for a given node
*
* @param node node name
* @param tags tag names
* @return a map of tag vs value
*/
Map<String, Object> getNodeValues(String node, Collection<String> tags);
/**
* Get the details of each replica in a node. It attempts to fetch as much details about
* the replica as mentioned in the keys list. It is not necessary to give al details
* <p>
* the format is {collection:shard :[{replicadetails}]}
*/
Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys);
Collection<String> getNodes();
/**Get the collection-specific policy
*/
String getPolicyNameByCollection(String coll);
@Override
default void close() throws IOException {
}
}

View File

@ -0,0 +1,83 @@
/*
* 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.List;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.autoscaling.Clause.Violation;
import org.apache.solr.cloud.autoscaling.Policy.ReplicaInfo;
import org.apache.solr.cloud.autoscaling.Policy.Suggester;
import org.apache.solr.common.util.Pair;
public class MoveReplicaSuggester extends Suggester {
@Override
SolrRequest init() {
SolrRequest operation = tryEachNode(true);
if (operation == null) operation = tryEachNode(false);
return operation;
}
SolrRequest tryEachNode(boolean strict) {
//iterate through elements and identify the least loaded
List<Clause.Violation> leastSeriousViolation = null;
Integer targetNodeIndex = null;
Integer fromNodeIndex = null;
ReplicaInfo fromReplicaInfo = null;
for (Pair<ReplicaInfo, Row> fromReplica : getValidReplicas(true, true, -1)) {
Row fromRow = fromReplica.second();
ReplicaInfo replicaInfo = fromReplica.first();
String coll = replicaInfo.collection;
String shard = replicaInfo.shard;
Pair<Row, ReplicaInfo> pair = fromRow.removeReplica(coll, shard);
Row tmpRow = pair.first();
if (tmpRow == null) {
//no such replica available
continue;
}
tmpRow.violations.clear();
final int i = getMatrix().indexOf(fromRow);
for (int j = getMatrix().size() - 1; j > i; j--) {
Row targetRow = getMatrix().get(j);
if (!isAllowed(targetRow.node, Hint.TARGET_NODE)) continue;
targetRow = targetRow.addReplica(coll, shard);
targetRow.violations.clear();
List<Violation> errs = testChangedMatrix(strict, getModifiedMatrix(getModifiedMatrix(getMatrix(), tmpRow, i), targetRow, j));
if (!containsNewErrors(errs) && isLessSerious(errs, leastSeriousViolation)) {
leastSeriousViolation = errs;
targetNodeIndex = j;
fromNodeIndex = i;
fromReplicaInfo = replicaInfo;
}
}
}
if (targetNodeIndex != null && fromNodeIndex != null) {
getMatrix().set(fromNodeIndex, getMatrix().get(fromNodeIndex).removeReplica(fromReplicaInfo.collection, fromReplicaInfo.shard).first());
getMatrix().set(targetNodeIndex, getMatrix().get(targetNodeIndex).addReplica(fromReplicaInfo.collection, fromReplicaInfo.shard));
return new CollectionAdminRequest.MoveReplica(
fromReplicaInfo.collection,
fromReplicaInfo.name,
getMatrix().get(targetNodeIndex).node);
}
return null;
}
}

View File

@ -0,0 +1,123 @@
/*
* 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.Objects;
import org.apache.solr.cloud.autoscaling.Clause.TestStatus;
import static org.apache.solr.cloud.autoscaling.Clause.TestStatus.FAIL;
import static org.apache.solr.cloud.autoscaling.Clause.TestStatus.NOT_APPLICABLE;
import static org.apache.solr.cloud.autoscaling.Clause.TestStatus.PASS;
import static org.apache.solr.cloud.autoscaling.Policy.ANY;
public enum Operand {
WILDCARD(ANY, Integer.MAX_VALUE) {
@Override
public TestStatus match(Object ruleVal, Object testVal) {
return testVal == null ? NOT_APPLICABLE : PASS;
}
},
EQUAL("", 0) {
@Override
public int _delta(int expected, int actual) {
return expected - actual;
}
},
NOT_EQUAL("!", 2) {
@Override
public TestStatus match(Object ruleVal, Object testVal) {
return super.match(ruleVal, testVal) == PASS ? FAIL : PASS;
}
@Override
public int _delta(int expected, int actual) {
return expected - actual;
}
},
GREATER_THAN(">", 1) {
@Override
public TestStatus match(Object ruleVal, Object testVal) {
if (testVal == null) return NOT_APPLICABLE;
if (ruleVal instanceof Double) {
return Double.compare(Clause.parseDouble("", testVal), (Double) ruleVal) == 1 ? PASS : FAIL;
}
return getLong(testVal) > getLong(ruleVal) ? PASS: FAIL ;
}
@Override
protected int _delta(int expected, int actual) {
return actual > expected ? 0 : (expected + 1) - actual;
}
},
LESS_THAN("<", 2) {
@Override
public TestStatus match(Object ruleVal, Object testVal) {
if (testVal == null) return NOT_APPLICABLE;
if (ruleVal instanceof Double) {
return Double.compare(Clause.parseDouble("", testVal), (Double) ruleVal) == -1 ? PASS : FAIL;
}
return getLong(testVal) < getLong(ruleVal) ? PASS: FAIL ;
}
@Override
protected int _delta(int expected, int actual) {
return actual < expected ? 0 : (expected ) - actual;
}
};
public final String operand;
final int priority;
Operand(String val, int priority) {
this.operand = val;
this.priority = priority;
}
public String toStr(Object expectedVal) {
return operand + expectedVal.toString();
}
public TestStatus match(Object ruleVal, Object testVal) {
return Objects.equals(ruleVal, testVal) ? PASS : FAIL;
}
Long getLong(Object o) {
if (o instanceof Long) return (Long) o;
if(o instanceof Number ) return ((Number) o).longValue();
return Long.parseLong(String.valueOf(o));
}
public Integer delta(Object expected, Object actual) {
try {
Integer expectedInt = Integer.parseInt(String.valueOf(expected));
Integer actualInt = Integer.parseInt(String.valueOf(actual));
return _delta(expectedInt, actualInt);
} catch (Exception e) {
return null;
}
}
protected int _delta(int expected, int actual) {
return 0;
}
}

View File

@ -0,0 +1,519 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumMap;
import java.util.HashMap;
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 java.util.SortedSet;
import java.util.TreeSet;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.cloud.autoscaling.Clause.Violation;
import org.apache.solr.common.IteratorWriter;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.stream.Collectors.toList;
/*The class that reads, parses and applies policies specified in
* autoscaling.json
*
* Create one instance of this class per unique autoscaling.json.
* This is immutable and is thread-safe
*
* Create a fresh new session for each use
*
*/
public class Policy implements MapWriter {
public static final String POLICY = "policy";
public static final String EACH = "#EACH";
public static final String ANY = "#ANY";
public static final String CLUSTER_POLICY = "cluster-policy";
public static final String CLUSTER_PREFERENCE = "cluster-preferences";
public static final Set<String> GLOBAL_ONLY_TAGS = Collections.singleton("cores");
final Map<String, List<Clause>> policies = new HashMap<>();
final List<Clause> clusterPolicy;
final List<Preference> clusterPreferences;
final List<String> params;
public Policy(Map<String, Object> jsonMap) {
clusterPreferences = ((List<Map<String, Object>>) jsonMap.getOrDefault(CLUSTER_PREFERENCE, emptyList())).stream()
.map(Preference::new)
.collect(toList());
for (int i = 0; i < clusterPreferences.size() - 1; i++) {
Preference preference = clusterPreferences.get(i);
preference.next = clusterPreferences.get(i + 1);
}
if (clusterPreferences.isEmpty()) {
clusterPreferences.add(new Preference((Map<String, Object>) Utils.fromJSONString("{minimize : cores, precision:1}")));
}
SortedSet<String> paramsOfInterest = new TreeSet<>();
for (Preference preference : clusterPreferences) {
if (paramsOfInterest.contains(preference.name.name())) {
throw new RuntimeException(preference.name + " is repeated");
}
paramsOfInterest.add(preference.name.toString());
}
this.params = new ArrayList<>(paramsOfInterest);
clusterPolicy = ((List<Map<String, Object>>) jsonMap.getOrDefault(CLUSTER_POLICY, emptyList())).stream()
.map(Clause::new)
.filter(clause -> {
clause.addTags(params);
return true;
})
.collect(Collectors.toList());
((Map<String, List<Map<String, Object>>>) jsonMap.getOrDefault("policies", emptyMap())).forEach((s, l1) ->
this.policies.put(s, l1.stream()
.map(Clause::new)
.filter(clause -> {
if (!clause.isPerCollectiontag())
throw new RuntimeException(clause.globalTag.name + " is only allowed in 'cluster-policy'");
clause.addTags(params);
return true;
})
.sorted()
.collect(toList())));
}
public List<Clause> getClusterPolicy() {
return clusterPolicy;
}
public List<Preference> getClusterPreferences() {
return clusterPreferences;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
if (!policies.isEmpty()) {
ew.put("policies", (MapWriter) ew1 -> {
for (Map.Entry<String, List<Clause>> e : policies.entrySet()) {
ew1.put(e.getKey(), e.getValue());
}
});
}
if (!clusterPreferences.isEmpty()) {
ew.put("preferences", (IteratorWriter) iw -> {
for (Preference p : clusterPreferences) iw.add(p);
});
}
}
/*This stores the logical state of the system, given a policy and
* a cluster state.
*
*/
public class Session implements MapWriter {
final List<String> nodes;
final ClusterDataProvider dataProvider;
final List<Row> matrix;
Set<String> collections = new HashSet<>();
List<Clause> expandedClauses;
List<Violation> violations = new ArrayList<>();
private Session(List<String> nodes, ClusterDataProvider dataProvider,
List<Row> matrix, List<Clause> expandedClauses) {
this.nodes = nodes;
this.dataProvider = dataProvider;
this.matrix = matrix;
this.expandedClauses = expandedClauses;
}
Session(ClusterDataProvider dataProvider) {
this.nodes = new ArrayList<>(dataProvider.getNodes());
this.dataProvider = dataProvider;
for (String node : nodes) {
collections.addAll(dataProvider.getReplicaInfo(node, Collections.emptyList()).keySet());
}
expandedClauses = clusterPolicy.stream()
.filter(clause -> !clause.isPerCollectiontag())
.collect(Collectors.toList());
for (String c : collections) {
addClausesForCollection(dataProvider, c);
}
Collections.sort(expandedClauses);
matrix = new ArrayList<>(nodes.size());
for (String node : nodes) matrix.add(new Row(node, params, dataProvider));
applyRules();
}
private void addClausesForCollection(ClusterDataProvider dataProvider, String c) {
String p = dataProvider.getPolicyNameByCollection(c);
if (p != null) {
List<Clause> perCollPolicy = policies.get(p);
if (perCollPolicy == null)
throw new RuntimeException(StrUtils.formatString("Policy for collection {0} is {1} . It does not exist", c, p));
}
expandedClauses.addAll(mergePolicies(c, policies.getOrDefault(p, emptyList()), clusterPolicy));
}
Session copy() {
return new Session(nodes, dataProvider, getMatrixCopy(), expandedClauses);
}
List<Row> getMatrixCopy() {
return matrix.stream()
.map(Row::copy)
.collect(Collectors.toList());
}
Policy getPolicy() {
return Policy.this;
}
/**
* Apply the preferences and conditions
*/
private void applyRules() {
if (!clusterPreferences.isEmpty()) {
//this is to set the approximate value according to the precision
ArrayList<Row> tmpMatrix = new ArrayList<>(matrix);
for (Preference p : clusterPreferences) {
Collections.sort(tmpMatrix, (r1, r2) -> p.compare(r1, r2, false));
p.setApproxVal(tmpMatrix);
}
//approximate values are set now. Let's do recursive sorting
Collections.sort(matrix, (r1, r2) -> {
int result = clusterPreferences.get(0).compare(r1, r2, true);
if (result == 0) result = clusterPreferences.get(0).compare(r1, r2, false);
return result;
});
}
for (Clause clause : expandedClauses) {
List<Violation> errs = clause.test(matrix);
violations.addAll(errs);
}
}
public List<Violation> getViolations() {
return violations;
}
public Suggester getSuggester(CollectionAction action) {
Suggester op = ops.get(action).get();
if (op == null) throw new UnsupportedOperationException(action.toString() + "is not supported");
op._init(this);
return op;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
for (int i = 0; i < matrix.size(); i++) {
Row row = matrix.get(i);
ew.put(row.node, row);
}
}
@Override
public String toString() {
return Utils.toJSONString(toMap(new LinkedHashMap<>()));
}
public List<Row> getSorted() {
return Collections.unmodifiableList(matrix);
}
}
public Session createSession(ClusterDataProvider dataProvider) {
return new Session(dataProvider);
}
enum SortParam {
freedisk(0, Integer.MAX_VALUE), cores(0, Integer.MAX_VALUE), heapUsage(0, Integer.MAX_VALUE), sysLoadAvg(0, 100);
public final int min,max;
SortParam(int min, int max) {
this.min = min;
this.max = max;
}
static SortParam get(String m) {
for (SortParam p : values()) if (p.name().equals(m)) return p;
throw new RuntimeException(StrUtils.formatString("Invalid sort {0} Sort must be on one of these {1}", m, Arrays.asList(values())));
}
}
enum Sort {
maximize(1), minimize(-1);
final int sortval;
Sort(int i) {
sortval = i;
}
static Sort get(Map<String, Object> m) {
if (m.containsKey(maximize.name()) && m.containsKey(minimize.name())) {
throw new RuntimeException("Cannot have both 'maximize' and 'minimize'");
}
if (m.containsKey(maximize.name())) return maximize;
if (m.containsKey(minimize.name())) return minimize;
throw new RuntimeException("must have either 'maximize' or 'minimize'");
}
}
public static class ReplicaInfo implements MapWriter {
final String name;
String core, collection, shard;
Map<String, Object> variables;
public ReplicaInfo(String name, String coll, String shard, Map<String, Object> vals) {
this.name = name;
this.variables = vals;
this.collection = coll;
this.shard = shard;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put(name, variables);
}
public String getCore() {
return core;
}
public String getCollection() {
return collection;
}
public String getShard() {
return shard;
}
}
/* A suggester is capable of suggesting a collection operation
* given a particular session. Before it suggests a new operation,
* it ensures that ,
* a) load is reduced on the most loaded node
* b) it causes no new violations
*
*/
public static abstract class Suggester {
protected final EnumMap<Hint, Object> hints = new EnumMap<>(Hint.class);
Policy.Session session;
SolrRequest operation;
protected List<Violation> originalViolations = new ArrayList<>();
private boolean isInitialized = false;
private void _init(Session session) {
this.session = session.copy();
}
public Suggester hint(Hint hint, Object value) {
if (hint == Hint.TARGET_NODE || hint == Hint.SRC_NODE) {
((Set) hints.computeIfAbsent(hint, h -> new HashSet<>())).add(value);
} else {
hints.put(hint, value);
}
return this;
}
abstract SolrRequest init();
public SolrRequest getOperation() {
if (!isInitialized) {
String coll = (String) hints.get(Hint.COLL);
String shard = (String) hints.get(Hint.SHARD);
// if this is not a known collection from the existing clusterstate,
// then add it
if (session.matrix.stream().noneMatch(row -> row.collectionVsShardVsReplicas.containsKey(coll))) {
session.addClausesForCollection(session.dataProvider, coll);
Collections.sort(session.expandedClauses);
}
if (coll != null) {
for (Row row : session.matrix) {
if (!row.collectionVsShardVsReplicas.containsKey(coll)) row.collectionVsShardVsReplicas.put(coll, new HashMap<>());
if (shard != null) {
Map<String, List<ReplicaInfo>> shardInfo = row.collectionVsShardVsReplicas.get(coll);
if (!shardInfo.containsKey(shard)) shardInfo.put(shard, new ArrayList<>());
}
}
}
session.applyRules();
originalViolations.addAll(session.getViolations());
this.operation = init();
isInitialized = true;
}
return operation;
}
public Session getSession() {
return session;
}
List<Row> getMatrix() {
return session.matrix;
}
//check if the fresh set of violations is less serious than the last set of violations
boolean isLessSerious(List<Violation> fresh, List<Violation> old) {
if (old == null || fresh.size() < old.size()) return true;
if (fresh.size() == old.size()) {
for (int i = 0; i < fresh.size(); i++) {
Violation freshViolation = fresh.get(i);
Violation oldViolation = null;
for (Violation v : old) {
if (v.equals(freshViolation)) oldViolation = v;
}
if (oldViolation != null && freshViolation.isLessSerious(oldViolation)) return true;
}
}
return false;
}
boolean containsNewErrors(List<Violation> violations) {
for (Violation v : violations) {
int idx = originalViolations.indexOf(v);
if (idx < 0 || originalViolations.get(idx).isLessSerious(v)) return true;
}
return false;
}
List<Pair<ReplicaInfo, Row>> getValidReplicas(boolean sortDesc, boolean isSource, int until) {
List<Pair<Policy.ReplicaInfo, Row>> allPossibleReplicas = new ArrayList<>();
if (sortDesc) {
if (until == -1) until = getMatrix().size();
for (int i = 0; i < until; i++) addReplicaToList(getMatrix().get(i), isSource, allPossibleReplicas);
} else {
if (until == -1) until = 0;
for (int i = getMatrix().size() - 1; i >= until; i--)
addReplicaToList(getMatrix().get(i), isSource, allPossibleReplicas);
}
return allPossibleReplicas;
}
void addReplicaToList(Row r, boolean isSource, List<Pair<Policy.ReplicaInfo, Row>> replicaList) {
if (!isAllowed(r.node, isSource ? Hint.SRC_NODE : Hint.TARGET_NODE)) return;
for (Map.Entry<String, Map<String, List<Policy.ReplicaInfo>>> e : r.collectionVsShardVsReplicas.entrySet()) {
if (!isAllowed(e.getKey(), Hint.COLL)) continue;
for (Map.Entry<String, List<Policy.ReplicaInfo>> shard : e.getValue().entrySet()) {
if (!isAllowed(e.getKey(), Hint.SHARD)) continue;
replicaList.add(new Pair<>(shard.getValue().get(0), r));
}
}
}
protected List<Violation> testChangedMatrix(boolean strict, List<Row> rows) {
List<Violation> errors = new ArrayList<>();
for (Clause clause : session.expandedClauses) {
if (strict || clause.strict) {
List<Violation> errs = clause.test(rows);
if (!errs.isEmpty()) {
errors.addAll(errs);
}
}
}
return errors;
}
ArrayList<Row> getModifiedMatrix(List<Row> matrix, Row tmpRow, int i) {
ArrayList<Row> copy = new ArrayList<>(matrix);
copy.set(i, tmpRow);
return copy;
}
protected boolean isAllowed(Object v, Hint hint) {
Object hintVal = hints.get(hint);
if (hint == Hint.TARGET_NODE || hint == Hint.SRC_NODE) {
Set set = (Set) hintVal;
return set == null || set.contains(v);
} else {
return hintVal == null || Objects.equals(v, hintVal);
}
}
public enum Hint {
COLL, SHARD, SRC_NODE, TARGET_NODE
}
}
static List<Clause> mergePolicies(String coll,
List<Clause> collPolicy,
List<Clause> globalPolicy) {
List<Clause> merged = insertColl(coll, collPolicy);
List<Clause> global = insertColl(coll, globalPolicy);
merged.addAll(global.stream()
.filter(clusterPolicyClause -> merged.stream().noneMatch(perCollPolicy -> perCollPolicy.doesOverride(clusterPolicyClause)))
.collect(Collectors.toList()));
return merged;
}
/**
* Insert the collection name into the clauses where collection is not specified
*/
static List<Clause> insertColl(String coll, Collection<Clause> conditions) {
return conditions.stream()
.filter(Clause::isPerCollectiontag)
.map(clause -> {
Map<String, Object> copy = new LinkedHashMap<>(clause.original);
if (!copy.containsKey("collection")) copy.put("collection", coll);
return new Clause(copy);
})
.filter(it -> (it.collection.isPass(coll)))
.collect(Collectors.toList());
}
private static final Map<CollectionAction, Supplier<Suggester>> ops = new HashMap<>();
static {
ops.put(CollectionAction.ADDREPLICA, () -> new AddReplicaSuggester());
ops.put(CollectionAction.MOVEREPLICA, () -> new MoveReplicaSuggester());
}
}

View File

@ -0,0 +1,96 @@
/*
* 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.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.cloud.autoscaling.Policy.Suggester.Hint;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.Utils;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
public class PolicyHelper {
public static Map<String, List<String>> getReplicaLocations(String collName, Map<String, Object> autoScalingJson,
ClusterDataProvider cdp,
Map<String, String> optionalPolicyMapping,
List<String> shardNames,
int repFactor,
List<String> nodesList) {
Map<String, List<String>> positionMapping = new HashMap<>();
for (String shardName : shardNames) positionMapping.put(shardName, new ArrayList<>(repFactor));
if (optionalPolicyMapping != null) {
final ClusterDataProvider delegate = cdp;
cdp = new ClusterDataProvider() {
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return delegate.getNodeValues(node, tags);
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return delegate.getReplicaInfo(node, keys);
}
@Override
public Collection<String> getNodes() {
return delegate.getNodes();
}
@Override
public String getPolicyNameByCollection(String coll) {
return optionalPolicyMapping.containsKey(coll) ?
optionalPolicyMapping.get(coll) :
delegate.getPolicyNameByCollection(coll);
}
};
}
Policy policy = new Policy(autoScalingJson);
Policy.Session session = policy.createSession(cdp);
for (String shardName : shardNames) {
for (int i = 0; i < repFactor; i++) {
Policy.Suggester suggester = session.getSuggester(ADDREPLICA)
.hint(Hint.COLL, collName)
.hint(Hint.SHARD, shardName);
if (nodesList != null) {
for (String nodeName : nodesList) {
suggester = suggester.hint(Hint.TARGET_NODE, nodeName);
}
}
SolrRequest op = suggester.getOperation();
if (op == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No node can satisfy the rules "+ Utils.toJSONString(Utils.getDeepCopy(session.expandedClauses, 4, true)));
}
session = suggester.getSession();
positionMapping.get(shardName).add(op.getParams().get(CoreAdminParams.NODE));
}
}
return positionMapping;
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.List;
import java.util.Map;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils;
class Preference implements MapWriter {
final Policy.SortParam name;
Integer precision;
final Policy.Sort sort;
Preference next;
public int idx;
private final Map original;
Preference(Map<String, Object> m) {
this.original = Utils.getDeepCopy(m,3);
sort = Policy.Sort.get(m);
name = Policy.SortParam.get(m.get(sort.name()).toString());
Object p = m.getOrDefault("precision", 0);
precision = p instanceof Number ? ((Number) p).intValue() : Integer.parseInt(p.toString());
if (precision < 0) {
throw new RuntimeException("precision must be a positive value ");
}
if(precision< name.min || precision> name.max){
throw new RuntimeException(StrUtils.formatString("invalid precision value {0} must lie between {1} and {1}",
precision, name.min, name.max ) );
}
}
// there are 2 modes of compare.
// recursive, it uses the precision to tie & when there is a tie use the next preference to compare
// in non-recursive mode, precision is not taken into consideration and sort is done on actual value
int compare(Row r1, Row r2, boolean useApprox) {
Object o1 = useApprox ? r1.cells[idx].approxVal : r1.cells[idx].val;
Object o2 = useApprox ? r2.cells[idx].approxVal : r2.cells[idx].val;
int result = 0;
if (o1 instanceof Integer && o2 instanceof Integer) result = ((Integer) o1).compareTo((Integer) o2);
else if (o1 instanceof Long && o2 instanceof Long) result = ((Long) o1).compareTo((Long) o2);
else if (o1 instanceof Float && o2 instanceof Float) result = ((Float) o1).compareTo((Float) o2);
else if (o1 instanceof Double && o2 instanceof Double) result = ((Double) o1).compareTo((Double) o2);
else if (!o1.getClass().getName().equals(o2.getClass().getName())) {
throw new RuntimeException("Unable to compare " + o1 + " of type: " + o1.getClass().getName() + " from " + r1.cells[idx].toString() + " and " + o2 + " of type: " + o2.getClass().getName() + " from " + r2.cells[idx].toString());
}
return result == 0 ? (next == null ? 0 : next.compare(r1, r2, useApprox)) : sort.sortval * result;
}
//sets the new value according to precision in val_
void setApproxVal(List<Row> tmpMatrix) {
Object prevVal = null;
for (Row row : tmpMatrix) {
prevVal = row.cells[idx].approxVal =
(prevVal == null || Double.compare(Math.abs(((Number) prevVal).doubleValue() - ((Number) row.cells[idx].val).doubleValue()), precision) > 0) ?
row.cells[idx].val :
prevVal;
}
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
for (Object o : original.entrySet()) {
Map.Entry e = (Map.Entry) o;
ew.put(String.valueOf(e.getKey()), e.getValue());
}
}
}

View File

@ -0,0 +1,115 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import org.apache.solr.common.IteratorWriter;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.util.Pair;
import org.apache.solr.common.util.Utils;
import org.apache.solr.cloud.autoscaling.Policy.ReplicaInfo;
import static org.apache.solr.common.params.CoreAdminParams.NODE;
class Row implements MapWriter {
public final String node;
final Cell[] cells;
Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas;
List<Clause> violations = new ArrayList<>();
boolean anyValueMissing = false;
Row(String node, List<String> params, ClusterDataProvider dataProvider) {
collectionVsShardVsReplicas = dataProvider.getReplicaInfo(node, params);
if (collectionVsShardVsReplicas == null) collectionVsShardVsReplicas = new HashMap<>();
this.node = node;
cells = new Cell[params.size()];
Map<String, Object> vals = dataProvider.getNodeValues(node, params);
for (int i = 0; i < params.size(); i++) {
String s = params.get(i);
cells[i] = new Cell(i, s, Clause.validate(s,vals.get(s), false));
if (NODE.equals(s)) cells[i].val = node;
if (cells[i].val == null) anyValueMissing = true;
}
}
Row(String node, Cell[] cells, boolean anyValueMissing, Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas, List<Clause> violations) {
this.node = node;
this.cells = new Cell[cells.length];
for (int i = 0; i < this.cells.length; i++) {
this.cells[i] = cells[i].copy();
}
this.anyValueMissing = anyValueMissing;
this.collectionVsShardVsReplicas = collectionVsShardVsReplicas;
this.violations = violations;
}
@Override
public void writeMap(EntryWriter ew) throws IOException {
ew.put(node, (IteratorWriter) iw -> {
iw.add((MapWriter) e -> e.put("replicas", collectionVsShardVsReplicas));
for (Cell cell : cells) iw.add(cell);
});
}
Row copy() {
return new Row(node, cells, anyValueMissing, Utils.getDeepCopy(collectionVsShardVsReplicas, 3), new ArrayList<>(violations));
}
Object getVal(String name) {
for (Cell cell : cells) if (cell.name.equals(name)) return cell.val;
return null;
}
@Override
public String toString() {
return node;
}
// this adds a replica to the replica info
Row addReplica(String coll, String shard) {
Row row = copy();
Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.get(coll);
if (c == null) row.collectionVsShardVsReplicas.put(coll, c = new HashMap<>());
List<ReplicaInfo> replicas = c.get(shard);
if (replicas == null) c.put(shard, replicas = new ArrayList<>());
replicas.add(new ReplicaInfo("" + new Random().nextInt(1000) + 1000, coll, shard, new HashMap<>()));
for (Cell cell : row.cells) {
if (cell.name.equals("cores")) cell.val = ((Number) cell.val).longValue() + 1;
}
return row;
}
Pair<Row, ReplicaInfo> removeReplica(String coll, String shard) {
Row row = copy();
Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.get(coll);
if (c == null) return null;
List<ReplicaInfo> s = c.get(shard);
if (s == null || s.isEmpty()) return null;
return new Pair(row, s.remove(0));
}
}

View File

@ -0,0 +1,22 @@
/*
* 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.
*/
/**
* Common classes for autoscaling parsing filtering nodes and sorting
*/
package org.apache.solr.cloud.autoscaling;

View File

@ -19,13 +19,16 @@ package org.apache.solr.common;
import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
/**
* Interface to help do push writing to an array
*/
public interface IteratorWriter {
/**
* @param iw after this method returns , the EntryWriter Object is invalid
* @param iw after this method returns , the ItemWriter Object is invalid
* Do not hold a reference to this object
*/
void writeIter(ItemWriter iw) throws IOException;
@ -62,4 +65,20 @@ public interface IteratorWriter {
return this;
}
}
default List toList( List l) {
try {
writeIter(new ItemWriter() {
@Override
public ItemWriter add(Object o) throws IOException {
if (o instanceof MapWriter) o = ((MapWriter) o).toMap(new LinkedHashMap<>());
if (o instanceof IteratorWriter) o = ((IteratorWriter) o).toList(new ArrayList<>());
l.add(o);
return this;
}
});
} catch (IOException e) {
throw new RuntimeException(e);
}
return l;
}
}

View File

@ -19,6 +19,8 @@ package org.apache.solr.common;
import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.Map;
/**
@ -34,6 +36,8 @@ public interface MapWriter extends MapSerializable {
writeMap(new EntryWriter() {
@Override
public EntryWriter put(String k, Object v) throws IOException {
if (v instanceof MapWriter) v = ((MapWriter) v).toMap(new LinkedHashMap<>());
if (v instanceof IteratorWriter) v = ((IteratorWriter) v).toList(new ArrayList<>());
map.put(k, v);
return this;
}
@ -60,6 +64,12 @@ public interface MapWriter extends MapSerializable {
*/
EntryWriter put(String k, Object v) throws IOException;
default EntryWriter putIfNotNull(String k, Object v) throws IOException {
if(v != null) put(k,v);
return this;
}
default EntryWriter put(String k, int v) throws IOException {
put(k, (Integer) v);
return this;

View File

@ -26,7 +26,9 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.BiConsumer;
import org.apache.solr.cloud.autoscaling.Policy;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.noggit.JSONUtil;
@ -66,6 +68,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
private final Integer numPullReplicas;
private final Integer maxShardsPerNode;
private final Boolean autoAddReplicas;
private final String policy;
public DocCollection(String name, Map<String, Slice> slices, Map<String, Object> props, DocRouter router) {
this(name, slices, props, router, Integer.MAX_VALUE, ZkStateReader.CLUSTER_STATE);
@ -92,6 +95,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
this.numPullReplicas = (Integer) verifyProp(props, PULL_REPLICAS);
this.maxShardsPerNode = (Integer) verifyProp(props, MAX_SHARDS_PER_NODE);
Boolean autoAddReplicas = (Boolean) verifyProp(props, AUTO_ADD_REPLICAS);
this.policy = (String) props.get(Policy.POLICY);
this.autoAddReplicas = autoAddReplicas == null ? Boolean.FALSE : autoAddReplicas;
verifyProp(props, RULE);
@ -170,6 +174,10 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
return slices.get(sliceName);
}
public void forEachReplica(BiConsumer<String, Replica> consumer) {
slices.forEach((shard, slice) -> slice.getReplicasMap().forEach((s, replica) -> consumer.accept(shard, replica)));
}
/**
* Gets the list of all slices for this collection.
*/
@ -363,4 +371,10 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
return numPullReplicas;
}
/**
* @return the policy associated with this collection if any
*/
public String getPolicyName() {
return policy;
}
}

View File

@ -93,6 +93,7 @@ public class ZkStateReader implements Closeable {
public static final String CLUSTER_PROPS = "/clusterprops.json";
public static final String REJOIN_AT_HEAD_PROP = "rejoinAtHead";
public static final String SOLR_SECURITY_CONF_PATH = "/security.json";
public static final String SOLR_AUTOSCALING_CONF_PATH = "/autoscaling.json";
public static final String REPLICATION_FACTOR = "replicationFactor";
public static final String MAX_SHARDS_PER_NODE = "maxShardsPerNode";

View File

@ -28,8 +28,10 @@ import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -46,12 +48,16 @@ public class ImplicitSnitch extends Snitch {
public static final String CORES = "cores";
public static final String DISK = "freedisk";
public static final String ROLE = "role";
public static final String NODEROLE = "nodeRole";
public static final String SYSPROP = "sysprop.";
public static final String SYSLOADAVG = "sysLoadAvg";
public static final String HEAPUSAGE = "heapUsage";
public static final List<String> IP_SNITCHES = Collections.unmodifiableList(Arrays.asList("ip_1", "ip_2", "ip_3", "ip_4"));
public static final Set<String> tags = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(NODE, PORT, HOST, CORES, DISK, ROLE, "ip_1", "ip_2", "ip_3", "ip_4")));
@Override
public void getTags(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
try {
if (requestedTags.contains(NODE)) ctx.getTags().put(NODE, solrNode);
if (requestedTags.contains(HOST)) {
Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
@ -61,9 +67,18 @@ public class ImplicitSnitch extends Snitch {
Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
if (hostAndPortMatcher.find()) ctx.getTags().put(PORT, hostAndPortMatcher.group(2));
}
if (requestedTags.contains(ROLE)) fillRole(solrNode, ctx);
if (requestedTags.contains(ROLE)) fillRole(solrNode, ctx, ROLE);
if (requestedTags.contains(NODEROLE)) fillRole(solrNode, ctx, NODEROLE);// for new policy framework
addIpTags(solrNode, requestedTags, ctx);
getRemoteInfo(solrNode, requestedTags, ctx);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
ModifiableSolrParams params = new ModifiableSolrParams();
if (requestedTags.contains(CORES)) params.add(CORES, "1");
if (requestedTags.contains(DISK)) params.add(DISK, "1");
@ -73,17 +88,25 @@ public class ImplicitSnitch extends Snitch {
if (params.size() > 0) ctx.invokeRemote(solrNode, params, "org.apache.solr.cloud.rule.ImplicitSnitch", null);
}
private void fillRole(String solrNode, SnitchContext ctx) {
private void fillRole(String solrNode, SnitchContext ctx, String key) throws KeeperException, InterruptedException {
Map roles = (Map) ctx.retrieve(ZkStateReader.ROLES); // we don't want to hit the ZK for each node
// so cache and reuse
try {
if (roles == null) roles = ctx.getZkJson(ZkStateReader.ROLES);
ctx.store(ZkStateReader.ROLES, roles == null ? Collections.emptyMap() : roles);
cacheRoles(solrNode, ctx, key, roles);
} catch (KeeperException.NoNodeException e) {
cacheRoles(solrNode, ctx, key, Collections.emptyMap());
}
}
private void cacheRoles(String solrNode, SnitchContext ctx, String key, Map roles) {
ctx.store(ZkStateReader.ROLES, roles);
if (roles != null) {
for (Object o : roles.entrySet()) {
Map.Entry e = (Map.Entry) o;
if (e.getValue() instanceof List) {
if (((List) e.getValue()).contains(solrNode)) {
ctx.getTags().put(ROLE, e.getKey());
ctx.getTags().put(key, e.getKey());
break;
}
}

View File

@ -16,18 +16,14 @@
*/
package org.apache.solr.common.cloud.rule;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
/**
*
*/
public abstract class Snitch {
public static final Set<Class> WELL_KNOWN_SNITCHES = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(ImplicitSnitch.class)));
public static final Set<Class> WELL_KNOWN_SNITCHES = Collections.singleton(ImplicitSnitch.class);
public abstract void getTags(String solrNode, Set<String> requestedTags, SnitchContext ctx);

View File

@ -22,6 +22,7 @@ import java.util.Map;
import java.util.Set;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -44,10 +45,6 @@ public abstract class SnitchContext implements RemoteCallback {
this.session = session;
}
public SnitchInfo getSnitchInfo() {
return snitchInfo;
}
public Map<String, Object> getTags() {
return tags;
}
@ -62,7 +59,7 @@ public abstract class SnitchContext implements RemoteCallback {
}
public abstract Map getZkJson(String path) ;
public abstract Map getZkJson(String path) throws KeeperException, InterruptedException;
public String getNode() {
return node;

View File

@ -181,6 +181,8 @@ public interface CommonParams {
String AUTHC_PATH = "/admin/authentication";
String ZK_PATH = "/admin/zookeeper";
String METRICS_PATH = "/admin/metrics";
String AUTOSCALING_PATH = "/admin/autoscaling";
String AUTOSCALING_DIAGNOSTICS_PATH = "/admin/autoscaling/diagnostics";
Set<String> ADMIN_PATHS = new HashSet<>(Arrays.asList(
CORES_HANDLER_PATH,
@ -188,7 +190,9 @@ public interface CommonParams {
CONFIGSETS_HANDLER_PATH,
AUTHC_PATH,
AUTHZ_PATH,
METRICS_PATH));
METRICS_PATH,
AUTOSCALING_PATH,
AUTOSCALING_DIAGNOSTICS_PATH));
/** valid values for: <code>echoParams</code> */
enum EchoParamStyle {

View File

@ -76,6 +76,10 @@ public class ModifiableSolrParams extends SolrParams
}
public ModifiableSolrParams setNonNull(String name, Object val) {
if (val != null) set(name, String.valueOf(val));
return this;
}
//----------------------------------------------------------------
//----------------------------------------------------------------

View File

@ -24,6 +24,7 @@ import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.solr.common.SolrException;
import org.noggit.JSONParser;
@ -62,6 +63,7 @@ public class CommandOperation {
String v = getStr(key, null);
return v == null ? def : Boolean.parseBoolean(v);
}
public void setCommandData(Object o) {
commandData = o;
}
@ -213,11 +215,19 @@ public class CommandOperation {
return errors;
}
public static List<CommandOperation> parse(Reader rdr) throws IOException {
return parse(rdr, Collections.emptySet());
}
/**
* Parse the command operations into command objects
*
* @param rdr The payload
* @param singletonCommands commands that cannot be repeated
* @return parsed list of commands
*/
public static List<CommandOperation> parse(Reader rdr) throws IOException {
public static List<CommandOperation> parse(Reader rdr, Set<String> singletonCommands) throws IOException {
JSONParser parser = new JSONParser(rdr);
parser.setFlags(parser.getFlags() |
JSONParser.ALLOW_MISSING_COLON_COMMA_BEFORE_OBJECT |
@ -236,7 +246,7 @@ public class CommandOperation {
Object key = ob.getKey();
ev = parser.nextEvent();
Object val = ob.getVal();
if (val instanceof List) {
if (val instanceof List && !singletonCommands.contains(key)) {
List list = (List) val;
for (Object o : list) {
if (!(o instanceof Map)) {
@ -274,7 +284,21 @@ public class CommandOperation {
return new String(toJSON(singletonMap(name, commandData)), StandardCharsets.UTF_8);
}
public static List<CommandOperation> readCommands(Iterable<ContentStream> streams, NamedList resp)
public static List<CommandOperation> readCommands(Iterable<ContentStream> streams, NamedList resp) throws IOException {
return readCommands(streams, resp, Collections.emptySet());
}
/**
* Read commands from request streams
*
* @param streams the streams
* @param resp solr query response
* @param singletonCommands , commands that cannot be repeated
* @return parsed list of commands
* @throws IOException if there is an error while parsing the stream
*/
public static List<CommandOperation> readCommands(Iterable<ContentStream> streams, NamedList resp, Set<String> singletonCommands)
throws IOException {
if (streams == null) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
@ -282,7 +306,7 @@ public class CommandOperation {
ArrayList<CommandOperation> ops = new ArrayList<>();
for (ContentStream stream : streams)
ops.addAll(parse(stream.getReader()));
ops.addAll(parse(stream.getReader(), singletonCommands));
List<Map> errList = CommandOperation.captureErrors(ops);
if (!errList.isEmpty()) {
resp.add(CommandOperation.ERR_MSGS, errList);

View File

@ -33,6 +33,8 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.solr.common.EnumFieldValue;
import org.apache.solr.common.IteratorWriter;
@ -390,7 +392,18 @@ public class JavaBinCodec implements PushWriter {
writeMap(((MapSerializable) val).toMap(new NamedList().asShallowMap()));
return true;
}
if (val instanceof AtomicInteger) {
writeInt(((AtomicInteger) val).get());
return true;
}
if (val instanceof AtomicLong) {
writeLong(((AtomicLong) val).get());
return true;
}
if (val instanceof AtomicBoolean) {
writeBoolean(((AtomicBoolean) val).get());
return true;
}
return false;
}

View File

@ -32,12 +32,19 @@ 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.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.http.HttpEntity;
import org.apache.http.util.EntityUtils;
import org.apache.solr.common.IteratorWriter;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkOperation;
import org.apache.zookeeper.KeeperException;
import org.noggit.CharArr;
import org.noggit.JSONParser;
import org.noggit.JSONWriter;
@ -52,31 +59,58 @@ public class Utils {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static Map getDeepCopy(Map map, int maxDepth) {
return getDeepCopy(map, maxDepth, true);
return getDeepCopy(map, maxDepth, true, false);
}
public static Map getDeepCopy(Map map, int maxDepth, boolean mutable) {
return getDeepCopy(map, maxDepth, mutable, false);
}
public static Map getDeepCopy(Map map, int maxDepth, boolean mutable, boolean sorted) {
if(map == null) return null;
if (maxDepth < 1) return map;
Map copy = new LinkedHashMap();
Map copy;
if (sorted) {
copy = new TreeMap();
} else {
copy = new LinkedHashMap();
}
for (Object o : map.entrySet()) {
Map.Entry e = (Map.Entry) o;
Object v = e.getValue();
if (v instanceof Map) v = getDeepCopy((Map) v, maxDepth - 1, mutable);
else if (v instanceof Collection) v = getDeepCopy((Collection) v, maxDepth - 1, mutable);
copy.put(e.getKey(), v);
copy.put(e.getKey(), makeDeepCopy(e.getValue(),maxDepth, mutable, sorted));
}
return mutable ? copy : Collections.unmodifiableMap(copy);
}
public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable) {
if (c == null || maxDepth < 1) return c;
Collection result = c instanceof Set ? new HashSet() : new ArrayList();
for (Object o : c) {
if (o instanceof Map) {
o = getDeepCopy((Map) o, maxDepth - 1, mutable);
private static Object makeDeepCopy(Object v, int maxDepth, boolean mutable, boolean sorted) {
if (v instanceof MapWriter && maxDepth > 1) {
v = ((MapWriter) v).toMap(new LinkedHashMap<>());
} else if (v instanceof IteratorWriter && maxDepth > 1) {
v = ((IteratorWriter) v).toList(new ArrayList<>());
if (sorted) {
Collections.sort((List)v);
}
result.add(o);
}
if (v instanceof Map) {
v = getDeepCopy((Map) v, maxDepth - 1, mutable, sorted);
} else if (v instanceof Collection) {
v = getDeepCopy((Collection) v, maxDepth - 1, mutable, sorted);
}
return v;
}
public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable) {
return getDeepCopy(c, maxDepth, mutable, false);
}
public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable, boolean sorted) {
if (c == null || maxDepth < 1) return c;
Collection result = c instanceof Set ?
( sorted? new TreeSet() : new HashSet()) : new ArrayList();
for (Object o : c) result.add(makeDeepCopy(o, maxDepth, mutable, sorted));
if (sorted && (result instanceof List)) {
Collections.sort((List)result);
}
return mutable ? result : result instanceof Set ? unmodifiableSet((Set) result) : unmodifiableList((List) result);
}
@ -84,6 +118,13 @@ public class Utils {
public static byte[] toJSON(Object o) {
if(o == null) return new byte[0];
CharArr out = new CharArr();
if (!(o instanceof List) && !(o instanceof Map)) {
if (o instanceof MapWriter) {
o = ((MapWriter)o).toMap(new LinkedHashMap<>());
} else if(o instanceof IteratorWriter){
o = ((IteratorWriter)o).toList(new ArrayList<>());
}
}
new JSONWriter(out, 2).write(o); // indentation by default
return toUTF8(out);
}
@ -116,12 +157,18 @@ public class Utils {
}
public static Map<String, Object> makeMap(Object... keyVals) {
return makeMap(false, keyVals);
}
public static Map<String, Object> makeMap(boolean skipNulls, Object... keyVals) {
if ((keyVals.length & 0x01) != 0) {
throw new IllegalArgumentException("arguments should be key,value");
}
Map<String, Object> propMap = new LinkedHashMap<>(keyVals.length >> 1);
for (int i = 0; i < keyVals.length; i += 2) {
propMap.put(keyVals[i].toString(), keyVals[i + 1]);
Object keyVal = keyVals[i + 1];
if (skipNulls && keyVal == null) continue;
propMap.put(keyVals[i].toString(), keyVal);
}
return propMap;
}
@ -162,6 +209,7 @@ public class Utils {
}
public static Object getObjectByPath(Map root, boolean onlyPrimitive, List<String> hierarchy) {
if(root == null) return null;
Map obj = root;
for (int i = 0; i < hierarchy.size(); i++) {
int idx = -1;
@ -232,6 +280,26 @@ public class Utils {
while (is.read() != -1) {}
}
/**
* Assumes data in ZooKeeper is a JSON string, deserializes it and returns as a Map
*
* @param zkClient the zookeeper client
* @param path the path to the znode being read
* @param retryOnConnLoss whether to retry the operation automatically on connection loss, see {@link org.apache.solr.common.cloud.ZkCmdExecutor#retryOperation(ZkOperation)}
* @return a Map if the node exists and contains valid JSON or an empty map if znode does not exist or has a null data
*/
public static Map<String, Object> getJson(SolrZkClient zkClient, String path, boolean retryOnConnLoss) throws KeeperException, InterruptedException {
try {
byte[] bytes = zkClient.getData(path, null, null, retryOnConnLoss);
if (bytes != null && bytes.length > 0) {
return (Map<String, Object>) Utils.fromJSON(bytes);
}
} catch (KeeperException.NoNodeException e) {
return Collections.emptyMap();
}
return Collections.emptyMap();
}
public static final Pattern ARRAY_ELEMENT_INDEX = Pattern
.compile("(\\S*?)\\[(\\d+)\\]");
}

View File

@ -0,0 +1,654 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
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 com.google.common.collect.ImmutableList;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.autoscaling.Clause.Violation;
import org.apache.solr.cloud.autoscaling.Policy.Suggester.Hint;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.Utils;
import org.apache.solr.common.util.ValidatingJsonMap;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
public class TestPolicy extends SolrTestCaseJ4 {
public static String clusterState = "{'gettingstarted':{" +
" 'router':{'name':'compositeId'}," +
" 'shards':{" +
" 'shard1':{" +
" 'range':'80000000-ffffffff'," +
" 'replicas':{" +
" 'r1':{" +
" 'core':r1," +
" 'base_url':'http://10.0.0.4:8983/solr'," +
" 'node_name':'node1'," +
" 'state':'active'," +
" 'leader':'true'}," +
" 'r2':{" +
" 'core':r2," +
" 'base_url':'http://10.0.0.4:7574/solr'," +
" 'node_name':'node2'," +
" 'state':'active'}}}," +
" 'shard2':{" +
" 'range':'0-7fffffff'," +
" 'replicas':{" +
" 'r3':{" +
" 'core':r3," +
" 'base_url':'http://10.0.0.4:8983/solr'," +
" 'node_name':'node1'," +
" 'state':'active'," +
" 'leader':'true'}," +
" 'r4':{" +
" 'core':r4," +
" 'base_url':'http://10.0.0.4:8987/solr'," +
" 'node_name':'node4'," +
" 'state':'active'}," +
" 'r6':{" +
" 'core':r6," +
" 'base_url':'http://10.0.0.4:8989/solr'," +
" 'node_name':'node3'," +
" 'state':'active'}," +
" 'r5':{" +
" 'core':r5," +
" 'base_url':'http://10.0.0.4:7574/solr'," +
" 'node_name':'node1'," +
" 'state':'active'}}}}}}";
public static Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaDetails(String node, String s) {
ValidatingJsonMap m = ValidatingJsonMap
.getDeepCopy((Map) Utils.fromJSONString(s), 6, true);
Map<String, Map<String, List<Policy.ReplicaInfo>>> result = new LinkedHashMap<>();
m.forEach((collName, o) -> {
ValidatingJsonMap coll = (ValidatingJsonMap) o;
coll.getMap("shards").forEach((shard, o1) -> {
ValidatingJsonMap sh = (ValidatingJsonMap) o1;
sh.getMap("replicas").forEach((replicaName, o2) -> {
ValidatingJsonMap r = (ValidatingJsonMap) o2;
String node_name = (String) r.get("node_name");
if (!node_name.equals(node)) return;
Map<String, List<Policy.ReplicaInfo>> shardVsReplicaStats = result.get(collName);
if (shardVsReplicaStats == null) result.put(collName, shardVsReplicaStats = new HashMap<>());
List<Policy.ReplicaInfo> replicaInfos = shardVsReplicaStats.get(shard);
if (replicaInfos == null) shardVsReplicaStats.put(shard, replicaInfos = new ArrayList<>());
replicaInfos.add(new Policy.ReplicaInfo(replicaName, collName, shard, new HashMap<>()));
});
});
});
return result;
}
public void testValidate() {
expectError("replica", -1, "must be greater than" );
expectError("replica","hello", "not a valid number" );
assertEquals( 1l, Clause.validate("replica", "1", true));
assertEquals("c", Clause.validate("collection", "c", true));
assertEquals( "s", Clause.validate("shard", "s",true));
assertEquals( "overseer", Clause.validate("nodeRole", "overseer",true));
expectError("nodeRole", "wrong","must be one of");
expectError("sysLoadAvg", "101","must be less than ");
expectError("sysLoadAvg", 101,"must be less than ");
expectError("sysLoadAvg", "-1","must be greater than");
expectError("sysLoadAvg", -1,"must be greater than");
assertEquals(12.46d,Clause.validate("sysLoadAvg", "12.46",true));
assertEquals(12.46,Clause.validate("sysLoadAvg", 12.46d,true));
expectError("ip_1", "300","must be less than ");
expectError("ip_1", 300,"must be less than ");
expectError("ip_1", "-1","must be greater than");
expectError("ip_1", -1,"must be greater than");
assertEquals(1l,Clause.validate("ip_1", "1",true));
expectError("heapUsage", "-1","must be greater than");
expectError("heapUsage", -1,"must be greater than");
assertEquals(69.9d,Clause.validate("heapUsage", "69.9",true));
assertEquals(69.9d,Clause.validate("heapUsage", 69.9d,true));
expectError("port", "70000","must be less than ");
expectError("port", 70000,"must be less than ");
expectError("port", "0","must be greater than");
expectError("port", 0,"must be greater than");
expectError("cores", "-1","must be greater than");
}
private static void expectError(String name, Object val, String msg){
try {
Clause.validate(name, val,true);
fail("expected exception containing "+msg);
} catch (Exception e) {
assertTrue("expected exception containing "+msg,e.getMessage().contains(msg));
}
}
public void testOperands() {
Clause c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:'<2', node:'#ANY'}"));
assertFalse(c.replica.isPass(3));
assertFalse(c.replica.isPass(2));
assertTrue(c.replica.isPass(1));
c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:'>2', node:'#ANY'}"));
assertTrue(c.replica.isPass(3));
assertFalse(c.replica.isPass(2));
assertFalse(c.replica.isPass(1));
c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:0, nodeRole:'!overseer'}"));
assertTrue(c.tag.isPass("OVERSEER"));
assertFalse(c.tag.isPass("overseer"));
c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:0, sysLoadAvg:'<12.7'}"));
assertTrue(c.tag.isPass("12.6"));
assertTrue(c.tag.isPass(12.6d));
assertFalse(c.tag.isPass("12.9"));
assertFalse(c.tag.isPass(12.9d));
c = new Clause((Map<String, Object>) Utils.fromJSONString("{replica:0, sysLoadAvg:'>12.7'}"));
assertTrue(c.tag.isPass("12.8"));
assertTrue(c.tag.isPass(12.8d));
assertFalse(c.tag.isPass("12.6"));
assertFalse(c.tag.isPass(12.6d));
}
public void testRow() {
Row row = new Row("nodex", new Cell[]{new Cell(0, "node", "nodex")}, false, new HashMap<>(), new ArrayList<>());
Row r1 = row.addReplica("c1", "s1");
Row r2 = r1.addReplica("c1", "s1");
assertEquals(1, r1.collectionVsShardVsReplicas.get("c1").get("s1").size());
assertEquals(2, r2.collectionVsShardVsReplicas.get("c1").get("s1").size());
assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(0) instanceof Policy.ReplicaInfo);
assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(1) instanceof Policy.ReplicaInfo);
}
public void testMerge() {
Map map = (Map) Utils.fromJSONString("{" +
" 'cluster-preferences': [" +
" { 'maximize': 'freedisk', 'precision': 50}," +
" { 'minimize': 'cores', 'precision': 50}" +
" ]," +
" 'cluster-policy': [" +
" { 'replica': 0, 'nodeRole': 'overseer'}," +
" { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}" +
" ]," +
" 'policies': {" +
" 'policy1': [" +
" { 'replica': '1', 'sysprop.fs': 'ssd', 'shard': '#EACH'}," +
" { 'replica': '<2', 'shard': '#ANY', 'node': '#ANY'}," +
" { 'replica': '<2', 'shard': '#EACH', 'sysprop.rack': 'rack1'}" +
" ]" +
" }" +
"}");
Policy policy = new Policy(map);
List<Clause> clauses = Policy.mergePolicies("mycoll", policy.policies.get("policy1"), policy.clusterPolicy);
Collections.sort(clauses);
assertEquals(clauses.size(), 4);
assertEquals("1", String.valueOf(clauses.get(0).original.get("replica")));
assertEquals("0", String.valueOf(clauses.get(1).original.get("replica")));
assertEquals("#ANY", clauses.get(3).original.get("shard"));
assertEquals("rack1", clauses.get(2).original.get("sysprop.rack"));
assertEquals("overseer", clauses.get(1).original.get("nodeRole"));
}
public void testConditionsSort() {
String rules = "{" +
" 'cluster-policy':[" +
" { 'nodeRole':'overseer', replica: 0, 'strict':false}," +
" { 'replica':'<1', 'node':'node3', 'shard':'#EACH'}," +
" { 'replica':'<2', 'node':'#ANY', 'shard':'#EACH'}," +
" { 'replica':1, 'sysprop.rack':'rack1'}]" +
" }";
Policy p = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
List<Clause> clauses = new ArrayList<>(p.getClusterPolicy());
Collections.sort(clauses);
assertEquals("nodeRole", clauses.get(1).tag.name);
assertEquals("sysprop.rack", clauses.get(0).tag.name);
}
public void testRules() throws IOException {
String rules = "{" +
"cluster-policy:[" +
"{nodeRole:'overseer',replica : 0 , strict:false}," +
"{replica:'<1',node:node3}," +
"{replica:'<2',node:'#ANY', shard:'#EACH'}]," +
" cluster-preferences:[" +
"{minimize:cores , precision:2}," +
"{maximize:freedisk, precision:50}, " +
"{minimize:heapUsage, precision:1000}]}";
Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
"node1:{cores:12, freedisk: 334, heapUsage:10480}," +
"node2:{cores:4, freedisk: 749, heapUsage:6873}," +
"node3:{cores:7, freedisk: 262, heapUsage:7834}," +
"node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer}" +
"}");
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
Policy.Session session;
session = policy.createSession(getClusterDataProvider(nodeValues, clusterState));
List<Row> l = session.getSorted();
assertEquals("node1", l.get(0).node);
assertEquals("node4", l.get(1).node);
assertEquals("node3", l.get(2).node);
assertEquals("node2", l.get(3).node);
List<Violation> violations = session.getViolations();
assertEquals(3, violations.size());
assertTrue(violations.stream().anyMatch(violation -> "node3".equals(violation.getClause().tag.val)));
assertTrue(violations.stream().anyMatch(violation -> "nodeRole".equals(violation.getClause().tag.name)));
assertTrue(violations.stream().anyMatch(violation -> (violation.getClause().replica.op == Operand.LESS_THAN && "node".equals(violation.getClause().tag.name))));
Policy.Suggester suggester = session.getSuggester(ADDREPLICA)
.hint(Hint.COLL, "gettingstarted")
.hint(Hint.SHARD, "r1");
SolrParams operation = suggester.getOperation().getParams();
assertEquals("node2", operation.get("node"));
nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
"node1:{cores:12, freedisk: 334, heapUsage:10480}," +
"node2:{cores:4, freedisk: 749, heapUsage:6873}," +
"node3:{cores:7, freedisk: 262, heapUsage:7834}," +
"node5:{cores:0, freedisk: 895, heapUsage:17834}," +
"node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer}" +
"}");
session = policy.createSession(getClusterDataProvider(nodeValues, clusterState));
SolrRequest opReq = session.getSuggester(MOVEREPLICA)
.hint(Hint.TARGET_NODE, "node5")
.getOperation();
assertNotNull(opReq);
assertEquals("node5", opReq.getParams().get("targetNode"));
}
public void testNegativeConditions() {
String autoscaleJson = "{" +
" 'cluster-policy':[" +
" {'replica':'<4','shard':'#EACH','node':'#ANY'}," +
" { 'replica': 0, 'sysprop.fs': '!ssd', 'shard': '#EACH'}," +//negative greedy condition
" {'nodeRole':'overseer','replica':'0'}]," +
" 'cluster-preferences':[" +
" {'minimize':'cores', 'precision':3}," +
" {'maximize':'freedisk','precision':100}]}";
Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
"node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4}," +
"node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3}," +
"node3:{cores:7, freedisk: 262, heapUsage:7834, rack: rack2, sysprop.fs : ssd}," +
"node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer, rack: rack1}" +
"}");
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson));
ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState);
Policy.Session session = policy.createSession(clusterDataProvider);
for (int i = 0; i < 3; i++) {
Policy.Suggester suggester = session.getSuggester(ADDREPLICA);
SolrRequest op = suggester
.hint(Hint.COLL, "newColl")
.hint(Hint.SHARD, "shard1")
.getOperation();
assertNotNull(op);
assertEquals("node3", op.getParams().get("node"));
session = suggester.getSession();
}
}
public void testGreedyConditions() {
String autoscaleJson = "{" +
" 'cluster-policy':[" +
" {'cores':'<10','node':'#ANY'}," +
" {'replica':'<3','shard':'#EACH','node':'#ANY'}," +
" { 'replica': 2, 'sysprop.fs': 'ssd', 'shard': '#EACH'}," +//greedy condition
" {'nodeRole':'overseer','replica':'0'}]," +
" 'cluster-preferences':[" +
" {'minimize':'cores', 'precision':3}," +
" {'maximize':'freedisk','precision':100}]}";
Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
"node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4}," +
"node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3}," +
"node3:{cores:7, freedisk: 262, heapUsage:7834, rack: rack2, sysprop.fs : ssd}," +
"node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer, rack: rack1}" +
"}");
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson));
ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState);
ClusterDataProvider cdp = new ClusterDataProvider() {
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return clusterDataProvider.getNodeValues(node, tags);
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return clusterDataProvider.getReplicaInfo(node, keys);
}
@Override
public Collection<String> getNodes() {
return clusterDataProvider.getNodes();
}
@Override
public String getPolicyNameByCollection(String coll) {
return null;
}
};
Policy.Session session = policy.createSession(cdp);
Policy.Suggester suggester = session.getSuggester(ADDREPLICA);
SolrRequest op = suggester
.hint(Hint.COLL, "newColl")
.hint(Hint.SHARD, "shard1")
.getOperation();
assertNotNull(op);
assertEquals("node3", op.getParams().get("node"));
suggester = suggester
.getSession()
.getSuggester(ADDREPLICA)
.hint(Hint.COLL, "newColl")
.hint(Hint.SHARD, "shard1");
op = suggester.getOperation();
assertNotNull(op);
assertEquals("node3", op.getParams().get("node"));
suggester = suggester
.getSession()
.getSuggester(ADDREPLICA)
.hint(Hint.COLL, "newColl")
.hint(Hint.SHARD, "shard1");
op = suggester.getOperation();
assertNotNull(op);
assertEquals("node2", op.getParams().get("node"));
}
public void testMoveReplica() {
String autoscaleJson = "{" +
" 'cluster-policy':[" +
" {'cores':'<10','node':'#ANY'}," +
" {'replica':'<3','shard':'#EACH','node':'#ANY'}," +
" {'nodeRole':'overseer','replica':'0'}]," +
" 'cluster-preferences':[" +
" {'minimize':'cores', 'precision':3}," +
" {'maximize':'freedisk','precision':100}]}";
Map replicaInfoMap = (Map) Utils.fromJSONString("{ '127.0.0.1:60099_solr':{}," +
" '127.0.0.1:60089_solr':{'compute_plan_action_test':{'shard1':[" +
" {'core_node1':{}}," +
" {'core_node2':{}}]}}}");
Map m = (Map) Utils.getObjectByPath(replicaInfoMap, false, "127.0.0.1:60089_solr/compute_plan_action_test");
m.put("shard1", Arrays.asList(
new Policy.ReplicaInfo("core_node1", "compute_plan_action_test", "shard1", Collections.emptyMap()),
new Policy.ReplicaInfo("core_node2", "compute_plan_action_test", "shard1", Collections.emptyMap())
));
Map<String, Map<String, Object>> tagsMap = (Map) Utils.fromJSONString("{" +
" '127.0.0.1:60099_solr':{" +
" 'cores':0," +
" 'freedisk':918005641216}," +
" '127.0.0.1:60089_solr':{" +
" 'cores':2," +
" 'freedisk':918005641216}}}");
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson));
Policy.Session session = policy.createSession(new ClusterDataProvider() {
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return tagsMap.get(node);
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return (Map<String, Map<String, List<Policy.ReplicaInfo>>>) replicaInfoMap.get(node);
}
@Override
public Collection<String> getNodes() {
return replicaInfoMap.keySet();
}
@Override
public String getPolicyNameByCollection(String coll) {
return null;
}
});
Policy.Suggester suggester = session.getSuggester(CollectionParams.CollectionAction.MOVEREPLICA)
.hint(Hint.TARGET_NODE, "127.0.0.1:60099_solr");
SolrParams op = suggester.getOperation().getParams();
assertNotNull(op);
session = suggester.getSession();
suggester = session.getSuggester(MOVEREPLICA).hint(Hint.TARGET_NODE, "127.0.0.1:60099_solr");
op = suggester.getOperation().getParams();
assertNotNull(op);
}
public void testOtherTag() {
String rules = "{" +
"'cluster-preferences':[" +
"{'minimize':'cores','precision':2}," +
"{'maximize':'freedisk','precision':50}," +
"{'minimize':'heapUsage','precision':1000}" +
"]," +
"'cluster-policy':[" +
"{replica:0, 'nodeRole':'overseer','strict':false}," +
"{'replica':'<1','node':'node3'}," +
"{'replica':'<2','node':'#ANY','shard':'#EACH'}" +
"]," +
"'policies':{" +
"'p1':[" +
"{replica:0, 'nodeRole':'overseer','strict':false}," +
"{'replica':'<1','node':'node3'}," +
"{'replica':'<2','node':'#ANY','shard':'#EACH'}," +
"{'replica':'<3','shard':'#EACH','sysprop.rack':'#ANY'}" +
"]" +
"}" +
"}";
Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
"node1:{cores:12, freedisk: 334, heapUsage:10480, rack: rack4}," +
"node2:{cores:4, freedisk: 749, heapUsage:6873, rack: rack3}," +
"node3:{cores:7, freedisk: 262, heapUsage:7834, rack: rack2}," +
"node4:{cores:8, freedisk: 375, heapUsage:16900, nodeRole:overseer, sysprop.rack: rack1}" +
"}");
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState);
ClusterDataProvider cdp = new ClusterDataProvider() {
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return clusterDataProvider.getNodeValues(node, tags);
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return clusterDataProvider.getReplicaInfo(node, keys);
}
@Override
public Collection<String> getNodes() {
return clusterDataProvider.getNodes();
}
@Override
public String getPolicyNameByCollection(String coll) {
return "p1";
}
};
Policy.Session session = policy.createSession(cdp);
CollectionAdminRequest.AddReplica op = (CollectionAdminRequest.AddReplica) session
.getSuggester(ADDREPLICA)
.hint(Hint.COLL, "newColl")
.hint(Hint.SHARD, "s1").getOperation();
assertNotNull(op);
assertEquals("node2", op.getNode());
}
private ClusterDataProvider getClusterDataProvider(final Map<String, Map> nodeValues, String clusterState) {
return new ClusterDataProvider() {
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
Map<String, Object> result = new LinkedHashMap<>();
tags.stream().forEach(s -> result.put(s, nodeValues.get(node).get(s)));
return result;
}
@Override
public Collection<String> getNodes() {
return nodeValues.keySet();
}
@Override
public String getPolicyNameByCollection(String coll) {
return null;
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return getReplicaDetails(node, clusterState);
}
};
}
public void testEmptyClusterState(){
String autoScaleJson = " {'policies':{'c1':[{" +
" 'replica':1," +
" 'shard':'#EACH'," +
" 'port':'50096'}]}}";
Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
" '127.0.0.1:50097_solr':{" +
" 'cores':0," +
" 'port':'50097'}," +
" '127.0.0.1:50096_solr':{" +
" 'cores':0," +
" 'port':'50096'}}");
ClusterDataProvider dataProvider = new ClusterDataProvider() {
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> keys) {
Map<String, Object> result = new LinkedHashMap<>();
keys.stream().forEach(s -> result.put(s, nodeValues.get(node).get(s)));
return result;
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return getReplicaDetails(node, clusterState);
}
@Override
public String getPolicyNameByCollection(String coll) {
return null;
}
@Override
public Collection<String> getNodes() {
return Arrays.asList( "127.0.0.1:50097_solr", "127.0.0.1:50096_solr");
}
};
Map<String, List<String>> locations = PolicyHelper.getReplicaLocations(
"newColl", (Map<String, Object>) Utils.fromJSONString(autoScaleJson),
dataProvider, Collections.singletonMap("newColl", "c1"), Arrays.asList("shard1", "shard2"), 1, null);
assertTrue(locations.get("shard1").containsAll(ImmutableList.of("127.0.0.1:50096_solr")));
assertTrue(locations.get("shard2").containsAll(ImmutableList.of("127.0.0.1:50096_solr")));
}
public void testMultiReplicaPlacement() {
String autoScaleJson = "{" +
" 'cluster-preferences': [" +
" { maximize : freedisk , precision: 50}," +
" { minimize : cores, precision: 2}" +
" ]," +
" 'cluster-policy': [" +
" { replica : '0' , 'nodeRole': 'overseer'}," +
" { 'replica': '<2', 'shard': '#ANY', 'node': '#ANY'" +
" }" +
" ]," +
" 'policies': {" +
" 'policy1': [" +
" { 'replica': '<2', 'shard': '#EACH', 'node': '#ANY'}," +
" { 'replica': '<2', 'shard': '#EACH', 'sysprop.rack': 'rack1'}" +
" ]" +
" }" +
"}";
Map<String, Map> nodeValues = (Map<String, Map>) Utils.fromJSONString("{" +
"node1:{cores:12, freedisk: 334, heap:10480, sysprop.rack:rack3}," +
"node2:{cores:4, freedisk: 749, heap:6873, sysprop.fs : ssd, sysprop.rack:rack1}," +
"node3:{cores:7, freedisk: 262, heap:7834, sysprop.rack:rack4}," +
"node4:{cores:0, freedisk: 900, heap:16900, nodeRole:overseer, sysprop.rack:rack2}" +
"}");
ClusterDataProvider dataProvider = new ClusterDataProvider() {
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> keys) {
Map<String, Object> result = new LinkedHashMap<>();
keys.stream().forEach(s -> result.put(s, nodeValues.get(node).get(s)));
return result;
}
@Override
public Map<String, Map<String, List<Policy.ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return getReplicaDetails(node, clusterState);
}
@Override
public String getPolicyNameByCollection(String coll) {
return null;
}
@Override
public Collection<String> getNodes() {
return Arrays.asList("node1", "node2", "node3", "node4");
}
};
Map<String, List<String>> locations = PolicyHelper.getReplicaLocations(
"newColl", (Map<String, Object>) Utils.fromJSONString(autoScaleJson),
dataProvider, Collections.singletonMap("newColl", "policy1"), Arrays.asList("shard1", "shard2"), 3, null);
assertTrue(locations.get("shard1").containsAll(ImmutableList.of("node2", "node1", "node3")));
assertTrue(locations.get("shard2").containsAll(ImmutableList.of("node2", "node1", "node3")));
}
}