Revert "SOLR-11285: Initial refactoring."

This reverts commit aee54ff7d1.
This commit is contained in:
Andrzej Bialecki 2017-08-24 15:10:29 +02:00
parent aee54ff7d1
commit ce7db65e6f
54 changed files with 553 additions and 976 deletions

View File

@ -18,7 +18,6 @@
package org.apache.solr.cloud; package org.apache.solr.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -72,7 +71,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
} }
ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete) ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
throws IOException, InterruptedException { throws KeeperException, InterruptedException {
log.debug("addReplica() : {}", Utils.toJSONString(message)); log.debug("addReplica() : {}", Utils.toJSONString(message));
String collection = message.getStr(COLLECTION_PROP); String collection = message.getStr(COLLECTION_PROP);
String node = message.getStr(CoreAdminParams.NODE); String node = message.getStr(CoreAdminParams.NODE);
@ -121,7 +120,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
} }
} else { } else {
node = getNodesForNewReplicas(clusterState, collection, shard, 1, node, node = getNodesForNewReplicas(clusterState, collection, shard, 1, node,
ocmh.overseer.getClusterDataProvider(), ocmh.overseer.getCoreContainer()).get(0).nodeName;// TODO: use replica type in this logic too ocmh.overseer.getZkController().getCoreContainer()).get(0).nodeName;// TODO: use replica type in this logic too
} }
} }
log.info("Node Identified {} for creating new replica", node); log.info("Node Identified {} for creating new replica", node);
@ -160,11 +159,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
if (coreNodeName != null) { if (coreNodeName != null) {
props = props.plus(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName); props = props.plus(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
} }
try { Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(props));
Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(props));
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Exception updating Overseer state queue", e);
}
} }
params.set(CoreAdminParams.CORE_NODE_NAME, params.set(CoreAdminParams.CORE_NODE_NAME,
ocmh.waitToSeeReplicasInState(collection, Collections.singletonList(coreName)).get(coreName).getName()); ocmh.waitToSeeReplicasInState(collection, Collections.singletonList(coreName)).get(coreName).getName());

View File

@ -31,7 +31,6 @@ import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper; import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider; import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
@ -244,10 +243,10 @@ public class Assign {
List<String> shardNames, List<String> shardNames,
int numNrtReplicas, int numNrtReplicas,
int numTlogReplicas, int numTlogReplicas,
int numPullReplicas) throws IOException, InterruptedException { int numPullReplicas) throws KeeperException, InterruptedException {
List<Map> rulesMap = (List) message.get("rule"); List<Map> rulesMap = (List) message.get("rule");
String policyName = message.getStr(POLICY); String policyName = message.getStr(POLICY);
AutoScalingConfig autoScalingConfig = ocmh.overseer.getClusterDataProvider().getAutoScalingConfig(); AutoScalingConfig autoScalingConfig = ocmh.zkStateReader.getAutoScalingConfig();
if (rulesMap == null && policyName == null && autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) { if (rulesMap == null && policyName == null && autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) {
log.debug("Identify nodes using default"); log.debug("Identify nodes using default");
@ -296,7 +295,7 @@ public class Assign {
PolicyHelper.SESSION_REF.set(ocmh.policySessionRef); PolicyHelper.SESSION_REF.set(ocmh.policySessionRef);
try { try {
return getPositionsUsingPolicy(collectionName, return getPositionsUsingPolicy(collectionName,
shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, ocmh.overseer.getClusterDataProvider(), nodeList); shardNames, numNrtReplicas, numTlogReplicas, numPullReplicas, policyName, ocmh.zkStateReader, nodeList);
} finally { } finally {
PolicyHelper.SESSION_REF.remove(); PolicyHelper.SESSION_REF.remove();
} }
@ -325,7 +324,7 @@ public class Assign {
// could be created on live nodes given maxShardsPerNode, Replication factor (if from createShard) etc. // could be created on live nodes given maxShardsPerNode, Replication factor (if from createShard) etc.
public static List<ReplicaCount> getNodesForNewReplicas(ClusterState clusterState, String collectionName, public static List<ReplicaCount> getNodesForNewReplicas(ClusterState clusterState, String collectionName,
String shard, int nrtReplicas, String shard, int nrtReplicas,
Object createNodeSet, ClusterDataProvider cdp, CoreContainer cc) throws IOException, InterruptedException { Object createNodeSet, CoreContainer cc) throws KeeperException, InterruptedException {
log.debug("getNodesForNewReplicas() shard: {} , replicas : {} , createNodeSet {}", shard, nrtReplicas, createNodeSet ); log.debug("getNodesForNewReplicas() shard: {} , replicas : {} , createNodeSet {}", shard, nrtReplicas, createNodeSet );
DocCollection coll = clusterState.getCollection(collectionName); DocCollection coll = clusterState.getCollection(collectionName);
Integer maxShardsPerNode = coll.getMaxShardsPerNode(); Integer maxShardsPerNode = coll.getMaxShardsPerNode();
@ -357,14 +356,13 @@ public class Assign {
List l = (List) coll.get(DocCollection.RULE); List l = (List) coll.get(DocCollection.RULE);
List<ReplicaPosition> replicaPositions = null; List<ReplicaPosition> replicaPositions = null;
if (l != null) { if (l != null) {
// TODO nocommit: make it so that this method doesn't require access to CC
replicaPositions = getNodesViaRules(clusterState, shard, nrtReplicas, cc, coll, createNodeList, l); replicaPositions = getNodesViaRules(clusterState, shard, nrtReplicas, cc, coll, createNodeList, l);
} }
String policyName = coll.getStr(POLICY); String policyName = coll.getStr(POLICY);
AutoScalingConfig autoScalingConfig = cdp.getAutoScalingConfig(); AutoScalingConfig autoScalingConfig = cc.getZkController().zkStateReader.getAutoScalingConfig();
if (policyName != null || !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) { if (policyName != null || !autoScalingConfig.getPolicy().getClusterPolicy().isEmpty()) {
replicaPositions = Assign.getPositionsUsingPolicy(collectionName, Collections.singletonList(shard), nrtReplicas, 0, 0, replicaPositions = Assign.getPositionsUsingPolicy(collectionName, Collections.singletonList(shard), nrtReplicas, 0, 0,
policyName, cdp, createNodeList); policyName, cc.getZkController().zkStateReader, createNodeList);
} }
if(replicaPositions != null){ if(replicaPositions != null){
@ -385,18 +383,21 @@ public class Assign {
int nrtReplicas, int nrtReplicas,
int tlogReplicas, int tlogReplicas,
int pullReplicas, int pullReplicas,
String policyName, ClusterDataProvider cdp, String policyName, ZkStateReader zkStateReader,
List<String> nodesList) throws IOException, InterruptedException { List<String> nodesList) throws KeeperException, InterruptedException {
log.debug("shardnames {} NRT {} TLOG {} PULL {} , policy {}, nodeList {}", shardNames, nrtReplicas, tlogReplicas, pullReplicas, policyName, nodesList); log.debug("shardnames {} NRT {} TLOG {} PULL {} , policy {}, nodeList {}", shardNames, nrtReplicas, tlogReplicas, pullReplicas, policyName, nodesList);
SolrClientDataProvider clientDataProvider = null; SolrClientDataProvider clientDataProvider = null;
List<ReplicaPosition> replicaPositions = null; List<ReplicaPosition> replicaPositions = null;
AutoScalingConfig autoScalingConfig = cdp.getAutoScalingConfig(); AutoScalingConfig autoScalingConfig = zkStateReader.getAutoScalingConfig();
try { try (CloudSolrClient csc = new CloudSolrClient.Builder()
.withClusterStateProvider(new ZkClientClusterStateProvider(zkStateReader))
.build()) {
clientDataProvider = new SolrClientDataProvider(csc);
Map<String, String> kvMap = Collections.singletonMap(collName, policyName); Map<String, String> kvMap = Collections.singletonMap(collName, policyName);
replicaPositions = PolicyHelper.getReplicaLocations( replicaPositions = PolicyHelper.getReplicaLocations(
collName, collName,
autoScalingConfig, autoScalingConfig,
cdp, clientDataProvider,
kvMap, kvMap,
shardNames, shardNames,
nrtReplicas, nrtReplicas,
@ -404,7 +405,7 @@ public class Assign {
pullReplicas, pullReplicas,
nodesList); nodesList);
return replicaPositions; return replicaPositions;
} catch (Exception e) { } catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error closing CloudSolrClient",e); throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error closing CloudSolrClient",e);
} finally { } finally {
if (log.isTraceEnabled()) { if (log.isTraceEnabled()) {

View File

@ -17,7 +17,6 @@
package org.apache.solr.cloud; package org.apache.solr.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@ -28,7 +27,6 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy; import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper; import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd; import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
@ -105,7 +103,7 @@ public class CreateShardCmd implements Cmd {
numPullReplicas); numPullReplicas);
} else { } else {
List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas, List<Assign.ReplicaCount> sortedNodeList = getNodesForNewReplicas(clusterState, collectionName, sliceName, totalReplicas,
createNodeSetStr, ocmh.overseer.getClusterDataProvider(), ocmh.overseer.getCoreContainer()); createNodeSetStr, ocmh.overseer.getZkController().getCoreContainer());
int i = 0; int i = 0;
positions = new ArrayList<>(); positions = new ArrayList<>();
for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas, for (Map.Entry<Replica.Type, Integer> e : ImmutableMap.of(Replica.Type.NRT, numNrtReplicas,
@ -175,8 +173,8 @@ public class CreateShardCmd implements Cmd {
} }
static boolean usePolicyFramework(DocCollection collection, OverseerCollectionMessageHandler ocmh) static boolean usePolicyFramework(DocCollection collection, OverseerCollectionMessageHandler ocmh)
throws IOException, InterruptedException { throws KeeperException, InterruptedException {
AutoScalingConfig autoScalingConfig = ocmh.overseer.getClusterDataProvider().getAutoScalingConfig(); Map autoScalingJson = Utils.getJson(ocmh.zkStateReader.getZkClient(), SOLR_AUTOSCALING_CONF_PATH, true);
return !autoScalingConfig.isEmpty() || collection.getPolicyName() != null; return autoScalingJson.get(Policy.CLUSTER_POLICY) != null || collection.getPolicyName() != null;
} }
} }

View File

@ -26,7 +26,6 @@ import java.util.Map;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd; import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;
import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;

View File

@ -30,7 +30,6 @@ import java.util.function.Predicate;
import com.codahale.metrics.Timer; import com.codahale.metrics.Timer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
@ -44,11 +43,11 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
/** /**
* A ZK-based distributed queue. Optimized for single-consumer, * A distributed queue. Optimized for single-consumer,
* multiple-producer: if there are multiple consumers on the same ZK queue, * multiple-producer: if there are multiple consumers on the same ZK queue,
* the results should be correct but inefficient * the results should be correct but inefficient
*/ */
public class ZkDistributedQueue implements DistributedQueue { public class DistributedQueue {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
static final String PREFIX = "qn-"; static final String PREFIX = "qn-";
@ -93,11 +92,11 @@ public class ZkDistributedQueue implements DistributedQueue {
private int watcherCount = 0; private int watcherCount = 0;
public ZkDistributedQueue(SolrZkClient zookeeper, String dir) { public DistributedQueue(SolrZkClient zookeeper, String dir) {
this(zookeeper, dir, new Overseer.Stats()); this(zookeeper, dir, new Overseer.Stats());
} }
public ZkDistributedQueue(SolrZkClient zookeeper, String dir, Overseer.Stats stats) { public DistributedQueue(SolrZkClient zookeeper, String dir, Overseer.Stats stats) {
this.dir = dir; this.dir = dir;
ZkCmdExecutor cmdExecutor = new ZkCmdExecutor(zookeeper.getZkClientTimeout()); ZkCmdExecutor cmdExecutor = new ZkCmdExecutor(zookeeper.getZkClientTimeout());
@ -120,7 +119,6 @@ public class ZkDistributedQueue implements DistributedQueue {
* *
* @return data at the first element of the queue, or null. * @return data at the first element of the queue, or null.
*/ */
@Override
public byte[] peek() throws KeeperException, InterruptedException { public byte[] peek() throws KeeperException, InterruptedException {
Timer.Context time = stats.time(dir + "_peek"); Timer.Context time = stats.time(dir + "_peek");
try { try {
@ -137,7 +135,6 @@ public class ZkDistributedQueue implements DistributedQueue {
* @param block if true, blocks until an element enters the queue * @param block if true, blocks until an element enters the queue
* @return data at the first element of the queue, or null. * @return data at the first element of the queue, or null.
*/ */
@Override
public byte[] peek(boolean block) throws KeeperException, InterruptedException { public byte[] peek(boolean block) throws KeeperException, InterruptedException {
return block ? peek(Long.MAX_VALUE) : peek(); return block ? peek(Long.MAX_VALUE) : peek();
} }
@ -149,7 +146,6 @@ public class ZkDistributedQueue implements DistributedQueue {
* @param wait max wait time in ms. * @param wait max wait time in ms.
* @return data at the first element of the queue, or null. * @return data at the first element of the queue, or null.
*/ */
@Override
public byte[] peek(long wait) throws KeeperException, InterruptedException { public byte[] peek(long wait) throws KeeperException, InterruptedException {
Preconditions.checkArgument(wait > 0); Preconditions.checkArgument(wait > 0);
Timer.Context time; Timer.Context time;
@ -181,7 +177,6 @@ public class ZkDistributedQueue implements DistributedQueue {
* *
* @return Head of the queue or null. * @return Head of the queue or null.
*/ */
@Override
public byte[] poll() throws KeeperException, InterruptedException { public byte[] poll() throws KeeperException, InterruptedException {
Timer.Context time = stats.time(dir + "_poll"); Timer.Context time = stats.time(dir + "_poll");
try { try {
@ -196,7 +191,6 @@ public class ZkDistributedQueue implements DistributedQueue {
* *
* @return The former head of the queue * @return The former head of the queue
*/ */
@Override
public byte[] remove() throws NoSuchElementException, KeeperException, InterruptedException { public byte[] remove() throws NoSuchElementException, KeeperException, InterruptedException {
Timer.Context time = stats.time(dir + "_remove"); Timer.Context time = stats.time(dir + "_remove");
try { try {
@ -215,7 +209,6 @@ public class ZkDistributedQueue implements DistributedQueue {
* *
* @return The former head of the queue * @return The former head of the queue
*/ */
@Override
public byte[] take() throws KeeperException, InterruptedException { public byte[] take() throws KeeperException, InterruptedException {
// Same as for element. Should refactor this. // Same as for element. Should refactor this.
Timer.Context timer = stats.time(dir + "_take"); Timer.Context timer = stats.time(dir + "_take");
@ -238,7 +231,6 @@ public class ZkDistributedQueue implements DistributedQueue {
* Inserts data into queue. If there are no other queue consumers, the offered element * Inserts data into queue. If there are no other queue consumers, the offered element
* will be immediately visible when this method returns. * will be immediately visible when this method returns.
*/ */
@Override
public void offer(byte[] data) throws KeeperException, InterruptedException { public void offer(byte[] data) throws KeeperException, InterruptedException {
Timer.Context time = stats.time(dir + "_offer"); Timer.Context time = stats.time(dir + "_offer");
try { try {
@ -334,8 +326,7 @@ public class ZkDistributedQueue implements DistributedQueue {
* <p/> * <p/>
* Package-private to support {@link OverseerTaskQueue} specifically. * Package-private to support {@link OverseerTaskQueue} specifically.
*/ */
@Override Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
List<String> foundChildren = new ArrayList<>(); List<String> foundChildren = new ArrayList<>();
long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis); long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
boolean first = true; boolean first = true;

View File

@ -223,11 +223,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
ZkStateReader.CORE_NAME_PROP, ZkStateReader.CORE_NAME_PROP,
leaderProps.getProperties().get(ZkStateReader.CORE_NAME_PROP), leaderProps.getProperties().get(ZkStateReader.CORE_NAME_PROP),
ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString()); ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
try { Overseer.getStateUpdateQueue(zkClient).offer(Utils.toJSON(m));
Overseer.getStateUpdateQueue(zkClient).offer(Utils.toJSON(m));
} catch (Exception e) {
throw new IOException("Overseer state update queue error", e);
}
} }
public LeaderElector getLeaderElector() { public LeaderElector getLeaderElector() {
@ -316,11 +312,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
// clear the leader in clusterstate // clear the leader in clusterstate
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(), ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(),
ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, collection); ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, collection);
try { Overseer.getStateUpdateQueue(zkClient).offer(Utils.toJSON(m));
Overseer.getStateUpdateQueue(zkClient).offer(Utils.toJSON(m));
} catch (Exception e) {
throw new IOException("Overseer state update queue error", e);
}
boolean allReplicasInLine = false; boolean allReplicasInLine = false;
if (!weAreReplacement) { if (!weAreReplacement) {
@ -502,7 +494,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
} }
} }
public void publishActiveIfRegisteredAndNotActive(SolrCore core) throws Exception { public void publishActiveIfRegisteredAndNotActive(SolrCore core) throws KeeperException, InterruptedException {
if (core.getCoreDescriptor().getCloudDescriptor().hasRegistered()) { if (core.getCoreDescriptor().getCloudDescriptor().hasRegistered()) {
ZkStateReader zkStateReader = zkController.getZkStateReader(); ZkStateReader zkStateReader = zkController.getZkStateReader();
zkStateReader.forceUpdateCollection(collection); zkStateReader.forceUpdateCollection(collection);

View File

@ -30,15 +30,9 @@ import java.util.concurrent.atomic.AtomicInteger;
import com.codahale.metrics.Timer; import com.codahale.metrics.Timer;
import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
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.AutoScaling; import org.apache.solr.cloud.autoscaling.AutoScaling;
import org.apache.solr.cloud.autoscaling.AutoScalingHandler; import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
import org.apache.solr.cloud.autoscaling.OverseerTriggerThread; import org.apache.solr.cloud.autoscaling.OverseerTriggerThread;
import org.apache.solr.cloud.autoscaling.ZkDistributedQueueFactory;
import org.apache.solr.cloud.overseer.ClusterStateMutator; import org.apache.solr.cloud.overseer.ClusterStateMutator;
import org.apache.solr.cloud.overseer.CollectionMutator; import org.apache.solr.cloud.overseer.CollectionMutator;
import org.apache.solr.cloud.overseer.NodeMutator; import org.apache.solr.cloud.overseer.NodeMutator;
@ -59,7 +53,6 @@ import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.ObjectReleaseTracker; import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.Utils; import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloudConfig; import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.admin.CollectionsHandler; import org.apache.solr.handler.admin.CollectionsHandler;
import org.apache.solr.handler.component.ShardHandler; import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.LocalSolrQueryRequest;
@ -95,10 +88,10 @@ public class Overseer implements Closeable {
private final SolrZkClient zkClient; private final SolrZkClient zkClient;
private final String myId; private final String myId;
//queue where everybody can throw tasks //queue where everybody can throw tasks
private final ZkDistributedQueue stateUpdateQueue; private final DistributedQueue stateUpdateQueue;
//Internal queue where overseer stores events that have not yet been published into cloudstate //Internal queue where overseer stores events that have not yet been published into cloudstate
//If Overseer dies while extracting the main queue a new overseer will start from this queue //If Overseer dies while extracting the main queue a new overseer will start from this queue
private final ZkDistributedQueue workQueue; private final DistributedQueue workQueue;
// Internal map which holds the information about running tasks. // Internal map which holds the information about running tasks.
private final DistributedMap runningMap; private final DistributedMap runningMap;
// Internal map which holds the information about successfully completed tasks. // Internal map which holds the information about successfully completed tasks.
@ -545,8 +538,7 @@ public class Overseer implements Closeable {
autoscalingTriggerCreator.start(); autoscalingTriggerCreator.start();
ThreadGroup triggerThreadGroup = new ThreadGroup("Overseer autoscaling triggers"); ThreadGroup triggerThreadGroup = new ThreadGroup("Overseer autoscaling triggers");
OverseerTriggerThread trigger = new OverseerTriggerThread(zkController.getCoreContainer().getResourceLoader(), OverseerTriggerThread trigger = new OverseerTriggerThread(zkController);
zkController.getClusterDataProvider());
triggerThread = new OverseerThread(triggerThreadGroup, trigger, "OverseerAutoScalingTriggerThread-" + id); triggerThread = new OverseerThread(triggerThreadGroup, trigger, "OverseerAutoScalingTriggerThread-" + id);
updaterThread.start(); updaterThread.start();
@ -562,15 +554,7 @@ public class Overseer implements Closeable {
ZkController getZkController(){ ZkController getZkController(){
return zkController; return zkController;
} }
public CoreContainer getCoreContainer() {
return zkController.getCoreContainer();
}
public ClusterDataProvider getClusterDataProvider() {
return zkController.getClusterDataProvider();
}
/** /**
* For tests. * For tests.
* *
@ -695,7 +679,7 @@ public class Overseer implements Closeable {
* This method will create the /overseer znode in ZooKeeper if it does not exist already. * This method will create the /overseer znode in ZooKeeper if it does not exist already.
* *
* @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue * @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue
* @return a {@link ZkDistributedQueue} object * @return a {@link DistributedQueue} object
*/ */
public static DistributedQueue getStateUpdateQueue(final SolrZkClient zkClient) { public static DistributedQueue getStateUpdateQueue(final SolrZkClient zkClient) {
return getStateUpdateQueue(zkClient, new Stats()); return getStateUpdateQueue(zkClient, new Stats());
@ -708,11 +692,11 @@ public class Overseer implements Closeable {
* *
* @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue * @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue
* @param zkStats a {@link Overseer.Stats} object which tracks statistics for all zookeeper operations performed by this queue * @param zkStats a {@link Overseer.Stats} object which tracks statistics for all zookeeper operations performed by this queue
* @return a {@link ZkDistributedQueue} object * @return a {@link DistributedQueue} object
*/ */
static ZkDistributedQueue getStateUpdateQueue(final SolrZkClient zkClient, Stats zkStats) { static DistributedQueue getStateUpdateQueue(final SolrZkClient zkClient, Stats zkStats) {
createOverseerNode(zkClient); createOverseerNode(zkClient);
return new ZkDistributedQueue(zkClient, "/overseer/queue", zkStats); return new DistributedQueue(zkClient, "/overseer/queue", zkStats);
} }
/** /**
@ -728,11 +712,11 @@ public class Overseer implements Closeable {
* *
* @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue * @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue
* @param zkStats a {@link Overseer.Stats} object which tracks statistics for all zookeeper operations performed by this queue * @param zkStats a {@link Overseer.Stats} object which tracks statistics for all zookeeper operations performed by this queue
* @return a {@link ZkDistributedQueue} object * @return a {@link DistributedQueue} object
*/ */
static ZkDistributedQueue getInternalWorkQueue(final SolrZkClient zkClient, Stats zkStats) { static DistributedQueue getInternalWorkQueue(final SolrZkClient zkClient, Stats zkStats) {
createOverseerNode(zkClient); createOverseerNode(zkClient);
return new ZkDistributedQueue(zkClient, "/overseer/queue-work", zkStats); return new DistributedQueue(zkClient, "/overseer/queue-work", zkStats);
} }
/* Internal map for failed tasks, not to be used outside of the Overseer */ /* Internal map for failed tasks, not to be used outside of the Overseer */
@ -766,7 +750,7 @@ public class Overseer implements Closeable {
* see {@link org.apache.solr.common.params.CollectionParams.CollectionAction#OVERSEERSTATUS}. * see {@link org.apache.solr.common.params.CollectionParams.CollectionAction#OVERSEERSTATUS}.
* *
* @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue * @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue
* @return a {@link ZkDistributedQueue} object * @return a {@link DistributedQueue} object
*/ */
static OverseerTaskQueue getCollectionQueue(final SolrZkClient zkClient) { static OverseerTaskQueue getCollectionQueue(final SolrZkClient zkClient) {
return getCollectionQueue(zkClient, new Stats()); return getCollectionQueue(zkClient, new Stats());
@ -784,7 +768,7 @@ public class Overseer implements Closeable {
* see {@link org.apache.solr.common.params.CollectionParams.CollectionAction#OVERSEERSTATUS}. * see {@link org.apache.solr.common.params.CollectionParams.CollectionAction#OVERSEERSTATUS}.
* *
* @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue * @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue
* @return a {@link ZkDistributedQueue} object * @return a {@link DistributedQueue} object
*/ */
static OverseerTaskQueue getCollectionQueue(final SolrZkClient zkClient, Stats zkStats) { static OverseerTaskQueue getCollectionQueue(final SolrZkClient zkClient, Stats zkStats) {
createOverseerNode(zkClient); createOverseerNode(zkClient);
@ -804,7 +788,7 @@ public class Overseer implements Closeable {
* see {@link org.apache.solr.common.params.CollectionParams.CollectionAction#OVERSEERSTATUS}. * see {@link org.apache.solr.common.params.CollectionParams.CollectionAction#OVERSEERSTATUS}.
* *
* @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue * @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue
* @return a {@link ZkDistributedQueue} object * @return a {@link DistributedQueue} object
*/ */
static OverseerTaskQueue getConfigSetQueue(final SolrZkClient zkClient) { static OverseerTaskQueue getConfigSetQueue(final SolrZkClient zkClient) {
return getConfigSetQueue(zkClient, new Stats()); return getConfigSetQueue(zkClient, new Stats());
@ -827,7 +811,7 @@ public class Overseer implements Closeable {
* {@link OverseerConfigSetMessageHandler}. * {@link OverseerConfigSetMessageHandler}.
* *
* @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue * @param zkClient the {@link SolrZkClient} to be used for reading/writing to the queue
* @return a {@link ZkDistributedQueue} object * @return a {@link DistributedQueue} object
*/ */
static OverseerTaskQueue getConfigSetQueue(final SolrZkClient zkClient, Stats zkStats) { static OverseerTaskQueue getConfigSetQueue(final SolrZkClient zkClient, Stats zkStats) {
// For now, we use the same queue as the collection queue, but ensure // For now, we use the same queue as the collection queue, but ensure

View File

@ -36,7 +36,6 @@ import com.google.common.collect.ImmutableMap;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper; import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException; import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
@ -322,7 +321,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
} }
private void processReplicaDeletePropertyCommand(ClusterState clusterState, ZkNodeProps message, NamedList results) private void processReplicaDeletePropertyCommand(ClusterState clusterState, ZkNodeProps message, NamedList results)
throws Exception { throws KeeperException, InterruptedException {
checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP); checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP);
SolrZkClient zkClient = zkStateReader.getZkClient(); SolrZkClient zkClient = zkStateReader.getZkClient();
DistributedQueue inQueue = Overseer.getStateUpdateQueue(zkClient); DistributedQueue inQueue = Overseer.getStateUpdateQueue(zkClient);
@ -333,7 +332,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
inQueue.offer(Utils.toJSON(m)); inQueue.offer(Utils.toJSON(m));
} }
private void balanceProperty(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception { private void balanceProperty(ClusterState clusterState, ZkNodeProps message, NamedList results) throws KeeperException, InterruptedException {
if (StringUtils.isBlank(message.getStr(COLLECTION_PROP)) || StringUtils.isBlank(message.getStr(PROPERTY_PROP))) { if (StringUtils.isBlank(message.getStr(COLLECTION_PROP)) || StringUtils.isBlank(message.getStr(PROPERTY_PROP))) {
throw new SolrException(ErrorCode.BAD_REQUEST, throw new SolrException(ErrorCode.BAD_REQUEST,
"The '" + COLLECTION_PROP + "' and '" + PROPERTY_PROP + "The '" + COLLECTION_PROP + "' and '" + PROPERTY_PROP +
@ -442,7 +441,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
} }
} }
void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws Exception { void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws KeeperException, InterruptedException {
ZkNodeProps m = new ZkNodeProps( ZkNodeProps m = new ZkNodeProps(
Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(), Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
ZkStateReader.CORE_NAME_PROP, core, ZkStateReader.CORE_NAME_PROP, core,
@ -463,7 +462,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
} }
//TODO should we not remove in the next release ? //TODO should we not remove in the next release ?
private void migrateStateFormat(ClusterState state, ZkNodeProps message, NamedList results) throws Exception { private void migrateStateFormat(ClusterState state, ZkNodeProps message, NamedList results)
throws KeeperException, InterruptedException {
final String collectionName = message.getStr(COLLECTION_PROP); final String collectionName = message.getStr(COLLECTION_PROP);
boolean firstLoop = true; boolean firstLoop = true;
@ -634,7 +634,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
private void modifyCollection(ClusterState clusterState, ZkNodeProps message, NamedList results) private void modifyCollection(ClusterState clusterState, ZkNodeProps message, NamedList results)
throws Exception { throws KeeperException, InterruptedException {
final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP); final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
//the rest of the processing is based on writing cluster state properties //the rest of the processing is based on writing cluster state properties
@ -712,7 +712,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
} }
ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete) ZkNodeProps addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
throws Exception { throws KeeperException, InterruptedException {
return ((AddReplicaCmd) commandMap.get(ADDREPLICA)).addReplica(clusterState, message, results, onComplete); return ((AddReplicaCmd) commandMap.get(ADDREPLICA)).addReplica(clusterState, message, results, onComplete);
} }

View File

@ -56,7 +56,7 @@ public class OverseerNodePrioritizer {
this.shardHandlerFactory = shardHandlerFactory; this.shardHandlerFactory = shardHandlerFactory;
} }
public synchronized void prioritizeOverseerNodes(String overseerId) throws Exception { public synchronized void prioritizeOverseerNodes(String overseerId) throws KeeperException, InterruptedException {
SolrZkClient zk = zkStateReader.getZkClient(); SolrZkClient zk = zkStateReader.getZkClient();
if(!zk.exists(ZkStateReader.ROLES,true))return; if(!zk.exists(ZkStateReader.ROLES,true))return;
Map m = (Map) Utils.fromJSON(zk.getData(ZkStateReader.ROLES, null, new Stat(), true)); Map m = (Map) Utils.fromJSON(zk.getData(ZkStateReader.ROLES, null, new Stat(), true));

View File

@ -35,11 +35,11 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
/** /**
* A {@link ZkDistributedQueue} augmented with helper methods specific to the overseer task queues. * A {@link DistributedQueue} augmented with helper methods specific to the overseer task queues.
* Methods specific to this subclass ignore superclass internal state and hit ZK directly. * Methods specific to this subclass ignore superclass internal state and hit ZK directly.
* This is inefficient! But the API on this class is kind of muddy.. * This is inefficient! But the API on this class is kind of muddy..
*/ */
public class OverseerTaskQueue extends ZkDistributedQueue { public class OverseerTaskQueue extends DistributedQueue {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String RESPONSE_PREFIX = "qnr-" ; private static final String RESPONSE_PREFIX = "qnr-" ;

View File

@ -174,7 +174,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
final private void recoveryFailed(final SolrCore core, final private void recoveryFailed(final SolrCore core,
final ZkController zkController, final String baseUrl, final ZkController zkController, final String baseUrl,
final String shardZkNodeName, final CoreDescriptor cd) throws Exception { final String shardZkNodeName, final CoreDescriptor cd) throws KeeperException, InterruptedException {
SolrException.log(LOG, "Recovery failed - I give up."); SolrException.log(LOG, "Recovery failed - I give up.");
try { try {
zkController.publish(cd, Replica.State.RECOVERY_FAILED); zkController.publish(cd, Replica.State.RECOVERY_FAILED);
@ -297,7 +297,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
} }
} }
final public void doRecovery(SolrCore core) throws Exception { final public void doRecovery(SolrCore core) throws KeeperException, InterruptedException {
if (core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog()) { if (core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog()) {
doSyncOrReplicateRecovery(core); doSyncOrReplicateRecovery(core);
} else { } else {
@ -440,7 +440,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
} }
// TODO: perhaps make this grab a new core each time through the loop to handle core reloads? // TODO: perhaps make this grab a new core each time through the loop to handle core reloads?
final public void doSyncOrReplicateRecovery(SolrCore core) throws Exception { final public void doSyncOrReplicateRecovery(SolrCore core) throws KeeperException, InterruptedException {
boolean replayed = false; boolean replayed = false;
boolean successfulRecovery = false; boolean successfulRecovery = false;

View File

@ -32,8 +32,8 @@ import java.util.Objects;
import java.util.Optional; import java.util.Optional;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper; import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.cloud.ReplicaPosition; import org.apache.solr.common.cloud.ReplicaPosition;

View File

@ -27,7 +27,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper; import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
import org.apache.solr.client.solrj.request.CoreAdminRequest; import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd; import org.apache.solr.cloud.OverseerCollectionMessageHandler.Cmd;

View File

@ -51,16 +51,10 @@ import java.util.concurrent.atomic.AtomicReference;
import com.google.common.base.Strings; import com.google.common.base.Strings;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder; import org.apache.solr.client.solrj.impl.HttpSolrClient.Builder;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
import org.apache.solr.client.solrj.impl.ZkClientClusterStateProvider;
import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState; import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.autoscaling.ZkDistributedQueueFactory;
import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.cloud.overseer.SliceMutator; import org.apache.solr.cloud.overseer.SliceMutator;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
@ -197,8 +191,6 @@ public class ZkController {
private final SolrZkClient zkClient; private final SolrZkClient zkClient;
private final ZkCmdExecutor cmdExecutor; private final ZkCmdExecutor cmdExecutor;
public final ZkStateReader zkStateReader; public final ZkStateReader zkStateReader;
private ClusterDataProvider clusterDataProvider;
private CloudSolrClient cloudSolrClient;
private final String zkServerAddress; // example: 127.0.0.1:54062/solr private final String zkServerAddress; // example: 127.0.0.1:54062/solr
@ -443,7 +435,7 @@ public class ZkController {
}); });
init(registerOnReconnect); init(registerOnReconnect);
assert ObjectReleaseTracker.track(this); assert ObjectReleaseTracker.track(this);
} }
@ -562,12 +554,6 @@ public class ZkController {
IOUtils.closeQuietly(overseerElector.getContext()); IOUtils.closeQuietly(overseerElector.getContext());
IOUtils.closeQuietly(overseer); IOUtils.closeQuietly(overseer);
} finally { } finally {
if (cloudSolrClient != null) {
IOUtils.closeQuietly(cloudSolrClient);
}
if (clusterDataProvider != null) {
IOUtils.closeQuietly(clusterDataProvider);
}
try { try {
try { try {
zkStateReader.close(); zkStateReader.close();
@ -602,22 +588,6 @@ public class ZkController {
return zkStateReader.getClusterState(); return zkStateReader.getClusterState();
} }
public ClusterDataProvider getClusterDataProvider() {
if (clusterDataProvider != null) {
return clusterDataProvider;
}
synchronized(this) {
if (clusterDataProvider != null) {
return clusterDataProvider;
}
cloudSolrClient = new CloudSolrClient.Builder()
.withClusterStateProvider(new ZkClientClusterStateProvider(zkStateReader))
.build();
clusterDataProvider = new SolrClientDataProvider(new ZkDistributedQueueFactory(zkClient), cloudSolrClient);
}
return clusterDataProvider;
}
/** /**
* Returns config file data (in bytes) * Returns config file data (in bytes)
*/ */
@ -1320,18 +1290,18 @@ public class ZkController {
return baseURL; return baseURL;
} }
public void publish(final CoreDescriptor cd, final Replica.State state) throws Exception { public void publish(final CoreDescriptor cd, final Replica.State state) throws KeeperException, InterruptedException {
publish(cd, state, true); publish(cd, state, true);
} }
public void publish(final CoreDescriptor cd, final Replica.State state, boolean updateLastState) throws Exception { public void publish(final CoreDescriptor cd, final Replica.State state, boolean updateLastState) throws KeeperException, InterruptedException {
publish(cd, state, updateLastState, false); publish(cd, state, updateLastState, false);
} }
/** /**
* Publish core state to overseer. * Publish core state to overseer.
*/ */
public void publish(final CoreDescriptor cd, final Replica.State state, boolean updateLastState, boolean forcePublish) throws Exception { public void publish(final CoreDescriptor cd, final Replica.State state, boolean updateLastState, boolean forcePublish) throws KeeperException, InterruptedException {
if (!forcePublish) { if (!forcePublish) {
try (SolrCore core = cc.getCore(cd.getName())) { try (SolrCore core = cc.getCore(cd.getName())) {
if (core == null || core.isClosed()) { if (core == null || core.isClosed()) {
@ -1440,7 +1410,7 @@ public class ZkController {
return true; return true;
} }
public void unregister(String coreName, CoreDescriptor cd) throws Exception { public void unregister(String coreName, CoreDescriptor cd) throws InterruptedException, KeeperException {
final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName(); final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
final String collection = cd.getCloudDescriptor().getCollectionName(); final String collection = cd.getCloudDescriptor().getCollectionName();
@ -1471,7 +1441,8 @@ public class ZkController {
overseerJobQueue.offer(Utils.toJSON(m)); overseerJobQueue.offer(Utils.toJSON(m));
} }
public void createCollection(String collection) throws Exception { public void createCollection(String collection) throws KeeperException,
InterruptedException {
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
CollectionParams.CollectionAction.CREATE.toLower(), ZkStateReader.NODE_NAME_PROP, getNodeName(), CollectionParams.CollectionAction.CREATE.toLower(), ZkStateReader.NODE_NAME_PROP, getNodeName(),
ZkStateReader.COLLECTION_PROP, collection); ZkStateReader.COLLECTION_PROP, collection);
@ -1596,9 +1567,6 @@ public class ZkController {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
log.error("", e); log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e); throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (Exception e) {
log.error("", e);
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} }
if (cd.getCloudDescriptor().getShardId() == null && needsToBeAssignedShardId(cd, zkStateReader.getClusterState(), coreNodeName)) { if (cd.getCloudDescriptor().getShardId() == null && needsToBeAssignedShardId(cd, zkStateReader.getClusterState(), coreNodeName)) {

View File

@ -20,7 +20,6 @@ package org.apache.solr.cloud.autoscaling;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.common.MapWriter; import org.apache.solr.common.MapWriter;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
@ -31,18 +30,18 @@ import org.apache.solr.core.CoreContainer;
*/ */
public class ActionContext implements MapWriter { public class ActionContext implements MapWriter {
private final ClusterDataProvider clusterDataProvider; private final CoreContainer coreContainer;
private final AutoScaling.Trigger source; private final AutoScaling.Trigger source;
private final Map<String, Object> properties; private final Map<String, Object> properties;
public ActionContext(ClusterDataProvider clusterDataProvider, AutoScaling.Trigger source, Map<String, Object> properties) { public ActionContext(CoreContainer coreContainer, AutoScaling.Trigger source, Map<String, Object> properties) {
this.clusterDataProvider = clusterDataProvider; this.coreContainer = coreContainer;
this.source = source; this.source = source;
this.properties = properties; this.properties = properties;
} }
public ClusterDataProvider getClusterDataProvider() { public CoreContainer getCoreContainer() {
return clusterDataProvider; return coreContainer;
} }
public AutoScaling.Trigger getSource() { public AutoScaling.Trigger getSource() {

View File

@ -18,12 +18,8 @@
package org.apache.solr.cloud.autoscaling; package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.NoneSuggester; import org.apache.solr.client.solrj.cloud.autoscaling.NoneSuggester;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy; import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
@ -31,20 +27,15 @@ import org.apache.solr.common.cloud.ZkStateReader;
public class AutoAddReplicasPlanAction extends ComputePlanAction { public class AutoAddReplicasPlanAction extends ComputePlanAction {
@Override @Override
protected Policy.Suggester getSuggester(Policy.Session session, TriggerEvent event, ClusterDataProvider cdp) { protected Policy.Suggester getSuggester(Policy.Session session, TriggerEvent event, ZkStateReader zkStateReader) {
// for backward compatibility // for backward compatibility
String autoAddReplicas = cdp.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, (String) null); String autoAddReplicas = zkStateReader.getClusterProperty(ZkStateReader.AUTO_ADD_REPLICAS, (String) null);
if (autoAddReplicas != null && autoAddReplicas.equals("false")) { if (autoAddReplicas != null && autoAddReplicas.equals("false")) {
return new NoneSuggester(); return new NoneSuggester();
} }
Policy.Suggester suggester = super.getSuggester(session, event, cdp); Policy.Suggester suggester = super.getSuggester(session, event, zkStateReader);
ClusterState clusterState; ClusterState clusterState = zkStateReader.getClusterState();
try {
clusterState = cdp.getClusterState();
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Exception getting cluster state", e);
}
boolean anyCollections = false; boolean anyCollections = false;
for (DocCollection collection: clusterState.getCollectionsMap().values()) { for (DocCollection collection: clusterState.getCollectionsMap().values()) {

View File

@ -24,10 +24,8 @@ import java.util.Map;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrResourceLoader;
public class AutoScaling { public class AutoScaling {
@ -112,13 +110,30 @@ public class AutoScaling {
void init(); void init();
} }
/** public static class TriggerFactory implements Closeable {
* Factory to produce instances of {@link Trigger}.
*/
public static abstract class TriggerFactory implements Closeable {
protected boolean isClosed = false;
public abstract Trigger create(TriggerEventType type, String name, Map<String, Object> props); private final CoreContainer coreContainer;
private boolean isClosed = false;
public TriggerFactory(CoreContainer coreContainer) {
Preconditions.checkNotNull(coreContainer);
this.coreContainer = coreContainer;
}
public synchronized Trigger create(TriggerEventType type, String name, Map<String, Object> props) {
if (isClosed) {
throw new AlreadyClosedException("TriggerFactory has already been closed, cannot create new triggers");
}
switch (type) {
case NODEADDED:
return new NodeAddedTrigger(name, props, coreContainer);
case NODELOST:
return new NodeLostTrigger(name, props, coreContainer);
default:
throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
}
}
@Override @Override
public void close() throws IOException { public void close() throws IOException {
@ -128,38 +143,6 @@ public class AutoScaling {
} }
} }
/**
* Default implementation of {@link TriggerFactory}.
*/
public static class TriggerFactoryImpl extends TriggerFactory {
private final ClusterDataProvider clusterDataProvider;
private final SolrResourceLoader loader;
public TriggerFactoryImpl(SolrResourceLoader loader, ClusterDataProvider clusterDataProvider) {
Preconditions.checkNotNull(clusterDataProvider);
Preconditions.checkNotNull(loader);
this.clusterDataProvider = clusterDataProvider;
this.loader = loader;
}
@Override
public synchronized Trigger create(TriggerEventType type, String name, Map<String, Object> props) {
if (isClosed) {
throw new AlreadyClosedException("TriggerFactory has already been closed, cannot create new triggers");
}
switch (type) {
case NODEADDED:
return new NodeAddedTrigger(name, props, loader, clusterDataProvider);
case NODELOST:
return new NodeLostTrigger(name, props, loader, clusterDataProvider);
default:
throw new IllegalArgumentException("Unknown event type: " + type + " in trigger: " + name);
}
}
}
public static final String AUTO_ADD_REPLICAS_TRIGGER_DSL = public static final String AUTO_ADD_REPLICAS_TRIGGER_DSL =
"{" + "{" +
" 'set-trigger' : {" + " 'set-trigger' : {" +

View File

@ -214,7 +214,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
try (CloudSolrClient build = new CloudSolrClient.Builder() try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient()) .withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) { .withZkHost(container.getZkController().getZkServerAddress()).build()) {
Policy.Session session = policy.createSession(new SolrClientDataProvider(new ZkDistributedQueueFactory(container.getZkController().getZkClient()), build)); Policy.Session session = policy.createSession(new SolrClientDataProvider(build));
List<Row> sorted = session.getSorted(); List<Row> sorted = session.getSorted();
List<Clause.Violation> violations = session.getViolations(); List<Clause.Violation> violations = session.getViolations();
@ -638,8 +638,7 @@ public class AutoScalingHandler extends RequestHandlerBase implements Permission
try (CloudSolrClient build = new CloudSolrClient.Builder() try (CloudSolrClient build = new CloudSolrClient.Builder()
.withHttpClient(container.getUpdateShardHandler().getHttpClient()) .withHttpClient(container.getUpdateShardHandler().getHttpClient())
.withZkHost(container.getZkController().getZkServerAddress()).build()) { .withZkHost(container.getZkController().getZkServerAddress()).build()) {
Policy.Session session = autoScalingConf.getPolicy() Policy.Session session = autoScalingConf.getPolicy().createSession(new SolrClientDataProvider(build));
.createSession(new SolrClientDataProvider(new ZkDistributedQueueFactory(container.getZkController().getZkClient()), build));
log.debug("Verified autoscaling configuration"); log.debug("Verified autoscaling configuration");
} }
} }

View File

@ -25,7 +25,6 @@ import java.util.Map;
import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.Policy; import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.SolrClientDataProvider; import org.apache.solr.client.solrj.impl.SolrClientDataProvider;
@ -49,30 +48,40 @@ public class ComputePlanAction extends TriggerActionBase {
@Override @Override
public void process(TriggerEvent event, ActionContext context) { public void process(TriggerEvent event, ActionContext context) {
log.debug("-- processing event: {} with context properties: {}", event, context.getProperties()); log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
ClusterDataProvider cdp = context.getClusterDataProvider(); CoreContainer container = context.getCoreContainer();
try { try {
AutoScalingConfig autoScalingConf = cdp.getAutoScalingConfig(); try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
if (autoScalingConf.isEmpty()) { .withZkHost(container.getZkController().getZkServerAddress())
log.error("Action: " + getName() + " executed but no policy is configured"); .withHttpClient(container.getUpdateShardHandler().getHttpClient())
return; .build()) {
} ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
Policy policy = autoScalingConf.getPolicy(); AutoScalingConfig autoScalingConf = zkStateReader.getAutoScalingConfig();
Policy.Session session = policy.createSession(cdp); if (autoScalingConf.isEmpty()) {
Policy.Suggester suggester = getSuggester(session, event, cdp); log.error("Action: " + getName() + " executed but no policy is configured");
while (true) { return;
SolrRequest operation = suggester.getOperation(); }
if (operation == null) break; Policy policy = autoScalingConf.getPolicy();
log.info("Computed Plan: {}", operation.getParams()); Policy.Session session = policy.createSession(new SolrClientDataProvider(cloudSolrClient));
Map<String, Object> props = context.getProperties(); Policy.Suggester suggester = getSuggester(session, event, zkStateReader);
props.compute("operations", (k, v) -> { while (true) {
List<SolrRequest> operations = (List<SolrRequest>) v; SolrRequest operation = suggester.getOperation();
if (operations == null) operations = new ArrayList<>(); if (operation == null) break;
operations.add(operation); log.info("Computed Plan: {}", operation.getParams());
return operations; Map<String, Object> props = context.getProperties();
}); props.compute("operations", (k, v) -> {
session = suggester.getSession(); List<SolrRequest> operations = (List<SolrRequest>) v;
suggester = getSuggester(session, event, cdp); if (operations == null) operations = new ArrayList<>();
operations.add(operation);
return operations;
});
session = suggester.getSession();
suggester = getSuggester(session, event, zkStateReader);
}
} }
} catch (KeeperException e) {
log.error("ZooKeeperException while processing event: " + event, e);
} catch (InterruptedException e) {
log.error("Interrupted while processing event: " + event, e);
} catch (IOException e) { } catch (IOException e) {
log.error("IOException while processing event: " + event, e); log.error("IOException while processing event: " + event, e);
} catch (Exception e) { } catch (Exception e) {
@ -80,7 +89,7 @@ public class ComputePlanAction extends TriggerActionBase {
} }
} }
protected Policy.Suggester getSuggester(Policy.Session session, TriggerEvent event, ClusterDataProvider cdp) { protected Policy.Suggester getSuggester(Policy.Session session, TriggerEvent event, ZkStateReader zkStateReader) {
Policy.Suggester suggester; Policy.Suggester suggester;
switch (event.getEventType()) { switch (event.getEventType()) {
case NODEADDED: case NODEADDED:

View File

@ -25,7 +25,6 @@ import java.util.List;
import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
@ -44,29 +43,32 @@ public class ExecutePlanAction extends TriggerActionBase {
@Override @Override
public void process(TriggerEvent event, ActionContext context) { public void process(TriggerEvent event, ActionContext context) {
log.debug("-- processing event: {} with context properties: {}", event, context.getProperties()); log.debug("-- processing event: {} with context properties: {}", event, context.getProperties());
ClusterDataProvider clusterDataProvider = context.getClusterDataProvider(); CoreContainer container = context.getCoreContainer();
List<SolrRequest> operations = (List<SolrRequest>) context.getProperty("operations"); List<SolrRequest> operations = (List<SolrRequest>) context.getProperty("operations");
if (operations == null || operations.isEmpty()) { if (operations == null || operations.isEmpty()) {
log.info("No operations to execute for event: {}", event); log.info("No operations to execute for event: {}", event);
return; return;
} }
try { try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
.withZkHost(container.getZkController().getZkServerAddress())
.withHttpClient(container.getUpdateShardHandler().getHttpClient())
.build()) {
for (SolrRequest operation : operations) { for (SolrRequest operation : operations) {
log.info("Executing operation: {}", operation.getParams()); log.info("Executing operation: {}", operation.getParams());
try { try {
SolrResponse response = clusterDataProvider.request(operation); SolrResponse response = operation.process(cloudSolrClient);
context.getProperties().compute("responses", (s, o) -> { context.getProperties().compute("responses", (s, o) -> {
List<NamedList<Object>> responses = (List<NamedList<Object>>) o; List<NamedList<Object>> responses = (List<NamedList<Object>>) o;
if (responses == null) responses = new ArrayList<>(operations.size()); if (responses == null) responses = new ArrayList<>(operations.size());
responses.add(response.getResponse()); responses.add(response.getResponse());
return responses; return responses;
}); });
} catch (Exception e) { } catch (SolrServerException | HttpSolrClient.RemoteSolrException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected exception executing operation: " + operation.getParams(), e); "Unexpected exception executing operation: " + operation.getParams(), e);
} }
} }
} catch (Exception e) { } catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Unexpected IOException while processing event: " + event, e); "Unexpected IOException while processing event: " + event, e);
} }

View File

@ -30,7 +30,6 @@ import org.apache.http.client.methods.HttpPost;
import org.apache.http.client.protocol.HttpClientContext; import org.apache.http.client.protocol.HttpClientContext;
import org.apache.http.entity.StringEntity; import org.apache.http.entity.StringEntity;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.impl.HttpClientUtil; import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.common.util.Utils; import org.apache.solr.common.util.Utils;
@ -73,9 +72,9 @@ public class HttpTriggerListener extends TriggerListenerBase {
private boolean followRedirects; private boolean followRedirects;
@Override @Override
public void init(ClusterDataProvider clusterDataProvider, AutoScalingConfig.TriggerListenerConfig config) { public void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) {
super.init(clusterDataProvider, config); super.init(coreContainer, config);
httpClient = clusterDataProvider.getHttpClient(); httpClient = coreContainer.getUpdateShardHandler().getHttpClient();
urlTemplate = (String)config.properties.get("url"); urlTemplate = (String)config.properties.get("url");
payloadTemplate = (String)config.properties.get("payload"); payloadTemplate = (String)config.properties.get("payload");
contentType = (String)config.properties.get("contentType"); contentType = (String)config.properties.get("contentType");

View File

@ -17,24 +17,29 @@
package org.apache.solr.cloud.autoscaling; package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.NoSuchElementException; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider; import org.apache.lucene.util.IOUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource; import org.apache.solr.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -44,28 +49,60 @@ import org.slf4j.LoggerFactory;
public class NodeAddedTrigger extends TriggerBase { public class NodeAddedTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final String name;
private final Map<String, Object> properties;
private final CoreContainer container;
private final List<TriggerAction> actions;
private final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef;
private final boolean enabled;
private final int waitForSecond;
private final TriggerEventType eventType;
private final TimeSource timeSource; private final TimeSource timeSource;
private boolean isClosed = false;
private Set<String> lastLiveNodes; private Set<String> lastLiveNodes;
private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>(); private Map<String, Long> nodeNameVsTimeAdded = new HashMap<>();
public NodeAddedTrigger(String name, Map<String, Object> properties, public NodeAddedTrigger(String name, Map<String, Object> properties,
SolrResourceLoader loader, CoreContainer container) {
ClusterDataProvider clusterDataProvider) { super(container.getZkController().getZkClient());
super(name, properties, loader, clusterDataProvider); this.name = name;
this.properties = properties;
this.container = container;
this.timeSource = TimeSource.CURRENT_TIME; this.timeSource = TimeSource.CURRENT_TIME;
lastLiveNodes = new HashSet<>(clusterDataProvider.getLiveNodes()); this.processorRef = new AtomicReference<>();
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
actions = new ArrayList<>(3);
for (Map<String, String> map : o) {
TriggerAction action = container.getResourceLoader().newInstance(map.get("class"), TriggerAction.class);
actions.add(action);
}
} else {
actions = Collections.emptyList();
}
lastLiveNodes = new HashSet<>(container.getZkController().getZkStateReader().getClusterState().getLiveNodes());
log.debug("Initial livenodes: {}", lastLiveNodes); log.debug("Initial livenodes: {}", lastLiveNodes);
this.enabled = Boolean.parseBoolean(String.valueOf(properties.getOrDefault("enabled", "true")));
this.waitForSecond = ((Long) properties.getOrDefault("waitFor", -1L)).intValue();
this.eventType = TriggerEventType.valueOf(properties.get("event").toString().toUpperCase(Locale.ROOT));
log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties); log.debug("NodeAddedTrigger {} instantiated with properties: {}", name, properties);
} }
@Override @Override
public void init() { public void init() {
super.init(); List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
for (int i = 0; i < o.size(); i++) {
Map<String, String> map = o.get(i);
actions.get(i).init(map);
}
}
// pick up added nodes for which marker paths were created // pick up added nodes for which marker paths were created
try { try {
List<String> added = clusterDataProvider.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH); List<String> added = container.getZkController().getZkClient().getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, null, true);
added.forEach(n -> { added.forEach(n -> {
// don't add nodes that have since gone away // don't add nodes that have since gone away
if (lastLiveNodes.contains(n)) { if (lastLiveNodes.contains(n)) {
@ -74,14 +111,77 @@ public class NodeAddedTrigger extends TriggerBase {
} }
removeMarker(n); removeMarker(n);
}); });
} catch (NoSuchElementException e) { } catch (KeeperException.NoNodeException e) {
// ignore // ignore
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Exception retrieving nodeLost markers", e); log.warn("Exception retrieving nodeLost markers", e);
} }
} }
@Override
public void setProcessor(AutoScaling.TriggerEventProcessor processor) {
processorRef.set(processor);
}
@Override
public AutoScaling.TriggerEventProcessor getProcessor() {
return processorRef.get();
}
@Override
public String getName() {
return name;
}
@Override
public TriggerEventType getEventType() {
return eventType;
}
@Override
public boolean isEnabled() {
return enabled;
}
@Override
public int getWaitForSecond() {
return waitForSecond;
}
@Override
public Map<String, Object> getProperties() {
return properties;
}
@Override
public List<TriggerAction> getActions() {
return actions;
}
@Override
public boolean equals(Object obj) {
if (obj instanceof NodeAddedTrigger) {
NodeAddedTrigger that = (NodeAddedTrigger) obj;
return this.name.equals(that.name)
&& this.properties.equals(that.properties);
}
return false;
}
@Override
public int hashCode() {
return Objects.hash(name, properties);
}
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
IOUtils.closeWhileHandlingException(actions);
}
}
@Override @Override
public void restoreState(AutoScaling.Trigger old) { public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed(); assert old.isClosed();
@ -129,7 +229,8 @@ public class NodeAddedTrigger extends TriggerBase {
} }
log.debug("Running NodeAddedTrigger {}", name); log.debug("Running NodeAddedTrigger {}", name);
Set<String> newLiveNodes = new HashSet<>(clusterDataProvider.getLiveNodes()); ZkStateReader reader = container.getZkController().getZkStateReader();
Set<String> newLiveNodes = reader.getClusterState().getLiveNodes();
log.debug("Found livenodes: {}", newLiveNodes); log.debug("Found livenodes: {}", newLiveNodes);
// have any nodes that we were tracking been removed from the cluster? // have any nodes that we were tracking been removed from the cluster?
@ -186,17 +287,24 @@ public class NodeAddedTrigger extends TriggerBase {
private void removeMarker(String nodeName) { private void removeMarker(String nodeName) {
String path = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName; String path = ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH + "/" + nodeName;
try { try {
if (clusterDataProvider.hasData(path)) { if (container.getZkController().getZkClient().exists(path, true)) {
clusterDataProvider.removeData(path, -1); container.getZkController().getZkClient().delete(path, -1, true);
} }
} catch (NoSuchElementException e) { } catch (KeeperException.NoNodeException e) {
// ignore // ignore
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.debug("Exception removing nodeAdded marker " + nodeName, e); log.debug("Exception removing nodeAdded marker " + nodeName, e);
} }
} }
@Override
public boolean isClosed() {
synchronized (this) {
return isClosed;
}
}
public static class NodeAddedEvent extends TriggerEvent { public static class NodeAddedEvent extends TriggerEvent {
public NodeAddedEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames) { public NodeAddedEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames) {

View File

@ -17,24 +17,29 @@
package org.apache.solr.cloud.autoscaling; package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.NoSuchElementException; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider; import org.apache.lucene.util.IOUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.TimeSource; import org.apache.solr.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -44,27 +49,59 @@ import org.slf4j.LoggerFactory;
public class NodeLostTrigger extends TriggerBase { public class NodeLostTrigger extends TriggerBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final String name;
private final Map<String, Object> properties;
private final CoreContainer container;
private final List<TriggerAction> actions;
private final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef;
private final boolean enabled;
private final int waitForSecond;
private final TriggerEventType eventType;
private final TimeSource timeSource; private final TimeSource timeSource;
private boolean isClosed = false;
private Set<String> lastLiveNodes; private Set<String> lastLiveNodes;
private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>(); private Map<String, Long> nodeNameVsTimeRemoved = new HashMap<>();
public NodeLostTrigger(String name, Map<String, Object> properties, public NodeLostTrigger(String name, Map<String, Object> properties,
SolrResourceLoader loader, CoreContainer container) {
ClusterDataProvider clusterDataProvider) { super(container.getZkController().getZkClient());
super(name, properties, loader, clusterDataProvider); this.name = name;
this.properties = properties;
this.container = container;
this.timeSource = TimeSource.CURRENT_TIME; this.timeSource = TimeSource.CURRENT_TIME;
lastLiveNodes = new HashSet<>(clusterDataProvider.getLiveNodes()); this.processorRef = new AtomicReference<>();
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
actions = new ArrayList<>(3);
for (Map<String, String> map : o) {
TriggerAction action = container.getResourceLoader().newInstance(map.get("class"), TriggerAction.class);
actions.add(action);
}
} else {
actions = Collections.emptyList();
}
lastLiveNodes = new HashSet<>(container.getZkController().getZkStateReader().getClusterState().getLiveNodes());
log.debug("Initial livenodes: {}", lastLiveNodes); log.debug("Initial livenodes: {}", lastLiveNodes);
this.enabled = Boolean.parseBoolean(String.valueOf(properties.getOrDefault("enabled", "true")));
this.waitForSecond = ((Long) properties.getOrDefault("waitFor", -1L)).intValue();
this.eventType = TriggerEventType.valueOf(properties.get("event").toString().toUpperCase(Locale.ROOT));
} }
@Override @Override
public void init() { public void init() {
super.init(); List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
for (int i = 0; i < o.size(); i++) {
Map<String, String> map = o.get(i);
actions.get(i).init(map);
}
}
// pick up lost nodes for which marker paths were created // pick up lost nodes for which marker paths were created
try { try {
List<String> lost = clusterDataProvider.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH); List<String> lost = container.getZkController().getZkClient().getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, null, true);
lost.forEach(n -> { lost.forEach(n -> {
// don't add nodes that have since came back // don't add nodes that have since came back
if (!lastLiveNodes.contains(n)) { if (!lastLiveNodes.contains(n)) {
@ -73,18 +110,76 @@ public class NodeLostTrigger extends TriggerBase {
} }
removeMarker(n); removeMarker(n);
}); });
} catch (NoSuchElementException e) { } catch (KeeperException.NoNodeException e) {
// ignore // ignore
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Exception retrieving nodeLost markers", e); log.warn("Exception retrieving nodeLost markers", e);
} }
} }
@Override
public void setProcessor(AutoScaling.TriggerEventProcessor processor) {
processorRef.set(processor);
}
@Override
public AutoScaling.TriggerEventProcessor getProcessor() {
return processorRef.get();
}
@Override
public String getName() {
return name;
}
@Override
public TriggerEventType getEventType() {
return eventType;
}
@Override
public boolean isEnabled() {
return enabled;
}
@Override
public int getWaitForSecond() {
return waitForSecond;
}
@Override
public Map<String, Object> getProperties() {
return properties;
}
@Override @Override
public List<TriggerAction> getActions() { public List<TriggerAction> getActions() {
return actions; return actions;
} }
@Override
public boolean equals(Object obj) {
if (obj instanceof NodeLostTrigger) {
NodeLostTrigger that = (NodeLostTrigger) obj;
return this.name.equals(that.name)
&& this.properties.equals(that.properties);
}
return false;
}
@Override
public int hashCode() {
return Objects.hash(name, properties);
}
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
IOUtils.closeWhileHandlingException(actions);
}
}
@Override @Override
public void restoreState(AutoScaling.Trigger old) { public void restoreState(AutoScaling.Trigger old) {
assert old.isClosed(); assert old.isClosed();
@ -131,7 +226,8 @@ public class NodeLostTrigger extends TriggerBase {
} }
} }
Set<String> newLiveNodes = new HashSet<>(clusterDataProvider.getLiveNodes()); ZkStateReader reader = container.getZkController().getZkStateReader();
Set<String> newLiveNodes = reader.getClusterState().getLiveNodes();
log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes); log.debug("Running NodeLostTrigger: {} with currently live nodes: {}", name, newLiveNodes);
// have any nodes that we were tracking been added to the cluster? // have any nodes that we were tracking been added to the cluster?
@ -190,16 +286,23 @@ public class NodeLostTrigger extends TriggerBase {
private void removeMarker(String nodeName) { private void removeMarker(String nodeName) {
String path = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + nodeName; String path = ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH + "/" + nodeName;
try { try {
if (clusterDataProvider.hasData(path)) { if (container.getZkController().getZkClient().exists(path, true)) {
clusterDataProvider.removeData(path, -1); container.getZkController().getZkClient().delete(path, -1, true);
} }
} catch (NoSuchElementException e) { } catch (KeeperException.NoNodeException e) {
// ignore // ignore
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Exception removing nodeLost marker " + nodeName, e); log.warn("Exception removing nodeLost marker " + nodeName, e);
} }
} }
@Override
public boolean isClosed() {
synchronized (this) {
return isClosed;
}
}
public static class NodeLostEvent extends TriggerEvent { public static class NodeLostEvent extends TriggerEvent {
public NodeLostEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames) { public NodeLostEvent(TriggerEventType eventType, String source, List<Long> times, List<String> nodeNames) {

View File

@ -20,19 +20,16 @@ package org.apache.solr.cloud.autoscaling;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.net.ConnectException;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set; import java.util.Set;
import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.ZkController; import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
@ -40,7 +37,6 @@ import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException; import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.util.IOUtils; import org.apache.solr.common.util.IOUtils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher;
@ -56,7 +52,11 @@ public class OverseerTriggerThread implements Runnable, Closeable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final ClusterDataProvider clusterDataProvider; private final ZkController zkController;
private final ZkStateReader zkStateReader;
private final SolrZkClient zkClient;
private final ScheduledTriggers scheduledTriggers; private final ScheduledTriggers scheduledTriggers;
@ -77,10 +77,12 @@ public class OverseerTriggerThread implements Runnable, Closeable {
private AutoScalingConfig autoScalingConfig; private AutoScalingConfig autoScalingConfig;
public OverseerTriggerThread(SolrResourceLoader loader, ClusterDataProvider clusterDataProvider) { public OverseerTriggerThread(ZkController zkController) {
this.clusterDataProvider = clusterDataProvider; this.zkController = zkController;
scheduledTriggers = new ScheduledTriggers(loader, clusterDataProvider); zkStateReader = zkController.getZkStateReader();
triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, clusterDataProvider); zkClient = zkController.getZkClient();
scheduledTriggers = new ScheduledTriggers(zkController);
triggerFactory = new AutoScaling.TriggerFactory(zkController.getCoreContainer());
} }
@Override @Override
@ -104,8 +106,11 @@ public class OverseerTriggerThread implements Runnable, Closeable {
try { try {
refreshAutoScalingConf(new AutoScalingWatcher()); refreshAutoScalingConf(new AutoScalingWatcher());
} catch (ConnectException e) { } catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
log.warn("ZooKeeper watch triggered for autoscaling conf, but Solr cannot talk to ZK: [{}]", e.getMessage()); log.warn("ZooKeeper watch triggered for autoscaling conf, but Solr cannot talk to ZK: [{}]", e.getMessage());
} catch (KeeperException e) {
log.error("A ZK error has occurred", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
} catch (InterruptedException e) { } catch (InterruptedException e) {
// Restore the interrupted status // Restore the interrupted status
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
@ -196,26 +201,26 @@ public class OverseerTriggerThread implements Runnable, Closeable {
if (cleanOldNodeLostMarkers) { if (cleanOldNodeLostMarkers) {
log.debug("-- clean old nodeLost markers"); log.debug("-- clean old nodeLost markers");
try { try {
List<String> markers = clusterDataProvider.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH); List<String> markers = zkClient.getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, null, true);
markers.forEach(n -> { markers.forEach(n -> {
removeNodeMarker(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, n); removeNodeMarker(ZkStateReader.SOLR_AUTOSCALING_NODE_LOST_PATH, n);
}); });
} catch (NoSuchElementException e) { } catch (KeeperException.NoNodeException e) {
// ignore // ignore
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Error removing old nodeLost markers", e); log.warn("Error removing old nodeLost markers", e);
} }
} }
if (cleanOldNodeAddedMarkers) { if (cleanOldNodeAddedMarkers) {
log.debug("-- clean old nodeAdded markers"); log.debug("-- clean old nodeAdded markers");
try { try {
List<String> markers = clusterDataProvider.listData(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH); List<String> markers = zkClient.getChildren(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, null, true);
markers.forEach(n -> { markers.forEach(n -> {
removeNodeMarker(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, n); removeNodeMarker(ZkStateReader.SOLR_AUTOSCALING_NODE_ADDED_PATH, n);
}); });
} catch (NoSuchElementException e) { } catch (KeeperException.NoNodeException e) {
// ignore // ignore
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Error removing old nodeAdded markers", e); log.warn("Error removing old nodeAdded markers", e);
} }
@ -226,11 +231,11 @@ public class OverseerTriggerThread implements Runnable, Closeable {
private void removeNodeMarker(String path, String nodeName) { private void removeNodeMarker(String path, String nodeName) {
path = path + "/" + nodeName; path = path + "/" + nodeName;
try { try {
clusterDataProvider.removeData(path, -1); zkClient.delete(path, -1, true);
log.debug(" -- deleted " + path); log.debug(" -- deleted " + path);
} catch (NoSuchElementException e) { } catch (KeeperException.NoNodeException e) {
// ignore // ignore
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Error removing old marker " + path, e); log.warn("Error removing old marker " + path, e);
} }
} }
@ -245,8 +250,11 @@ public class OverseerTriggerThread implements Runnable, Closeable {
try { try {
refreshAutoScalingConf(this); refreshAutoScalingConf(this);
} catch (ConnectException e) { } catch (KeeperException.ConnectionLossException | KeeperException.SessionExpiredException e) {
log.warn("ZooKeeper watch triggered for autoscaling conf, but we cannot talk to ZK: [{}]", e.getMessage()); log.warn("ZooKeeper watch triggered for autoscaling conf, but Solr cannot talk to ZK: [{}]", e.getMessage());
} catch (KeeperException e) {
log.error("A ZK error has occurred", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
} catch (InterruptedException e) { } catch (InterruptedException e) {
// Restore the interrupted status // Restore the interrupted status
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
@ -258,13 +266,13 @@ public class OverseerTriggerThread implements Runnable, Closeable {
} }
private void refreshAutoScalingConf(Watcher watcher) throws ConnectException, InterruptedException, IOException { private void refreshAutoScalingConf(Watcher watcher) throws KeeperException, InterruptedException {
updateLock.lock(); updateLock.lock();
try { try {
if (isClosed) { if (isClosed) {
return; return;
} }
AutoScalingConfig currentConfig = clusterDataProvider.getAutoScalingConfig(watcher); AutoScalingConfig currentConfig = zkStateReader.getAutoScalingConfig(watcher);
log.debug("Refreshing {} with znode version {}", ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, currentConfig.getZkVersion()); log.debug("Refreshing {} with znode version {}", ZkStateReader.SOLR_AUTOSCALING_CONF_PATH, currentConfig.getZkVersion());
if (znodeVersion >= currentConfig.getZkVersion()) { if (znodeVersion >= currentConfig.getZkVersion()) {
// protect against reordered watcher fires by ensuring that we only move forward // protect against reordered watcher fires by ensuring that we only move forward

View File

@ -41,18 +41,15 @@ import java.util.stream.Collectors;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.cloud.ActionThrottle; import org.apache.solr.cloud.ActionThrottle;
import org.apache.solr.cloud.Overseer; import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController; import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils; import org.apache.solr.common.util.IOUtils;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.DefaultSolrThreadFactory; import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op; import org.apache.zookeeper.Op;
@ -88,17 +85,17 @@ public class ScheduledTriggers implements Closeable {
private final ActionThrottle actionThrottle; private final ActionThrottle actionThrottle;
private final ClusterDataProvider clusterDataProvider; private final SolrZkClient zkClient;
private final SolrResourceLoader loader;
private final Overseer.Stats queueStats; private final Overseer.Stats queueStats;
private final CoreContainer coreContainer;
private final TriggerListeners listeners; private final TriggerListeners listeners;
private AutoScalingConfig autoScalingConfig; private AutoScalingConfig autoScalingConfig;
public ScheduledTriggers(SolrResourceLoader loader, ClusterDataProvider clusterDataProvider) { public ScheduledTriggers(ZkController zkController) {
// todo make the core pool size configurable // todo make the core pool size configurable
// it is important to use more than one because a time taking trigger can starve other scheduled triggers // it is important to use more than one because a time taking trigger can starve other scheduled triggers
// ideally we should have as many core threads as the number of triggers but firstly, we don't know beforehand // ideally we should have as many core threads as the number of triggers but firstly, we don't know beforehand
@ -111,8 +108,8 @@ public class ScheduledTriggers implements Closeable {
actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor")); actionExecutor = ExecutorUtil.newMDCAwareSingleThreadExecutor(new DefaultSolrThreadFactory("AutoscalingActionExecutor"));
// todo make the wait time configurable // todo make the wait time configurable
actionThrottle = new ActionThrottle("action", DEFAULT_MIN_MS_BETWEEN_ACTIONS); actionThrottle = new ActionThrottle("action", DEFAULT_MIN_MS_BETWEEN_ACTIONS);
this.clusterDataProvider = clusterDataProvider; coreContainer = zkController.getCoreContainer();
this.loader = loader; zkClient = zkController.getZkClient();
queueStats = new Overseer.Stats(); queueStats = new Overseer.Stats();
listeners = new TriggerListeners(); listeners = new TriggerListeners();
} }
@ -139,12 +136,7 @@ public class ScheduledTriggers implements Closeable {
if (isClosed) { if (isClosed) {
throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore"); throw new AlreadyClosedException("ScheduledTriggers has been closed and cannot be used anymore");
} }
ScheduledTrigger scheduledTrigger; ScheduledTrigger scheduledTrigger = new ScheduledTrigger(newTrigger, zkClient, queueStats);
try {
scheduledTrigger = new ScheduledTrigger(newTrigger, clusterDataProvider, queueStats);
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "exception creating scheduled trigger", e);
}
ScheduledTrigger old = scheduledTriggers.putIfAbsent(newTrigger.getName(), scheduledTrigger); ScheduledTrigger old = scheduledTriggers.putIfAbsent(newTrigger.getName(), scheduledTrigger);
if (old != null) { if (old != null) {
if (old.trigger.equals(newTrigger)) { if (old.trigger.equals(newTrigger)) {
@ -190,7 +182,7 @@ public class ScheduledTriggers implements Closeable {
// let the action executor thread wait instead of the trigger thread so we use the throttle here // let the action executor thread wait instead of the trigger thread so we use the throttle here
actionThrottle.minimumWaitBetweenActions(); actionThrottle.minimumWaitBetweenActions();
actionThrottle.markAttemptingAction(); actionThrottle.markAttemptingAction();
ActionContext actionContext = new ActionContext(clusterDataProvider, newTrigger, new HashMap<>()); ActionContext actionContext = new ActionContext(coreContainer, newTrigger, new HashMap<>());
for (TriggerAction action : actions) { for (TriggerAction action : actions) {
List<String> beforeActions = (List<String>)actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.BEFORE_ACTION.toString(), k -> new ArrayList<String>()); List<String> beforeActions = (List<String>)actionContext.getProperties().computeIfAbsent(TriggerEventProcessorStage.BEFORE_ACTION.toString(), k -> new ArrayList<String>());
beforeActions.add(action.getName()); beforeActions.add(action.getName());
@ -252,23 +244,23 @@ public class ScheduledTriggers implements Closeable {
String statePath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + triggerName; String statePath = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + triggerName;
String eventsPath = ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName; String eventsPath = ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName;
try { try {
if (clusterDataProvider.hasData(statePath)) { if (zkClient.exists(statePath, true)) {
clusterDataProvider.removeData(statePath, -1); zkClient.delete(statePath, -1, true);
} }
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Failed to remove state for removed trigger " + statePath, e); log.warn("Failed to remove state for removed trigger " + statePath, e);
} }
try { try {
if (clusterDataProvider.hasData(eventsPath)) { if (zkClient.exists(eventsPath, true)) {
List<String> events = clusterDataProvider.listData(eventsPath); List<String> events = zkClient.getChildren(eventsPath, null, true);
List<Op> ops = new ArrayList<>(events.size() + 1); List<Op> ops = new ArrayList<>(events.size() + 1);
events.forEach(ev -> { events.forEach(ev -> {
ops.add(Op.delete(eventsPath + "/" + ev, -1)); ops.add(Op.delete(eventsPath + "/" + ev, -1));
}); });
ops.add(Op.delete(eventsPath, -1)); ops.add(Op.delete(eventsPath, -1));
clusterDataProvider.multi(ops); zkClient.multi(ops, true);
} }
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
log.warn("Failed to remove events for removed trigger " + eventsPath, e); log.warn("Failed to remove events for removed trigger " + eventsPath, e);
} }
} }
@ -305,9 +297,9 @@ public class ScheduledTriggers implements Closeable {
boolean replay; boolean replay;
volatile boolean isClosed; volatile boolean isClosed;
ScheduledTrigger(AutoScaling.Trigger trigger, ClusterDataProvider clusterDataProvider, Overseer.Stats stats) throws IOException { ScheduledTrigger(AutoScaling.Trigger trigger, SolrZkClient zkClient, Overseer.Stats stats) {
this.trigger = trigger; this.trigger = trigger;
this.queue = new TriggerEventQueue(clusterDataProvider, trigger.getName(), stats); this.queue = new TriggerEventQueue(zkClient, trigger.getName(), stats);
this.replay = true; this.replay = true;
this.isClosed = false; this.isClosed = false;
} }
@ -434,13 +426,13 @@ public class ScheduledTriggers implements Closeable {
if (listener == null) { // create new instance if (listener == null) { // create new instance
String clazz = config.listenerClass; String clazz = config.listenerClass;
try { try {
listener = loader.newInstance(clazz, TriggerListener.class); listener = coreContainer.getResourceLoader().newInstance(clazz, TriggerListener.class);
} catch (Exception e) { } catch (Exception e) {
log.warn("Invalid TriggerListener class name '" + clazz + "', skipping...", e); log.warn("Invalid TriggerListener class name '" + clazz + "', skipping...", e);
} }
if (listener != null) { if (listener != null) {
try { try {
listener.init(clusterDataProvider, config); listener.init(coreContainer, config);
listenersPerName.put(config.name, listener); listenersPerName.put(config.name, listener);
} catch (Exception e) { } catch (Exception e) {
log.warn("Error initializing TriggerListener " + config, e); log.warn("Error initializing TriggerListener " + config, e);

View File

@ -31,7 +31,6 @@ import java.util.StringJoiner;
import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.client.solrj.request.UpdateRequest;
@ -74,8 +73,8 @@ public class SystemLogListener extends TriggerListenerBase {
private boolean enabled = true; private boolean enabled = true;
@Override @Override
public void init(ClusterDataProvider clusterDataProvider, AutoScalingConfig.TriggerListenerConfig config) { public void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) {
super.init(clusterDataProvider, config); super.init(coreContainer, config);
collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL); collection = (String)config.properties.getOrDefault(CollectionAdminParams.COLLECTION, CollectionAdminParams.SYSTEM_COLL);
enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true))); enabled = Boolean.parseBoolean(String.valueOf(config.properties.getOrDefault("enabled", true)));
} }
@ -86,7 +85,10 @@ public class SystemLogListener extends TriggerListenerBase {
if (!enabled) { if (!enabled) {
return; return;
} }
try { try (CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder()
.withZkHost(coreContainer.getZkController().getZkServerAddress())
.withHttpClient(coreContainer.getUpdateShardHandler().getHttpClient())
.build()) {
SolrInputDocument doc = new SolrInputDocument(); SolrInputDocument doc = new SolrInputDocument();
doc.addField(CommonParams.TYPE, DOC_TYPE); doc.addField(CommonParams.TYPE, DOC_TYPE);
doc.addField(SOURCE_FIELD, SOURCE); doc.addField(SOURCE_FIELD, SOURCE);
@ -120,8 +122,7 @@ public class SystemLogListener extends TriggerListenerBase {
} }
UpdateRequest req = new UpdateRequest(); UpdateRequest req = new UpdateRequest();
req.add(doc); req.add(doc);
req.setParam(CollectionAdminParams.COLLECTION, collection); cloudSolrClient.request(req, collection);
clusterDataProvider.request(req);
} catch (Exception e) { } catch (Exception e) {
if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) { if ((e instanceof SolrException) && e.getMessage().contains("Collection not found")) {
// relatively benign // relatively benign

View File

@ -16,26 +16,14 @@
*/ */
package org.apache.solr.cloud.autoscaling; package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.Objects;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.util.IOUtils;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils; import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.data.Stat;
@ -49,130 +37,19 @@ import org.slf4j.LoggerFactory;
public abstract class TriggerBase implements AutoScaling.Trigger { public abstract class TriggerBase implements AutoScaling.Trigger {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected final String name; protected SolrZkClient zkClient;
protected final ClusterDataProvider clusterDataProvider;
protected final Map<String, Object> properties = new HashMap<>();
protected final TriggerEventType eventType;
protected final int waitForSecond;
protected Map<String,Object> lastState; protected Map<String,Object> lastState;
protected final AtomicReference<AutoScaling.TriggerEventProcessor> processorRef = new AtomicReference<>();
protected final List<TriggerAction> actions;
protected final boolean enabled;
protected boolean isClosed;
protected TriggerBase(String name, Map<String, Object> properties, SolrResourceLoader loader, ClusterDataProvider clusterDataProvider) { protected TriggerBase(SolrZkClient zkClient) {
this.name = name; this.zkClient = zkClient;
if (properties != null) {
this.properties.putAll(properties);
}
this.clusterDataProvider = clusterDataProvider;
this.enabled = Boolean.parseBoolean(String.valueOf(this.properties.getOrDefault("enabled", "true")));
this.eventType = TriggerEventType.valueOf(this.properties.getOrDefault("event", TriggerEventType.INVALID.toString()).toString().toUpperCase(Locale.ROOT));
this.waitForSecond = ((Long) this.properties.getOrDefault("waitFor", -1L)).intValue();
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
actions = new ArrayList<>(3);
for (Map<String, String> map : o) {
TriggerAction action = loader.newInstance(map.get("class"), TriggerAction.class);
actions.add(action);
}
} else {
actions = Collections.emptyList();
}
try { try {
if (!clusterDataProvider.hasData(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH)) { zkClient.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, false, true);
clusterDataProvider.makePath(ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH); } catch (KeeperException | InterruptedException e) {
}
} catch (IOException e) {
LOG.warn("Exception checking ZK path " + ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, e); LOG.warn("Exception checking ZK path " + ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH, e);
} }
} }
@Override
public void init() {
List<Map<String, String>> o = (List<Map<String, String>>) properties.get("actions");
if (o != null && !o.isEmpty()) {
for (int i = 0; i < o.size(); i++) {
Map<String, String> map = o.get(i);
actions.get(i).init(map);
}
}
}
@Override
public void setProcessor(AutoScaling.TriggerEventProcessor processor) {
processorRef.set(processor);
}
@Override
public AutoScaling.TriggerEventProcessor getProcessor() {
return processorRef.get();
}
@Override
public String getName() {
return name;
}
@Override
public TriggerEventType getEventType() {
return eventType;
}
@Override
public boolean isEnabled() {
return enabled;
}
@Override
public int getWaitForSecond() {
return waitForSecond;
}
@Override
public Map<String, Object> getProperties() {
return properties;
}
@Override
public List<TriggerAction> getActions() {
return actions;
}
@Override
public boolean isClosed() {
synchronized (this) {
return isClosed;
}
}
@Override
public void close() throws IOException {
synchronized (this) {
isClosed = true;
IOUtils.closeWhileHandlingException(actions);
}
}
@Override
public int hashCode() {
return Objects.hash(name, properties);
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj.getClass().equals(this.getClass())) {
TriggerBase that = (TriggerBase) obj;
return this.name.equals(that.name)
&& this.properties.equals(that.properties);
}
return false;
}
/** /**
* Prepare and return internal state of this trigger in a format suitable for persisting in ZK. * Prepare and return internal state of this trigger in a format suitable for persisting in ZK.
* @return map of internal state properties. Note: values must be supported by {@link Utils#toJSON(Object)}. * @return map of internal state properties. Note: values must be supported by {@link Utils#toJSON(Object)}.
@ -195,15 +72,15 @@ public abstract class TriggerBase implements AutoScaling.Trigger {
byte[] data = Utils.toJSON(state); byte[] data = Utils.toJSON(state);
String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName(); String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName();
try { try {
if (clusterDataProvider.hasData(path)) { if (zkClient.exists(path, true)) {
// update // update
clusterDataProvider.setData(path, data, -1); zkClient.setData(path, data, -1, true);
} else { } else {
// create // create
clusterDataProvider.createData(path, data, CreateMode.PERSISTENT); zkClient.create(path, data, CreateMode.PERSISTENT, true);
} }
lastState = state; lastState = state;
} catch (IOException e) { } catch (KeeperException | InterruptedException e) {
LOG.warn("Exception updating trigger state '" + path + "'", e); LOG.warn("Exception updating trigger state '" + path + "'", e);
} }
} }
@ -213,10 +90,10 @@ public abstract class TriggerBase implements AutoScaling.Trigger {
byte[] data = null; byte[] data = null;
String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName(); String path = ZkStateReader.SOLR_AUTOSCALING_TRIGGER_STATE_PATH + "/" + getName();
try { try {
if (clusterDataProvider.hasData(path)) { if (zkClient.exists(path, true)) {
ClusterDataProvider.VersionedData versionedDat = clusterDataProvider.getData(path); data = zkClient.getData(path, null, new Stat(), true);
} }
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
LOG.warn("Exception getting trigger state '" + path + "'", e); LOG.warn("Exception getting trigger state '" + path + "'", e);
} }
if (data != null) { if (data != null) {

View File

@ -1,24 +1,23 @@
package org.apache.solr.cloud.autoscaling; package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.Map; import java.util.Map;
import java.util.Queue;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.cloud.DistributedQueue;
import org.apache.solr.cloud.Overseer; import org.apache.solr.cloud.Overseer;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils; import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeSource; import org.apache.solr.util.TimeSource;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
/** /**
* *
*/ */
public class TriggerEventQueue { public class TriggerEventQueue extends DistributedQueue {
private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String ENQUEUE_TIME = "_enqueue_time_"; public static final String ENQUEUE_TIME = "_enqueue_time_";
@ -26,11 +25,9 @@ public class TriggerEventQueue {
private final String triggerName; private final String triggerName;
private final TimeSource timeSource; private final TimeSource timeSource;
private final DistributedQueue delegate;
public TriggerEventQueue(ClusterDataProvider clusterDataProvider, String triggerName, Overseer.Stats stats) throws IOException { public TriggerEventQueue(SolrZkClient zookeeper, String triggerName, Overseer.Stats stats) {
// TODO: collect stats super(zookeeper, ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName, stats);
this.delegate = clusterDataProvider.getDistributedQueueFactory().makeQueue(ZkStateReader.SOLR_AUTOSCALING_EVENTS_PATH + "/" + triggerName);
this.triggerName = triggerName; this.triggerName = triggerName;
this.timeSource = TimeSource.CURRENT_TIME; this.timeSource = TimeSource.CURRENT_TIME;
} }
@ -39,9 +36,9 @@ public class TriggerEventQueue {
event.getProperties().put(ENQUEUE_TIME, timeSource.getTime()); event.getProperties().put(ENQUEUE_TIME, timeSource.getTime());
try { try {
byte[] data = Utils.toJSON(event); byte[] data = Utils.toJSON(event);
delegate.offer(data); offer(data);
return true; return true;
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
LOG.warn("Exception adding event " + event + " to queue " + triggerName, e); LOG.warn("Exception adding event " + event + " to queue " + triggerName, e);
return false; return false;
} }
@ -50,7 +47,7 @@ public class TriggerEventQueue {
public TriggerEvent peekEvent() { public TriggerEvent peekEvent() {
byte[] data; byte[] data;
try { try {
while ((data = delegate.peek()) != null) { while ((data = peek()) != null) {
if (data.length == 0) { if (data.length == 0) {
LOG.warn("ignoring empty data..."); LOG.warn("ignoring empty data...");
continue; continue;
@ -63,7 +60,7 @@ public class TriggerEventQueue {
continue; continue;
} }
} }
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
LOG.warn("Exception peeking queue of trigger " + triggerName, e); LOG.warn("Exception peeking queue of trigger " + triggerName, e);
} }
return null; return null;
@ -72,7 +69,7 @@ public class TriggerEventQueue {
public TriggerEvent pollEvent() { public TriggerEvent pollEvent() {
byte[] data; byte[] data;
try { try {
while ((data = delegate.poll()) != null) { while ((data = poll()) != null) {
if (data.length == 0) { if (data.length == 0) {
LOG.warn("ignoring empty data..."); LOG.warn("ignoring empty data...");
continue; continue;
@ -85,7 +82,7 @@ public class TriggerEventQueue {
continue; continue;
} }
} }
} catch (Exception e) { } catch (KeeperException | InterruptedException e) {
LOG.warn("Exception polling queue of trigger " + triggerName, e); LOG.warn("Exception polling queue of trigger " + triggerName, e);
} }
return null; return null;

View File

@ -19,7 +19,6 @@ package org.apache.solr.cloud.autoscaling;
import java.io.Closeable; import java.io.Closeable;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
@ -29,7 +28,7 @@ import org.apache.solr.core.CoreContainer;
*/ */
public interface TriggerListener extends Closeable { public interface TriggerListener extends Closeable {
void init(ClusterDataProvider clusterDataProvider, AutoScalingConfig.TriggerListenerConfig config) throws Exception; void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) throws Exception;
AutoScalingConfig.TriggerListenerConfig getConfig(); AutoScalingConfig.TriggerListenerConfig getConfig();

View File

@ -19,7 +19,7 @@ package org.apache.solr.cloud.autoscaling;
import java.io.IOException; import java.io.IOException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider; import org.apache.solr.core.CoreContainer;
/** /**
* Base class for implementations of {@link TriggerListener}. * Base class for implementations of {@link TriggerListener}.
@ -27,11 +27,11 @@ import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
public abstract class TriggerListenerBase implements TriggerListener { public abstract class TriggerListenerBase implements TriggerListener {
protected AutoScalingConfig.TriggerListenerConfig config; protected AutoScalingConfig.TriggerListenerConfig config;
protected ClusterDataProvider clusterDataProvider; protected CoreContainer coreContainer;
@Override @Override
public void init(ClusterDataProvider clusterDataProvider, AutoScalingConfig.TriggerListenerConfig config) { public void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) {
this.clusterDataProvider = clusterDataProvider; this.coreContainer = coreContainer;
this.config = config; this.config = config;
} }

View File

@ -1,28 +0,0 @@
package org.apache.solr.cloud.autoscaling;
import java.io.IOException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.cloud.ZkDistributedQueue;
import org.apache.solr.common.cloud.SolrZkClient;
/**
*
*/
public class ZkDistributedQueueFactory implements ClusterDataProvider.DistributedQueueFactory {
private final SolrZkClient zkClient;
public ZkDistributedQueueFactory(SolrZkClient zkClient) {
this.zkClient = zkClient;
}
@Override
public DistributedQueue makeQueue(String path) throws IOException {
return new ZkDistributedQueue(zkClient, path);
}
@Override
public void removeQueue(String path) throws IOException {
}
}

View File

@ -956,8 +956,6 @@ public class CoreContainer {
SolrException.log(log, null, e); SolrException.log(log, null, e);
} catch (KeeperException e) { } catch (KeeperException e) {
SolrException.log(log, null, e); SolrException.log(log, null, e);
} catch (Exception e) {
SolrException.log(log, null, e);
} }
} }
@ -1379,8 +1377,6 @@ public class CoreContainer {
throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while unregistering core [" + name + "] from cloud state"); throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while unregistering core [" + name + "] from cloud state");
} catch (KeeperException e) { } catch (KeeperException e) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Error unregistering core [" + name + "] from cloud state", e); throw new SolrException(ErrorCode.SERVER_ERROR, "Error unregistering core [" + name + "] from cloud state", e);
} catch (Exception e) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Error unregistering core [" + name + "] from cloud state", e);
} }
} }
} }

View File

@ -227,8 +227,6 @@ public class ZkContainer {
} catch (InterruptedException e) { } catch (InterruptedException e) {
Thread.interrupted(); Thread.interrupted();
ZkContainer.log.error("", e); ZkContainer.log.error("", e);
} catch (Exception e) {
ZkContainer.log.error("", e);
} }
} }
} }

View File

@ -46,7 +46,7 @@ import org.apache.solr.client.solrj.request.GenericSolrRequest;
import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.SimpleSolrResponse; import org.apache.solr.client.solrj.response.SimpleSolrResponse;
import org.apache.solr.cloud.CloudDescriptor; import org.apache.solr.cloud.CloudDescriptor;
import org.apache.solr.client.solrj.cloud.DistributedQueue; import org.apache.solr.cloud.DistributedQueue;
import org.apache.solr.cloud.Overseer; import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.ZkController; import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.cloud.overseer.OverseerAction;

View File

@ -21,7 +21,6 @@ import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CoreStatus; import org.apache.solr.client.solrj.request.CoreStatus;
@ -83,7 +82,8 @@ public class DeleteShardTest extends SolrCloudTestCase {
} }
protected void setSliceState(String collection, String slice, State state) throws Exception { protected void setSliceState(String collection, String slice, State state) throws SolrServerException, IOException,
KeeperException, InterruptedException {
CloudSolrClient client = cluster.getSolrClient(); CloudSolrClient client = cluster.getSolrClient();

View File

@ -25,7 +25,6 @@ import java.util.concurrent.TimeoutException;
import java.util.function.Predicate; import java.util.function.Predicate;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.SolrjNamedThreadFactory; import org.apache.solr.common.util.SolrjNamedThreadFactory;
@ -96,7 +95,7 @@ public class DistributedQueueTest extends SolrTestCaseJ4 {
String dqZNode = "/distqueue/test"; String dqZNode = "/distqueue/test";
byte[] data = "hello world".getBytes(UTF8); byte[] data = "hello world".getBytes(UTF8);
ZkDistributedQueue consumer = makeDistributedQueue(dqZNode); DistributedQueue consumer = makeDistributedQueue(dqZNode);
DistributedQueue producer = makeDistributedQueue(dqZNode); DistributedQueue producer = makeDistributedQueue(dqZNode);
DistributedQueue producer2 = makeDistributedQueue(dqZNode); DistributedQueue producer2 = makeDistributedQueue(dqZNode);
@ -125,7 +124,7 @@ public class DistributedQueueTest extends SolrTestCaseJ4 {
String dqZNode = "/distqueue/test"; String dqZNode = "/distqueue/test";
String testData = "hello world"; String testData = "hello world";
ZkDistributedQueue dq = makeDistributedQueue(dqZNode); DistributedQueue dq = makeDistributedQueue(dqZNode);
assertNull(dq.peek()); assertNull(dq.peek());
Future<String> future = executor.submit(() -> new String(dq.peek(true), UTF8)); Future<String> future = executor.submit(() -> new String(dq.peek(true), UTF8));
@ -172,7 +171,7 @@ public class DistributedQueueTest extends SolrTestCaseJ4 {
@Test @Test
public void testLeakChildWatcher() throws Exception { public void testLeakChildWatcher() throws Exception {
String dqZNode = "/distqueue/test"; String dqZNode = "/distqueue/test";
ZkDistributedQueue dq = makeDistributedQueue(dqZNode); DistributedQueue dq = makeDistributedQueue(dqZNode);
assertTrue(dq.peekElements(1, 1, s1 -> true).isEmpty()); assertTrue(dq.peekElements(1, 1, s1 -> true).isEmpty());
assertEquals(1, dq.watcherCount()); assertEquals(1, dq.watcherCount());
assertFalse(dq.isDirty()); assertFalse(dq.isDirty());
@ -281,8 +280,8 @@ public class DistributedQueueTest extends SolrTestCaseJ4 {
assertFalse(sessionId == zkClient.getSolrZooKeeper().getSessionId()); assertFalse(sessionId == zkClient.getSolrZooKeeper().getSessionId());
} }
protected ZkDistributedQueue makeDistributedQueue(String dqZNode) throws Exception { protected DistributedQueue makeDistributedQueue(String dqZNode) throws Exception {
return new ZkDistributedQueue(zkClient, setupNewDistributedQueueZNode(dqZNode)); return new DistributedQueue(zkClient, setupNewDistributedQueueZNode(dqZNode));
} }
private static class QueueChangerThread extends Thread { private static class QueueChangerThread extends Thread {

View File

@ -26,7 +26,6 @@ import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrRequest.METHOD; import org.apache.solr.client.solrj.SolrRequest.METHOD;
import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@ -231,7 +230,8 @@ public class ForceLeaderTest extends HttpPartitionTest {
} }
} }
protected void setReplicaState(String collection, String slice, Replica replica, Replica.State state) throws Exception { protected void setReplicaState(String collection, String slice, Replica replica, Replica.State state) throws SolrServerException, IOException,
KeeperException, InterruptedException {
DistributedQueue inQueue = Overseer.getStateUpdateQueue(cloudClient.getZkStateReader().getZkClient()); DistributedQueue inQueue = Overseer.getStateUpdateQueue(cloudClient.getZkStateReader().getZkClient());
ZkStateReader zkStateReader = cloudClient.getZkStateReader(); ZkStateReader zkStateReader = cloudClient.getZkStateReader();

View File

@ -23,7 +23,6 @@ import java.util.Random;
import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create; import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
@ -69,7 +68,7 @@ public class MultiThreadedOCPTest extends AbstractFullDistribZkTestBase {
private void testFillWorkQueue() throws Exception { private void testFillWorkQueue() throws Exception {
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) { try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
DistributedQueue distributedQueue = new ZkDistributedQueue(cloudClient.getZkStateReader().getZkClient(), DistributedQueue distributedQueue = new DistributedQueue(cloudClient.getZkStateReader().getZkClient(),
"/overseer/collection-queue-work", new Overseer.Stats()); "/overseer/collection-queue-work", new Overseer.Stats());
//fill the work queue with blocked tasks by adding more than the no:of parallel tasks //fill the work queue with blocked tasks by adding more than the no:of parallel tasks
for (int i = 0; i < MAX_PARALLEL_TASKS+5; i++) { for (int i = 0; i < MAX_PARALLEL_TASKS+5; i++) {
@ -150,7 +149,7 @@ public class MultiThreadedOCPTest extends AbstractFullDistribZkTestBase {
private void testTaskExclusivity() throws Exception, SolrServerException { private void testTaskExclusivity() throws Exception, SolrServerException {
DistributedQueue distributedQueue = new ZkDistributedQueue(cloudClient.getZkStateReader().getZkClient(), DistributedQueue distributedQueue = new DistributedQueue(cloudClient.getZkStateReader().getZkClient(),
"/overseer/collection-queue-work", new Overseer.Stats()); "/overseer/collection-queue-work", new Overseer.Stats());
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) { try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {

View File

@ -37,7 +37,6 @@ import com.codahale.metrics.Snapshot;
import com.codahale.metrics.Timer; import com.codahale.metrics.Timer;
import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.DocCollection;
@ -134,7 +133,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
zkClient.close(); zkClient.close();
} }
public void createCollection(String collection, int numShards) throws Exception { public void createCollection(String collection, int numShards) throws KeeperException, InterruptedException {
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.CREATE.toLower(), ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.CREATE.toLower(),
"name", collection, "name", collection,
@ -147,7 +146,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
} }
public String publishState(String collection, String coreName, String coreNodeName, String shard, Replica.State stateName, int numShards) public String publishState(String collection, String coreName, String coreNodeName, String shard, Replica.State stateName, int numShards)
throws Exception { throws KeeperException, InterruptedException, IOException {
if (stateName == null) { if (stateName == null) {
ElectionContext ec = electionContext.remove(coreName); ElectionContext ec = electionContext.remove(coreName);
if (ec != null) { if (ec != null) {

View File

@ -31,7 +31,6 @@ import org.apache.lucene.util.TestUtil;
import org.apache.solr.BaseDistributedSearchTestCase; import org.apache.solr.BaseDistributedSearchTestCase;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.CollectionAdminRequest;

View File

@ -150,7 +150,7 @@ public class AutoAddReplicasPlanActionTest extends SolrCloudTestCase{
private List<SolrRequest> getOperations(JettySolrRunner actionJetty, String lostNodeName) { private List<SolrRequest> getOperations(JettySolrRunner actionJetty, String lostNodeName) {
AutoAddReplicasPlanAction action = new AutoAddReplicasPlanAction(); AutoAddReplicasPlanAction action = new AutoAddReplicasPlanAction();
TriggerEvent lostNode = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST, ".auto_add_replicas", Collections.singletonList(System.currentTimeMillis()), Collections.singletonList(lostNodeName)); TriggerEvent lostNode = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST, ".auto_add_replicas", Collections.singletonList(System.currentTimeMillis()), Collections.singletonList(lostNodeName));
ActionContext context = new ActionContext(actionJetty.getCoreContainer().getZkController().getClusterDataProvider(), null, new HashMap<>()); ActionContext context = new ActionContext(actionJetty.getCoreContainer(), null, new HashMap<>());
action.process(lostNode, context); action.process(lostNode, context);
List<SolrRequest> operations = (List) context.getProperty("operations"); List<SolrRequest> operations = (List) context.getProperty("operations");
return operations; return operations;

View File

@ -110,7 +110,7 @@ public class ExecutePlanActionTest extends SolrCloudTestCase {
NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST, NodeLostTrigger.NodeLostEvent nodeLostEvent = new NodeLostTrigger.NodeLostEvent(TriggerEventType.NODELOST,
"mock_trigger_name", Collections.singletonList(TimeSource.CURRENT_TIME.getTime()), "mock_trigger_name", Collections.singletonList(TimeSource.CURRENT_TIME.getTime()),
Collections.singletonList(sourceNodeName)); Collections.singletonList(sourceNodeName));
ActionContext actionContext = new ActionContext(survivor.getCoreContainer().getZkController().getClusterDataProvider(), null, ActionContext actionContext = new ActionContext(survivor.getCoreContainer(), null,
new HashMap<>(Collections.singletonMap("operations", operations))); new HashMap<>(Collections.singletonMap("operations", operations)));
action.process(nodeLostEvent, actionContext); action.process(nodeLostEvent, actionContext);

View File

@ -30,7 +30,6 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.TimeSource; import org.apache.solr.util.TimeSource;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -73,8 +72,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
long waitForSeconds = 1 + random().nextInt(5); long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds); Map<String, Object> props = createTriggerProps(waitForSeconds);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(), try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
trigger.run(); trigger.run();
@ -114,8 +112,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
// add a new node but remove it before the waitFor period expires // add a new node but remove it before the waitFor period expires
// and assert that the trigger doesn't fire at all // and assert that the trigger doesn't fire at all
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(), try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
final long waitTime = 2; final long waitTime = 2;
props.put("waitFor", waitTime); props.put("waitFor", waitTime);
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
@ -160,8 +157,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
action.put("name", "testActionInit"); action.put("name", "testActionInit");
action.put("class", NodeAddedTriggerTest.AssertInitTriggerAction.class.getName()); action.put("class", NodeAddedTriggerTest.AssertInitTriggerAction.class.getName());
actions.add(action); actions.add(action);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(), try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
assertEquals(true, actionConstructorCalled.get()); assertEquals(true, actionConstructorCalled.get());
assertEquals(false, actionInitCalled.get()); assertEquals(false, actionInitCalled.get());
assertEquals(false, actionCloseCalled.get()); assertEquals(false, actionCloseCalled.get());
@ -202,8 +198,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
public void testListenerAcceptance() throws Exception { public void testListenerAcceptance() throws Exception {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer(); CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
Map<String, Object> props = createTriggerProps(0); Map<String, Object> props = createTriggerProps(0);
try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(), try (NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
trigger.run(); // starts tracking live nodes trigger.run(); // starts tracking live nodes
@ -239,8 +234,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
// add a new node but update the trigger before the waitFor period expires // add a new node but update the trigger before the waitFor period expires
// and assert that the new trigger still fires // and assert that the new trigger still fires
NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(), NodeAddedTrigger trigger = new NodeAddedTrigger("node_added_trigger", props, container);
container.getZkController().getClusterDataProvider());
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
trigger.run(); trigger.run();
@ -248,8 +242,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
trigger.run(); // this run should detect the new node trigger.run(); // this run should detect the new node
trigger.close(); // close the old trigger trigger.close(); // close the old trigger
try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name", props, container.getResourceLoader(), try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("some_different_name", props, container)) {
container.getZkController().getClusterDataProvider())) {
try { try {
newTrigger.restoreState(trigger); newTrigger.restoreState(trigger);
fail("Trigger should only be able to restore state from an old trigger of the same name"); fail("Trigger should only be able to restore state from an old trigger of the same name");
@ -258,8 +251,7 @@ public class NodeAddedTriggerTest extends SolrCloudTestCase {
} }
} }
try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger", props, container.getResourceLoader(), try (NodeAddedTrigger newTrigger = new NodeAddedTrigger("node_added_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
AtomicBoolean fired = new AtomicBoolean(false); AtomicBoolean fired = new AtomicBoolean(false);
AtomicReference<TriggerEvent> eventRef = new AtomicReference<>(); AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
newTrigger.setProcessor(event -> { newTrigger.setProcessor(event -> {

View File

@ -73,8 +73,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
long waitForSeconds = 1 + random().nextInt(5); long waitForSeconds = 1 + random().nextInt(5);
Map<String, Object> props = createTriggerProps(waitForSeconds); Map<String, Object> props = createTriggerProps(waitForSeconds);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container.getResourceLoader(), try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
trigger.run(); trigger.run();
String lostNodeName1 = cluster.getJettySolrRunner(1).getNodeName(); String lostNodeName1 = cluster.getJettySolrRunner(1).getNodeName();
@ -118,8 +117,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
// remove a node but add it back before the waitFor period expires // remove a node but add it back before the waitFor period expires
// and assert that the trigger doesn't fire at all // and assert that the trigger doesn't fire at all
try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container.getResourceLoader(), try (NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
final long waitTime = 2; final long waitTime = 2;
props.put("waitFor", waitTime); props.put("waitFor", waitTime);
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
@ -175,8 +173,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
action.put("name", "testActionInit"); action.put("name", "testActionInit");
action.put("class", AssertInitTriggerAction.class.getName()); action.put("class", AssertInitTriggerAction.class.getName());
actions.add(action); actions.add(action);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, container.getResourceLoader(), try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
assertEquals(true, actionConstructorCalled.get()); assertEquals(true, actionConstructorCalled.get());
assertEquals(false, actionInitCalled.get()); assertEquals(false, actionInitCalled.get());
assertEquals(false, actionCloseCalled.get()); assertEquals(false, actionCloseCalled.get());
@ -217,8 +214,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
public void testListenerAcceptance() throws Exception { public void testListenerAcceptance() throws Exception {
CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer(); CoreContainer container = cluster.getJettySolrRunners().get(0).getCoreContainer();
Map<String, Object> props = createTriggerProps(0); Map<String, Object> props = createTriggerProps(0);
try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, container.getResourceLoader(), try (NodeLostTrigger trigger = new NodeLostTrigger("node_added_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
JettySolrRunner newNode = cluster.startJettySolrRunner(); JettySolrRunner newNode = cluster.startJettySolrRunner();
@ -271,8 +267,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
// remove a node but update the trigger before the waitFor period expires // remove a node but update the trigger before the waitFor period expires
// and assert that the new trigger still fires // and assert that the new trigger still fires
NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container.getResourceLoader(), NodeLostTrigger trigger = new NodeLostTrigger("node_lost_trigger", props, container);
container.getZkController().getClusterDataProvider());
trigger.setProcessor(noFirstRunProcessor); trigger.setProcessor(noFirstRunProcessor);
trigger.run(); trigger.run();
@ -289,8 +284,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
trigger.run(); // this run should detect the lost node trigger.run(); // this run should detect the lost node
trigger.close(); // close the old trigger trigger.close(); // close the old trigger
try (NodeLostTrigger newTrigger = new NodeLostTrigger("some_different_name", props, container.getResourceLoader(), try (NodeLostTrigger newTrigger = new NodeLostTrigger("some_different_name", props, container)) {
container.getZkController().getClusterDataProvider())) {
try { try {
newTrigger.restoreState(trigger); newTrigger.restoreState(trigger);
fail("Trigger should only be able to restore state from an old trigger of the same name"); fail("Trigger should only be able to restore state from an old trigger of the same name");
@ -299,8 +293,7 @@ public class NodeLostTriggerTest extends SolrCloudTestCase {
} }
} }
try (NodeLostTrigger newTrigger = new NodeLostTrigger("node_lost_trigger", props, container.getResourceLoader(), try (NodeLostTrigger newTrigger = new NodeLostTrigger("node_lost_trigger", props, container)) {
container.getZkController().getClusterDataProvider())) {
AtomicBoolean fired = new AtomicBoolean(false); AtomicBoolean fired = new AtomicBoolean(false);
AtomicReference<TriggerEvent> eventRef = new AtomicReference<>(); AtomicReference<TriggerEvent> eventRef = new AtomicReference<>();
newTrigger.setProcessor(event -> { newTrigger.setProcessor(event -> {

View File

@ -168,7 +168,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
CollectionAdminRequest.createCollection("metricsTest", "conf", 1, 1) CollectionAdminRequest.createCollection("metricsTest", "conf", 1, 1)
.process(cluster.getSolrClient()); .process(cluster.getSolrClient());
DocCollection collection = getCollectionState("metricsTest"); DocCollection collection = getCollectionState("metricsTest");
SolrClientDataProvider provider = new SolrClientDataProvider(new ZkDistributedQueueFactory(cluster.getZkClient()), solrClient); SolrClientDataProvider provider = new SolrClientDataProvider(solrClient);
List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count", List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
"metrics:solr.jvm:buffers.direct.Count"); "metrics:solr.jvm:buffers.direct.Count");
Map<String, Object> val = provider.getNodeValues(collection .getReplicas().get(0).getNodeName(), tags); Map<String, Object> val = provider.getNodeValues(collection .getReplicas().get(0).getNodeName(), tags);
@ -268,7 +268,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2) CollectionAdminRequest.createCollectionWithImplicitRouter("policiesTest", "conf", "shard1", 2)
.process(cluster.getSolrClient()); .process(cluster.getSolrClient());
DocCollection rulesCollection = getCollectionState("policiesTest"); DocCollection rulesCollection = getCollectionState("policiesTest");
SolrClientDataProvider provider = new SolrClientDataProvider(new ZkDistributedQueueFactory(cluster.getZkClient()), cluster.getSolrClient()); SolrClientDataProvider provider = new SolrClientDataProvider(cluster.getSolrClient());
Map<String, Object> val = provider.getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList( Map<String, Object> val = provider.getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
"freedisk", "freedisk",
"cores", "cores",

View File

@ -34,7 +34,6 @@ import java.util.concurrent.locks.ReentrantLock;
import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig; import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType; import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@ -966,8 +965,8 @@ public class TriggerIntegrationTest extends SolrCloudTestCase {
public static class TestTriggerListener extends TriggerListenerBase { public static class TestTriggerListener extends TriggerListenerBase {
@Override @Override
public void init(ClusterDataProvider clusterDataProvider, AutoScalingConfig.TriggerListenerConfig config) { public void init(CoreContainer coreContainer, AutoScalingConfig.TriggerListenerConfig config) {
super.init(clusterDataProvider, config); super.init(coreContainer, config);
listenerCreated.countDown(); listenerCreated.countDown();
} }

View File

@ -1,44 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.client.solrj.cloud;
import java.util.Collection;
import java.util.function.Predicate;
import org.apache.solr.common.util.Pair;
/**
*
*/
public interface DistributedQueue {
byte[] peek() throws Exception;
byte[] peek(boolean block) throws Exception;
byte[] peek(long wait) throws Exception;
byte[] poll() throws Exception;
byte[] remove() throws Exception;
byte[] take() throws Exception;
void offer(byte[] data) throws Exception;
Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws Exception;
}

View File

@ -19,28 +19,12 @@ package org.apache.solr.client.solrj.cloud.autoscaling;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.net.ConnectException;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NoSuchElementException;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.client.solrj.cloud.DistributedQueue;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.Watcher;
/**
* This interface abstracts the details of dealing with Zookeeper and Solr from the autoscaling framework.
*/
public interface ClusterDataProvider extends Closeable { public interface ClusterDataProvider extends Closeable {
/** /**Get the value of each tag for a given node
* Get the value of each tag for a given node
* *
* @param node node name * @param node node name
* @param tags tag names * @param tags tag names
@ -50,87 +34,19 @@ public interface ClusterDataProvider extends Closeable {
/** /**
* Get the details of each replica in a node. It attempts to fetch as much details about * 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 all details * the replica as mentioned in the keys list. It is not necessary to give al details
* <p> * <p>
* the format is {collection:shard :[{replicadetails}]} * the format is {collection:shard :[{replicadetails}]}
*/ */
Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys); Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys);
/** Collection<String> getNodes();
* Get the current set of live nodes.
*/
Collection<String> getLiveNodes();
ClusterState getClusterState() throws IOException; /**Get the collection-specific policy
Map<String, Object> getClusterProperties();
default <T> T getClusterProperty(String key, T defaultValue) {
T value = (T) getClusterProperties().get(key);
if (value == null)
return defaultValue;
return value;
}
AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws ConnectException, InterruptedException, IOException;
default AutoScalingConfig getAutoScalingConfig() throws ConnectException, InterruptedException, IOException {
return getAutoScalingConfig(null);
}
/**
* Get the collection-specific policy
*/ */
String getPolicyNameByCollection(String coll); String getPolicyNameByCollection(String coll);
@Override @Override
default void close() throws IOException { default void close() throws IOException {
} }
// ZK-like methods
boolean hasData(String path) throws IOException;
List<String> listData(String path) throws NoSuchElementException, IOException;
class VersionedData {
public final int version;
public final byte[] data;
public VersionedData(int version, byte[] data) {
this.version = version;
this.data = data;
}
}
VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException;
default VersionedData getData(String path) throws NoSuchElementException, IOException {
return getData(path, null);
}
// mutators
void makePath(String path) throws IOException;
void createData(String path, byte[] data, CreateMode mode) throws IOException;
void removeData(String path, int version) throws NoSuchElementException, IOException;
void setData(String path, byte[] data, int version) throws NoSuchElementException, IOException;
List<OpResult> multi(final Iterable<Op> ops) throws IOException;
// Solr-like methods
SolrResponse request(SolrRequest req) throws IOException;
HttpClient getHttpClient();
interface DistributedQueueFactory {
DistributedQueue makeQueue(String path) throws IOException;
void removeQueue(String path) throws IOException;
}
DistributedQueueFactory getDistributedQueueFactory();
} }

View File

@ -1,118 +0,0 @@
package org.apache.solr.client.solrj.cloud.autoscaling;
import java.io.IOException;
import java.net.ConnectException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.Watcher;
/**
*
*/
public class DelegatingClusterDataProvider implements ClusterDataProvider {
protected ClusterDataProvider delegate;
public DelegatingClusterDataProvider(ClusterDataProvider delegate) {
this.delegate = delegate;
}
@Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return delegate.getNodeValues(node, tags);
}
@Override
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return delegate.getReplicaInfo(node, keys);
}
@Override
public Collection<String> getLiveNodes() {
return delegate.getLiveNodes();
}
@Override
public Map<String, Object> getClusterProperties() {
return delegate.getClusterProperties();
}
@Override
public ClusterState getClusterState() throws IOException {
return delegate.getClusterState();
}
@Override
public AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws ConnectException, InterruptedException, IOException {
return delegate.getAutoScalingConfig(watcher);
}
@Override
public String getPolicyNameByCollection(String coll) {
return delegate.getPolicyNameByCollection(coll);
}
@Override
public boolean hasData(String path) throws IOException {
return delegate.hasData(path);
}
@Override
public List<String> listData(String path) throws NoSuchElementException, IOException {
return delegate.listData(path);
}
@Override
public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException {
return delegate.getData(path, watcher);
}
@Override
public void makePath(String path) throws IOException {
delegate.makePath(path);
}
@Override
public void createData(String path, byte[] data, CreateMode mode) throws IOException {
delegate.createData(path, data, mode);
}
@Override
public void removeData(String path, int version) throws NoSuchElementException, IOException {
delegate.removeData(path, version);
}
@Override
public void setData(String path, byte[] data, int version) throws NoSuchElementException, IOException {
delegate.setData(path, data, version);
}
@Override
public List<OpResult> multi(Iterable<Op> ops) throws IOException {
return delegate.multi(ops);
}
@Override
public SolrResponse request(SolrRequest req) throws IOException {
return delegate.request(req);
}
@Override
public HttpClient getHttpClient() {
return delegate.getHttpClient();
}
@Override
public DistributedQueueFactory getDistributedQueueFactory() {
return delegate.getDistributedQueueFactory();
}
}

View File

@ -201,7 +201,7 @@ public class Policy implements MapWriter {
} }
Session(ClusterDataProvider dataProvider) { Session(ClusterDataProvider dataProvider) {
this.nodes = new ArrayList<>(dataProvider.getLiveNodes()); this.nodes = new ArrayList<>(dataProvider.getNodes());
this.dataProvider = dataProvider; this.dataProvider = dataProvider;
for (String node : nodes) { for (String node : nodes) {
collections.addAll(dataProvider.getReplicaInfo(node, Collections.emptyList()).keySet()); collections.addAll(dataProvider.getReplicaInfo(node, Collections.emptyList()).keySet());

View File

@ -49,7 +49,22 @@ public class PolicyHelper {
List<String> nodesList) { List<String> nodesList) {
List<ReplicaPosition> positions = new ArrayList<>(); List<ReplicaPosition> positions = new ArrayList<>();
final ClusterDataProvider delegate = cdp; final ClusterDataProvider delegate = cdp;
cdp = new DelegatingClusterDataProvider(delegate) { 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<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
return delegate.getReplicaInfo(node, keys);
}
@Override
public Collection<String> getNodes() {
return delegate.getNodes();
}
@Override @Override
public String getPolicyNameByCollection(String coll) { public String getPolicyNameByCollection(String coll) {
return policyMapping.get() != null && policyMapping.get().containsKey(coll) ? return policyMapping.get() != null && policyMapping.get().containsKey(coll) ?

View File

@ -46,7 +46,7 @@ public class Row implements MapWriter {
if (collectionVsShardVsReplicas == null) collectionVsShardVsReplicas = new HashMap<>(); if (collectionVsShardVsReplicas == null) collectionVsShardVsReplicas = new HashMap<>();
this.node = node; this.node = node;
cells = new Cell[params.size()]; cells = new Cell[params.size()];
isLive = dataProvider.getLiveNodes().contains(node); isLive = dataProvider.getNodes().contains(node);
Map<String, Object> vals = isLive ? dataProvider.getNodeValues(node, params) : Collections.emptyMap(); Map<String, Object> vals = isLive ? dataProvider.getNodeValues(node, params) : Collections.emptyMap();
for (int i = 0; i < params.size(); i++) { for (int i = 0; i < params.size(); i++) {
String s = params.get(i); String s = params.get(i);

View File

@ -28,15 +28,10 @@ import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Queue;
import java.util.Set; import java.util.Set;
import org.apache.http.client.HttpClient;
import org.apache.solr.client.solrj.SolrRequest; 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.SolrServerException;
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider; import org.apache.solr.client.solrj.cloud.autoscaling.ClusterDataProvider;
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo; import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
import org.apache.solr.client.solrj.request.GenericSolrRequest; import org.apache.solr.client.solrj.request.GenericSolrRequest;
@ -45,7 +40,6 @@ import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.rule.ImplicitSnitch; import org.apache.solr.common.cloud.rule.ImplicitSnitch;
import org.apache.solr.common.cloud.rule.RemoteCallback; import org.apache.solr.common.cloud.rule.RemoteCallback;
@ -56,12 +50,7 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils; import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.Utils; import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -74,19 +63,16 @@ public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final CloudSolrClient solrClient; private final CloudSolrClient solrClient;
private final DistributedQueueFactory queueFactory;
private final ZkStateReader zkStateReader;
private final SolrZkClient zkClient;
private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> data = new HashMap<>(); 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, Object> snitchSession = new HashMap<>();
private Map<String, Map> nodeVsTags = new HashMap<>(); private Map<String, Map> nodeVsTags = new HashMap<>();
public SolrClientDataProvider(DistributedQueueFactory queueFactory, CloudSolrClient solrClient) { public SolrClientDataProvider(CloudSolrClient solrClient) {
this.queueFactory = queueFactory;
this.solrClient = solrClient; this.solrClient = solrClient;
this.zkStateReader = solrClient.getZkStateReader(); ZkStateReader zkStateReader = solrClient.getZkStateReader();
this.zkClient = zkStateReader.getZkClient();
ClusterState clusterState = zkStateReader.getClusterState(); ClusterState clusterState = zkStateReader.getClusterState();
this.liveNodes = clusterState.getLiveNodes();
Map<String, ClusterState.CollectionRef> all = clusterState.getCollectionStates(); Map<String, ClusterState.CollectionRef> all = clusterState.getCollectionStates();
all.forEach((collName, ref) -> { all.forEach((collName, ref) -> {
DocCollection coll = ref.get(); DocCollection coll = ref.get();
@ -121,135 +107,16 @@ public class SolrClientDataProvider implements ClusterDataProvider, MapWriter {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return solrClient.getZkStateReader().getClusterState().getLiveNodes(); return liveNodes;
} }
@Override @Override
public void writeMap(EntryWriter ew) throws IOException { public void writeMap(EntryWriter ew) throws IOException {
ew.put("liveNodes", zkStateReader.getClusterState().getLiveNodes()); ew.put("liveNodes", liveNodes);
ew.put("replicaInfo", Utils.getDeepCopy(data, 5)); ew.put("replicaInfo", Utils.getDeepCopy(data, 5));
ew.put("nodeValues", nodeVsTags); ew.put("nodeValues", nodeVsTags);
}
@Override
public Map<String, Object> getClusterProperties() {
return zkStateReader.getClusterProperties();
}
@Override
public ClusterState getClusterState() {
return zkStateReader.getClusterState();
}
@Override
public AutoScalingConfig getAutoScalingConfig(Watcher watcher) throws IOException {
try {
return zkStateReader.getAutoScalingConfig(watcher);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public boolean hasData(String path) throws IOException {
try {
return zkClient.exists(path, true);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public List<String> listData(String path) throws NoSuchElementException, IOException {
try {
return zkClient.getChildren(path, null, true);
} catch (KeeperException.NoNodeException e) {
throw new NoSuchElementException(path);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public VersionedData getData(String path, Watcher watcher) throws NoSuchElementException, IOException {
Stat stat = new Stat();
try {
byte[] bytes = zkClient.getData(path, watcher, stat, true);
return new VersionedData(stat.getVersion(), bytes);
} catch (KeeperException.NoNodeException e) {
throw new NoSuchElementException(path);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public void makePath(String path) throws IOException {
try {
zkClient.makePath(path, true);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public void createData(String path, byte[] data, CreateMode mode) throws IOException {
try {
zkClient.create(path, data, mode, true);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public void removeData(String path, int version) throws NoSuchElementException, IOException {
try {
zkClient.delete(path, version, true);
} catch (KeeperException.NoNodeException e) {
throw new NoSuchElementException(path);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public void setData(String path, byte[] data, int version) throws NoSuchElementException, IOException {
try {
zkClient.setData(path, data, version, true);
} catch (KeeperException.NoNodeException e) {
throw new NoSuchElementException(path);
} catch (KeeperException | InterruptedException e) {
throw new IOException(e);
}
}
@Override
public SolrResponse request(SolrRequest req) throws IOException {
try {
return req.process(solrClient);
} catch (SolrServerException e) {
throw new IOException(e);
}
}
@Override
public List<OpResult> multi(Iterable<Op> ops) throws IOException {
try {
return zkClient.multi(ops, true);
} catch (Exception e) {
throw new IOException(e);
}
}
@Override
public HttpClient getHttpClient() {
return solrClient.getHttpClient();
}
@Override
public DistributedQueueFactory getDistributedQueueFactory() {
return queueFactory;
} }
static class ClientSnitchCtx static class ClientSnitchCtx

View File

@ -27,6 +27,7 @@ import java.util.HashMap;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
@ -421,7 +422,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
}); });
}); });
return new DelegatingClusterDataProvider(null) { return new ClusterDataProvider(){
@Override @Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) { public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return (Map<String, Object>) Utils.getObjectByPath(m,false, Arrays.asList("nodeValues", node)); return (Map<String, Object>) Utils.getObjectByPath(m,false, Arrays.asList("nodeValues", node));
@ -433,7 +434,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return (Collection<String>) m.get("liveNodes"); return (Collection<String>) m.get("liveNodes");
} }
@ -962,7 +963,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson)); Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson));
ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState); ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState);
ClusterDataProvider cdp = new DelegatingClusterDataProvider(null) { ClusterDataProvider cdp = new ClusterDataProvider() {
@Override @Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) { public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return clusterDataProvider.getNodeValues(node, tags); return clusterDataProvider.getNodeValues(node, tags);
@ -974,8 +975,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return clusterDataProvider.getLiveNodes(); return clusterDataProvider.getNodes();
} }
@Override @Override
@ -1040,7 +1041,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
" 'freedisk':918005641216}}}"); " 'freedisk':918005641216}}}");
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson)); Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(autoscaleJson));
Policy.Session session = policy.createSession(new DelegatingClusterDataProvider(null) { Policy.Session session = policy.createSession(new ClusterDataProvider() {
@Override @Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) { public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return tagsMap.get(node); return tagsMap.get(node);
@ -1052,7 +1053,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return replicaInfoMap.keySet(); return replicaInfoMap.keySet();
} }
@ -1098,7 +1099,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
"}"); "}");
Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(rules)); Policy policy = new Policy((Map<String, Object>) Utils.fromJSONString(rules));
ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState); ClusterDataProvider clusterDataProvider = getClusterDataProvider(nodeValues, clusterState);
ClusterDataProvider cdp = new DelegatingClusterDataProvider(null) { ClusterDataProvider cdp = new ClusterDataProvider() {
@Override @Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) { public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
return clusterDataProvider.getNodeValues(node, tags); return clusterDataProvider.getNodeValues(node, tags);
@ -1110,8 +1111,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return clusterDataProvider.getLiveNodes(); return clusterDataProvider.getNodes();
} }
@Override @Override
@ -1130,7 +1131,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
private ClusterDataProvider getClusterDataProvider(final Map<String, Map> nodeValues, String clusterState) { private ClusterDataProvider getClusterDataProvider(final Map<String, Map> nodeValues, String clusterState) {
return new DelegatingClusterDataProvider(null) { return new ClusterDataProvider() {
@Override @Override
public Map<String, Object> getNodeValues(String node, Collection<String> tags) { public Map<String, Object> getNodeValues(String node, Collection<String> tags) {
Map<String, Object> result = new LinkedHashMap<>(); Map<String, Object> result = new LinkedHashMap<>();
@ -1139,7 +1140,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return nodeValues.keySet(); return nodeValues.keySet();
} }
@ -1167,7 +1168,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
" '127.0.0.1:50096_solr':{" + " '127.0.0.1:50096_solr':{" +
" 'cores':0," + " 'cores':0," +
" 'port':'50096'}}"); " 'port':'50096'}}");
ClusterDataProvider dataProvider = new DelegatingClusterDataProvider(null) { ClusterDataProvider dataProvider = new ClusterDataProvider() {
@Override @Override
public Map<String, Object> getNodeValues(String node, Collection<String> keys) { public Map<String, Object> getNodeValues(String node, Collection<String> keys) {
Map<String, Object> result = new LinkedHashMap<>(); Map<String, Object> result = new LinkedHashMap<>();
@ -1186,7 +1187,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return Arrays.asList( "127.0.0.1:50097_solr", "127.0.0.1:50096_solr"); return Arrays.asList( "127.0.0.1:50097_solr", "127.0.0.1:50096_solr");
} }
}; };
@ -1224,7 +1225,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
"node4:{cores:0, freedisk: 900, heap:16900, nodeRole:overseer, sysprop.rack:rack2}" + "node4:{cores:0, freedisk: 900, heap:16900, nodeRole:overseer, sysprop.rack:rack2}" +
"}"); "}");
ClusterDataProvider dataProvider = new DelegatingClusterDataProvider(null) { ClusterDataProvider dataProvider = new ClusterDataProvider() {
@Override @Override
public Map<String, Object> getNodeValues(String node, Collection<String> keys) { public Map<String, Object> getNodeValues(String node, Collection<String> keys) {
Map<String, Object> result = new LinkedHashMap<>(); Map<String, Object> result = new LinkedHashMap<>();
@ -1243,7 +1244,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
} }
@Override @Override
public Collection<String> getLiveNodes() { public Collection<String> getNodes() {
return Arrays.asList("node1", "node2", "node3", "node4"); return Arrays.asList("node1", "node2", "node3", "node4");
} }
}; };