mirror of https://github.com/apache/lucene.git
SOLR-14244: Remove ReplicaInfo.
This commit is contained in:
parent
e3c2a93edb
commit
a0488c1cf1
|
@ -93,7 +93,9 @@ Other Changes
|
|||
* LUCENE-9411: Fail complation on warnings, 9x gradle-only (Erick Erickson, Dawid Weiss)
|
||||
Deserves mention here as well as Lucene CHANGES.txt since it affects both.
|
||||
|
||||
* SOLR-12847: Remove support for maxShardsPerNode.
|
||||
* SOLR-12847: Remove support for maxShardsPerNode. (ab)
|
||||
|
||||
* SOLR-14244: Remove ReplicaInfo. (ab)
|
||||
|
||||
Bug Fixes
|
||||
---------------------
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.solr.client.solrj.cloud.DistribStateManager;
|
|||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
|
||||
import org.apache.solr.client.solrj.request.CoreAdminRequest;
|
||||
|
@ -638,17 +637,17 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
|
|||
NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
|
||||
Map<String, Object> nodeValues = nodeStateProvider.getNodeValues(parentShardLeader.getNodeName(),
|
||||
Collections.singletonList(ImplicitSnitch.DISK));
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = nodeStateProvider.getReplicaInfo(parentShardLeader.getNodeName(),
|
||||
Map<String, Map<String, List<Replica>>> infos = nodeStateProvider.getReplicaInfo(parentShardLeader.getNodeName(),
|
||||
Collections.singletonList(Type.CORE_IDX.metricsAttribute));
|
||||
if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing replica information for parent shard leader");
|
||||
}
|
||||
// find the leader
|
||||
List<ReplicaInfo> lst = infos.get(collection).get(shard);
|
||||
List<Replica> lst = infos.get(collection).get(shard);
|
||||
Double indexSize = null;
|
||||
for (ReplicaInfo info : lst) {
|
||||
if (info.getCore().equals(parentShardLeader.getCoreName())) {
|
||||
Number size = (Number)info.getVariable(Type.CORE_IDX.metricsAttribute);
|
||||
for (Replica info : lst) {
|
||||
if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
|
||||
Number size = (Number)info.get(Type.CORE_IDX.metricsAttribute);
|
||||
if (size == null) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing index size information for parent shard leader");
|
||||
}
|
||||
|
|
|
@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import java.util.Locale;
|
||||
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
|
||||
import org.apache.solr.common.SolrException;
|
||||
|
@ -263,14 +262,14 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
}
|
||||
|
||||
// replica name / info + size, retrieved from leaders only
|
||||
Map<String, ReplicaInfo> currentSizes = new HashMap<>();
|
||||
Map<String, Replica> currentSizes = new HashMap<>();
|
||||
|
||||
try {
|
||||
ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
|
||||
for (String node : clusterState.getLiveNodes()) {
|
||||
Map<String, ReplicaInfo> metricTags = new HashMap<>();
|
||||
Map<String, Replica> metricTags = new HashMap<>();
|
||||
// coll, shard, replica
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
|
||||
Map<String, Map<String, List<Replica>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
|
||||
infos.forEach((coll, shards) -> {
|
||||
if (!collections.isEmpty() && !collections.contains(coll)) {
|
||||
return;
|
||||
|
@ -293,9 +292,9 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
return;
|
||||
}
|
||||
// find ReplicaInfo
|
||||
ReplicaInfo info = null;
|
||||
for (ReplicaInfo ri : replicas) {
|
||||
if (r.getCoreName().equals(ri.getCore())) {
|
||||
Replica info = null;
|
||||
for (Replica ri : replicas) {
|
||||
if (r.getCoreName().equals(ri.getCoreName())) {
|
||||
info = ri;
|
||||
break;
|
||||
}
|
||||
|
@ -305,7 +304,7 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
return;
|
||||
}
|
||||
// we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
|
||||
String replicaName = Utils.parseMetricsReplicaName(coll, info.getCore());
|
||||
String replicaName = Utils.parseMetricsReplicaName(coll, info.getCoreName());
|
||||
if (replicaName == null) { // should never happen???
|
||||
replicaName = info.getName(); // which is actually coreNode name...
|
||||
}
|
||||
|
@ -325,7 +324,7 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
}
|
||||
Map<String, Object> sizes = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
|
||||
sizes.forEach((tag, size) -> {
|
||||
final ReplicaInfo info = metricTags.get(tag);
|
||||
final Replica info = metricTags.get(tag);
|
||||
if (info == null) {
|
||||
log.warn("Missing replica info for response tag {}", tag);
|
||||
} else {
|
||||
|
@ -335,15 +334,15 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
return;
|
||||
}
|
||||
|
||||
ReplicaInfo currentInfo = currentSizes.computeIfAbsent(info.getCore(), k -> (ReplicaInfo)info.clone());
|
||||
Replica currentInfo = currentSizes.computeIfAbsent(info.getCoreName(), k -> (Replica) info.clone());
|
||||
if (tag.contains("INDEX")) {
|
||||
currentInfo.getVariables().put(TOTAL_BYTES_SIZE_KEY, ((Number) size).longValue());
|
||||
currentInfo.getProperties().put(TOTAL_BYTES_SIZE_KEY, ((Number) size).longValue());
|
||||
} else if (tag.endsWith("SEARCHER.searcher.numDocs")) {
|
||||
currentInfo.getVariables().put(DOCS_SIZE_KEY, ((Number) size).longValue());
|
||||
currentInfo.getProperties().put(DOCS_SIZE_KEY, ((Number) size).longValue());
|
||||
} else if (tag.endsWith("SEARCHER.searcher.maxDoc")) {
|
||||
currentInfo.getVariables().put(MAX_DOC_KEY, ((Number) size).longValue());
|
||||
currentInfo.getProperties().put(MAX_DOC_KEY, ((Number) size).longValue());
|
||||
} else if (tag.endsWith("SEARCHER.searcher.indexCommitSize")) {
|
||||
currentInfo.getVariables().put(COMMIT_SIZE_KEY, ((Number) size).longValue());
|
||||
currentInfo.getProperties().put(COMMIT_SIZE_KEY, ((Number) size).longValue());
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -358,31 +357,31 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
// now check thresholds
|
||||
|
||||
// collection / list(info)
|
||||
Map<String, List<ReplicaInfo>> aboveSize = new HashMap<>();
|
||||
Map<String, List<Replica>> aboveSize = new HashMap<>();
|
||||
|
||||
Set<String> splittable = new HashSet<>();
|
||||
|
||||
currentSizes.forEach((coreName, info) -> {
|
||||
// calculate estimated bytes
|
||||
long maxDoc = (Long)info.getVariable(MAX_DOC_KEY);
|
||||
long numDocs = (Long)info.getVariable(DOCS_SIZE_KEY);
|
||||
long commitSize = (Long)info.getVariable(COMMIT_SIZE_KEY, 0L);
|
||||
long maxDoc = (Long)info.get(MAX_DOC_KEY);
|
||||
long numDocs = (Long)info.get(DOCS_SIZE_KEY);
|
||||
long commitSize = (Long)info.get(COMMIT_SIZE_KEY, 0L);
|
||||
if (commitSize <= 0) {
|
||||
commitSize = (Long)info.getVariable(TOTAL_BYTES_SIZE_KEY);
|
||||
commitSize = (Long)info.get(TOTAL_BYTES_SIZE_KEY);
|
||||
}
|
||||
// calculate estimated size as a side-effect
|
||||
commitSize = estimatedSize(maxDoc, numDocs, commitSize);
|
||||
info.getVariables().put(BYTES_SIZE_KEY, commitSize);
|
||||
info.getProperties().put(BYTES_SIZE_KEY, commitSize);
|
||||
|
||||
if ((Long)info.getVariable(BYTES_SIZE_KEY) > aboveBytes ||
|
||||
(Long)info.getVariable(DOCS_SIZE_KEY) > aboveDocs) {
|
||||
if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes ||
|
||||
(Long)info.get(DOCS_SIZE_KEY) > aboveDocs) {
|
||||
if (waitForElapsed(coreName, now, lastAboveEventMap)) {
|
||||
List<ReplicaInfo> infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
||||
List<Replica> infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
||||
if (!infos.contains(info)) {
|
||||
if ((Long)info.getVariable(BYTES_SIZE_KEY) > aboveBytes) {
|
||||
info.getVariables().put(VIOLATION_KEY, ABOVE_BYTES_PROP);
|
||||
if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes) {
|
||||
info.getProperties().put(VIOLATION_KEY, ABOVE_BYTES_PROP);
|
||||
} else {
|
||||
info.getVariables().put(VIOLATION_KEY, ABOVE_DOCS_PROP);
|
||||
info.getProperties().put(VIOLATION_KEY, ABOVE_DOCS_PROP);
|
||||
}
|
||||
infos.add(info);
|
||||
splittable.add(info.getName());
|
||||
|
@ -395,20 +394,20 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
});
|
||||
|
||||
// collection / list(info)
|
||||
Map<String, List<ReplicaInfo>> belowSize = new HashMap<>();
|
||||
Map<String, List<Replica>> belowSize = new HashMap<>();
|
||||
|
||||
currentSizes.forEach((coreName, info) -> {
|
||||
if (((Long)info.getVariable(BYTES_SIZE_KEY) < belowBytes ||
|
||||
(Long)info.getVariable(DOCS_SIZE_KEY) < belowDocs) &&
|
||||
if (((Long)info.get(BYTES_SIZE_KEY) < belowBytes ||
|
||||
(Long)info.get(DOCS_SIZE_KEY) < belowDocs) &&
|
||||
// make sure we don't produce conflicting ops
|
||||
!splittable.contains(info.getName())) {
|
||||
if (waitForElapsed(coreName, now, lastBelowEventMap)) {
|
||||
List<ReplicaInfo> infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
||||
List<Replica> infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
||||
if (!infos.contains(info)) {
|
||||
if ((Long)info.getVariable(BYTES_SIZE_KEY) < belowBytes) {
|
||||
info.getVariables().put(VIOLATION_KEY, BELOW_BYTES_PROP);
|
||||
if ((Long)info.get(BYTES_SIZE_KEY) < belowBytes) {
|
||||
info.getProperties().put(VIOLATION_KEY, BELOW_BYTES_PROP);
|
||||
} else {
|
||||
info.getVariables().put(VIOLATION_KEY, BELOW_DOCS_PROP);
|
||||
info.getProperties().put(VIOLATION_KEY, BELOW_DOCS_PROP);
|
||||
}
|
||||
infos.add(info);
|
||||
}
|
||||
|
@ -435,7 +434,7 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
// sort by decreasing size to first split the largest ones
|
||||
// XXX see the comment below about using DOCS_SIZE_PROP in lieu of BYTES_SIZE_PROP
|
||||
replicas.sort((r1, r2) -> {
|
||||
long delta = (Long) r1.getVariable(DOCS_SIZE_KEY) - (Long) r2.getVariable(DOCS_SIZE_KEY);
|
||||
long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
|
||||
if (delta > 0) {
|
||||
return -1;
|
||||
} else if (delta < 0) {
|
||||
|
@ -458,7 +457,7 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
params.put(SPLIT_BY_PREFIX, splitByPrefix);
|
||||
op.addHint(Suggester.Hint.PARAMS, params);
|
||||
ops.add(op);
|
||||
Long time = lastAboveEventMap.get(r.getCore());
|
||||
Long time = lastAboveEventMap.get(r.getCoreName());
|
||||
if (time != null && eventTime.get() > time) {
|
||||
eventTime.set(time);
|
||||
}
|
||||
|
@ -477,7 +476,7 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
// then we should be sorting by BYTES_SIZE_PROP. However, since DOCS and BYTES are
|
||||
// loosely correlated it's simpler to sort just by docs (which better reflects the "too small"
|
||||
// condition than index size, due to possibly existing deleted docs that still occupy space)
|
||||
long delta = (Long) r1.getVariable(DOCS_SIZE_KEY) - (Long) r2.getVariable(DOCS_SIZE_KEY);
|
||||
long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
|
||||
if (delta > 0) {
|
||||
return 1;
|
||||
} else if (delta < 0) {
|
||||
|
@ -495,11 +494,11 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(0).getShard()));
|
||||
op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(1).getShard()));
|
||||
ops.add(op);
|
||||
Long time = lastBelowEventMap.get(replicas.get(0).getCore());
|
||||
Long time = lastBelowEventMap.get(replicas.get(0).getCoreName());
|
||||
if (time != null && eventTime.get() > time) {
|
||||
eventTime.set(time);
|
||||
}
|
||||
time = lastBelowEventMap.get(replicas.get(1).getCore());
|
||||
time = lastBelowEventMap.get(replicas.get(1).getCoreName());
|
||||
if (time != null && eventTime.get() > time) {
|
||||
eventTime.set(time);
|
||||
}
|
||||
|
@ -511,14 +510,14 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
if (processor.process(new IndexSizeEvent(getName(), eventTime.get(), ops, aboveSize, belowSize))) {
|
||||
// update last event times
|
||||
aboveSize.forEach((coll, replicas) -> {
|
||||
replicas.forEach(r -> lastAboveEventMap.put(r.getCore(), now));
|
||||
replicas.forEach(r -> lastAboveEventMap.put(r.getCoreName(), now));
|
||||
});
|
||||
belowSize.forEach((coll, replicas) -> {
|
||||
if (replicas.size() < 2) {
|
||||
return;
|
||||
}
|
||||
lastBelowEventMap.put(replicas.get(0).getCore(), now);
|
||||
lastBelowEventMap.put(replicas.get(1).getCore(), now);
|
||||
lastBelowEventMap.put(replicas.get(0).getCoreName(), now);
|
||||
lastBelowEventMap.put(replicas.get(1).getCoreName(), now);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -544,18 +543,18 @@ public class IndexSizeTrigger extends TriggerBase {
|
|||
}
|
||||
|
||||
public static class IndexSizeEvent extends TriggerEvent {
|
||||
public IndexSizeEvent(String source, long eventTime, List<Op> ops, Map<String, List<ReplicaInfo>> aboveSize,
|
||||
Map<String, List<ReplicaInfo>> belowSize) {
|
||||
public IndexSizeEvent(String source, long eventTime, List<Op> ops, Map<String, List<Replica>> aboveSize,
|
||||
Map<String, List<Replica>> belowSize) {
|
||||
super(TriggerEventType.INDEXSIZE, source, eventTime, null);
|
||||
properties.put(TriggerEvent.REQUESTED_OPS, ops);
|
||||
// avoid passing very large amounts of data here - just use replica names
|
||||
TreeMap<String, String> above = new TreeMap<>();
|
||||
aboveSize.forEach((coll, replicas) ->
|
||||
replicas.forEach(r -> above.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_KEY) + ", bytes=" + r.getVariable(BYTES_SIZE_KEY))));
|
||||
replicas.forEach(r -> above.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
|
||||
properties.put(ABOVE_SIZE_KEY, above);
|
||||
TreeMap<String, String> below = new TreeMap<>();
|
||||
belowSize.forEach((coll, replicas) ->
|
||||
replicas.forEach(r -> below.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_KEY) + ", bytes=" + r.getVariable(BYTES_SIZE_KEY))));
|
||||
replicas.forEach(r -> below.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
|
||||
properties.put(BELOW_SIZE_KEY, below);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.util.concurrent.AtomicDouble;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
|
||||
|
@ -331,7 +330,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
}
|
||||
|
||||
// collection, shard, list(replica + rate)
|
||||
Map<String, Map<String, List<ReplicaInfo>>> collectionRates = new HashMap<>();
|
||||
Map<String, Map<String, List<Replica>>> collectionRates = new HashMap<>();
|
||||
// node, rate
|
||||
Map<String, AtomicDouble> nodeRates = new HashMap<>();
|
||||
// this replication factor only considers replica types that are searchable
|
||||
|
@ -346,9 +345,9 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
return;
|
||||
}
|
||||
for (String node : cloudManager.getClusterStateProvider().getLiveNodes()) {
|
||||
Map<String, ReplicaInfo> metricTags = new HashMap<>();
|
||||
Map<String, Replica> metricTags = new HashMap<>();
|
||||
// coll, shard, replica
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
|
||||
Map<String, Map<String, List<Replica>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
|
||||
infos.forEach((coll, shards) -> {
|
||||
Map<String, AtomicInteger> replPerShard = searchableReplicationFactors.computeIfAbsent(coll, c -> new HashMap<>());
|
||||
shards.forEach((sh, replicas) -> {
|
||||
|
@ -360,7 +359,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
}
|
||||
repl.incrementAndGet();
|
||||
// we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
|
||||
String replicaName = Utils.parseMetricsReplicaName(coll, replica.getCore());
|
||||
String replicaName = Utils.parseMetricsReplicaName(coll, replica.getCoreName());
|
||||
if (replicaName == null) { // should never happen???
|
||||
replicaName = replica.getName(); // which is actually coreNode name...
|
||||
}
|
||||
|
@ -379,14 +378,14 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
rates.forEach((tag, rate) -> log.debug("### " + tag + "\t" + rate)); // logOk
|
||||
}
|
||||
rates.forEach((tag, rate) -> {
|
||||
ReplicaInfo info = metricTags.get(tag);
|
||||
Replica info = metricTags.get(tag);
|
||||
if (info == null) {
|
||||
log.warn("Missing replica info for response tag {}", tag);
|
||||
} else {
|
||||
Map<String, List<ReplicaInfo>> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>());
|
||||
List<ReplicaInfo> perShard = perCollection.computeIfAbsent(info.getShard(), s -> new ArrayList<>());
|
||||
info = (ReplicaInfo)info.clone();
|
||||
info.getVariables().put(AutoScalingParams.RATE, ((Number)rate).doubleValue());
|
||||
Map<String, List<Replica>> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>());
|
||||
List<Replica> perShard = perCollection.computeIfAbsent(info.getShard(), s -> new ArrayList<>());
|
||||
info = (Replica)info.clone();
|
||||
info.getProperties().put(AutoScalingParams.RATE, ((Number)rate).doubleValue());
|
||||
perShard.add(info);
|
||||
AtomicDouble perNode = nodeRates.computeIfAbsent(node, s -> new AtomicDouble());
|
||||
perNode.addAndGet(((Number)rate).doubleValue());
|
||||
|
@ -399,7 +398,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
log.debug("## Collection: {}", coll);
|
||||
collRates.forEach((s, replicas) -> {
|
||||
log.debug("## - {}", s);
|
||||
replicas.forEach(ri -> log.debug("## {} {}", ri.getCore(), ri.getVariable(AutoScalingParams.RATE))); //logOk
|
||||
replicas.forEach(ri -> log.debug("## {} {}", ri.getCoreName(), ri.get(AutoScalingParams.RATE))); //logOk
|
||||
});
|
||||
});
|
||||
}
|
||||
|
@ -428,18 +427,18 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
|
||||
Map<String, Map<String, Double>> hotShards = new HashMap<>();
|
||||
Map<String, Map<String, Double>> coldShards = new HashMap<>();
|
||||
List<ReplicaInfo> hotReplicas = new ArrayList<>();
|
||||
List<ReplicaInfo> coldReplicas = new ArrayList<>();
|
||||
List<Replica> hotReplicas = new ArrayList<>();
|
||||
List<Replica> coldReplicas = new ArrayList<>();
|
||||
collectionRates.forEach((coll, shardRates) -> {
|
||||
shardRates.forEach((sh, replicaRates) -> {
|
||||
double totalShardRate = replicaRates.stream()
|
||||
.map(r -> {
|
||||
String elapsedKey = r.getCollection() + "." + r.getCore();
|
||||
if ((Double)r.getVariable(AutoScalingParams.RATE) > aboveRate) {
|
||||
String elapsedKey = r.getCollection() + "." + r.getCoreName();
|
||||
if ((Double)r.get(AutoScalingParams.RATE) > aboveRate) {
|
||||
if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
|
||||
hotReplicas.add(r);
|
||||
}
|
||||
} else if ((Double)r.getVariable(AutoScalingParams.RATE) < belowRate) {
|
||||
} else if ((Double)r.get(AutoScalingParams.RATE) < belowRate) {
|
||||
if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
|
||||
coldReplicas.add(r);
|
||||
}
|
||||
|
@ -449,7 +448,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
}
|
||||
return r;
|
||||
})
|
||||
.mapToDouble(r -> (Double)r.getVariable(AutoScalingParams.RATE)).sum();
|
||||
.mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum();
|
||||
// calculate average shard rate over all searchable replicas (see SOLR-12470)
|
||||
double shardRate = totalShardRate / searchableReplicationFactors.get(coll).get(sh).doubleValue();
|
||||
String elapsedKey = coll + "." + sh;
|
||||
|
@ -486,7 +485,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
collectionRates.forEach((coll, shardRates) -> {
|
||||
double total = shardRates.entrySet().stream()
|
||||
.mapToDouble(e -> e.getValue().stream()
|
||||
.mapToDouble(r -> (Double)r.getVariable(AutoScalingParams.RATE)).sum()).sum();
|
||||
.mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum()).sum();
|
||||
if (collections.isEmpty() || collections.contains(coll)) {
|
||||
if (total > aboveRate) {
|
||||
if (waitForElapsed(coll, now, lastCollectionEvent)) {
|
||||
|
@ -547,13 +546,13 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
});
|
||||
});
|
||||
hotReplicas.forEach(r -> {
|
||||
long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCore());
|
||||
long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
|
||||
if (eventTime.get() > time) {
|
||||
eventTime.set(time);
|
||||
}
|
||||
});
|
||||
coldReplicas.forEach(r -> {
|
||||
long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCore());
|
||||
long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
|
||||
if (eventTime.get() > time) {
|
||||
eventTime.set(time);
|
||||
}
|
||||
|
@ -593,8 +592,8 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
.forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
|
||||
coldShards.entrySet().forEach(e -> e.getValue()
|
||||
.forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
|
||||
hotReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCore(), now));
|
||||
coldReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCore(), now));
|
||||
hotReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
|
||||
coldReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -604,7 +603,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
Map<String, Double> hotNodes,
|
||||
Map<String, Double> hotCollections,
|
||||
Map<String, Map<String, Double>> hotShards,
|
||||
List<ReplicaInfo> hotReplicas) {
|
||||
List<Replica> hotReplicas) {
|
||||
// calculate the number of replicas to add to each hot shard, based on how much the rate was
|
||||
// exceeded - but within limits.
|
||||
|
||||
|
@ -678,7 +677,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
Map<String, Double> coldNodes,
|
||||
Map<String, Double> coldCollections,
|
||||
Map<String, Map<String, Double>> coldShards,
|
||||
List<ReplicaInfo> coldReplicas) {
|
||||
List<Replica> coldReplicas) {
|
||||
// COLD COLLECTIONS
|
||||
// Probably can't do anything reasonable about whole cold collections
|
||||
// because they may be needed even if not used.
|
||||
|
@ -694,7 +693,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
// replicas still available (additional non-searchable replicas may exist, too)
|
||||
// NOTE: do this before adding ops for DELETENODE because we don't want to attempt
|
||||
// deleting replicas that have been already moved elsewhere
|
||||
Map<String, Map<String, List<ReplicaInfo>>> byCollectionByShard = new HashMap<>();
|
||||
Map<String, Map<String, List<Replica>>> byCollectionByShard = new HashMap<>();
|
||||
coldReplicas.forEach(ri -> {
|
||||
byCollectionByShard.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
|
||||
.computeIfAbsent(ri.getShard(), s -> new ArrayList<>())
|
||||
|
@ -702,7 +701,7 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
});
|
||||
coldShards.forEach((coll, perShard) -> {
|
||||
perShard.forEach((shard, rate) -> {
|
||||
List<ReplicaInfo> replicas = byCollectionByShard
|
||||
List<Replica> replicas = byCollectionByShard
|
||||
.getOrDefault(coll, Collections.emptyMap())
|
||||
.getOrDefault(shard, Collections.emptyList());
|
||||
if (replicas.isEmpty()) {
|
||||
|
@ -784,11 +783,11 @@ public class SearchRateTrigger extends TriggerBase {
|
|||
Map<String, Double> hotNodes,
|
||||
Map<String, Double> hotCollections,
|
||||
Map<String, Map<String, Double>> hotShards,
|
||||
List<ReplicaInfo> hotReplicas,
|
||||
List<Replica> hotReplicas,
|
||||
Map<String, Double> coldNodes,
|
||||
Map<String, Double> coldCollections,
|
||||
Map<String, Map<String, Double>> coldShards,
|
||||
List<ReplicaInfo> coldReplicas,
|
||||
List<Replica> coldReplicas,
|
||||
Set<String> violations) {
|
||||
super(TriggerEventType.SEARCHRATE, source, eventTime, null);
|
||||
properties.put(TriggerEvent.REQUESTED_OPS, ops);
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
|
|||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
|
||||
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
|
||||
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
|
||||
|
@ -320,7 +319,7 @@ public class SimCloudManager implements SolrCloudManager {
|
|||
for (String node : other.getClusterStateProvider().getLiveNodes()) {
|
||||
SimClusterStateProvider simClusterStateProvider = cloudManager.getSimClusterStateProvider();
|
||||
cloudManager.getSimNodeStateProvider().simSetNodeValues(node, other.getNodeStateProvider().getNodeValues(node, nodeTags));
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
|
||||
Map<String, Map<String, List<Replica>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
|
||||
simClusterStateProvider.simSetReplicaValues(node, infos, true);
|
||||
}
|
||||
SimUtils.checkConsistency(cloudManager, config);
|
||||
|
@ -397,7 +396,7 @@ public class SimCloudManager implements SolrCloudManager {
|
|||
Map<String, Map<Replica.State, AtomicInteger>> replicaStates = new TreeMap<>();
|
||||
int numReplicas = 0;
|
||||
for (String node : getLiveNodesSet().get()) {
|
||||
List<ReplicaInfo> replicas = getSimClusterStateProvider().simGetReplicaInfos(node);
|
||||
List<Replica> replicas = getSimClusterStateProvider().simGetReplicaInfos(node);
|
||||
numReplicas += replicas.size();
|
||||
if (replicas.size() > maxReplicas) {
|
||||
maxReplicas = replicas.size();
|
||||
|
@ -405,7 +404,7 @@ public class SimCloudManager implements SolrCloudManager {
|
|||
if (minReplicas > replicas.size()) {
|
||||
minReplicas = replicas.size();
|
||||
}
|
||||
for (ReplicaInfo ri : replicas) {
|
||||
for (Replica ri : replicas) {
|
||||
replicaStates.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>())
|
||||
.computeIfAbsent(ri.getState(), s -> new AtomicInteger())
|
||||
.incrementAndGet();
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
|||
import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
|
||||
|
@ -139,8 +138,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
private final SimDistribStateManager stateManager;
|
||||
private final SimCloudManager cloudManager;
|
||||
|
||||
private final Map<String, List<ReplicaInfo>> nodeReplicaMap = new ConcurrentHashMap<>();
|
||||
private final Map<String, Map<String, List<ReplicaInfo>>> colShardReplicaMap = new ConcurrentHashMap<>();
|
||||
private final Map<String, List<Replica>> nodeReplicaMap = new ConcurrentHashMap<>();
|
||||
private final Map<String, Map<String, List<Replica>>> colShardReplicaMap = new ConcurrentHashMap<>();
|
||||
private final Map<String, Object> clusterProperties = new ConcurrentHashMap<>();
|
||||
private final Map<String, Map<String, Object>> collProperties = new ConcurrentHashMap<>();
|
||||
private final Map<String, Map<String, Map<String, Object>>> sliceProperties = new ConcurrentHashMap<>();
|
||||
|
@ -190,10 +189,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
}
|
||||
Map<String, Object> props;
|
||||
synchronized (ri) {
|
||||
props = new HashMap<>(ri.getVariables());
|
||||
props = new HashMap<>(ri.getProperties());
|
||||
}
|
||||
props.put(ZkStateReader.NODE_NAME_PROP, n);
|
||||
props.put(ZkStateReader.CORE_NAME_PROP, ri.getCore());
|
||||
props.put(ZkStateReader.CORE_NAME_PROP, ri.getCoreName());
|
||||
props.put(ZkStateReader.REPLICA_TYPE, ri.getType().toString());
|
||||
props.put(ZkStateReader.STATE_PROP, ri.getState().toString());
|
||||
Replica r = new Replica(ri.getName(), props, ri.getCollection(), ri.getShard());
|
||||
|
@ -360,9 +359,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
if (leader != null && r.getName().equals(leader.getName())) {
|
||||
props.put("leader", "true");
|
||||
}
|
||||
ReplicaInfo ri = new ReplicaInfo(r.getName(), r.getCoreName(), dc.getName(), s.getName(), r.getType(), r.getNodeName(), props);
|
||||
Replica ri = new Replica(r.getName(), r.getNodeName(), dc.getName(), s.getName(), r.getCoreName(),
|
||||
r.getState(), r.getType(), props);
|
||||
if (leader != null && r.getName().equals(leader.getName())) {
|
||||
ri.getVariables().put("leader", "true");
|
||||
ri.getProperties().put("leader", "true");
|
||||
}
|
||||
if (liveNodes.get().contains(r.getNodeName())) {
|
||||
nodeReplicaMap.computeIfAbsent(r.getNodeName(), Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN).add(ri);
|
||||
|
@ -412,13 +412,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
return nodes.get(random.nextInt(nodes.size()));
|
||||
}
|
||||
|
||||
private ReplicaInfo getReplicaInfo(Replica r) {
|
||||
private Replica getReplicaInfo(Replica r) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
final List<ReplicaInfo> list = nodeReplicaMap.computeIfAbsent
|
||||
final List<Replica> list = nodeReplicaMap.computeIfAbsent
|
||||
(r.getNodeName(), Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
|
||||
synchronized (list) {
|
||||
for (ReplicaInfo ri : list) {
|
||||
if (r.getCoreName().equals(ri.getCore()) && r.getName().equals(ri.getName())) {
|
||||
for (Replica ri : list) {
|
||||
if (r.getCoreName().equals(ri.getCoreName()) && r.getName().equals(ri.getName())) {
|
||||
return ri;
|
||||
}
|
||||
}
|
||||
|
@ -529,12 +529,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
// this method needs to be called under a lock
|
||||
private void setReplicaStates(String nodeId, Replica.State state, Set<String> changedCollections) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent(nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
|
||||
List<Replica> replicas = nodeReplicaMap.computeIfAbsent(nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
|
||||
synchronized (replicas) {
|
||||
replicas.forEach(r -> {
|
||||
r.getVariables().put(ZkStateReader.STATE_PROP, state.toString());
|
||||
r.setState(state);
|
||||
//r.getProperties().put(ZkStateReader.STATE_PROP, state.toString());
|
||||
if (state != Replica.State.ACTIVE) {
|
||||
r.getVariables().remove(ZkStateReader.LEADER_PROP);
|
||||
r.getProperties().remove(ZkStateReader.LEADER_PROP);
|
||||
}
|
||||
changedCollections.add(r.getCollection());
|
||||
});
|
||||
|
@ -628,16 +629,17 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
if (message.getStr(CoreAdminParams.CORE_NODE_NAME) == null) {
|
||||
createReplica.coreNodeName = Assign.assignCoreNodeName(stateManager, coll);
|
||||
}
|
||||
ReplicaInfo ri = new ReplicaInfo(
|
||||
Replica ri = new Replica(
|
||||
createReplica.coreNodeName,
|
||||
createReplica.coreName,
|
||||
createReplica.node,
|
||||
createReplica.collectionName,
|
||||
createReplica.sliceName,
|
||||
createReplica.coreName,
|
||||
Replica.State.DOWN,
|
||||
createReplica.replicaType,
|
||||
createReplica.node,
|
||||
message.getProperties()
|
||||
);
|
||||
simAddReplica(ri.getNode(), ri, true);
|
||||
simAddReplica(ri.getNodeName(), ri, true);
|
||||
}
|
||||
if (sessionWrapper.get() != null) {
|
||||
sessionWrapper.get().release();
|
||||
|
@ -653,17 +655,17 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
* @param runLeaderElection if true then run a leader election after adding the replica.
|
||||
*/
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public void simAddReplica(String nodeId, ReplicaInfo replicaInfo, boolean runLeaderElection) throws Exception {
|
||||
public void simAddReplica(String nodeId, Replica replicaInfo, boolean runLeaderElection) throws Exception {
|
||||
ensureNotClosed();
|
||||
lock.lockInterruptibly();
|
||||
try {
|
||||
|
||||
// make sure SolrCore name is unique across cluster and coreNodeName within collection
|
||||
for (Map.Entry<String, List<ReplicaInfo>> e : nodeReplicaMap.entrySet()) {
|
||||
final List<ReplicaInfo> replicas = e.getValue();
|
||||
for (Map.Entry<String, List<Replica>> e : nodeReplicaMap.entrySet()) {
|
||||
final List<Replica> replicas = e.getValue();
|
||||
synchronized (replicas) {
|
||||
for (ReplicaInfo ri : replicas) {
|
||||
if (ri.getCore().equals(replicaInfo.getCore())) {
|
||||
for (Replica ri : replicas) {
|
||||
if (ri.getCoreName().equals(replicaInfo.getCoreName())) {
|
||||
throw new Exception("Duplicate SolrCore name for existing=" + ri + " on node " + e.getKey() + " and new=" + replicaInfo);
|
||||
}
|
||||
if (ri.getName().equals(replicaInfo.getName()) && ri.getCollection().equals(replicaInfo.getCollection())) {
|
||||
|
@ -676,29 +678,29 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
throw new Exception("Target node " + nodeId + " is not live: " + liveNodes);
|
||||
}
|
||||
// verify info
|
||||
if (replicaInfo.getCore() == null) {
|
||||
if (replicaInfo.getCoreName() == null) {
|
||||
throw new Exception("Missing core: " + replicaInfo);
|
||||
}
|
||||
// XXX replica info is not supposed to have this as a variable
|
||||
replicaInfo.getVariables().remove(ZkStateReader.SHARD_ID_PROP);
|
||||
replicaInfo.getProperties().remove(ZkStateReader.SHARD_ID_PROP);
|
||||
if (replicaInfo.getName() == null) {
|
||||
throw new Exception("Missing name: " + replicaInfo);
|
||||
}
|
||||
if (replicaInfo.getNode() == null) {
|
||||
if (replicaInfo.getNodeName() == null) {
|
||||
throw new Exception("Missing node: " + replicaInfo);
|
||||
}
|
||||
if (!replicaInfo.getNode().equals(nodeId)) {
|
||||
if (!replicaInfo.getNodeName().equals(nodeId)) {
|
||||
throw new Exception("Wrong node (not " + nodeId + "): " + replicaInfo);
|
||||
}
|
||||
|
||||
opDelay(replicaInfo.getCollection(), CollectionParams.CollectionAction.ADDREPLICA.name());
|
||||
|
||||
// mark replica as active
|
||||
replicaInfo.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
|
||||
replicaInfo.setState(Replica.State.ACTIVE);
|
||||
// add a property expected in Policy calculations, if missing
|
||||
if (replicaInfo.getVariable(Type.CORE_IDX.metricsAttribute) == null) {
|
||||
replicaInfo.getVariables().put(Type.CORE_IDX.metricsAttribute, new AtomicLong(SimCloudManager.DEFAULT_IDX_SIZE_BYTES));
|
||||
replicaInfo.getVariables().put(Variable.coreidxsize,
|
||||
if (replicaInfo.get(Type.CORE_IDX.metricsAttribute) == null) {
|
||||
replicaInfo.getProperties().put(Type.CORE_IDX.metricsAttribute, new AtomicLong(SimCloudManager.DEFAULT_IDX_SIZE_BYTES));
|
||||
replicaInfo.getProperties().put(Variable.coreidxsize,
|
||||
new AtomicDouble((Double)Type.CORE_IDX.convertVal(SimCloudManager.DEFAULT_IDX_SIZE_BYTES)));
|
||||
}
|
||||
nodeReplicaMap.computeIfAbsent(nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN).add(replicaInfo);
|
||||
|
@ -719,13 +721,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
throw new Exception("Missing '" + ImplicitSnitch.DISK + "' in node metrics for node " + nodeId);
|
||||
//disk = SimCloudManager.DEFAULT_FREE_DISK;
|
||||
}
|
||||
long replicaSize = ((Number)replicaInfo.getVariable(Type.CORE_IDX.metricsAttribute)).longValue();
|
||||
long replicaSize = ((Number)replicaInfo.get(Type.CORE_IDX.metricsAttribute)).longValue();
|
||||
Number replicaSizeGB = (Number)Type.CORE_IDX.convertVal(replicaSize);
|
||||
cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk.doubleValue() - replicaSizeGB.doubleValue());
|
||||
// fake metrics
|
||||
String registry = SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, replicaInfo.getCollection(),
|
||||
replicaInfo.getShard(),
|
||||
Utils.parseMetricsReplicaName(replicaInfo.getCollection(), replicaInfo.getCore()));
|
||||
Utils.parseMetricsReplicaName(replicaInfo.getCollection(), replicaInfo.getCoreName()));
|
||||
cloudManager.getMetricManager().registry(registry).counter("UPDATE./update.requests");
|
||||
cloudManager.getMetricManager().registry(registry).counter("QUERY./select.requests");
|
||||
cloudManager.getMetricManager().registerGauge(null, registry,
|
||||
|
@ -752,12 +754,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
lock.lockInterruptibly();
|
||||
try {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
final List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent
|
||||
final List<Replica> replicas = nodeReplicaMap.computeIfAbsent
|
||||
(nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
|
||||
synchronized (replicas) {
|
||||
for (int i = 0; i < replicas.size(); i++) {
|
||||
if (collection.equals(replicas.get(i).getCollection()) && coreNodeName.equals(replicas.get(i).getName())) {
|
||||
ReplicaInfo ri = replicas.remove(i);
|
||||
Replica ri = replicas.remove(i);
|
||||
colShardReplicaMap.computeIfAbsent(ri.getCollection(), c -> new ConcurrentHashMap<>())
|
||||
.computeIfAbsent(ri.getShard(), s -> new ArrayList<>())
|
||||
.remove(ri);
|
||||
|
@ -776,10 +778,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
if (disk == null || disk.doubleValue() == 0.0) {
|
||||
throw new Exception("Unexpected value of 'freedisk' (" + disk + ") on node: " + nodeId);
|
||||
}
|
||||
if (ri.getVariable(Type.CORE_IDX.metricsAttribute) == null) {
|
||||
if (ri.get(Type.CORE_IDX.metricsAttribute) == null) {
|
||||
throw new RuntimeException("Missing replica size: " + ri);
|
||||
}
|
||||
long replicaSize = ((Number)ri.getVariable(Type.CORE_IDX.metricsAttribute)).longValue();
|
||||
long replicaSize = ((Number)ri.get(Type.CORE_IDX.metricsAttribute)).longValue();
|
||||
Number replicaSizeGB = (Number)Type.CORE_IDX.convertVal(replicaSize);
|
||||
cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk.doubleValue() + replicaSizeGB.doubleValue());
|
||||
}
|
||||
|
@ -907,17 +909,17 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
ActionThrottle lt = getThrottle(collection, s.getName());
|
||||
synchronized (lt) {
|
||||
// collect all active and live
|
||||
List<ReplicaInfo> active = new ArrayList<>();
|
||||
List<Replica> active = new ArrayList<>();
|
||||
AtomicBoolean alreadyHasLeader = new AtomicBoolean(false);
|
||||
s.getReplicas().forEach(r -> {
|
||||
// find our ReplicaInfo for this replica
|
||||
ReplicaInfo ri = getReplicaInfo(r);
|
||||
Replica ri = getReplicaInfo(r);
|
||||
if (ri == null) {
|
||||
throw new IllegalStateException("-- could not find ReplicaInfo for replica " + r);
|
||||
}
|
||||
synchronized (ri) {
|
||||
if (r.isActive(liveNodes.get())) {
|
||||
if (ri.getVariables().get(ZkStateReader.LEADER_PROP) != null) {
|
||||
if (ri.getProperties().get(ZkStateReader.LEADER_PROP) != null) {
|
||||
if (log.isTraceEnabled()) {
|
||||
log.trace("-- found existing leader {} / {}: {}, {}", collection, s.getName(), ri, r);
|
||||
}
|
||||
|
@ -931,8 +933,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
log.trace("-- replica not active on live nodes: {}, {}", liveNodes.get(), r);
|
||||
}
|
||||
if (!liveNodes.contains(r.getNodeName())) {
|
||||
ri.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
|
||||
ri.getVariables().remove(ZkStateReader.LEADER_PROP);
|
||||
ri.getProperties().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
|
||||
ri.getProperties().remove(ZkStateReader.LEADER_PROP);
|
||||
stateChanged.set(true);
|
||||
}
|
||||
}
|
||||
|
@ -954,8 +956,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
return;
|
||||
}
|
||||
// pick first active one
|
||||
ReplicaInfo ri = null;
|
||||
for (ReplicaInfo a : active) {
|
||||
Replica ri = null;
|
||||
for (Replica a : active) {
|
||||
if (!a.getType().equals(Replica.Type.PULL)) {
|
||||
ri = a;
|
||||
break;
|
||||
|
@ -969,7 +971,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
lt.minimumWaitBetweenActions();
|
||||
lt.markAttemptingAction();
|
||||
synchronized (ri) {
|
||||
ri.getVariables().put(ZkStateReader.LEADER_PROP, "true");
|
||||
ri.getProperties().put(ZkStateReader.LEADER_PROP, "true");
|
||||
}
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug("-- elected new leader for {} / {} (currentVersion={}): {}", collection,
|
||||
|
@ -1106,8 +1108,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
replicaProps.put("SEARCHER.searcher.deletedDocs", new AtomicLong(0));
|
||||
replicaProps.put("SEARCHER.searcher.numDocs", new AtomicLong(0));
|
||||
replicaProps.put("SEARCHER.searcher.maxDoc", new AtomicLong(0));
|
||||
ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, withCollection, 0),
|
||||
coreName, withCollection, withCollectionShard, pos.type, pos.node, replicaProps);
|
||||
Replica ri = new Replica("core_node" + Assign.incAndGetId(stateManager, withCollection, 0),
|
||||
pos.node, withCollection, withCollectionShard, coreName, Replica.State.DOWN,
|
||||
pos.type, replicaProps);
|
||||
cloudManager.submit(() -> {
|
||||
simAddReplica(pos.node, ri, false);
|
||||
// do not count down the latch here
|
||||
|
@ -1129,8 +1132,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
replicaProps.put("SEARCHER.searcher.deletedDocs", new AtomicLong(0));
|
||||
replicaProps.put("SEARCHER.searcher.numDocs", new AtomicLong(0));
|
||||
replicaProps.put("SEARCHER.searcher.maxDoc", new AtomicLong(0));
|
||||
ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
|
||||
coreName, collectionName, pos.shard, pos.type, pos.node, replicaProps);
|
||||
Replica ri = new Replica("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
|
||||
pos.node, collectionName, pos.shard, coreName, Replica.State.DOWN,
|
||||
pos.type, replicaProps);
|
||||
cloudManager.submit(() -> {
|
||||
simAddReplica(pos.node, ri, true);
|
||||
finalStateLatch.countDown();
|
||||
|
@ -1187,8 +1191,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
opDelays.remove(collection);
|
||||
nodeReplicaMap.forEach((n, replicas) -> {
|
||||
synchronized (replicas) {
|
||||
for (Iterator<ReplicaInfo> it = replicas.iterator(); it.hasNext(); ) {
|
||||
ReplicaInfo ri = it.next();
|
||||
for (Iterator<Replica> it = replicas.iterator(); it.hasNext(); ) {
|
||||
Replica ri = it.next();
|
||||
if (ri.getCollection().equals(collection)) {
|
||||
it.remove();
|
||||
// update the number of cores in node values
|
||||
|
@ -1292,11 +1296,11 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
}
|
||||
|
||||
opDelay(collection, CollectionParams.CollectionAction.MOVEREPLICA.name());
|
||||
ReplicaInfo ri = getReplicaInfo(replica);
|
||||
Replica ri = getReplicaInfo(replica);
|
||||
if (ri != null) {
|
||||
if (ri.getVariable(Type.CORE_IDX.tagName) != null) {
|
||||
if (ri.get(Type.CORE_IDX.tagName) != null) {
|
||||
// simulate very large replicas - add additional delay of 5s / GB
|
||||
long sizeInGB = ((Number)ri.getVariable(Type.CORE_IDX.tagName)).longValue();
|
||||
long sizeInGB = ((Number)ri.get(Type.CORE_IDX.tagName)).longValue();
|
||||
long opDelay = opDelays.getOrDefault(ri.getCollection(), Collections.emptyMap())
|
||||
.getOrDefault(CollectionParams.CollectionAction.MOVEREPLICA.name(), defaultOpDelays.get(CollectionParams.CollectionAction.MOVEREPLICA.name()));
|
||||
opDelay = TimeUnit.MILLISECONDS.toSeconds(opDelay);
|
||||
|
@ -1315,7 +1319,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
Map<String, Object> props = replica.getProperties().entrySet().stream()
|
||||
.filter(e -> !NO_COPY_PROPS.contains(e.getKey()))
|
||||
.collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
|
||||
ReplicaInfo newReplica = new ReplicaInfo(coreNodeName, newSolrCoreName, collection, slice.getName(), replica.getType(), targetNode, props);
|
||||
Replica newReplica = new Replica(coreNodeName, targetNode, collection, slice.getName(), newSolrCoreName,
|
||||
Replica.State.DOWN, replica.getType(), props);
|
||||
log.debug("-- new replica: {}", newReplica);
|
||||
// xxx should run leader election here already?
|
||||
simAddReplica(targetNode, newReplica, false);
|
||||
|
@ -1458,7 +1463,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
if (sessionWrapper != null) sessionWrapper.release();
|
||||
|
||||
// adjust numDocs / deletedDocs / maxDoc
|
||||
String numDocsStr = String.valueOf(getReplicaInfo(leader).getVariable("SEARCHER.searcher.numDocs", "0"));
|
||||
String numDocsStr = String.valueOf(getReplicaInfo(leader).get("SEARCHER.searcher.numDocs", "0"));
|
||||
long numDocs = Long.parseLong(numDocsStr);
|
||||
long newNumDocs = numDocs / subSlices.size();
|
||||
long remainderDocs = numDocs % subSlices.size();
|
||||
|
@ -1504,8 +1509,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
replicaProps.put(Type.CORE_IDX.metricsAttribute, new AtomicLong(replicasIndexSize));
|
||||
replicaProps.put(Variable.coreidxsize, new AtomicDouble((Double)Type.CORE_IDX.convertVal(replicasIndexSize)));
|
||||
|
||||
ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
|
||||
solrCoreName, collectionName, replicaPosition.shard, replicaPosition.type, subShardNodeName, replicaProps);
|
||||
Replica ri = new Replica("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
|
||||
subShardNodeName, collectionName, replicaPosition.shard, solrCoreName,
|
||||
Replica.State.DOWN, replicaPosition.type, replicaProps);
|
||||
simAddReplica(replicaPosition.node, ri, false);
|
||||
}
|
||||
simRunLeaderElection(Collections.singleton(collectionName), true);
|
||||
|
@ -1620,9 +1626,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
colShardReplicaMap.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>()).remove(sliceName);
|
||||
nodeReplicaMap.forEach((n, replicas) -> {
|
||||
synchronized (replicas) {
|
||||
Iterator<ReplicaInfo> it = replicas.iterator();
|
||||
Iterator<Replica> it = replicas.iterator();
|
||||
while (it.hasNext()) {
|
||||
ReplicaInfo ri = it.next();
|
||||
Replica ri = it.next();
|
||||
if (ri.getCollection().equals(collectionName) && ri.getShard().equals(sliceName)) {
|
||||
it.remove();
|
||||
}
|
||||
|
@ -1700,8 +1706,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
throw new IOException("-- no leader in " + s);
|
||||
}
|
||||
cloudManager.getMetricManager().registry(createRegistryName(collection, s.getName(), leader)).counter("UPDATE./update.requests").inc();
|
||||
ReplicaInfo ri = getReplicaInfo(leader);
|
||||
Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
|
||||
Replica ri = getReplicaInfo(leader);
|
||||
Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
|
||||
if (numDocs == null || numDocs.intValue() <= 0) {
|
||||
if (log.isDebugEnabled()) {
|
||||
log.debug("-- attempting to delete nonexistent doc {} from {}", id, s.getLeader());
|
||||
|
@ -1727,7 +1733,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
continue;
|
||||
}
|
||||
deletesPerShard.computeIfAbsent(s.getName(), slice -> new AtomicLong(0)).incrementAndGet();
|
||||
Number indexSize = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
|
||||
Number indexSize = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
|
||||
if (indexSize != null) {
|
||||
indexSizePerShard.put(s.getName(), indexSize);
|
||||
}
|
||||
|
@ -1775,14 +1781,14 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
}
|
||||
|
||||
cloudManager.getMetricManager().registry(createRegistryName(collection, s.getName(), leader)).counter("UPDATE./update.requests").inc();
|
||||
ReplicaInfo ri = getReplicaInfo(leader);
|
||||
Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
|
||||
Replica ri = getReplicaInfo(leader);
|
||||
Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
|
||||
if (numDocs == null || numDocs.intValue() == 0) {
|
||||
continue;
|
||||
}
|
||||
lock.lockInterruptibly();
|
||||
try {
|
||||
Number indexSize = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
|
||||
Number indexSize = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
|
||||
if (indexSize != null) {
|
||||
long delta = indexSize.longValue() < SimCloudManager.DEFAULT_IDX_SIZE_BYTES ? 0 :
|
||||
indexSize.longValue() - SimCloudManager.DEFAULT_IDX_SIZE_BYTES;
|
||||
|
@ -1974,8 +1980,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
try {
|
||||
coll.getSlices().forEach(s -> {
|
||||
Replica leader = s.getLeader();
|
||||
ReplicaInfo ri = getReplicaInfo(leader);
|
||||
Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
|
||||
Replica ri = getReplicaInfo(leader);
|
||||
Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
|
||||
if (numDocs == null || numDocs.intValue() == 0) {
|
||||
numDocs = 0;
|
||||
}
|
||||
|
@ -2015,8 +2021,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
if (r == null) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, collection + "/" + s.getName() + " has no leader");
|
||||
}
|
||||
ReplicaInfo ri = getReplicaInfo(r);
|
||||
Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs", 0L);
|
||||
Replica ri = getReplicaInfo(r);
|
||||
Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs", 0L);
|
||||
count.addAndGet(numDocs.longValue());
|
||||
AtomicLong bufferedUpdates = (AtomicLong)sliceProperties.get(collection).get(s.getName()).get(BUFFERED_UPDATES);
|
||||
if (bufferedUpdates != null) {
|
||||
|
@ -2212,7 +2218,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
* divided by the number of replicas.
|
||||
*/
|
||||
public void simSetShardValue(String collection, String shard, String key, Object value, boolean delta, boolean divide) throws Exception {
|
||||
final List<ReplicaInfo> infos;
|
||||
final List<Replica> infos;
|
||||
if (shard == null) {
|
||||
infos = new ArrayList<>();
|
||||
colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>())
|
||||
|
@ -2231,13 +2237,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
value = ((Number) value).doubleValue() / infos.size();
|
||||
}
|
||||
}
|
||||
for (ReplicaInfo r : infos) {
|
||||
for (Replica r : infos) {
|
||||
synchronized (r) {
|
||||
if (value == null) {
|
||||
r.getVariables().remove(key);
|
||||
r.getProperties().remove(key);
|
||||
} else {
|
||||
if (delta) {
|
||||
Object prevValue = r.getVariables().get(key);
|
||||
Object prevValue = r.getProperties().get(key);
|
||||
if (prevValue != null) {
|
||||
if ((prevValue instanceof Number) && (value instanceof Number)) {
|
||||
if (((prevValue instanceof Long) || (prevValue instanceof Integer) ||
|
||||
|
@ -2250,14 +2256,14 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
} else if (prevValue instanceof AtomicInteger) {
|
||||
((AtomicInteger)prevValue).set(((Number)prevValue).intValue() + ((Number)value).intValue());
|
||||
} else {
|
||||
r.getVariables().put(key, newValue);
|
||||
r.getProperties().put(key, newValue);
|
||||
}
|
||||
} else {
|
||||
double newValue = ((Number)prevValue).doubleValue() + ((Number)value).doubleValue();
|
||||
if (prevValue instanceof AtomicDouble) {
|
||||
((AtomicDouble)prevValue).set(newValue);
|
||||
} else {
|
||||
r.getVariables().put(key, newValue);
|
||||
r.getProperties().put(key, newValue);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
|
@ -2265,24 +2271,24 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
}
|
||||
} else {
|
||||
if (value instanceof Integer) {
|
||||
r.getVariables().put(key, new AtomicInteger((Integer)value));
|
||||
r.getProperties().put(key, new AtomicInteger((Integer)value));
|
||||
} else if (value instanceof Long) {
|
||||
r.getVariables().put(key, new AtomicLong((Long)value));
|
||||
r.getProperties().put(key, new AtomicLong((Long)value));
|
||||
} else if (value instanceof Double) {
|
||||
r.getVariables().put(key, new AtomicDouble((Double)value));
|
||||
r.getProperties().put(key, new AtomicDouble((Double)value));
|
||||
} else {
|
||||
r.getVariables().put(key, value);
|
||||
r.getProperties().put(key, value);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (value instanceof Integer) {
|
||||
r.getVariables().put(key, new AtomicInteger((Integer)value));
|
||||
r.getProperties().put(key, new AtomicInteger((Integer)value));
|
||||
} else if (value instanceof Long) {
|
||||
r.getVariables().put(key, new AtomicLong((Long)value));
|
||||
r.getProperties().put(key, new AtomicLong((Long)value));
|
||||
} else if (value instanceof Double) {
|
||||
r.getVariables().put(key, new AtomicDouble((Double)value));
|
||||
r.getProperties().put(key, new AtomicDouble((Double)value));
|
||||
} else {
|
||||
r.getVariables().put(key, value);
|
||||
r.getProperties().put(key, value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2291,26 +2297,26 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public void simSetReplicaValues(String node, Map<String, Map<String, List<ReplicaInfo>>> source, boolean overwrite) {
|
||||
List<ReplicaInfo> infos = nodeReplicaMap.get(node);
|
||||
public void simSetReplicaValues(String node, Map<String, Map<String, List<Replica>>> source, boolean overwrite) {
|
||||
List<Replica> infos = nodeReplicaMap.get(node);
|
||||
if (infos == null) {
|
||||
throw new RuntimeException("Node not present: " + node);
|
||||
}
|
||||
// core_node_name is not unique across collections
|
||||
Map<String, Map<String, ReplicaInfo>> infoMap = new HashMap<>();
|
||||
Map<String, Map<String, Replica>> infoMap = new HashMap<>();
|
||||
infos.forEach(ri -> infoMap.computeIfAbsent(ri.getCollection(), Utils.NEW_HASHMAP_FUN).put(ri.getName(), ri));
|
||||
source.forEach((coll, shards) -> shards.forEach((shard, replicas) -> replicas.forEach(r -> {
|
||||
ReplicaInfo target = infoMap.getOrDefault(coll, Collections.emptyMap()).get(r.getName());
|
||||
Replica target = infoMap.getOrDefault(coll, Collections.emptyMap()).get(r.getName());
|
||||
if (target == null) {
|
||||
throw new RuntimeException("Unable to find simulated replica of " + r);
|
||||
}
|
||||
r.getVariables().forEach((k, v) -> {
|
||||
if (target.getVariables().containsKey(k)) {
|
||||
r.getProperties().forEach((k, v) -> {
|
||||
if (target.getProperties().containsKey(k)) {
|
||||
if (overwrite) {
|
||||
target.getVariables().put(k, v);
|
||||
target.getProperties().put(k, v);
|
||||
}
|
||||
} else {
|
||||
target.getVariables().put(k, v);
|
||||
target.getProperties().put(k, v);
|
||||
}
|
||||
});
|
||||
})));
|
||||
|
@ -2321,29 +2327,29 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
* @param node node id
|
||||
* @return copy of the list of replicas on that node, or empty list if none
|
||||
*/
|
||||
public List<ReplicaInfo> simGetReplicaInfos(String node) {
|
||||
public List<Replica> simGetReplicaInfos(String node) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
final List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent
|
||||
final List<Replica> replicas = nodeReplicaMap.computeIfAbsent
|
||||
(node, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
|
||||
// make a defensive copy to avoid ConcurrentModificationException
|
||||
return Arrays.asList(replicas.toArray(new ReplicaInfo[replicas.size()]));
|
||||
return Arrays.asList(replicas.toArray(new Replica[replicas.size()]));
|
||||
}
|
||||
|
||||
public List<ReplicaInfo> simGetReplicaInfos(String collection, String shard) {
|
||||
List<ReplicaInfo> replicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>())
|
||||
public List<Replica> simGetReplicaInfos(String collection, String shard) {
|
||||
List<Replica> replicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>())
|
||||
.computeIfAbsent(shard, s -> new ArrayList<>());
|
||||
if (replicas == null) {
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
// make a defensive copy to avoid ConcurrentModificationException
|
||||
return Arrays.asList(replicas.toArray(new ReplicaInfo[replicas.size()]));
|
||||
return Arrays.asList(replicas.toArray(new Replica[replicas.size()]));
|
||||
}
|
||||
}
|
||||
|
||||
public ReplicaInfo simGetReplicaInfo(String collection, String coreNode) {
|
||||
Map<String, List<ReplicaInfo>> shardsReplicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>());
|
||||
for (List<ReplicaInfo> replicas : shardsReplicas.values()) {
|
||||
for (ReplicaInfo ri : replicas) {
|
||||
public Replica simGetReplicaInfo(String collection, String coreNode) {
|
||||
Map<String, List<Replica>> shardsReplicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>());
|
||||
for (List<Replica> replicas : shardsReplicas.values()) {
|
||||
for (Replica ri : replicas) {
|
||||
if (ri.getName().equals(coreNode)) {
|
||||
return ri;
|
||||
}
|
||||
|
@ -2388,10 +2394,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
totalReplicas += s.getReplicas().size();
|
||||
if (s.getState() != Slice.State.ACTIVE) {
|
||||
if (!s.getReplicas().isEmpty()) {
|
||||
ReplicaInfo ri = getReplicaInfo(s.getReplicas().iterator().next());
|
||||
Replica ri = getReplicaInfo(s.getReplicas().iterator().next());
|
||||
if (ri != null) {
|
||||
Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
|
||||
Number numBytes = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
|
||||
Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
|
||||
Number numBytes = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
|
||||
if (numDocs != null) {
|
||||
inactiveDocs.addValue(numDocs.doubleValue());
|
||||
}
|
||||
|
@ -2421,7 +2427,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
leader = s.getReplicas().iterator().next();
|
||||
}
|
||||
}
|
||||
ReplicaInfo ri = null;
|
||||
Replica ri = null;
|
||||
if (leader != null) {
|
||||
ri = getReplicaInfo(leader);
|
||||
if (ri == null) {
|
||||
|
@ -2429,9 +2435,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
}
|
||||
}
|
||||
if (ri != null) {
|
||||
Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
|
||||
Number delDocs = (Number)ri.getVariable("SEARCHER.searcher.deleteDocs");
|
||||
Number numBytes = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
|
||||
Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
|
||||
Number delDocs = (Number)ri.get("SEARCHER.searcher.deleteDocs");
|
||||
Number numBytes = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
|
||||
if (numDocs != null) {
|
||||
docs.addValue(numDocs.doubleValue());
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ import java.util.regex.Pattern;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -291,14 +291,14 @@ public class SimNodeStateProvider implements NodeStateProvider {
|
|||
String collection = m.group(1);
|
||||
String shard = m.group(2);
|
||||
String replica = m.group(3);
|
||||
List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(collection, shard);
|
||||
List<Replica> replicas = clusterStateProvider.simGetReplicaInfos(collection, shard);
|
||||
replicas.forEach(r -> {
|
||||
if (r.getNode().equals(node) && r.getCore().endsWith(replica)) {
|
||||
Object value = r.getVariables().get(key);
|
||||
if (r.getNodeName().equals(node) && r.getCoreName().endsWith(replica)) {
|
||||
Object value = r.getProperties().get(key);
|
||||
if (value != null) {
|
||||
values.put(tag, value);
|
||||
} else {
|
||||
value = r.getVariables().get(tag);
|
||||
value = r.getProperties().get(tag);
|
||||
if (value != null) {
|
||||
values.put(tag, value);
|
||||
}
|
||||
|
@ -334,18 +334,18 @@ public class SimNodeStateProvider implements NodeStateProvider {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
List<Replica> replicas = clusterStateProvider.simGetReplicaInfos(node);
|
||||
if (replicas == null || replicas.isEmpty()) {
|
||||
return new HashMap<>();
|
||||
}
|
||||
Map<String, Map<String, List<ReplicaInfo>>> res = new HashMap<>();
|
||||
Map<String, Map<String, List<Replica>>> res = new HashMap<>();
|
||||
// TODO: probably needs special treatment for "metrics:solr.core..." tags
|
||||
for (ReplicaInfo r : replicas) {
|
||||
for (Replica r : replicas) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Map<String, List<ReplicaInfo>> perCollection = res.computeIfAbsent(r.getCollection(), Utils.NEW_HASHMAP_FUN);
|
||||
Map<String, List<Replica>> perCollection = res.computeIfAbsent(r.getCollection(), Utils.NEW_HASHMAP_FUN);
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<ReplicaInfo> perShard = perCollection.computeIfAbsent(r.getShard(), Utils.NEW_ARRAYLIST_FUN);
|
||||
List<Replica> perShard = perCollection.computeIfAbsent(r.getShard(), Utils.NEW_ARRAYLIST_FUN);
|
||||
// XXX filter out some properties?
|
||||
perShard.add(r);
|
||||
}
|
||||
|
|
|
@ -49,7 +49,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
|||
import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
|
||||
|
@ -66,6 +65,7 @@ import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
|
|||
import org.apache.solr.cloud.autoscaling.TriggerEvent;
|
||||
import org.apache.solr.cloud.autoscaling.TriggerListener;
|
||||
import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.params.AutoScalingParams;
|
||||
import org.apache.solr.common.params.CollectionAdminParams;
|
||||
import org.apache.solr.common.params.CollectionParams;
|
||||
|
@ -561,14 +561,14 @@ public class SimScenario implements AutoCloseable {
|
|||
}
|
||||
Map<String, Object> paramsMap = new LinkedHashMap<>();
|
||||
params.toMap(paramsMap);
|
||||
ReplicaInfo info = scenario.cluster.getSimClusterStateProvider().simGetReplicaInfo(
|
||||
Replica info = scenario.cluster.getSimClusterStateProvider().simGetReplicaInfo(
|
||||
params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
|
||||
if (info == null) {
|
||||
log.error("Could not find ReplicaInfo for params: {}", params);
|
||||
} else if (scenario.verbose) {
|
||||
paramsMap.put("replicaInfo", info);
|
||||
} else if (info.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
paramsMap.put(Variable.Type.CORE_IDX.tagName, info.getVariable(Variable.Type.CORE_IDX.tagName));
|
||||
} else if (info.get(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
paramsMap.put(Variable.Type.CORE_IDX.tagName, info.get(Variable.Type.CORE_IDX.tagName));
|
||||
}
|
||||
try {
|
||||
scenario.cluster.request(operation);
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
|||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Cell;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Row;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
|
||||
import org.apache.solr.client.solrj.request.CollectionApiMapping;
|
||||
|
@ -110,9 +109,9 @@ public class SimUtils {
|
|||
}
|
||||
});
|
||||
});
|
||||
Map<String, Map<String, ReplicaInfo>> allReplicaInfos = new HashMap<>();
|
||||
Map<String, Map<String, Replica>> allReplicaInfos = new HashMap<>();
|
||||
solrCloudManager.getClusterStateProvider().getLiveNodes().forEach(n -> {
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = solrCloudManager.getNodeStateProvider().getReplicaInfo(n, replicaTags);
|
||||
Map<String, Map<String, List<Replica>>> infos = solrCloudManager.getNodeStateProvider().getReplicaInfo(n, replicaTags);
|
||||
infos.forEach((coll, shards) -> shards.forEach((shard, replicas) -> replicas.forEach(r -> {
|
||||
if (allReplicaInfos.containsKey(r.getName())) {
|
||||
throw new RuntimeException("duplicate core_node name in NodeStateProvider: " + allReplicaInfos.get(r.getName()) + " versus " + r);
|
||||
|
@ -152,9 +151,9 @@ public class SimUtils {
|
|||
});
|
||||
// verify all replicas have size info
|
||||
allReplicaInfos.forEach((coll, replicas) -> replicas.forEach((core, ri) -> {
|
||||
Number size = (Number) ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute);
|
||||
Number size = (Number) ri.get(Variable.Type.CORE_IDX.metricsAttribute);
|
||||
if (size == null) {
|
||||
size = (Number) ri.getVariable(Variable.Type.CORE_IDX.tagName);
|
||||
size = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
|
||||
if (size == null) {
|
||||
// for (String node : solrCloudManager.getClusterStateProvider().getLiveNodes()) {
|
||||
// log.error("Check for missing values: {}: {}", node, solrCloudManager.getNodeStateProvider().getReplicaInfo(node, SnapshotNodeStateProvider.REPLICA_TAGS));
|
||||
|
@ -261,25 +260,25 @@ public class SimUtils {
|
|||
}
|
||||
row.forEachReplica(ri -> {
|
||||
Map<String, Object> perReplica = collReplicas.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>())
|
||||
.computeIfAbsent(ri.getCore().substring(ri.getCollection().length() + 1), core -> new LinkedHashMap<>());
|
||||
.computeIfAbsent(ri.getCoreName().substring(ri.getCollection().length() + 1), core -> new LinkedHashMap<>());
|
||||
// if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
// perReplica.put(Variable.Type.CORE_IDX.tagName, ri.getVariable(Variable.Type.CORE_IDX.tagName));
|
||||
// }
|
||||
if (ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute) != null) {
|
||||
perReplica.put(Variable.Type.CORE_IDX.metricsAttribute, ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute));
|
||||
if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
perReplica.put(Variable.Type.CORE_IDX.tagName, ri.getVariable(Variable.Type.CORE_IDX.tagName));
|
||||
if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) != null) {
|
||||
perReplica.put(Variable.Type.CORE_IDX.metricsAttribute, ri.get(Variable.Type.CORE_IDX.metricsAttribute));
|
||||
if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
perReplica.put(Variable.Type.CORE_IDX.tagName, ri.get(Variable.Type.CORE_IDX.tagName));
|
||||
} else {
|
||||
perReplica.put(Variable.Type.CORE_IDX.tagName,
|
||||
Variable.Type.CORE_IDX.convertVal(ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute)));
|
||||
Variable.Type.CORE_IDX.convertVal(ri.get(Variable.Type.CORE_IDX.metricsAttribute)));
|
||||
}
|
||||
}
|
||||
perReplica.put("coreNode", ri.getName());
|
||||
if (ri.isLeader || ri.getBool("leader", false)) {
|
||||
if (ri.isLeader() || ri.getBool("leader", false)) {
|
||||
perReplica.put("leader", true);
|
||||
Double totalSize = (Double)collStats.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
|
||||
.computeIfAbsent("avgShardSize", size -> 0.0);
|
||||
Number riSize = (Number)ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute);
|
||||
Number riSize = (Number)ri.get(Variable.Type.CORE_IDX.metricsAttribute);
|
||||
if (riSize != null) {
|
||||
totalSize += riSize.doubleValue();
|
||||
collStats.get(ri.getCollection()).put("avgShardSize", totalSize);
|
||||
|
|
|
@ -42,11 +42,11 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
|||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
|
||||
import org.apache.solr.client.solrj.impl.ClusterStateProvider;
|
||||
import org.apache.solr.client.solrj.request.V2Request;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.params.CollectionAdminParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ObjectCache;
|
||||
|
@ -193,7 +193,7 @@ public class SnapshotCloudManager implements SolrCloudManager {
|
|||
if (s.getOperation() instanceof V2Request) {
|
||||
params = SimUtils.v2AdminRequestToV1Params((V2Request)s.getOperation());
|
||||
}
|
||||
ReplicaInfo info = nodeStateProvider.getReplicaInfo(
|
||||
Replica info = nodeStateProvider.getReplicaInfo(
|
||||
params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
|
||||
if (info == null) {
|
||||
log.warn("Can't find ReplicaInfo for suggested operation: {}", s);
|
||||
|
|
|
@ -30,15 +30,15 @@ import java.util.Set;
|
|||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
|
||||
/**
|
||||
* Read-only snapshot of another {@link NodeStateProvider}.
|
||||
*/
|
||||
public class SnapshotNodeStateProvider implements NodeStateProvider {
|
||||
private Map<String, Map<String, Object>> nodeValues = new LinkedHashMap<>();
|
||||
private Map<String, Map<String, Map<String, List<ReplicaInfo>>>> replicaInfos = new LinkedHashMap<>();
|
||||
private Map<String, Map<String, Map<String, List<Replica>>>> replicaInfos = new LinkedHashMap<>();
|
||||
|
||||
private static double GB = 1024.0d * 1024.0d * 1024.0d;
|
||||
|
||||
|
@ -59,29 +59,29 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
|
|||
replicaTags.addAll(config.getPolicy().getPerReplicaAttributes());
|
||||
for (String node : other.getClusterStateProvider().getLiveNodes()) {
|
||||
nodeValues.put(node, new LinkedHashMap<>(other.getNodeStateProvider().getNodeValues(node, nodeTags)));
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
|
||||
Map<String, Map<String, List<Replica>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
|
||||
infos.forEach((collection, shards) -> {
|
||||
shards.forEach((shard, replicas) -> {
|
||||
replicas.forEach(r -> {
|
||||
List<ReplicaInfo> myReplicas = replicaInfos
|
||||
List<Replica> myReplicas = replicaInfos
|
||||
.computeIfAbsent(node, n -> new LinkedHashMap<>())
|
||||
.computeIfAbsent(collection, c -> new LinkedHashMap<>())
|
||||
.computeIfAbsent(shard, s -> new ArrayList<>());
|
||||
Map<String, Object> rMap = new LinkedHashMap<>();
|
||||
r.toMap(rMap);
|
||||
if (r.isLeader) { // ReplicaInfo.toMap doesn't write this!!!
|
||||
if (r.isLeader()) { // ReplicaInfo.toMap doesn't write this!!!
|
||||
((Map<String, Object>)rMap.values().iterator().next()).put("leader", "true");
|
||||
}
|
||||
ReplicaInfo ri = new ReplicaInfo(rMap);
|
||||
Replica ri = new Replica(rMap);
|
||||
// put in "leader" again if present
|
||||
if (r.isLeader) {
|
||||
ri.getVariables().put("leader", "true");
|
||||
if (r.isLeader()) {
|
||||
ri.getProperties().put("leader", "true");
|
||||
}
|
||||
// externally produced snapshots may not include the right units
|
||||
if (ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute) == null) {
|
||||
if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
Number indexSizeGB = (Number) ri.getVariable(Variable.Type.CORE_IDX.tagName);
|
||||
ri.getVariables().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
|
||||
if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) == null) {
|
||||
if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
Number indexSizeGB = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
|
||||
ri.getProperties().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
|
||||
} else {
|
||||
throw new RuntimeException("Missing size information for replica: " + ri);
|
||||
}
|
||||
|
@ -102,21 +102,21 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
|
|||
Objects.requireNonNull(snapshot);
|
||||
nodeValues = (Map<String, Map<String, Object>>)snapshot.getOrDefault("nodeValues", Collections.emptyMap());
|
||||
((Map<String, Object>)snapshot.getOrDefault("replicaInfos", Collections.emptyMap())).forEach((node, v) -> {
|
||||
Map<String, Map<String, List<ReplicaInfo>>> perNode = replicaInfos.computeIfAbsent(node, n -> new LinkedHashMap<>());
|
||||
Map<String, Map<String, List<Replica>>> perNode = replicaInfos.computeIfAbsent(node, n -> new LinkedHashMap<>());
|
||||
((Map<String, Object>)v).forEach((collection, shards) -> {
|
||||
Map<String, List<ReplicaInfo>> perColl = perNode.computeIfAbsent(collection, c -> new LinkedHashMap<>());
|
||||
Map<String, List<Replica>> perColl = perNode.computeIfAbsent(collection, c -> new LinkedHashMap<>());
|
||||
((Map<String, Object>)shards).forEach((shard, replicas) -> {
|
||||
List<ReplicaInfo> infos = perColl.computeIfAbsent(shard, s -> new ArrayList<>());
|
||||
List<Replica> infos = perColl.computeIfAbsent(shard, s -> new ArrayList<>());
|
||||
((List<Map<String, Object>>)replicas).forEach(replicaMap -> {
|
||||
ReplicaInfo ri = new ReplicaInfo(new LinkedHashMap<>(replicaMap)); // constructor modifies this map
|
||||
if (ri.isLeader) {
|
||||
ri.getVariables().put("leader", "true");
|
||||
Replica ri = new Replica(new LinkedHashMap<>(replicaMap)); // constructor modifies this map
|
||||
if (ri.isLeader()) {
|
||||
ri.getProperties().put("leader", "true");
|
||||
}
|
||||
// externally produced snapshots may not include the right units
|
||||
if (ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute) == null) {
|
||||
if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
Number indexSizeGB = (Number) ri.getVariable(Variable.Type.CORE_IDX.tagName);
|
||||
ri.getVariables().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
|
||||
if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) == null) {
|
||||
if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
Number indexSizeGB = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
|
||||
ri.getProperties().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
|
||||
} else {
|
||||
throw new RuntimeException("Missing size information for replica: " + ri);
|
||||
}
|
||||
|
@ -148,7 +148,7 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
|
|||
.computeIfAbsent(shard, s -> new ArrayList<>());
|
||||
Map<String, Object> rMap = new LinkedHashMap<>();
|
||||
r.toMap(rMap);
|
||||
if (r.isLeader) { // ReplicaInfo.toMap doesn't write this!!!
|
||||
if (r.isLeader()) { // ReplicaInfo.toMap doesn't write this!!!
|
||||
((Map<String, Object>)rMap.values().iterator().next()).put("leader", "true");
|
||||
}
|
||||
myReplicas.add(rMap);
|
||||
|
@ -165,17 +165,17 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
Map<String, Map<String, List<ReplicaInfo>>> result = new LinkedHashMap<>();
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = replicaInfos.getOrDefault(node, Collections.emptyMap());
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
Map<String, Map<String, List<Replica>>> result = new LinkedHashMap<>();
|
||||
Map<String, Map<String, List<Replica>>> infos = replicaInfos.getOrDefault(node, Collections.emptyMap());
|
||||
// deep copy
|
||||
infos.forEach((coll, shards) -> {
|
||||
shards.forEach((shard, replicas) -> {
|
||||
replicas.forEach(ri -> {
|
||||
List<ReplicaInfo> myReplicas = result
|
||||
List<Replica> myReplicas = result
|
||||
.computeIfAbsent(coll, c -> new LinkedHashMap<>())
|
||||
.computeIfAbsent(shard, s -> new ArrayList<>());
|
||||
ReplicaInfo myReplica = (ReplicaInfo)ri.clone();
|
||||
Replica myReplica = (Replica)ri.clone();
|
||||
myReplicas.add(myReplica);
|
||||
});
|
||||
});
|
||||
|
@ -183,12 +183,12 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
|
|||
return result;
|
||||
}
|
||||
|
||||
public ReplicaInfo getReplicaInfo(String collection, String coreNode) {
|
||||
for (Map<String, Map<String, List<ReplicaInfo>>> perNode : replicaInfos.values()) {
|
||||
for (List<ReplicaInfo> perShard : perNode.getOrDefault(collection, Collections.emptyMap()).values()) {
|
||||
for (ReplicaInfo ri : perShard) {
|
||||
public Replica getReplicaInfo(String collection, String coreNode) {
|
||||
for (Map<String, Map<String, List<Replica>>> perNode : replicaInfos.values()) {
|
||||
for (List<Replica> perShard : perNode.getOrDefault(collection, Collections.emptyMap()).values()) {
|
||||
for (Replica ri : perShard) {
|
||||
if (ri.getName().equals(coreNode)) {
|
||||
return (ReplicaInfo)ri.clone();
|
||||
return (Replica)ri.clone();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,8 +64,8 @@ public class NodeMutator {
|
|||
if (rNodeName.equals(nodeName)) {
|
||||
log.debug("Update replica state for {} to {}", replica, Replica.State.DOWN);
|
||||
Map<String, Object> props = replica.shallowCopy();
|
||||
props.put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
|
||||
Replica newReplica = new Replica(replica.getName(), props, collection, slice.getName());
|
||||
Replica newReplica = new Replica(replica.getName(), replica.node, replica.collection, slice.getName(), replica.core,
|
||||
Replica.State.DOWN, replica.type, props);
|
||||
newReplicas.put(replica.getName(), newReplica);
|
||||
needToUpdateCollection = true;
|
||||
}
|
||||
|
|
|
@ -72,7 +72,7 @@ public class ReplicaMutator {
|
|||
|
||||
Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
|
||||
replicaProps.put(key, value);
|
||||
return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice());
|
||||
return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getShard());
|
||||
}
|
||||
|
||||
protected Replica unsetProperty(Replica replica, String key) {
|
||||
|
@ -81,7 +81,7 @@ public class ReplicaMutator {
|
|||
if (!replica.containsKey(key)) return replica;
|
||||
Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
|
||||
replicaProps.remove(key);
|
||||
return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice());
|
||||
return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getShard());
|
||||
}
|
||||
|
||||
protected Replica setLeader(Replica replica) {
|
||||
|
|
|
@ -60,7 +60,6 @@ import org.apache.solr.client.solrj.SolrQuery;
|
|||
import org.apache.solr.client.solrj.SolrRequest;
|
||||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
|
||||
import org.apache.solr.client.solrj.impl.HttpClientUtil;
|
||||
|
@ -488,7 +487,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
|
|||
return;
|
||||
}
|
||||
// add core-level stats
|
||||
Map<String, Map<String, List<ReplicaInfo>>> infos = nodeStateProvider.getReplicaInfo(node, collTags);
|
||||
Map<String, Map<String, List<Replica>>> infos = nodeStateProvider.getReplicaInfo(node, collTags);
|
||||
infos.forEach((coll, shards) -> {
|
||||
shards.forEach((sh, replicas) -> {
|
||||
String registry = SolrMetricManager.getRegistryName(Group.collection, coll);
|
||||
|
@ -497,7 +496,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
|
|||
.computeIfAbsent(registry, r -> new HashMap<>());
|
||||
replicas.forEach(ri -> {
|
||||
collTags.forEach(tag -> {
|
||||
double value = ((Number)ri.getVariable(tag, 0.0)).doubleValue();
|
||||
double value = ((Number)ri.get(tag, 0.0)).doubleValue();
|
||||
DoubleAdder adder = (DoubleAdder)perReg.computeIfAbsent(tag, t -> new DoubleAdder());
|
||||
adder.add(value);
|
||||
});
|
||||
|
|
|
@ -100,7 +100,6 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
|||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
|
@ -1171,14 +1170,14 @@ public class SolrCLI implements CLIO {
|
|||
}
|
||||
Map<String, Object> paramsMap = new LinkedHashMap<>();
|
||||
params.toMap(paramsMap);
|
||||
ReplicaInfo info = simCloudManager.getSimClusterStateProvider().simGetReplicaInfo(
|
||||
Replica info = simCloudManager.getSimClusterStateProvider().simGetReplicaInfo(
|
||||
params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
|
||||
if (info == null) {
|
||||
CLIO.err("Could not find ReplicaInfo for params: " + params);
|
||||
} else if (verbose) {
|
||||
paramsMap.put("replicaInfo", info);
|
||||
} else if (info.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
paramsMap.put(Variable.Type.CORE_IDX.tagName, info.getVariable(Variable.Type.CORE_IDX.tagName));
|
||||
} else if (info.get(Variable.Type.CORE_IDX.tagName) != null) {
|
||||
paramsMap.put(Variable.Type.CORE_IDX.tagName, info.get(Variable.Type.CORE_IDX.tagName));
|
||||
}
|
||||
if (withSuggestions) {
|
||||
perStepOps.add(paramsMap);
|
||||
|
|
|
@ -232,7 +232,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
|
|||
boolean firstReplica = true;
|
||||
for (Replica replica : shard) {
|
||||
coresCompared++;
|
||||
assertEquals(shard.getName(), replica.getSlice()); // sanity check
|
||||
assertEquals(shard.getName(), replica.getShard()); // sanity check
|
||||
final String core = replica.getCoreName();
|
||||
final ReplicaData initData = initReplicaData.get(core);
|
||||
final ReplicaData finalData = finalReplicaData.get(core);
|
||||
|
@ -301,7 +301,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
|
|||
"rows", "0",
|
||||
"_trace", "counting_docs"))).process(client).getResults().getNumFound();
|
||||
|
||||
final ReplicaData data = new ReplicaData(replica.getSlice(),coreName,(Long)version,numDocs);
|
||||
final ReplicaData data = new ReplicaData(replica.getShard(),coreName,(Long)version,numDocs);
|
||||
log.info("{}", data);
|
||||
results.put(coreName, data);
|
||||
|
||||
|
|
|
@ -582,7 +582,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
|
|||
}
|
||||
|
||||
protected int getReplicaPort(Replica replica) {
|
||||
String replicaNode = replica.getNodeName();
|
||||
String replicaNode = replica.getNodeName();
|
||||
String tmp = replicaNode.substring(replicaNode.indexOf(':')+1);
|
||||
if (tmp.indexOf('_') != -1)
|
||||
tmp = tmp.substring(0,tmp.indexOf('_'));
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.solr.client.solrj.SolrRequest;
|
||||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
|
@ -540,7 +539,7 @@ public class ComputePlanActionTest extends SolrCloudTestCase {
|
|||
NodeStateProvider stateProvider = cloudManager.getNodeStateProvider();
|
||||
List<String> nodes = new ArrayList<>();
|
||||
cloudManager.getClusterStateProvider().getLiveNodes().forEach(n -> {
|
||||
Map<String, Map<String, List<ReplicaInfo>>> map = stateProvider.getReplicaInfo(n, ImplicitSnitch.tags);
|
||||
Map<String, Map<String, List<Replica>>> map = stateProvider.getReplicaInfo(n, ImplicitSnitch.tags);
|
||||
if (map.containsKey("testSelected3") && map.containsKey("testSelected2") && map.containsKey("testSelected1")) {
|
||||
nodes.add(n);
|
||||
}
|
||||
|
|
|
@ -31,7 +31,6 @@ import com.google.common.util.concurrent.AtomicDouble;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
|
@ -243,13 +242,13 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
|
|||
assertNotNull("nodeRates", nodeRates);
|
||||
// no node violations because node rates weren't set in the config
|
||||
assertTrue(nodeRates.toString(), nodeRates.isEmpty());
|
||||
List<ReplicaInfo> replicaRates = (List<ReplicaInfo>) ev.event.getProperties().get(SearchRateTrigger.HOT_REPLICAS);
|
||||
List<Replica> replicaRates = (List<Replica>) ev.event.getProperties().get(SearchRateTrigger.HOT_REPLICAS);
|
||||
assertNotNull("replicaRates", replicaRates);
|
||||
assertTrue(replicaRates.toString(), replicaRates.size() > 0);
|
||||
AtomicDouble totalReplicaRate = new AtomicDouble();
|
||||
replicaRates.forEach(r -> {
|
||||
assertTrue(r.toString(), r.getVariable("rate") != null);
|
||||
totalReplicaRate.addAndGet((Double) r.getVariable("rate"));
|
||||
assertTrue(r.toString(), r.get("rate") != null);
|
||||
totalReplicaRate.addAndGet((Double) r.get("rate"));
|
||||
});
|
||||
Map<String, Object> shardRates = (Map<String, Object>) ev.event.getProperties().get(SearchRateTrigger.HOT_SHARDS);
|
||||
assertNotNull("shardRates", shardRates);
|
||||
|
|
|
@ -30,7 +30,6 @@ import com.codahale.metrics.MetricRegistry;
|
|||
import com.google.common.util.concurrent.AtomicDouble;
|
||||
|
||||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
|
@ -42,6 +41,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
|||
import org.apache.solr.cloud.CloudUtil;
|
||||
import org.apache.solr.cloud.SolrCloudTestCase;
|
||||
import org.apache.solr.cloud.ZkDistributedQueueFactory;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.params.AutoScalingParams;
|
||||
import org.apache.solr.common.params.CollectionParams;
|
||||
|
@ -145,11 +145,11 @@ public class SearchRateTriggerTest extends SolrCloudTestCase {
|
|||
assertEquals(1, events.size());
|
||||
TriggerEvent event = events.get(0);
|
||||
assertEquals(TriggerEventType.SEARCHRATE, event.eventType);
|
||||
List<ReplicaInfo> infos = (List<ReplicaInfo>)event.getProperty(SearchRateTrigger.HOT_REPLICAS);
|
||||
List<Replica> infos = (List<Replica>)event.getProperty(SearchRateTrigger.HOT_REPLICAS);
|
||||
assertEquals(1, infos.size());
|
||||
ReplicaInfo info = infos.get(0);
|
||||
assertEquals(coreName, info.getCore());
|
||||
assertTrue((Double)info.getVariable(AutoScalingParams.RATE) > rate);
|
||||
Replica info = infos.get(0);
|
||||
assertEquals(coreName, info.getCoreName());
|
||||
assertTrue((Double)info.get(AutoScalingParams.RATE) > rate);
|
||||
}
|
||||
// close that jetty to remove the violation - alternatively wait for 1 min...
|
||||
JettySolrRunner j = cluster.stopJettySolrRunner(1);
|
||||
|
@ -263,11 +263,11 @@ public class SearchRateTriggerTest extends SolrCloudTestCase {
|
|||
TriggerEvent event = events.get(0);
|
||||
assertEquals(event.toString(), TriggerEventType.SEARCHRATE, event.eventType);
|
||||
Map<String, Object> hotNodes, hotCollections, hotShards;
|
||||
List<ReplicaInfo> hotReplicas;
|
||||
List<Replica> hotReplicas;
|
||||
hotNodes = (Map<String, Object>)event.properties.get(SearchRateTrigger.HOT_NODES);
|
||||
hotCollections = (Map<String, Object>)event.properties.get(SearchRateTrigger.HOT_COLLECTIONS);
|
||||
hotShards = (Map<String, Object>)event.properties.get(SearchRateTrigger.HOT_SHARDS);
|
||||
hotReplicas = (List<ReplicaInfo>)event.properties.get(SearchRateTrigger.HOT_REPLICAS);
|
||||
hotReplicas = (List<Replica>)event.properties.get(SearchRateTrigger.HOT_REPLICAS);
|
||||
assertTrue("no hot nodes?", hotNodes.isEmpty());
|
||||
assertFalse("no hot collections?", hotCollections.isEmpty());
|
||||
assertFalse("no hot shards?", hotShards.isEmpty());
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
|
|||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Row;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
|
@ -157,9 +156,9 @@ public class TestPolicyCloud extends SolrCloudTestCase {
|
|||
try (SolrCloudManager cloudManager = new SolrClientCloudManager(new ZkDistributedQueueFactory(cluster.getZkClient()), cluster.getSolrClient())) {
|
||||
String nodeName = cloudManager.getClusterStateProvider().getLiveNodes().iterator().next();
|
||||
SolrClientNodeStateProvider nodeStateProvider = (SolrClientNodeStateProvider) cloudManager.getNodeStateProvider();
|
||||
Map<String, Map<String, List<ReplicaInfo>>> result = nodeStateProvider.getReplicaInfo(nodeName, Collections.singleton("UPDATE./update.requests"));
|
||||
Map<String, Map<String, List<Replica>>> result = nodeStateProvider.getReplicaInfo(nodeName, Collections.singleton("UPDATE./update.requests"));
|
||||
nodeStateProvider.forEachReplica(nodeName, replicaInfo -> {
|
||||
if (replicaInfo.getVariables().containsKey("UPDATE./update.requests")) count.incrementAndGet();
|
||||
if (replicaInfo.getProperties().containsKey("UPDATE./update.requests")) count.incrementAndGet();
|
||||
});
|
||||
assertTrue(count.get() > 0);
|
||||
|
||||
|
@ -177,8 +176,8 @@ public class TestPolicyCloud extends SolrCloudTestCase {
|
|||
count .set(0);
|
||||
for (Row row : session.getSortedNodes()) {
|
||||
row.collectionVsShardVsReplicas.forEach((c, shardVsReplicas) -> shardVsReplicas.forEach((s, replicaInfos) -> {
|
||||
for (ReplicaInfo replicaInfo : replicaInfos) {
|
||||
if (replicaInfo.getVariables().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
|
||||
for (Replica replicaInfo : replicaInfos) {
|
||||
if (replicaInfo.getProperties().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
|
|
@ -33,7 +33,6 @@ import java.util.stream.Collectors;
|
|||
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.cloud.CloudTestUtils;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
|
@ -226,15 +225,15 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public static void assertReplicaInfoEquals(ReplicaInfo one, ReplicaInfo two) {
|
||||
public static void assertReplicaInfoEquals(Replica one, Replica two) {
|
||||
assertEquals(one.getName(), two.getName());
|
||||
assertEquals(one.getNode(), two.getNode());
|
||||
assertEquals(one.getNodeName(), two.getNodeName());
|
||||
assertEquals(one.getState(), two.getState());
|
||||
assertEquals(one.getType(), two.getType());
|
||||
assertEquals(one.getCore(), two.getCore());
|
||||
assertEquals(one.getCoreName(), two.getCoreName());
|
||||
assertEquals(one.getCollection(), two.getCollection());
|
||||
assertEquals(one.getShard(), two.getShard());
|
||||
assertEquals(one.isLeader, two.isLeader);
|
||||
assertEquals(one.isLeader(), two.isLeader());
|
||||
Map<String, Object> oneMap = new HashMap<>();
|
||||
Map<String, Object> twoMap = new HashMap<>();
|
||||
one.toMap(oneMap);
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.util.concurrent.TimeUnit;
|
|||
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Preference;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
|
@ -64,7 +63,7 @@ public class TestSimClusterStateProvider extends SolrCloudTestCase {
|
|||
private static Collection<String> liveNodes;
|
||||
private static Map<String, Object> clusterProperties;
|
||||
private static AutoScalingConfig autoScalingConfig;
|
||||
private static Map<String, Map<String, Map<String, List<ReplicaInfo>>>> replicas;
|
||||
private static Map<String, Map<String, Map<String, List<Replica>>>> replicas;
|
||||
private static Map<String, Map<String, Object>> nodeValues;
|
||||
private static ClusterState realState;
|
||||
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.solr.client.solrj.SolrRequest;
|
|||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
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.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Row;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
|
@ -105,8 +104,8 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
|
|||
AtomicInteger count = new AtomicInteger(0);
|
||||
for (Row row : session.getSortedNodes()) {
|
||||
row.collectionVsShardVsReplicas.forEach((c, shardVsReplicas) -> shardVsReplicas.forEach((s, replicaInfos) -> {
|
||||
for (ReplicaInfo replicaInfo : replicaInfos) {
|
||||
if (replicaInfo.getVariables().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
|
||||
for (Replica replicaInfo : replicaInfos) {
|
||||
if (replicaInfo.getProperties().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
|
|
@ -38,7 +38,6 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
import org.apache.solr.client.solrj.SolrClient;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
|
@ -61,6 +60,7 @@ import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
|
|||
import org.apache.solr.cloud.autoscaling.TriggerValidationException;
|
||||
import org.apache.solr.common.MapWriter;
|
||||
import org.apache.solr.common.cloud.LiveNodesListener;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.util.TimeSource;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
|
@ -1467,13 +1467,13 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
|
|||
assertTrue(nodeRates.toString(), nodeRates.size() > 0);
|
||||
AtomicDouble totalNodeRate = new AtomicDouble();
|
||||
nodeRates.forEach((n, r) -> totalNodeRate.addAndGet(r));
|
||||
List<ReplicaInfo> replicaRates = (List<ReplicaInfo>)ev.event.getProperties().get(SearchRateTrigger.HOT_REPLICAS);
|
||||
List<Replica> replicaRates = (List<Replica>)ev.event.getProperties().get(SearchRateTrigger.HOT_REPLICAS);
|
||||
assertNotNull("replicaRates", replicaRates);
|
||||
assertTrue(replicaRates.toString(), replicaRates.size() > 0);
|
||||
AtomicDouble totalReplicaRate = new AtomicDouble();
|
||||
replicaRates.forEach(r -> {
|
||||
assertTrue(r.toString(), r.getVariable("rate") != null);
|
||||
totalReplicaRate.addAndGet((Double)r.getVariable("rate"));
|
||||
assertTrue(r.toString(), r.get("rate") != null);
|
||||
totalReplicaRate.addAndGet((Double)r.get("rate"));
|
||||
});
|
||||
Map<String, Object> shardRates = (Map<String, Object>)ev.event.getProperties().get(SearchRateTrigger.HOT_SHARDS);
|
||||
assertNotNull("shardRates", shardRates);
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.solr.client.solrj.cloud.DistribStateManager;
|
|||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggestion;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
|
||||
|
@ -202,22 +201,22 @@ public class TestSnapshotCloudManager extends SolrCloudTestCase {
|
|||
}
|
||||
}
|
||||
assertEquals(Utils.toJSONString(oneVals), Utils.toJSONString(twoVals));
|
||||
Map<String, Map<String, List<ReplicaInfo>>> oneInfos = one.getReplicaInfo(node, SimUtils.COMMON_REPLICA_TAGS);
|
||||
Map<String, Map<String, List<ReplicaInfo>>> twoInfos = two.getReplicaInfo(node, SimUtils.COMMON_REPLICA_TAGS);
|
||||
Map<String, Map<String, List<Replica>>> oneInfos = one.getReplicaInfo(node, SimUtils.COMMON_REPLICA_TAGS);
|
||||
Map<String, Map<String, List<Replica>>> twoInfos = two.getReplicaInfo(node, SimUtils.COMMON_REPLICA_TAGS);
|
||||
assertEquals("collections on node" + node, oneInfos.keySet(), twoInfos.keySet());
|
||||
oneInfos.forEach((coll, oneShards) -> {
|
||||
Map<String, List<ReplicaInfo>> twoShards = twoInfos.get(coll);
|
||||
Map<String, List<Replica>> twoShards = twoInfos.get(coll);
|
||||
assertEquals("shards on node " + node, oneShards.keySet(), twoShards.keySet());
|
||||
oneShards.forEach((shard, oneReplicas) -> {
|
||||
List<ReplicaInfo> twoReplicas = twoShards.get(shard);
|
||||
List<Replica> twoReplicas = twoShards.get(shard);
|
||||
assertEquals("num replicas on node " + node, oneReplicas.size(), twoReplicas.size());
|
||||
Map<String, ReplicaInfo> oneMap = oneReplicas.stream()
|
||||
.collect(Collectors.toMap(ReplicaInfo::getName, Function.identity()));
|
||||
Map<String, ReplicaInfo> twoMap = twoReplicas.stream()
|
||||
.collect(Collectors.toMap(ReplicaInfo::getName, Function.identity()));
|
||||
Map<String, Replica> oneMap = oneReplicas.stream()
|
||||
.collect(Collectors.toMap(Replica::getName, Function.identity()));
|
||||
Map<String, Replica> twoMap = twoReplicas.stream()
|
||||
.collect(Collectors.toMap(Replica::getName, Function.identity()));
|
||||
assertEquals("replica coreNodeNames on node " + node, oneMap.keySet(), twoMap.keySet());
|
||||
oneMap.forEach((coreNode, oneReplica) -> {
|
||||
ReplicaInfo twoReplica = twoMap.get(coreNode);
|
||||
Replica twoReplica = twoMap.get(coreNode);
|
||||
SimSolrCloudTestCase.assertReplicaInfoEquals(oneReplica, twoReplica);
|
||||
});
|
||||
});
|
||||
|
|
|
@ -31,8 +31,8 @@ import com.google.common.collect.ImmutableList;
|
|||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.DelegatingCloudManager;
|
||||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.ReplicaPosition;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.cloud.rule.Snitch;
|
||||
|
@ -212,7 +212,7 @@ public class RuleEngineTest extends SolrTestCaseJ4{
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -185,7 +185,7 @@ public class CoreSorterTest extends SolrTestCaseJ4 {
|
|||
private CoreDescriptor newCoreDescriptor(Replica r) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Map<String,String> props = map(
|
||||
CoreDescriptor.CORE_SHARD, r.getSlice(),
|
||||
CoreDescriptor.CORE_SHARD, r.getShard(),
|
||||
CoreDescriptor.CORE_COLLECTION, r.getCollection(),
|
||||
CoreDescriptor.CORE_NODE_NAME, r.getNodeName()
|
||||
);
|
||||
|
|
|
@ -20,8 +20,8 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.common.SolrCloseable;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
|
||||
/**
|
||||
* This interface models the access to node and replica information.
|
||||
|
@ -40,6 +40,7 @@ public interface NodeStateProvider extends SolrCloseable {
|
|||
* 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
|
||||
* <p>The format is {collection:shard :[{replicadetails}]}.</p>
|
||||
* @return map of replica infos per collection/shard
|
||||
*/
|
||||
Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys);
|
||||
Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys);
|
||||
}
|
||||
|
|
|
@ -700,7 +700,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
|
|||
}
|
||||
|
||||
|
||||
public boolean isMatch(ReplicaInfo r, String collection, String shard) {
|
||||
public boolean isMatch(Replica r, String collection, String shard) {
|
||||
if (type != null && r.getType() != type) return false;
|
||||
if (r.getCollection().equals(collection)) {
|
||||
if (this.shard == null || this.shard.val.equals(Policy.ANY)) return true;
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.util.Collection;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
|
||||
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
|
||||
|
||||
public class CoresVariable extends VariableBase {
|
||||
|
@ -59,12 +61,12 @@ public class CoresVariable extends VariableBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
|
||||
public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
|
||||
cell.val = cell.val == null ? 0 : ((Number) cell.val).doubleValue() + 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
public void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
cell.val = cell.val == null ? 0 : ((Number) cell.val).doubleValue() - 1;
|
||||
}
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
|
||||
/**
|
||||
* Base class for overriding some behavior of {@link NodeStateProvider}.
|
||||
|
@ -40,7 +41,7 @@ public class DelegatingNodeStateProvider implements NodeStateProvider {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return delegate.getReplicaInfo(node, keys);
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.function.Consumer;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Suggester.Hint;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.util.Pair;
|
||||
|
||||
import static org.apache.solr.client.solrj.cloud.autoscaling.Suggestion.suggestNegativeViolations;
|
||||
|
@ -90,31 +91,31 @@ public class FreeDiskVariable extends VariableBase {
|
|||
List<Row> matchingNodes = ctx.session.matrix.stream().filter(
|
||||
row -> ctx.violation.getViolatingReplicas()
|
||||
.stream()
|
||||
.anyMatch(p -> row.node.equals(p.replicaInfo.getNode())))
|
||||
.anyMatch(p -> row.node.equals(p.replicaInfo.getNodeName())))
|
||||
.sorted(Comparator.comparing(r -> ((Double) r.getVal(DISK, 0d))))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
for (Row node : matchingNodes) {
|
||||
//lets try to start moving the smallest cores off of the node
|
||||
ArrayList<ReplicaInfo> replicas = new ArrayList<>();
|
||||
ArrayList<Replica> replicas = new ArrayList<>();
|
||||
node.forEachReplica(replicas::add);
|
||||
replicas.sort((r1, r2) -> {
|
||||
Long s1 = Clause.parseLong(CORE_IDX.tagName, r1.getVariables().get(CORE_IDX.tagName));
|
||||
Long s2 = Clause.parseLong(CORE_IDX.tagName, r2.getVariables().get(CORE_IDX.tagName));
|
||||
Long s1 = Clause.parseLong(CORE_IDX.tagName, r1.getProperties().get(CORE_IDX.tagName));
|
||||
Long s2 = Clause.parseLong(CORE_IDX.tagName, r2.getProperties().get(CORE_IDX.tagName));
|
||||
if (s1 != null && s2 != null) return s1.compareTo(s2);
|
||||
return 0;
|
||||
});
|
||||
double currentDelta = ctx.violation.getClause().tag.delta(node.getVal(DISK));
|
||||
for (ReplicaInfo replica : replicas) {
|
||||
for (Replica replica : replicas) {
|
||||
if (currentDelta < 1) break;
|
||||
if (replica.getVariables().get(CORE_IDX.tagName) == null) continue;
|
||||
if (replica.getProperties().get(CORE_IDX.tagName) == null) continue;
|
||||
Suggester suggester = ctx.session.getSuggester(MOVEREPLICA)
|
||||
.hint(Hint.COLL_SHARD, new Pair<>(replica.getCollection(), replica.getShard()))
|
||||
.hint(Hint.SRC_NODE, node.node)
|
||||
.forceOperation(true);
|
||||
ctx.addSuggestion(suggester);
|
||||
currentDelta -= Clause.parseLong(CORE_IDX.tagName, replica.getVariable(CORE_IDX.tagName));
|
||||
currentDelta -= Clause.parseLong(CORE_IDX.tagName, replica.get(CORE_IDX.tagName));
|
||||
}
|
||||
}
|
||||
} else if (ctx.violation.replicaCountDelta < 0) {
|
||||
|
@ -131,7 +132,7 @@ public class FreeDiskVariable extends VariableBase {
|
|||
node.forEachShard(coll, (s, ri) -> {
|
||||
if (result.get() != null) return;
|
||||
if (s.equals(shard1) && ri.size() > 0) {
|
||||
Number sz = ((Number) ri.get(0).getVariable(CORE_IDX.tagName));
|
||||
Number sz = ((Number) ri.get(0).get(CORE_IDX.tagName));
|
||||
if (sz != null) result.set(new Pair<>(shard1, sz.longValue()));
|
||||
}
|
||||
});
|
||||
|
@ -146,28 +147,28 @@ public class FreeDiskVariable extends VariableBase {
|
|||
|
||||
//When a replica is added, freedisk should be incremented
|
||||
@Override
|
||||
public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
|
||||
public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
|
||||
//go through other replicas of this shard and copy the index size value into this
|
||||
for (Row row : cell.getRow().session.matrix) {
|
||||
row.forEachReplica(replicaInfo -> {
|
||||
if (ri != replicaInfo &&
|
||||
ri.getCollection().equals(replicaInfo.getCollection()) &&
|
||||
ri.getShard().equals(replicaInfo.getShard()) &&
|
||||
ri.getVariable(CORE_IDX.tagName) == null &&
|
||||
replicaInfo.getVariable(CORE_IDX.tagName) != null) {
|
||||
ri.getVariables().put(CORE_IDX.tagName, validate(CORE_IDX.tagName, replicaInfo.getVariable(CORE_IDX.tagName), false));
|
||||
ri.get(CORE_IDX.tagName) == null &&
|
||||
replicaInfo.get(CORE_IDX.tagName) != null) {
|
||||
ri.getProperties().put(CORE_IDX.tagName, validate(CORE_IDX.tagName, replicaInfo.get(CORE_IDX.tagName), false));
|
||||
}
|
||||
});
|
||||
}
|
||||
Double idxSize = (Double) validate(CORE_IDX.tagName, ri.getVariable(CORE_IDX.tagName), false);
|
||||
Double idxSize = (Double) validate(CORE_IDX.tagName, ri.get(CORE_IDX.tagName), false);
|
||||
if (idxSize == null) return;
|
||||
Double currFreeDisk = cell.val == null ? 0.0d : (Double) cell.val;
|
||||
cell.val = currFreeDisk - idxSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
Double idxSize = (Double) validate(CORE_IDX.tagName, ri.getVariable(CORE_IDX.tagName), false);
|
||||
public void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
Double idxSize = (Double) validate(CORE_IDX.tagName, ri.get(CORE_IDX.tagName), false);
|
||||
if (idxSize == null) return;
|
||||
Double currFreeDisk = cell.val == null ? 0.0d : (Double) cell.val;
|
||||
cell.val = currFreeDisk + idxSize;
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.solr.client.solrj.SolrRequest;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.params.CollectionParams;
|
||||
import org.apache.solr.common.util.Pair;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -47,14 +48,14 @@ public class MoveReplicaSuggester extends Suggester {
|
|||
List<Violation> leastSeriousViolation = null;
|
||||
Row bestSrcRow = null;
|
||||
Row bestTargetRow = null;
|
||||
ReplicaInfo sourceReplicaInfo = null;
|
||||
List<Pair<ReplicaInfo, Row>> validReplicas = getValidReplicas(true, true, -1);
|
||||
Replica sourceReplicaInfo = null;
|
||||
List<Pair<Replica, Row>> validReplicas = getValidReplicas(true, true, -1);
|
||||
validReplicas.sort(leaderLast);
|
||||
for (int i1 = 0; i1 < validReplicas.size(); i1++) {
|
||||
lastBestDeviation = null;
|
||||
Pair<ReplicaInfo, Row> fromReplica = validReplicas.get(i1);
|
||||
Pair<Replica, Row> fromReplica = validReplicas.get(i1);
|
||||
Row fromRow = fromReplica.second();
|
||||
ReplicaInfo ri = fromReplica.first();
|
||||
Replica ri = fromReplica.first();
|
||||
if (ri == null) continue;
|
||||
final int i = session.indexOf(fromRow.node);
|
||||
int stopAt = force ? 0 : i;
|
||||
|
@ -97,8 +98,8 @@ public class MoveReplicaSuggester extends Suggester {
|
|||
return null;
|
||||
}
|
||||
|
||||
static Comparator<Pair<ReplicaInfo, Row>> leaderLast = (r1, r2) -> {
|
||||
int leaderCompare = Boolean.compare(r1.first().isLeader, r2.first().isLeader);
|
||||
static Comparator<Pair<Replica, Row>> leaderLast = (r1, r2) -> {
|
||||
int leaderCompare = Boolean.compare(r1.first().isLeader(), r2.first().isLeader());
|
||||
if ( leaderCompare != 0 ) {
|
||||
return leaderCompare;
|
||||
} else {
|
||||
|
|
|
@ -145,11 +145,11 @@ public class PolicyHelper {
|
|||
for (String shardName : shardNames) {
|
||||
Replica ldr = coll.getLeader(shardName);
|
||||
if (ldr != null && cloudManager.getClusterStateProvider().getLiveNodes().contains(ldr.getNodeName())) {
|
||||
Map<String, Map<String, List<ReplicaInfo>>> details = cloudManager.getNodeStateProvider().getReplicaInfo(ldr.getNodeName(),
|
||||
Map<String, Map<String, List<Replica>>> details = cloudManager.getNodeStateProvider().getReplicaInfo(ldr.getNodeName(),
|
||||
Collections.singleton(FREEDISK.perReplicaValue));
|
||||
ReplicaInfo replicaInfo = details.getOrDefault(collName, emptyMap()).getOrDefault(shardName, singletonList(null)).get(0);
|
||||
Replica replicaInfo = details.getOrDefault(collName, emptyMap()).getOrDefault(shardName, singletonList(null)).get(0);
|
||||
if (replicaInfo != null) {
|
||||
Object idxSz = replicaInfo.getVariables().get(FREEDISK.perReplicaValue);
|
||||
Object idxSz = replicaInfo.getProperties().get(FREEDISK.perReplicaValue);
|
||||
if (idxSz != null) {
|
||||
diskSpaceReqd.put(shardName, 1.5 * (Double) Variable.Type.FREEDISK.validate(null, idxSz, false));
|
||||
}
|
||||
|
|
|
@ -63,14 +63,14 @@ class ReplicaCount implements MapWriter {
|
|||
return total();
|
||||
}
|
||||
|
||||
public void increment(List<ReplicaInfo> infos) {
|
||||
public void increment(List<Replica> infos) {
|
||||
if (infos == null) return;
|
||||
for (ReplicaInfo info : infos) {
|
||||
for (Replica info : infos) {
|
||||
increment(info);
|
||||
}
|
||||
}
|
||||
|
||||
void increment(ReplicaInfo info) {
|
||||
void increment(Replica info) {
|
||||
increment(info.getType());
|
||||
}
|
||||
|
||||
|
|
|
@ -1,224 +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.autoscaling;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.BiPredicate;
|
||||
|
||||
import org.apache.solr.common.MapWriter;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
|
||||
import static org.apache.solr.common.ConditionalMapWriter.NON_NULL_VAL;
|
||||
import static org.apache.solr.common.ConditionalMapWriter.dedupeKeyPredicate;
|
||||
import static org.apache.solr.common.cloud.ZkStateReader.LEADER_PROP;
|
||||
|
||||
public class ReplicaInfo implements MapWriter {
|
||||
private final String name;
|
||||
private final String core, collection, shard;
|
||||
private final Replica.Type type;
|
||||
private final String node;
|
||||
public final boolean isLeader;
|
||||
private final Map<String, Object> variables = new HashMap<>();
|
||||
|
||||
public ReplicaInfo(String coll, String shard, Replica r, Map<String, Object> vals) {
|
||||
this.name = r.getName();
|
||||
this.core = r.getCoreName();
|
||||
this.collection = coll;
|
||||
this.shard = shard;
|
||||
this.type = r.getType();
|
||||
this.node = r.getNodeName();
|
||||
boolean maybeLeader = r.getBool(LEADER_PROP, false);
|
||||
if (vals != null) {
|
||||
this.variables.putAll(vals);
|
||||
maybeLeader = "true".equals(String.valueOf(vals.getOrDefault(LEADER_PROP, maybeLeader)));
|
||||
}
|
||||
this.isLeader = maybeLeader;
|
||||
validate();
|
||||
}
|
||||
|
||||
public ReplicaInfo(String name, String core, String coll, String shard, Replica.Type type, String node, Map<String, Object> vals) {
|
||||
if (vals == null) vals = Collections.emptyMap();
|
||||
this.name = name;
|
||||
if (vals != null) {
|
||||
this.variables.putAll(vals);
|
||||
}
|
||||
this.isLeader = "true".equals(String.valueOf(vals.getOrDefault(LEADER_PROP, "false")));
|
||||
this.collection = coll;
|
||||
this.shard = shard;
|
||||
this.type = type;
|
||||
this.core = core;
|
||||
this.node = node;
|
||||
validate();
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public ReplicaInfo(Map<String, Object> map) {
|
||||
this.name = map.keySet().iterator().next();
|
||||
@SuppressWarnings({"rawtypes"})Map details = (Map) map.get(name);
|
||||
details = Utils.getDeepCopy(details, 4);
|
||||
this.collection = (String) details.remove("collection");
|
||||
this.shard = (String) details.remove("shard");
|
||||
this.core = (String) details.remove("core");
|
||||
this.node = (String) details.remove("node_name");
|
||||
this.isLeader = Boolean.parseBoolean((String) details.getOrDefault("leader", "false"));
|
||||
details.remove("leader");
|
||||
type = Replica.Type.valueOf((String) details.getOrDefault("type", "NRT"));
|
||||
details.remove("type");
|
||||
this.variables.putAll(details);
|
||||
validate();
|
||||
}
|
||||
|
||||
private final void validate() {
|
||||
Objects.requireNonNull(this.name, "'name' must not be null");
|
||||
Objects.requireNonNull(this.core, "'core' must not be null");
|
||||
Objects.requireNonNull(this.collection, "'collection' must not be null");
|
||||
Objects.requireNonNull(this.shard, "'shard' must not be null");
|
||||
Objects.requireNonNull(this.type, "'type' must not be null");
|
||||
Objects.requireNonNull(this.node, "'node' must not be null");
|
||||
}
|
||||
|
||||
public Object clone() {
|
||||
return new ReplicaInfo(name, core, collection, shard, type, node, new HashMap<>(variables));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeMap(EntryWriter ew) throws IOException {
|
||||
BiPredicate<CharSequence, Object> p = dedupeKeyPredicate(new HashSet<>())
|
||||
.and(NON_NULL_VAL);
|
||||
ew.put(name, (MapWriter) ew1 -> {
|
||||
ew1.put(ZkStateReader.CORE_NAME_PROP, core, p)
|
||||
.put(ZkStateReader.SHARD_ID_PROP, shard, p)
|
||||
.put(ZkStateReader.COLLECTION_PROP, collection, p)
|
||||
.put(ZkStateReader.NODE_NAME_PROP, node, p)
|
||||
.put(ZkStateReader.REPLICA_TYPE, type.toString(), p);
|
||||
for (Map.Entry<String, Object> e : variables.entrySet()) ew1.put(e.getKey(), e.getValue(), p);
|
||||
});
|
||||
}
|
||||
|
||||
/** Replica "coreNode" name. */
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
/** SolrCore name. */
|
||||
public String getCore() {
|
||||
return core;
|
||||
}
|
||||
|
||||
public String getCollection() {
|
||||
return collection;
|
||||
}
|
||||
|
||||
public String getShard() {
|
||||
return shard;
|
||||
}
|
||||
|
||||
public Replica.Type getType() {
|
||||
Object o = type == null ? variables.get(ZkStateReader.REPLICA_TYPE) : type;
|
||||
if (o == null) {
|
||||
return Replica.Type.NRT;
|
||||
} else if (o instanceof Replica.Type) {
|
||||
return (Replica.Type)o;
|
||||
} else {
|
||||
Replica.Type type = Replica.Type.get(String.valueOf(o).toUpperCase(Locale.ROOT));
|
||||
return type;
|
||||
}
|
||||
}
|
||||
|
||||
public Replica.State getState() {
|
||||
if (variables.get(ZkStateReader.STATE_PROP) != null) {
|
||||
return Replica.State.getState((String) variables.get(ZkStateReader.STATE_PROP));
|
||||
} else {
|
||||
// default to ACTIVE
|
||||
variables.put(ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
|
||||
return Replica.State.ACTIVE;
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, Object> getVariables() {
|
||||
return variables;
|
||||
}
|
||||
|
||||
public Object getVariable(String name) {
|
||||
return variables.get(name);
|
||||
}
|
||||
|
||||
public Object getVariable(String name, Object defValue) {
|
||||
Object o = variables.get(name);
|
||||
if (o != null) {
|
||||
return o;
|
||||
} else {
|
||||
return defValue;
|
||||
}
|
||||
}
|
||||
|
||||
public boolean getBool(String name, boolean defValue) {
|
||||
Object o = getVariable(name, defValue);
|
||||
if (o instanceof Boolean) {
|
||||
return (Boolean)o;
|
||||
} else {
|
||||
return Boolean.parseBoolean(String.valueOf(o));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null) {
|
||||
return false;
|
||||
}
|
||||
if (!(o instanceof ReplicaInfo)) {
|
||||
return false;
|
||||
}
|
||||
ReplicaInfo other = (ReplicaInfo)o;
|
||||
if (
|
||||
name.equals(other.name) &&
|
||||
collection.equals(other.collection) &&
|
||||
core.equals(other.core) &&
|
||||
isLeader == other.isLeader &&
|
||||
node.equals(other.node) &&
|
||||
shard.equals(other.shard) &&
|
||||
type == other.type &&
|
||||
variables.equals(other.variables)) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(name, core, collection, shard, type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Utils.toJSONString(this);
|
||||
}
|
||||
|
||||
public String getNode() {
|
||||
return node;
|
||||
}
|
||||
}
|
|
@ -38,7 +38,6 @@ import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
|||
import org.apache.solr.client.solrj.cloud.SolrCloudManager;
|
||||
import org.apache.solr.common.MapWriter;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.util.Pair;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -55,7 +54,7 @@ public class Row implements MapWriter {
|
|||
public final String node;
|
||||
final Cell[] cells;
|
||||
//this holds the details of each replica in the node
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas;
|
||||
public Map<String, Map<String, List<Replica>>> collectionVsShardVsReplicas;
|
||||
|
||||
boolean anyValueMissing = false;
|
||||
boolean isLive = true;
|
||||
|
@ -130,7 +129,7 @@ public class Row implements MapWriter {
|
|||
}
|
||||
|
||||
|
||||
public void forEachShard(String collection, BiConsumer<String, List<ReplicaInfo>> consumer) {
|
||||
public void forEachShard(String collection, BiConsumer<String, List<Replica>> consumer) {
|
||||
collectionVsShardVsReplicas
|
||||
.getOrDefault(collection, Collections.emptyMap())
|
||||
.forEach(consumer);
|
||||
|
@ -172,7 +171,7 @@ public class Row implements MapWriter {
|
|||
|
||||
public Row(String node, Cell[] cells, boolean anyValueMissing,
|
||||
@SuppressWarnings({"rawtypes"}) Map<String,
|
||||
Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas, boolean isLive, Policy.Session session,
|
||||
Map<String, List<Replica>>> collectionVsShardVsReplicas, boolean isLive, Policy.Session session,
|
||||
@SuppressWarnings({"rawtypes"}) Map perRowCache,
|
||||
@SuppressWarnings({"rawtypes"})Map globalCache) {
|
||||
this.session = session;
|
||||
|
@ -251,11 +250,11 @@ public class Row implements MapWriter {
|
|||
row = session.copy().getNode(this.node);
|
||||
if (row == null) throw new RuntimeException("couldn't get a row");
|
||||
row.lazyCopyReplicas(coll, shard);
|
||||
Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.computeIfAbsent(coll, k -> new HashMap<>());
|
||||
List<ReplicaInfo> replicas = c.computeIfAbsent(shard, k -> new ArrayList<>());
|
||||
Map<String, List<Replica>> c = row.collectionVsShardVsReplicas.computeIfAbsent(coll, k -> new HashMap<>());
|
||||
List<Replica> replicas = c.computeIfAbsent(shard, k -> new ArrayList<>());
|
||||
String replicaname = "SYNTHETIC." + new Random().nextInt(1000) + 1000;
|
||||
ReplicaInfo ri = new ReplicaInfo(replicaname, replicaname, coll, shard, type, this.node,
|
||||
Utils.makeMap(ZkStateReader.REPLICA_TYPE, type != null ? type.toString() : Replica.Type.NRT.toString()));
|
||||
Replica ri = new Replica(replicaname, this.node, coll, shard, replicaname,
|
||||
Replica.State.ACTIVE, type != null ? type : Replica.Type.NRT, Collections.emptyMap());
|
||||
replicas.add(ri);
|
||||
for (Cell cell : row.cells) {
|
||||
cell.type.projectAddReplica(cell, ri, opCollector, strictMode);
|
||||
|
@ -281,8 +280,8 @@ public class Row implements MapWriter {
|
|||
perCollCache = cacheCopy;
|
||||
if (isAlreadyCopied) return;//caches need to be invalidated but the rest can remain as is
|
||||
|
||||
Map<String, Map<String, List<ReplicaInfo>>> replicasCopy = new HashMap<>(collectionVsShardVsReplicas);
|
||||
Map<String, List<ReplicaInfo>> oneColl = replicasCopy.get(coll);
|
||||
Map<String, Map<String, List<Replica>>> replicasCopy = new HashMap<>(collectionVsShardVsReplicas);
|
||||
Map<String, List<Replica>> oneColl = replicasCopy.get(coll);
|
||||
if (oneColl != null) {
|
||||
replicasCopy.put(coll, Utils.getDeepCopy(oneColl, 2));
|
||||
}
|
||||
|
@ -296,7 +295,7 @@ public class Row implements MapWriter {
|
|||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public void createCollShard(Pair<String, String> collShard) {
|
||||
Map<String, List<ReplicaInfo>> shardInfo = collectionVsShardVsReplicas.computeIfAbsent(collShard.first(), Utils.NEW_HASHMAP_FUN);
|
||||
Map<String, List<Replica>> shardInfo = collectionVsShardVsReplicas.computeIfAbsent(collShard.first(), Utils.NEW_HASHMAP_FUN);
|
||||
if (collShard.second() != null) shardInfo.computeIfAbsent(collShard.second(), Utils.NEW_ARRAYLIST_FUN);
|
||||
}
|
||||
|
||||
|
@ -318,14 +317,14 @@ public class Row implements MapWriter {
|
|||
}
|
||||
|
||||
|
||||
public ReplicaInfo getReplica(String coll, String shard, Replica.Type type) {
|
||||
Map<String, List<ReplicaInfo>> c = collectionVsShardVsReplicas.get(coll);
|
||||
public Replica getReplica(String coll, String shard, Replica.Type type) {
|
||||
Map<String, List<Replica>> c = collectionVsShardVsReplicas.get(coll);
|
||||
if (c == null) return null;
|
||||
List<ReplicaInfo> r = c.get(shard);
|
||||
List<Replica> r = c.get(shard);
|
||||
if (r == null) return null;
|
||||
int idx = -1;
|
||||
for (int i = 0; i < r.size(); i++) {
|
||||
ReplicaInfo info = r.get(i);
|
||||
Replica info = r.get(i);
|
||||
if (type == null || info.getType() == type) {
|
||||
idx = i;
|
||||
break;
|
||||
|
@ -350,20 +349,20 @@ public class Row implements MapWriter {
|
|||
Consumer<OperationInfo> opCollector = it -> furtherOps.add(it);
|
||||
Row row = session.copy().getNode(this.node);
|
||||
row.lazyCopyReplicas(coll, shard);
|
||||
Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.get(coll);
|
||||
Map<String, List<Replica>> c = row.collectionVsShardVsReplicas.get(coll);
|
||||
if (c == null) return null;
|
||||
List<ReplicaInfo> r = c.get(shard);
|
||||
List<Replica> r = c.get(shard);
|
||||
if (r == null) return null;
|
||||
int idx = -1;
|
||||
for (int i = 0; i < r.size(); i++) {
|
||||
ReplicaInfo info = r.get(i);
|
||||
Replica info = r.get(i);
|
||||
if (type == null || info.getType() == type) {
|
||||
idx = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (idx == -1) return null;
|
||||
ReplicaInfo removed = r.remove(idx);
|
||||
Replica removed = r.remove(idx);
|
||||
for (Cell cell : row.cells) {
|
||||
cell.type.projectRemoveReplica(cell, removed, opCollector);
|
||||
}
|
||||
|
@ -379,23 +378,23 @@ public class Row implements MapWriter {
|
|||
return isLive;
|
||||
}
|
||||
|
||||
public void forEachReplica(Consumer<ReplicaInfo> consumer) {
|
||||
public void forEachReplica(Consumer<Replica> consumer) {
|
||||
forEachReplica(collectionVsShardVsReplicas, consumer);
|
||||
}
|
||||
|
||||
public void forEachReplica(String coll, Consumer<ReplicaInfo> consumer) {
|
||||
public void forEachReplica(String coll, Consumer<Replica> consumer) {
|
||||
collectionVsShardVsReplicas.getOrDefault(coll, Collections.emptyMap()).forEach((shard, replicaInfos) -> {
|
||||
for (ReplicaInfo replicaInfo : replicaInfos) {
|
||||
for (Replica replicaInfo : replicaInfos) {
|
||||
consumer.accept(replicaInfo);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static void forEachReplica(Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas, Consumer<ReplicaInfo> consumer) {
|
||||
public static void forEachReplica(Map<String, Map<String, List<Replica>>> collectionVsShardVsReplicas, Consumer<Replica> consumer) {
|
||||
collectionVsShardVsReplicas.forEach((coll, shardVsReplicas) -> shardVsReplicas
|
||||
.forEach((shard, replicaInfos) -> {
|
||||
for (int i = 0; i < replicaInfos.size(); i++) {
|
||||
ReplicaInfo r = replicaInfos.get(i);
|
||||
Replica r = replicaInfos.get(i);
|
||||
consumer.accept(r);
|
||||
}
|
||||
}));
|
||||
|
|
|
@ -296,8 +296,8 @@ public abstract class Suggester implements MapWriter {
|
|||
return false;
|
||||
}
|
||||
|
||||
List<Pair<ReplicaInfo, Row>> getValidReplicas(boolean sortDesc, boolean isSource, int until) {
|
||||
List<Pair<ReplicaInfo, Row>> allPossibleReplicas = new ArrayList<>();
|
||||
List<Pair<Replica, Row>> getValidReplicas(boolean sortDesc, boolean isSource, int until) {
|
||||
List<Pair<Replica, Row>> allPossibleReplicas = new ArrayList<>();
|
||||
|
||||
if (sortDesc) {
|
||||
if (until == -1) until = getMatrix().size();
|
||||
|
@ -310,14 +310,14 @@ public abstract class Suggester implements MapWriter {
|
|||
return allPossibleReplicas;
|
||||
}
|
||||
|
||||
void addReplicaToList(Row r, boolean isSource, List<Pair<ReplicaInfo, Row>> replicaList) {
|
||||
void addReplicaToList(Row r, boolean isSource, List<Pair<Replica, Row>> replicaList) {
|
||||
if (!isAllowed(r.node, isSource ? Hint.SRC_NODE : Hint.TARGET_NODE)) return;
|
||||
for (Map.Entry<String, Map<String, List<ReplicaInfo>>> e : r.collectionVsShardVsReplicas.entrySet()) {
|
||||
for (Map.Entry<String, Map<String, List<Replica>>> e : r.collectionVsShardVsReplicas.entrySet()) {
|
||||
if (!isAllowed(e.getKey(), Hint.COLL)) continue;
|
||||
for (Map.Entry<String, List<ReplicaInfo>> shard : e.getValue().entrySet()) {
|
||||
for (Map.Entry<String, List<Replica>> shard : e.getValue().entrySet()) {
|
||||
if (!isAllowed(new Pair<>(e.getKey(), shard.getKey()), Hint.COLL_SHARD)) continue;//todo fix
|
||||
if (shard.getValue() == null || shard.getValue().isEmpty()) continue;
|
||||
for (ReplicaInfo replicaInfo : shard.getValue()) {
|
||||
for (Replica replicaInfo : shard.getValue()) {
|
||||
if (replicaInfo.getName().startsWith("SYNTHETIC.")) continue;
|
||||
replicaList.add(new Pair<>(shard.getValue().get(0), r));
|
||||
break;
|
||||
|
@ -481,7 +481,7 @@ public abstract class Suggester implements MapWriter {
|
|||
row.forEachReplica(r -> {
|
||||
if (withCollection.equals(r.getCollection()) &&
|
||||
"shard1".equals(r.getShard())) {
|
||||
withCollectionNodes.add(r.getNode());
|
||||
withCollectionNodes.add(r.getNodeName());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -123,7 +123,7 @@ public class Suggestion {
|
|||
Suggester suggester = ctx.session.getSuggester(MOVEREPLICA)
|
||||
.forceOperation(true)
|
||||
.hint(Suggester.Hint.COLL_SHARD, new Pair<>(e.replicaInfo.getCollection(), e.replicaInfo.getShard()))
|
||||
.hint(Suggester.Hint.SRC_NODE, e.replicaInfo.getNode());
|
||||
.hint(Suggester.Hint.SRC_NODE, e.replicaInfo.getNodeName());
|
||||
if (ctx.addSuggestion(suggester) != null) {
|
||||
currentDelta--;
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
|
||||
|
||||
import static java.util.Collections.emptySet;
|
||||
|
@ -48,7 +49,7 @@ public interface Variable {
|
|||
return val;
|
||||
}
|
||||
|
||||
default void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
|
||||
default void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
|
||||
}
|
||||
|
||||
default boolean addViolatingReplicas(Violation.Ctx ctx) {
|
||||
|
@ -77,7 +78,7 @@ public interface Variable {
|
|||
|
||||
int compareViolation(Violation v1, Violation v2);
|
||||
|
||||
default void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
default void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
}
|
||||
|
||||
default String postValidate(Condition condition) {
|
||||
|
@ -329,11 +330,11 @@ public interface Variable {
|
|||
/**
|
||||
* Simulate a replica addition to a node in the cluster
|
||||
*/
|
||||
public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
|
||||
public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
|
||||
impl.projectAddReplica(cell, ri, opCollector, strictMode);
|
||||
}
|
||||
|
||||
public void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
public void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
|
||||
impl.projectRemoveReplica(cell, ri, opCollector);
|
||||
}
|
||||
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.util.function.Function;
|
|||
|
||||
import org.apache.solr.common.IteratorWriter;
|
||||
import org.apache.solr.common.MapWriter;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
|
||||
public class Violation implements MapWriter {
|
||||
|
@ -109,10 +110,10 @@ public class Violation implements MapWriter {
|
|||
}
|
||||
|
||||
static class ReplicaInfoAndErr implements MapWriter{
|
||||
final ReplicaInfo replicaInfo;
|
||||
final Replica replicaInfo;
|
||||
Double delta;
|
||||
|
||||
ReplicaInfoAndErr(ReplicaInfo replicaInfo) {
|
||||
ReplicaInfoAndErr(Replica replicaInfo) {
|
||||
this.replicaInfo = replicaInfo;
|
||||
}
|
||||
|
||||
|
|
|
@ -56,7 +56,7 @@ public class WithCollectionVariable extends VariableBase {
|
|||
return true;
|
||||
}
|
||||
|
||||
public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
|
||||
public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
|
||||
if (strictMode) {
|
||||
// we do not want to add a replica of the 'withCollection' in strict mode
|
||||
return;
|
||||
|
@ -138,14 +138,14 @@ public class WithCollectionVariable extends VariableBase {
|
|||
|
||||
if (hostedCollections.contains(withCollection) && !hostedCollections.contains(collection)) {
|
||||
// find the candidate replicas that we can move
|
||||
List<ReplicaInfo> movableReplicas = new ArrayList<>();
|
||||
List<Replica> movableReplicas = new ArrayList<>();
|
||||
row.forEachReplica(replicaInfo -> {
|
||||
if (replicaInfo.getCollection().equals(withCollection)) {
|
||||
movableReplicas.add(replicaInfo);
|
||||
}
|
||||
});
|
||||
|
||||
for (ReplicaInfo toMove : movableReplicas) {
|
||||
for (Replica toMove : movableReplicas) {
|
||||
// candidate source node for a move replica operation
|
||||
Suggester suggester = ctx.session.getSuggester(MOVEREPLICA)
|
||||
.forceOperation(true)
|
||||
|
|
|
@ -34,7 +34,6 @@ import java.util.stream.Collectors;
|
|||
import org.apache.solr.client.solrj.SolrRequest;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.client.solrj.cloud.NodeStateProvider;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Row;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
|
||||
import org.apache.solr.client.solrj.cloud.autoscaling.VariableBase;
|
||||
|
@ -45,6 +44,7 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.rule.ImplicitSnitch;
|
||||
import org.apache.solr.common.cloud.rule.SnitchContext;
|
||||
import org.apache.solr.common.params.CollectionAdminParams;
|
||||
|
@ -76,7 +76,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
|
|||
|
||||
|
||||
private final CloudSolrClient solrClient;
|
||||
protected final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> nodeVsCollectionVsShardVsReplicaInfo = new HashMap<>();
|
||||
protected final Map<String, Map<String, Map<String, List<Replica>>>> nodeVsCollectionVsShardVsReplicaInfo = new HashMap<>();
|
||||
private Map<String, Object> snitchSession = new HashMap<>();
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
private Map<String, Map> nodeVsTags = new HashMap<>();
|
||||
|
@ -110,10 +110,10 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
|
|||
withCollectionsMap.put(coll.getName(), (String) coll.getProperties().get(CollectionAdminParams.WITH_COLLECTION));
|
||||
}
|
||||
coll.forEachReplica((shard, replica) -> {
|
||||
Map<String, Map<String, List<ReplicaInfo>>> nodeData = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(replica.getNodeName(), k -> new HashMap<>());
|
||||
Map<String, List<ReplicaInfo>> collData = nodeData.computeIfAbsent(collName, k -> new HashMap<>());
|
||||
List<ReplicaInfo> replicas = collData.computeIfAbsent(shard, k -> new ArrayList<>());
|
||||
replicas.add(new ReplicaInfo(collName, shard, replica, new HashMap<>(replica.getProperties())));
|
||||
Map<String, Map<String, List<Replica>>> nodeData = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(replica.getNodeName(), k -> new HashMap<>());
|
||||
Map<String, List<Replica>> collData = nodeData.computeIfAbsent(collName, k -> new HashMap<>());
|
||||
List<Replica> replicas = collData.computeIfAbsent(shard, k -> new ArrayList<>());
|
||||
replicas.add((Replica) replica.clone());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
@ -141,21 +141,21 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
|
|||
return ctx.getTags();
|
||||
}
|
||||
|
||||
public void forEachReplica(String node, Consumer<ReplicaInfo> consumer){
|
||||
public void forEachReplica(String node, Consumer<Replica> consumer){
|
||||
Row.forEachReplica(nodeVsCollectionVsShardVsReplicaInfo.get(node), consumer);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Map<String, Map<String, List<ReplicaInfo>>> result = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(node, Utils.NEW_HASHMAP_FUN);
|
||||
Map<String, Map<String, List<Replica>>> result = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(node, Utils.NEW_HASHMAP_FUN);
|
||||
if (!keys.isEmpty()) {
|
||||
Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica = new HashMap<>();
|
||||
Map<String, Pair<String, Replica>> metricsKeyVsTagReplica = new HashMap<>();
|
||||
Row.forEachReplica(result, r -> {
|
||||
for (String key : keys) {
|
||||
if (r.getVariables().containsKey(key)) continue;// it's already collected
|
||||
String perReplicaMetricsKey = "solr.core." + r.getCollection() + "." + r.getShard() + "." + Utils.parseMetricsReplicaName(r.getCollection(), r.getCore()) + ":";
|
||||
if (r.getProperties().containsKey(key)) continue;// it's already collected
|
||||
String perReplicaMetricsKey = "solr.core." + r.getCollection() + "." + r.getShard() + "." + Utils.parseMetricsReplicaName(r.getCollection(), r.getCoreName()) + ":";
|
||||
Type tagType = VariableBase.getTagType(key);
|
||||
String perReplicaValue = key;
|
||||
if (tagType != null) {
|
||||
|
@ -170,10 +170,10 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
|
|||
if (!metricsKeyVsTagReplica.isEmpty()) {
|
||||
Map<String, Object> tagValues = fetchReplicaMetrics(node, metricsKeyVsTagReplica);
|
||||
tagValues.forEach((k, o) -> {
|
||||
Pair<String, ReplicaInfo> p = metricsKeyVsTagReplica.get(k);
|
||||
Pair<String, Replica> p = metricsKeyVsTagReplica.get(k);
|
||||
Type validator = VariableBase.getTagType(p.first());
|
||||
if (validator != null) o = validator.convertVal(o);
|
||||
if (p.second() != null) p.second().getVariables().put(p.first(), o);
|
||||
if (p.second() != null) p.second().getProperties().put(p.first(), o);
|
||||
});
|
||||
|
||||
}
|
||||
|
@ -181,7 +181,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
|
|||
return result;
|
||||
}
|
||||
|
||||
protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
|
||||
protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
|
||||
Map<String, Object> collect = metricsKeyVsTagReplica.entrySet().stream()
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getKey));
|
||||
ClientSnitchCtx ctx = new ClientSnitchCtx(null, null, emptyMap(), solrClient);
|
||||
|
|
|
@ -16,13 +16,24 @@
|
|||
*/
|
||||
package org.apache.solr.common.cloud;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.function.BiPredicate;
|
||||
|
||||
import org.apache.solr.common.MapWriter;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
public class Replica extends ZkNodeProps {
|
||||
import org.noggit.JSONWriter;
|
||||
|
||||
import static org.apache.solr.common.ConditionalMapWriter.NON_NULL_VAL;
|
||||
import static org.apache.solr.common.ConditionalMapWriter.dedupeKeyPredicate;
|
||||
|
||||
public class Replica extends ZkNodeProps implements MapWriter {
|
||||
|
||||
/**
|
||||
* The replica's state. In general, if the node the replica is hosted on is
|
||||
|
@ -101,45 +112,105 @@ public class Replica extends ZkNodeProps {
|
|||
*/
|
||||
PULL;
|
||||
|
||||
public static Type get(String name){
|
||||
public static Type get(String name) {
|
||||
return name == null ? Type.NRT : Type.valueOf(name.toUpperCase(Locale.ROOT));
|
||||
}
|
||||
}
|
||||
|
||||
private final String name;
|
||||
private final String nodeName;
|
||||
private final String core;
|
||||
private final State state;
|
||||
private final Type type;
|
||||
public final String slice, collection;
|
||||
// immutable
|
||||
public final String name; // coreNode name
|
||||
public final String node;
|
||||
public final String core;
|
||||
public final Type type;
|
||||
public final String shard, collection;
|
||||
|
||||
public Replica(String name, Map<String,Object> propMap, String collection, String slice) {
|
||||
super(propMap);
|
||||
// mutable
|
||||
private State state;
|
||||
|
||||
public Replica(String name, Map<String,Object> map, String collection, String shard) {
|
||||
super(new HashMap<>());
|
||||
propMap.putAll(map);
|
||||
this.collection = collection;
|
||||
this.slice = slice;
|
||||
this.shard = shard;
|
||||
this.name = name;
|
||||
this.nodeName = (String) propMap.get(ZkStateReader.NODE_NAME_PROP);
|
||||
this.node = (String) propMap.get(ZkStateReader.NODE_NAME_PROP);
|
||||
this.core = (String) propMap.get(ZkStateReader.CORE_NAME_PROP);
|
||||
type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
|
||||
Objects.requireNonNull(this.collection, "'collection' must not be null");
|
||||
Objects.requireNonNull(this.slice, "'slice' must not be null");
|
||||
Objects.requireNonNull(this.name, "'name' must not be null");
|
||||
Objects.requireNonNull(this.nodeName, "'node_name' must not be null");
|
||||
Objects.requireNonNull(this.core, "'core' must not be null");
|
||||
Objects.requireNonNull(this.type, "'type' must not be null");
|
||||
if (propMap.get(ZkStateReader.STATE_PROP) != null) {
|
||||
this.state = State.getState((String) propMap.get(ZkStateReader.STATE_PROP));
|
||||
} else {
|
||||
this.state = State.ACTIVE; //Default to ACTIVE
|
||||
propMap.put(ZkStateReader.STATE_PROP, state.toString());
|
||||
}
|
||||
this.type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
|
||||
// default to ACTIVE
|
||||
this.state = State.getState(String.valueOf(propMap.getOrDefault(ZkStateReader.STATE_PROP, State.ACTIVE.toString())));
|
||||
validate();
|
||||
}
|
||||
|
||||
public String getCollection(){
|
||||
// clone constructor
|
||||
public Replica(String name, String node, String collection, String shard, String core,
|
||||
State state, Type type, Map<String, Object> props) {
|
||||
super(new HashMap<>());
|
||||
this.name = name;
|
||||
this.node = node;
|
||||
this.state = state;
|
||||
this.type = type;
|
||||
this.collection = collection;
|
||||
this.shard = shard;
|
||||
this.core = core;
|
||||
if (props != null) {
|
||||
this.propMap.putAll(props);
|
||||
}
|
||||
validate();
|
||||
}
|
||||
|
||||
/**
|
||||
* This constructor uses a map with one key (coreNode name) and a value that
|
||||
* is a map containing all replica properties.
|
||||
* @param nestedMap nested map containing replica properties
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public Replica(Map<String, Object> nestedMap) {
|
||||
this.name = nestedMap.keySet().iterator().next();
|
||||
Map<String, Object> details = (Map<String, Object>) nestedMap.get(name);
|
||||
Objects.requireNonNull(details);
|
||||
details = Utils.getDeepCopy(details, 4);
|
||||
this.collection = String.valueOf(details.get("collection"));
|
||||
this.shard = String.valueOf(details.get("shard"));
|
||||
this.core = String.valueOf(details.get("core"));
|
||||
this.node = String.valueOf(details.get("node_name"));
|
||||
type = Replica.Type.valueOf(String.valueOf(details.getOrDefault(ZkStateReader.REPLICA_TYPE, "NRT")));
|
||||
state = State.getState(String.valueOf(details.getOrDefault(ZkStateReader.STATE_PROP, "active")));
|
||||
this.propMap.putAll(details);
|
||||
validate();
|
||||
|
||||
}
|
||||
|
||||
private final void validate() {
|
||||
Objects.requireNonNull(this.name, "'name' must not be null");
|
||||
Objects.requireNonNull(this.core, "'core' must not be null");
|
||||
Objects.requireNonNull(this.collection, "'collection' must not be null");
|
||||
Objects.requireNonNull(this.shard, "'shard' must not be null");
|
||||
Objects.requireNonNull(this.type, "'type' must not be null");
|
||||
Objects.requireNonNull(this.state, "'state' must not be null");
|
||||
Objects.requireNonNull(this.node, "'node' must not be null");
|
||||
// make sure all declared props are in the propMap
|
||||
propMap.put(ZkStateReader.COLLECTION_PROP, collection);
|
||||
propMap.put(ZkStateReader.SHARD_ID_PROP, shard);
|
||||
propMap.put(ZkStateReader.CORE_NODE_NAME_PROP, name);
|
||||
propMap.put(ZkStateReader.NODE_NAME_PROP, node);
|
||||
propMap.put(ZkStateReader.CORE_NAME_PROP, core);
|
||||
propMap.put(ZkStateReader.REPLICA_TYPE, type.toString());
|
||||
propMap.put(ZkStateReader.STATE_PROP, state.toString());
|
||||
}
|
||||
|
||||
|
||||
|
||||
public String getCollection() {
|
||||
return collection;
|
||||
}
|
||||
public String getSlice(){
|
||||
return slice;
|
||||
|
||||
public String getShard() {
|
||||
return shard;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getProperties() {
|
||||
return super.getProperties();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -148,9 +219,9 @@ public class Replica extends ZkNodeProps {
|
|||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (!super.equals(o)) return false;
|
||||
|
||||
Replica replica = (Replica) o;
|
||||
Replica other = (Replica) o;
|
||||
|
||||
return name.equals(replica.name);
|
||||
return name.equals(other.name);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -166,7 +237,8 @@ public class Replica extends ZkNodeProps {
|
|||
public String getCoreUrl() {
|
||||
return ZkCoreNodeProps.getCoreUrl(getStr(ZkStateReader.BASE_URL_PROP), core);
|
||||
}
|
||||
public String getBaseUrl(){
|
||||
|
||||
public String getBaseUrl() {
|
||||
return getStr(ZkStateReader.BASE_URL_PROP);
|
||||
}
|
||||
|
||||
|
@ -177,7 +249,7 @@ public class Replica extends ZkNodeProps {
|
|||
|
||||
/** The name of the node this replica resides on */
|
||||
public String getNodeName() {
|
||||
return nodeName;
|
||||
return node;
|
||||
}
|
||||
|
||||
/** Returns the {@link State} of this replica. */
|
||||
|
@ -185,14 +257,32 @@ public class Replica extends ZkNodeProps {
|
|||
return state;
|
||||
}
|
||||
|
||||
public void setState(State state) {
|
||||
this.state = state;
|
||||
propMap.put(ZkStateReader.STATE_PROP, this.state.toString());
|
||||
}
|
||||
|
||||
public boolean isActive(Set<String> liveNodes) {
|
||||
return this.nodeName != null && liveNodes.contains(this.nodeName) && this.state == State.ACTIVE;
|
||||
return this.node != null && liveNodes.contains(this.node) && this.state == State.ACTIVE;
|
||||
}
|
||||
|
||||
public Type getType() {
|
||||
return this.type;
|
||||
}
|
||||
|
||||
public boolean isLeader() {
|
||||
return getBool(ZkStateReader.LEADER_PROP, false);
|
||||
}
|
||||
|
||||
public Object get(String key, Object defValue) {
|
||||
Object o = get(key);
|
||||
if (o != null) {
|
||||
return o;
|
||||
} else {
|
||||
return defValue;
|
||||
}
|
||||
}
|
||||
|
||||
public String getProperty(String propertyName) {
|
||||
final String propertyKey;
|
||||
if (!propertyName.startsWith(ZkStateReader.PROPERTY_PROP_PREFIX)) {
|
||||
|
@ -204,6 +294,45 @@ public class Replica extends ZkNodeProps {
|
|||
return propertyValue;
|
||||
}
|
||||
|
||||
public Object clone() {
|
||||
return new Replica(name, node, collection, shard, core, state, type,
|
||||
propMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeMap(MapWriter.EntryWriter ew) throws IOException {
|
||||
ew.put(name, _allPropsWriter());
|
||||
}
|
||||
|
||||
|
||||
private MapWriter _allPropsWriter() {
|
||||
BiPredicate<CharSequence, Object> p = dedupeKeyPredicate(new HashSet<>())
|
||||
.and(NON_NULL_VAL);
|
||||
return writer -> {
|
||||
// XXX this is why this class should be immutable - it's a mess !!!
|
||||
|
||||
// propMap takes precedence because it's mutable and we can't control its
|
||||
// contents, so a third party may override some declared fields
|
||||
for (Map.Entry<String, Object> e : propMap.entrySet()) {
|
||||
writer.put(e.getKey(), e.getValue(), p);
|
||||
}
|
||||
writer.put(ZkStateReader.CORE_NAME_PROP, core, p)
|
||||
.put(ZkStateReader.SHARD_ID_PROP, shard, p)
|
||||
.put(ZkStateReader.COLLECTION_PROP, collection, p)
|
||||
.put(ZkStateReader.NODE_NAME_PROP, node, p)
|
||||
.put(ZkStateReader.REPLICA_TYPE, type.toString(), p)
|
||||
.put(ZkStateReader.STATE_PROP, state.toString(), p);
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(JSONWriter jsonWriter) {
|
||||
Map<String, Object> map = new LinkedHashMap<>();
|
||||
// this serializes also our declared properties
|
||||
_allPropsWriter().toMap(map);
|
||||
jsonWriter.write(map);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return name + ':' + Utils.toJSONString(propMap); // small enough, keep it on one line (i.e. no indent)
|
||||
|
|
|
@ -39,27 +39,27 @@ public class MoveReplicaSuggesterTest extends SolrTestCaseJ4 {
|
|||
private Replica.Type REPLICA_TYPE = Replica.Type.NRT;
|
||||
private Row ROW = null;
|
||||
|
||||
private ReplicaInfo REPLICA_INFO_ONE = new ReplicaInfo("core_node1", CORE, COLLECTION, SHARD, REPLICA_TYPE, NODE, IS_LEADER);
|
||||
private ReplicaInfo REPLICA_INFO_TWO = new ReplicaInfo("core_node2", CORE, COLLECTION, SHARD, REPLICA_TYPE, NODE, null);
|
||||
private ReplicaInfo REPLICA_INFO_THREE = new ReplicaInfo("core_node3", CORE, COLLECTION, SHARD, REPLICA_TYPE, NODE, IS_LEADER);
|
||||
private ReplicaInfo REPLICA_INFO_FOUR = new ReplicaInfo("core_node4", CORE, COLLECTION, SHARD, REPLICA_TYPE, NODE, null);
|
||||
private Replica REPLICA_INFO_ONE = new Replica("core_node1", NODE, COLLECTION, SHARD, CORE, Replica.State.ACTIVE, REPLICA_TYPE, IS_LEADER);
|
||||
private Replica REPLICA_INFO_TWO = new Replica("core_node2", NODE, COLLECTION, SHARD, CORE, Replica.State.ACTIVE, REPLICA_TYPE, null);
|
||||
private Replica REPLICA_INFO_THREE = new Replica("core_node3", NODE, COLLECTION, SHARD, CORE, Replica.State.ACTIVE, REPLICA_TYPE, IS_LEADER);
|
||||
private Replica REPLICA_INFO_FOUR = new Replica("core_node4", NODE, COLLECTION, SHARD, CORE, Replica.State.ACTIVE, REPLICA_TYPE, null);
|
||||
|
||||
private Pair<ReplicaInfo, Row> PAIR_ONE = new Pair<>(REPLICA_INFO_ONE, ROW);
|
||||
private Pair<ReplicaInfo, Row> PAIR_TWO = new Pair<>(REPLICA_INFO_TWO, ROW);
|
||||
private Pair<ReplicaInfo, Row> PAIR_THREE = new Pair<>(REPLICA_INFO_THREE, ROW);
|
||||
private Pair<ReplicaInfo, Row> PAIR_FOUR = new Pair<>(REPLICA_INFO_FOUR, ROW);
|
||||
private Pair<Replica, Row> PAIR_ONE = new Pair<>(REPLICA_INFO_ONE, ROW);
|
||||
private Pair<Replica, Row> PAIR_TWO = new Pair<>(REPLICA_INFO_TWO, ROW);
|
||||
private Pair<Replica, Row> PAIR_THREE = new Pair<>(REPLICA_INFO_THREE, ROW);
|
||||
private Pair<Replica, Row> PAIR_FOUR = new Pair<>(REPLICA_INFO_FOUR, ROW);
|
||||
|
||||
@Test
|
||||
public void assertLeaderProperties() {
|
||||
assertTrue(REPLICA_INFO_ONE.isLeader);
|
||||
assertFalse(REPLICA_INFO_TWO.isLeader);
|
||||
assertTrue(REPLICA_INFO_THREE.isLeader);
|
||||
assertFalse(REPLICA_INFO_FOUR.isLeader);
|
||||
assertTrue(REPLICA_INFO_ONE.isLeader());
|
||||
assertFalse(REPLICA_INFO_TWO.isLeader());
|
||||
assertTrue(REPLICA_INFO_THREE.isLeader());
|
||||
assertFalse(REPLICA_INFO_FOUR.isLeader());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void sortReplicasValidate() {
|
||||
List<Pair<ReplicaInfo, Row>> validReplicas = new ArrayList<Pair<ReplicaInfo, Row>>() {
|
||||
List<Pair<Replica, Row>> validReplicas = new ArrayList<Pair<Replica, Row>>() {
|
||||
{
|
||||
add(PAIR_ONE);
|
||||
add(PAIR_FOUR);
|
||||
|
@ -78,7 +78,7 @@ public class MoveReplicaSuggesterTest extends SolrTestCaseJ4 {
|
|||
|
||||
@Test
|
||||
public void sortReplicasValidateLeadersMultipleLeadersComeLast() {
|
||||
List<Pair<ReplicaInfo, Row>> validReplicas = new ArrayList<Pair<ReplicaInfo, Row>>() {
|
||||
List<Pair<Replica, Row>> validReplicas = new ArrayList<Pair<Replica, Row>>() {
|
||||
{
|
||||
add(PAIR_THREE);
|
||||
add(PAIR_ONE);
|
||||
|
@ -97,8 +97,8 @@ public class MoveReplicaSuggesterTest extends SolrTestCaseJ4 {
|
|||
assertTrue(isReplicaLeader(validReplicas, 3));
|
||||
}
|
||||
|
||||
private boolean isReplicaLeader(List<Pair<ReplicaInfo, Row>> replicas, int index) {
|
||||
return replicas.get(index).first().isLeader;
|
||||
private boolean isReplicaLeader(List<Pair<Replica, Row>> replicas, int index) {
|
||||
return replicas.get(index).first().isLeader();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -119,11 +119,11 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
|
||||
public static String clusterState = Utils.toJSONString(loadFromResource("testPolicy.json"));
|
||||
|
||||
public static Map<String, Map<String, List<ReplicaInfo>>> getReplicaDetails(String node,
|
||||
public static Map<String, Map<String, List<Replica>>> getReplicaDetails(String node,
|
||||
@SuppressWarnings({"rawtypes"})Map clusterState) {
|
||||
ValidatingJsonMap m = ValidatingJsonMap
|
||||
.getDeepCopy(clusterState, 6, true);
|
||||
Map<String, Map<String, List<ReplicaInfo>>> result = new LinkedHashMap<>();
|
||||
Map<String, Map<String, List<Replica>>> result = new LinkedHashMap<>();
|
||||
|
||||
m.forEach((collName, o) -> {
|
||||
ValidatingJsonMap coll = (ValidatingJsonMap) o;
|
||||
|
@ -133,10 +133,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
ValidatingJsonMap r = (ValidatingJsonMap) o2;
|
||||
String node_name = (String) r.get("node_name");
|
||||
if (!node_name.equals(node)) return;
|
||||
Map<String, List<ReplicaInfo>> shardVsReplicaStats = result.computeIfAbsent(collName, k -> new HashMap<>());
|
||||
List<ReplicaInfo> replicaInfos = shardVsReplicaStats.computeIfAbsent(shard, k -> new ArrayList<>());
|
||||
replicaInfos.add(new ReplicaInfo(replicaName, (String) r.get("core"), collName, shard,
|
||||
Replica.Type.get((String) r.get(ZkStateReader.REPLICA_TYPE)), node, r));
|
||||
Map<String, List<Replica>> shardVsReplicaStats = result.computeIfAbsent(collName, k -> new HashMap<>());
|
||||
List<Replica> replicaInfos = shardVsReplicaStats.computeIfAbsent(shard, k -> new ArrayList<>());
|
||||
replicaInfos.add(new Replica(replicaName, node, collName, shard, (String) r.get("core"),
|
||||
Replica.State.ACTIVE, Replica.Type.get((String) r.get(ZkStateReader.REPLICA_TYPE)), r));
|
||||
});
|
||||
});
|
||||
});
|
||||
|
@ -173,7 +173,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
|
||||
//e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
|
||||
Map<String, Object> result = new HashMap<>();
|
||||
metricsKeyVsTagReplica.forEach((k, v) -> {
|
||||
|
@ -270,7 +270,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
|
||||
//e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
|
||||
Map<String, Object> result = new HashMap<>();
|
||||
metricsKeyVsTagReplica.forEach((k, v) -> {
|
||||
|
@ -365,7 +365,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
|
||||
//e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
|
||||
Map<String, Object> result = new HashMap<>();
|
||||
metricsKeyVsTagReplica.forEach((k, v) -> {
|
||||
|
@ -475,7 +475,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
|
||||
protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
|
||||
//e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
|
||||
Map<String, Object> result = new HashMap<>();
|
||||
metricsKeyVsTagReplica.forEach((k, v) -> {
|
||||
|
@ -854,7 +854,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
|
||||
protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
|
||||
//e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
|
||||
Map<String, Object> result = new HashMap<>();
|
||||
metricsKeyVsTagReplica.forEach((k, v) -> {
|
||||
|
@ -905,7 +905,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
assertEquals(3, violation.getViolatingReplicas().size());
|
||||
Set<String> expected = ImmutableSet.of("r1", "r3", "r5");
|
||||
for (Violation.ReplicaInfoAndErr replicaInfoAndErr : violation.getViolatingReplicas()) {
|
||||
assertTrue(expected.contains(replicaInfoAndErr.replicaInfo.getCore()));
|
||||
assertTrue(expected.contains(replicaInfoAndErr.replicaInfo.getCoreName()));
|
||||
}
|
||||
} else if (violation.node.equals("node5")) {
|
||||
assertEquals(-1, violation.replicaCountDelta.doubleValue(), 0.01);
|
||||
|
@ -1191,8 +1191,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
String name = m3.keySet().iterator().next().toString();
|
||||
m3 = (Map) m3.get(name);
|
||||
Replica.Type type = Replica.Type.get((String) m3.get("type"));
|
||||
l3.set(i, new ReplicaInfo(name, name
|
||||
, coll.toString(), shard.toString(), type, (String) node, m3));
|
||||
l3.set(i, new Replica(name, (String) node, coll.toString(), shard.toString(), name, Replica.State.ACTIVE, type, m3));
|
||||
}
|
||||
});
|
||||
|
||||
|
@ -1239,9 +1238,9 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Map<String, Map<String, List<ReplicaInfo>>> result = (Map<String, Map<String, List<ReplicaInfo>>>) Utils.getObjectByPath(m, false, Arrays.asList("replicaInfo", node));
|
||||
Map<String, Map<String, List<Replica>>> result = (Map<String, Map<String, List<Replica>>>) Utils.getObjectByPath(m, false, Arrays.asList("replicaInfo", node));
|
||||
return result == null ? new HashMap<>() : result;
|
||||
}
|
||||
};
|
||||
|
@ -1481,10 +1480,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
public NodeStateProvider getNodeStateProvider() {
|
||||
return new DelegatingNodeStateProvider(null) {
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Map<String, Map<String, List<ReplicaInfo>>> o = (Map<String, Map<String, List<ReplicaInfo>>>) Utils.fromJSONString("{c1: {s0:[{}]}}");
|
||||
Utils.setObjectByPath(o, "c1/s0[0]", new ReplicaInfo("r0", "c1.s0", "c1", "s0", Replica.Type.NRT, "nodex", new HashMap<>()));
|
||||
Map<String, Map<String, List<Replica>>> o = (Map<String, Map<String, List<Replica>>>) Utils.fromJSONString("{c1: {s0:[{}]}}");
|
||||
Utils.setObjectByPath(o, "c1/s0[0]", new Replica("r0", "nodex", "c1", "s0", "c1.s0", Replica.State.ACTIVE, Replica.Type.NRT, new HashMap<>()));
|
||||
return o;
|
||||
}
|
||||
|
||||
|
@ -1516,8 +1515,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
Row r2 = r1.addReplica("c1", "s1", Replica.Type.NRT);
|
||||
assertEquals(1, r1.collectionVsShardVsReplicas.get("c1").get("s1").size());
|
||||
assertEquals(2, r2.collectionVsShardVsReplicas.get("c1").get("s1").size());
|
||||
assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(0) instanceof ReplicaInfo);
|
||||
assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(1) instanceof ReplicaInfo);
|
||||
assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(0) instanceof Replica);
|
||||
assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(1) instanceof Replica);
|
||||
}
|
||||
|
||||
public void testMerge() {
|
||||
|
@ -2019,8 +2018,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
" {'core_node2':{}}]}}}");
|
||||
Map m = (Map) Utils.getObjectByPath(replicaInfoMap, false, "127.0.0.1:60089_solr/compute_plan_action_test");
|
||||
m.put("shard1", Arrays.asList(
|
||||
new ReplicaInfo("core_node1", "core_node1", "compute_plan_action_test", "shard1", Replica.Type.NRT, "127.0.0.1:60089_solr", Collections.emptyMap()),
|
||||
new ReplicaInfo("core_node2", "core_node2", "compute_plan_action_test", "shard1", Replica.Type.NRT, "127.0.0.1:60089_solr", Collections.emptyMap())));
|
||||
new Replica("core_node1", "127.0.0.1:60089_solr", "compute_plan_action_test", "shard1", "core_node1",
|
||||
Replica.State.ACTIVE, Replica.Type.NRT, Collections.emptyMap()),
|
||||
new Replica("core_node2", "127.0.0.1:60089_solr", "compute_plan_action_test", "shard1", "core_node2",
|
||||
Replica.State.ACTIVE, Replica.Type.NRT, Collections.emptyMap())));
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
Map<String, Map<String, Object>> tagsMap = (Map) Utils.fromJSONString("{" +
|
||||
|
@ -2056,8 +2057,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
|
||||
@Override
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return (Map<String, Map<String, List<ReplicaInfo>>>) replicaInfoMap.get(node);
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return (Map<String, Map<String, List<Replica>>>) replicaInfoMap.get(node);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -2136,7 +2137,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return cloudManager.getNodeStateProvider().getReplicaInfo(node, keys);
|
||||
}
|
||||
};
|
||||
|
@ -2214,7 +2215,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return getReplicaDetails(node, clusterS);
|
||||
}
|
||||
};
|
||||
|
@ -2277,7 +2278,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return getReplicaDetails(node, (Map)Utils.fromJSONString(clusterState));
|
||||
}
|
||||
};
|
||||
|
@ -2351,7 +2352,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return getReplicaDetails(node, (Map)Utils.fromJSONString(clusterState));
|
||||
}
|
||||
};
|
||||
|
@ -2805,18 +2806,16 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
|
||||
@Override
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
if (node.equals("node1")) {
|
||||
Map m = Utils.makeMap("newColl",
|
||||
Utils.makeMap("shard1", Collections.singletonList(new ReplicaInfo("r1", "shard1",
|
||||
new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node1", ZkStateReader.CORE_NAME_PROP, "core1"), "newColl", "shard1"),
|
||||
Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
|
||||
Utils.makeMap("shard1", Collections.singletonList(new Replica("r1", "node1", "newColl", "shard1", "core1",
|
||||
Replica.State.ACTIVE, Replica.Type.NRT, Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
|
||||
return m;
|
||||
} else if (node.equals("node2")) {
|
||||
Map m = Utils.makeMap("newColl",
|
||||
Utils.makeMap("shard2", Collections.singletonList(new ReplicaInfo("r1", "shard2",
|
||||
new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node2", ZkStateReader.CORE_NAME_PROP, "core2"),"newColl", "shard2"),
|
||||
Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
|
||||
Utils.makeMap("shard2", Collections.singletonList(new Replica("r1", "node2", "newColl", "shard2", "core2",
|
||||
Replica.State.ACTIVE, Replica.Type.NRT, Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
|
||||
return m;
|
||||
}
|
||||
return new HashMap<>();
|
||||
|
@ -2857,27 +2856,34 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
|
||||
public void testMoveReplicaLeaderlast() {
|
||||
|
||||
List<Pair<ReplicaInfo, Row>> validReplicas = new ArrayList<>();
|
||||
List<Pair<Replica, Row>> validReplicas = new ArrayList<>();
|
||||
Map<String, Object> propMap = Utils.makeMap(
|
||||
"leader", "true",
|
||||
ZkStateReader.NODE_NAME_PROP, "node1",
|
||||
ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.toString(),
|
||||
ZkStateReader.CORE_NAME_PROP, "core1");
|
||||
Replica replica = new Replica("r1", propMap, "c1", "s1");
|
||||
ReplicaInfo replicaInfo = new ReplicaInfo(replica.collection, replica.slice ,replica, new HashMap<>());
|
||||
Replica replicaInfo = new Replica(replica.name, replica.node, replica.collection, replica.shard, replica.core,
|
||||
replica.getState(), replica.type, replica.getProperties());
|
||||
validReplicas.add(new Pair<>(replicaInfo, null));
|
||||
|
||||
replicaInfo = new ReplicaInfo("r4", "c1_s2_r1", "c1", "s2", Replica.Type.NRT, "n1", Collections.singletonMap("leader", "true"));
|
||||
replicaInfo = new Replica("r4", "n1", "c1_s2_r1", "c1", "s2", Replica.State.ACTIVE, Replica.Type.NRT,
|
||||
Utils.makeMap(ZkStateReader.LEADER_PROP, "true"));
|
||||
validReplicas.add(new Pair<>(replicaInfo, null));
|
||||
|
||||
|
||||
propMap.put("leader", false);
|
||||
propMap.put("core", "r2");
|
||||
propMap.put("node_name", "n1");
|
||||
replica = new Replica("r2", propMap,"c1","s1");
|
||||
replicaInfo = new ReplicaInfo(replica.collection, replica.slice, replica, new HashMap<>());
|
||||
replicaInfo = new Replica(replica.name, replica.node, replica.collection, replica.shard, replica.core,
|
||||
replica.getState(), replica.type, new HashMap<>());
|
||||
validReplicas.add(new Pair<>(replicaInfo, null));
|
||||
|
||||
propMap.put("core", "r2");
|
||||
replica = new Replica("r3", propMap,"c1","s1");
|
||||
replicaInfo = new ReplicaInfo(replica.collection,replica.slice, replica, new HashMap<>());
|
||||
replicaInfo = new Replica(replica.name, replica.node, replica.collection, replica.shard, replica.core,
|
||||
replica.getState(), replica.type, new HashMap<>());
|
||||
validReplicas.add(new Pair<>(replicaInfo, null));
|
||||
|
||||
|
||||
|
@ -3128,8 +3134,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
String name = m3.keySet().iterator().next().toString();
|
||||
m3 = (Map) m3.get(name);
|
||||
Replica.Type type = Replica.Type.get((String) m3.get("type"));
|
||||
l3.set(i, new ReplicaInfo(name, name
|
||||
, coll.toString(), shard.toString(), type, (String) node, m3));
|
||||
l3.set(i, new Replica(name, (String) node, coll.toString(), shard.toString(),
|
||||
name, Replica.State.ACTIVE, type, m3));
|
||||
}
|
||||
});
|
||||
|
||||
|
@ -3181,9 +3187,9 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Map<String, Map<String, List<ReplicaInfo>>> result = (Map<String, Map<String, List<ReplicaInfo>>>) Utils.getObjectByPath(m, false, Arrays.asList("replicaInfo", node));
|
||||
Map<String, Map<String, List<Replica>>> result = (Map<String, Map<String, List<Replica>>>) Utils.getObjectByPath(m, false, Arrays.asList("replicaInfo", node));
|
||||
return result == null ? new HashMap<>() : result;
|
||||
}
|
||||
};
|
||||
|
@ -3262,7 +3268,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
|
|||
|
||||
@Override
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
//return Collections.EMPTY_MAP;
|
||||
return replicaInfoMap;
|
||||
}
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.solr.client.solrj.impl.ClusterStateProvider;
|
|||
import org.apache.solr.client.solrj.impl.SolrClientNodeStateProvider;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.ReplicaPosition;
|
||||
import org.apache.solr.common.util.Utils;
|
||||
import org.junit.Ignore;
|
||||
|
@ -230,17 +231,17 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Map<String, Map<String, List<ReplicaInfo>>> result = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(node, Utils.NEW_HASHMAP_FUN);
|
||||
Map<String, Map<String, List<Replica>>> result = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(node, Utils.NEW_HASHMAP_FUN);
|
||||
if (!keys.isEmpty()) {
|
||||
Row.forEachReplica(result, replicaInfo -> {
|
||||
for (String key : keys) {
|
||||
if (!replicaInfo.getVariables().containsKey(key)) {
|
||||
if (!replicaInfo.getProperties().containsKey(key)) {
|
||||
replicaVals.stream()
|
||||
.filter(it -> replicaInfo.getCore().equals(it.get("core")))
|
||||
.filter(it -> replicaInfo.getCoreName().equals(it.get("core")))
|
||||
.findFirst()
|
||||
.ifPresent(map -> replicaInfo.getVariables().put(key, map.get(key)));
|
||||
.ifPresent(map -> replicaInfo.getProperties().put(key, map.get(key)));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -279,17 +280,17 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
|
|||
Map<String, Map<String, List<Map>>> replicas = (Map<String, Map<String, List<Map>>>) ((Map) o).get("replicas");
|
||||
replicas.forEach((coll, shardVsReplicas) -> shardVsReplicas
|
||||
.forEach((shard, repDetails) -> {
|
||||
List<ReplicaInfo> reps = (List) ((Map) nodeDetails
|
||||
List<Replica> reps = (List) ((Map) nodeDetails
|
||||
.computeIfAbsent(coll, o1 -> new LinkedHashMap<>()))
|
||||
.computeIfAbsent(shard, o12 -> new ArrayList<ReplicaInfo>());
|
||||
for (Map map : repDetails) reps.add(new ReplicaInfo(map));
|
||||
.computeIfAbsent(shard, o12 -> new ArrayList<Replica>());
|
||||
for (Map map : repDetails) reps.add(new Replica(map));
|
||||
}));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
|
||||
return nodeVsCollectionVsShardVsReplicaInfo.get(node) == null ?
|
||||
Collections.emptyMap() :
|
||||
nodeVsCollectionVsShardVsReplicaInfo.get(node);
|
||||
|
|
Loading…
Reference in New Issue