mirror of https://github.com/apache/lucene.git
SOLR-13264: IndexSizeTrigger aboveOp / belowOp properties not in valid properties.
This commit is contained in:
parent
c266044d06
commit
0f10b5f042
|
@ -61,7 +61,8 @@ Optimizations
|
||||||
|
|
||||||
Bug Fixes
|
Bug Fixes
|
||||||
---------------------
|
---------------------
|
||||||
(No changes)
|
* SOLR-13264: IndexSizeTrigger aboveOp / belowOp properties not in valid properties.
|
||||||
|
(Christine Poerschke, ab)
|
||||||
|
|
||||||
Other Changes
|
Other Changes
|
||||||
---------------------
|
---------------------
|
||||||
|
|
|
@ -60,6 +60,7 @@ import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.CORE_
|
||||||
public class IndexSizeTrigger extends TriggerBase {
|
public class IndexSizeTrigger extends TriggerBase {
|
||||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
||||||
|
// configuration properties
|
||||||
public static final String ABOVE_BYTES_PROP = "aboveBytes";
|
public static final String ABOVE_BYTES_PROP = "aboveBytes";
|
||||||
public static final String ABOVE_DOCS_PROP = "aboveDocs";
|
public static final String ABOVE_DOCS_PROP = "aboveDocs";
|
||||||
public static final String ABOVE_OP_PROP = "aboveOp";
|
public static final String ABOVE_OP_PROP = "aboveOp";
|
||||||
|
@ -72,14 +73,15 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
public static final String SPLIT_METHOD_PROP = CommonAdminParams.SPLIT_METHOD;
|
public static final String SPLIT_METHOD_PROP = CommonAdminParams.SPLIT_METHOD;
|
||||||
public static final String SPLIT_BY_PREFIX = CommonAdminParams.SPLIT_BY_PREFIX;
|
public static final String SPLIT_BY_PREFIX = CommonAdminParams.SPLIT_BY_PREFIX;
|
||||||
|
|
||||||
public static final String BYTES_SIZE_PROP = "__bytes__";
|
// event properties
|
||||||
public static final String TOTAL_BYTES_SIZE_PROP = "__total_bytes__";
|
public static final String BYTES_SIZE_KEY = "__bytes__";
|
||||||
public static final String DOCS_SIZE_PROP = "__docs__";
|
public static final String TOTAL_BYTES_SIZE_KEY = "__total_bytes__";
|
||||||
public static final String MAX_DOC_PROP = "__maxDoc__";
|
public static final String DOCS_SIZE_KEY = "__docs__";
|
||||||
public static final String COMMIT_SIZE_PROP = "__commitBytes__";
|
public static final String MAX_DOC_KEY = "__maxDoc__";
|
||||||
public static final String ABOVE_SIZE_PROP = "aboveSize";
|
public static final String COMMIT_SIZE_KEY = "__commitBytes__";
|
||||||
public static final String BELOW_SIZE_PROP = "belowSize";
|
public static final String ABOVE_SIZE_KEY = "aboveSize";
|
||||||
public static final String VIOLATION_PROP = "violationType";
|
public static final String BELOW_SIZE_KEY = "belowSize";
|
||||||
|
public static final String VIOLATION_KEY = "violationType";
|
||||||
|
|
||||||
public static final int DEFAULT_MAX_OPS = 10;
|
public static final int DEFAULT_MAX_OPS = 10;
|
||||||
|
|
||||||
|
@ -98,8 +100,10 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
public IndexSizeTrigger(String name) {
|
public IndexSizeTrigger(String name) {
|
||||||
super(TriggerEventType.INDEXSIZE, name);
|
super(TriggerEventType.INDEXSIZE, name);
|
||||||
TriggerUtils.validProperties(validProperties,
|
TriggerUtils.validProperties(validProperties,
|
||||||
ABOVE_BYTES_PROP, ABOVE_DOCS_PROP, BELOW_BYTES_PROP, BELOW_DOCS_PROP,
|
ABOVE_BYTES_PROP, ABOVE_DOCS_PROP, ABOVE_OP_PROP,
|
||||||
COLLECTIONS_PROP, MAX_OPS_PROP, SPLIT_METHOD_PROP, SPLIT_FUZZ_PROP, SPLIT_BY_PREFIX);
|
BELOW_BYTES_PROP, BELOW_DOCS_PROP, BELOW_OP_PROP,
|
||||||
|
COLLECTIONS_PROP, MAX_OPS_PROP,
|
||||||
|
SPLIT_METHOD_PROP, SPLIT_FUZZ_PROP, SPLIT_BY_PREFIX);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -331,13 +335,13 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
|
|
||||||
ReplicaInfo currentInfo = currentSizes.computeIfAbsent(info.getCore(), k -> (ReplicaInfo)info.clone());
|
ReplicaInfo currentInfo = currentSizes.computeIfAbsent(info.getCore(), k -> (ReplicaInfo)info.clone());
|
||||||
if (tag.contains("INDEX")) {
|
if (tag.contains("INDEX")) {
|
||||||
currentInfo.getVariables().put(TOTAL_BYTES_SIZE_PROP, ((Number) size).longValue());
|
currentInfo.getVariables().put(TOTAL_BYTES_SIZE_KEY, ((Number) size).longValue());
|
||||||
} else if (tag.endsWith("SEARCHER.searcher.numDocs")) {
|
} else if (tag.endsWith("SEARCHER.searcher.numDocs")) {
|
||||||
currentInfo.getVariables().put(DOCS_SIZE_PROP, ((Number) size).longValue());
|
currentInfo.getVariables().put(DOCS_SIZE_KEY, ((Number) size).longValue());
|
||||||
} else if (tag.endsWith("SEARCHER.searcher.maxDoc")) {
|
} else if (tag.endsWith("SEARCHER.searcher.maxDoc")) {
|
||||||
currentInfo.getVariables().put(MAX_DOC_PROP, ((Number) size).longValue());
|
currentInfo.getVariables().put(MAX_DOC_KEY, ((Number) size).longValue());
|
||||||
} else if (tag.endsWith("SEARCHER.searcher.indexCommitSize")) {
|
} else if (tag.endsWith("SEARCHER.searcher.indexCommitSize")) {
|
||||||
currentInfo.getVariables().put(COMMIT_SIZE_PROP, ((Number) size).longValue());
|
currentInfo.getVariables().put(COMMIT_SIZE_KEY, ((Number) size).longValue());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -358,25 +362,25 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
|
|
||||||
currentSizes.forEach((coreName, info) -> {
|
currentSizes.forEach((coreName, info) -> {
|
||||||
// calculate estimated bytes
|
// calculate estimated bytes
|
||||||
long maxDoc = (Long)info.getVariable(MAX_DOC_PROP);
|
long maxDoc = (Long)info.getVariable(MAX_DOC_KEY);
|
||||||
long numDocs = (Long)info.getVariable(DOCS_SIZE_PROP);
|
long numDocs = (Long)info.getVariable(DOCS_SIZE_KEY);
|
||||||
long commitSize = (Long)info.getVariable(COMMIT_SIZE_PROP, 0L);
|
long commitSize = (Long)info.getVariable(COMMIT_SIZE_KEY, 0L);
|
||||||
if (commitSize <= 0) {
|
if (commitSize <= 0) {
|
||||||
commitSize = (Long)info.getVariable(TOTAL_BYTES_SIZE_PROP);
|
commitSize = (Long)info.getVariable(TOTAL_BYTES_SIZE_KEY);
|
||||||
}
|
}
|
||||||
// calculate estimated size as a side-effect
|
// calculate estimated size as a side-effect
|
||||||
commitSize = estimatedSize(maxDoc, numDocs, commitSize);
|
commitSize = estimatedSize(maxDoc, numDocs, commitSize);
|
||||||
info.getVariables().put(BYTES_SIZE_PROP, commitSize);
|
info.getVariables().put(BYTES_SIZE_KEY, commitSize);
|
||||||
|
|
||||||
if ((Long)info.getVariable(BYTES_SIZE_PROP) > aboveBytes ||
|
if ((Long)info.getVariable(BYTES_SIZE_KEY) > aboveBytes ||
|
||||||
(Long)info.getVariable(DOCS_SIZE_PROP) > aboveDocs) {
|
(Long)info.getVariable(DOCS_SIZE_KEY) > aboveDocs) {
|
||||||
if (waitForElapsed(coreName, now, lastAboveEventMap)) {
|
if (waitForElapsed(coreName, now, lastAboveEventMap)) {
|
||||||
List<ReplicaInfo> infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
List<ReplicaInfo> infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
||||||
if (!infos.contains(info)) {
|
if (!infos.contains(info)) {
|
||||||
if ((Long)info.getVariable(BYTES_SIZE_PROP) > aboveBytes) {
|
if ((Long)info.getVariable(BYTES_SIZE_KEY) > aboveBytes) {
|
||||||
info.getVariables().put(VIOLATION_PROP, ABOVE_BYTES_PROP);
|
info.getVariables().put(VIOLATION_KEY, ABOVE_BYTES_PROP);
|
||||||
} else {
|
} else {
|
||||||
info.getVariables().put(VIOLATION_PROP, ABOVE_DOCS_PROP);
|
info.getVariables().put(VIOLATION_KEY, ABOVE_DOCS_PROP);
|
||||||
}
|
}
|
||||||
infos.add(info);
|
infos.add(info);
|
||||||
splittable.add(info.getName());
|
splittable.add(info.getName());
|
||||||
|
@ -392,17 +396,17 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
Map<String, List<ReplicaInfo>> belowSize = new HashMap<>();
|
Map<String, List<ReplicaInfo>> belowSize = new HashMap<>();
|
||||||
|
|
||||||
currentSizes.forEach((coreName, info) -> {
|
currentSizes.forEach((coreName, info) -> {
|
||||||
if (((Long)info.getVariable(BYTES_SIZE_PROP) < belowBytes ||
|
if (((Long)info.getVariable(BYTES_SIZE_KEY) < belowBytes ||
|
||||||
(Long)info.getVariable(DOCS_SIZE_PROP) < belowDocs) &&
|
(Long)info.getVariable(DOCS_SIZE_KEY) < belowDocs) &&
|
||||||
// make sure we don't produce conflicting ops
|
// make sure we don't produce conflicting ops
|
||||||
!splittable.contains(info.getName())) {
|
!splittable.contains(info.getName())) {
|
||||||
if (waitForElapsed(coreName, now, lastBelowEventMap)) {
|
if (waitForElapsed(coreName, now, lastBelowEventMap)) {
|
||||||
List<ReplicaInfo> infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
List<ReplicaInfo> infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
|
||||||
if (!infos.contains(info)) {
|
if (!infos.contains(info)) {
|
||||||
if ((Long)info.getVariable(BYTES_SIZE_PROP) < belowBytes) {
|
if ((Long)info.getVariable(BYTES_SIZE_KEY) < belowBytes) {
|
||||||
info.getVariables().put(VIOLATION_PROP, BELOW_BYTES_PROP);
|
info.getVariables().put(VIOLATION_KEY, BELOW_BYTES_PROP);
|
||||||
} else {
|
} else {
|
||||||
info.getVariables().put(VIOLATION_PROP, BELOW_DOCS_PROP);
|
info.getVariables().put(VIOLATION_KEY, BELOW_DOCS_PROP);
|
||||||
}
|
}
|
||||||
infos.add(info);
|
infos.add(info);
|
||||||
}
|
}
|
||||||
|
@ -429,7 +433,7 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
// sort by decreasing size to first split the largest ones
|
// 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
|
// XXX see the comment below about using DOCS_SIZE_PROP in lieu of BYTES_SIZE_PROP
|
||||||
replicas.sort((r1, r2) -> {
|
replicas.sort((r1, r2) -> {
|
||||||
long delta = (Long) r1.getVariable(DOCS_SIZE_PROP) - (Long) r2.getVariable(DOCS_SIZE_PROP);
|
long delta = (Long) r1.getVariable(DOCS_SIZE_KEY) - (Long) r2.getVariable(DOCS_SIZE_KEY);
|
||||||
if (delta > 0) {
|
if (delta > 0) {
|
||||||
return -1;
|
return -1;
|
||||||
} else if (delta < 0) {
|
} else if (delta < 0) {
|
||||||
|
@ -471,7 +475,7 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
// then we should be sorting by BYTES_SIZE_PROP. However, since DOCS and BYTES are
|
// 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"
|
// 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)
|
// condition than index size, due to possibly existing deleted docs that still occupy space)
|
||||||
long delta = (Long) r1.getVariable(DOCS_SIZE_PROP) - (Long) r2.getVariable(DOCS_SIZE_PROP);
|
long delta = (Long) r1.getVariable(DOCS_SIZE_KEY) - (Long) r2.getVariable(DOCS_SIZE_KEY);
|
||||||
if (delta > 0) {
|
if (delta > 0) {
|
||||||
return 1;
|
return 1;
|
||||||
} else if (delta < 0) {
|
} else if (delta < 0) {
|
||||||
|
@ -545,12 +549,12 @@ public class IndexSizeTrigger extends TriggerBase {
|
||||||
// avoid passing very large amounts of data here - just use replica names
|
// avoid passing very large amounts of data here - just use replica names
|
||||||
TreeMap<String, String> above = new TreeMap<>();
|
TreeMap<String, String> above = new TreeMap<>();
|
||||||
aboveSize.forEach((coll, replicas) ->
|
aboveSize.forEach((coll, replicas) ->
|
||||||
replicas.forEach(r -> above.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_PROP) + ", bytes=" + r.getVariable(BYTES_SIZE_PROP))));
|
replicas.forEach(r -> above.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_KEY) + ", bytes=" + r.getVariable(BYTES_SIZE_KEY))));
|
||||||
properties.put(ABOVE_SIZE_PROP, above);
|
properties.put(ABOVE_SIZE_KEY, above);
|
||||||
TreeMap<String, String> below = new TreeMap<>();
|
TreeMap<String, String> below = new TreeMap<>();
|
||||||
belowSize.forEach((coll, replicas) ->
|
belowSize.forEach((coll, replicas) ->
|
||||||
replicas.forEach(r -> below.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_PROP) + ", bytes=" + r.getVariable(BYTES_SIZE_PROP))));
|
replicas.forEach(r -> below.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_KEY) + ", bytes=" + r.getVariable(BYTES_SIZE_KEY))));
|
||||||
properties.put(BELOW_SIZE_PROP, below);
|
properties.put(BELOW_SIZE_KEY, below);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -81,10 +81,24 @@ public abstract class TriggerBase implements AutoScaling.Trigger {
|
||||||
this.eventType = eventType;
|
this.eventType = eventType;
|
||||||
this.name = name;
|
this.name = name;
|
||||||
|
|
||||||
// subclasses may modify this set to include other supported properties
|
// subclasses may further modify this set to include other supported properties
|
||||||
TriggerUtils.validProperties(validProperties, "name", "class", "event", "enabled", "waitFor", "actions");
|
TriggerUtils.validProperties(validProperties, "name", "class", "event", "enabled", "waitFor", "actions");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return a set of valid property names supported by this trigger.
|
||||||
|
*/
|
||||||
|
public final Set<String> getValidProperties() {
|
||||||
|
return Collections.unmodifiableSet(this.validProperties);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return a set of required property names supported by this trigger.
|
||||||
|
*/
|
||||||
|
public final Set<String> getRequiredProperties() {
|
||||||
|
return Collections.unmodifiableSet(this.requiredProperties);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
|
public void configure(SolrResourceLoader loader, SolrCloudManager cloudManager, Map<String, Object> properties) throws TriggerValidationException {
|
||||||
this.cloudManager = cloudManager;
|
this.cloudManager = cloudManager;
|
||||||
|
|
|
@ -20,8 +20,10 @@ package org.apache.solr.cloud.autoscaling;
|
||||||
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
|
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
|
import java.lang.reflect.Field;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -737,6 +739,23 @@ public class IndexSizeTriggerTest extends SolrCloudTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// make sure all defined properties are added to valid properties (SOLR-13264)
|
||||||
|
@Test
|
||||||
|
public void testValidProperties() throws Exception {
|
||||||
|
|
||||||
|
final Set<String> propFields = new HashSet<>();
|
||||||
|
|
||||||
|
final TriggerBase trigger = new IndexSizeTrigger("index_size_trigger");
|
||||||
|
for (final Field field : trigger.getClass().getFields()) {
|
||||||
|
if (field.getName().endsWith("_PROP")) {
|
||||||
|
propFields.add(field.get(trigger).toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
propFields.removeAll(trigger.getValidProperties());
|
||||||
|
|
||||||
|
assertTrue("Invalid _PROP constants: "+propFields.toString(), propFields.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
private Map<String, Object> createTriggerProps(long waitForSeconds) {
|
private Map<String, Object> createTriggerProps(long waitForSeconds) {
|
||||||
Map<String, Object> props = new HashMap<>();
|
Map<String, Object> props = new HashMap<>();
|
||||||
props.put("event", "indexSize");
|
props.put("event", "indexSize");
|
||||||
|
|
Loading…
Reference in New Issue