From 5fb800f01819a2bfcebf8ba04fb1fd7d28ba6b23 Mon Sep 17 00:00:00 2001
From: Adrien Grand
Date: Mon, 7 Aug 2017 13:17:53 +0200
Subject: [PATCH 01/26] LUCENE-7655: Speed up geo-distance queries that match
most documents.
Closes #226
---
lucene/CHANGES.txt | 3 +
.../document/LatLonPointDistanceQuery.java | 232 +++++++++++++-----
.../lucene/search/TestLatLonPointQueries.java | 37 ++-
3 files changed, 205 insertions(+), 67 deletions(-)
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c0f263c9fd4..d6083b91317 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -21,6 +21,9 @@ Optimizations
SortedSetDocValuesFacetCounts and others) builds its map (Robert
Muir, Adrien Grand, Mike McCandless)
+* LUCENE-7655: Speed up geo-distance queries in case of dense single-valued
+ fields when most documents match. (Maciej Zasada via Adrien Grand)
+
Bug Fixes
* LUCENE-7914: Add a maximum recursion level in automaton recursive
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
index 71ddf3d1fdb..f48c816b101 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
@@ -29,12 +29,15 @@ import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.search.ConstantScoreScorer;
import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.ScorerSupplier;
import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BitSetIterator;
import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.StringHelper;
@@ -128,75 +131,11 @@ final class LatLonPointDistanceQuery extends Query {
return null;
}
LatLonPoint.checkCompatible(fieldInfo);
-
+
// matching docids
DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
- final IntersectVisitor visitor =
- new IntersectVisitor() {
+ final IntersectVisitor visitor = getIntersectVisitor(result);
- DocIdSetBuilder.BulkAdder adder;
-
- @Override
- public void grow(int count) {
- adder = result.grow(count);
- }
-
- @Override
- public void visit(int docID) {
- adder.add(docID);
- }
-
- @Override
- public void visit(int docID, byte[] packedValue) {
- // bounding box check
- if (StringHelper.compare(Integer.BYTES, packedValue, 0, maxLat, 0) > 0 ||
- StringHelper.compare(Integer.BYTES, packedValue, 0, minLat, 0) < 0) {
- // latitude out of bounding box range
- return;
- }
-
- if ((StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, maxLon, 0) > 0 ||
- StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, minLon, 0) < 0)
- && StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, minLon2, 0) < 0) {
- // longitude out of bounding box range
- return;
- }
-
- int docLatitude = NumericUtils.sortableBytesToInt(packedValue, 0);
- int docLongitude = NumericUtils.sortableBytesToInt(packedValue, Integer.BYTES);
- if (distancePredicate.test(docLatitude, docLongitude)) {
- adder.add(docID);
- }
- }
-
- // algorithm: we create a bounding box (two bounding boxes if we cross the dateline).
- // 1. check our bounding box(es) first. if the subtree is entirely outside of those, bail.
- // 2. check if the subtree is disjoint. it may cross the bounding box but not intersect with circle
- // 3. see if the subtree is fully contained. if the subtree is enormous along the x axis, wrapping half way around the world, etc: then this can't work, just go to step 4.
- // 4. recurse naively (subtrees crossing over circle edge)
- @Override
- public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
- if (StringHelper.compare(Integer.BYTES, minPackedValue, 0, maxLat, 0) > 0 ||
- StringHelper.compare(Integer.BYTES, maxPackedValue, 0, minLat, 0) < 0) {
- // latitude out of bounding box range
- return Relation.CELL_OUTSIDE_QUERY;
- }
-
- if ((StringHelper.compare(Integer.BYTES, minPackedValue, Integer.BYTES, maxLon, 0) > 0 ||
- StringHelper.compare(Integer.BYTES, maxPackedValue, Integer.BYTES, minLon, 0) < 0)
- && StringHelper.compare(Integer.BYTES, maxPackedValue, Integer.BYTES, minLon2, 0) < 0) {
- // longitude out of bounding box range
- return Relation.CELL_OUTSIDE_QUERY;
- }
-
- double latMin = decodeLatitude(minPackedValue, 0);
- double lonMin = decodeLongitude(minPackedValue, Integer.BYTES);
- double latMax = decodeLatitude(maxPackedValue, 0);
- double lonMax = decodeLongitude(maxPackedValue, Integer.BYTES);
-
- return GeoUtils.relate(latMin, latMax, lonMin, lonMax, latitude, longitude, sortKey, axisLat);
- }
- };
final Weight weight = this;
return new ScorerSupplier() {
@@ -204,6 +143,19 @@ final class LatLonPointDistanceQuery extends Query {
@Override
public Scorer get(boolean randomAccess) throws IOException {
+ if (values.getDocCount() == reader.maxDoc()
+ && values.getDocCount() == values.size()
+ && cost() > reader.maxDoc() / 2) {
+ // If all docs have exactly one value and the cost is greater
+ // than half the leaf size then maybe we can make things faster
+ // by computing the set of documents that do NOT match the range
+ final FixedBitSet result = new FixedBitSet(reader.maxDoc());
+ result.set(0, reader.maxDoc());
+ int[] cost = new int[]{reader.maxDoc()};
+ values.intersect(getInverseIntersectVisitor(result, cost));
+ final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
+ return new ConstantScoreScorer(weight, score(), iterator);
+ }
values.intersect(visitor);
return new ConstantScoreScorer(weight, score(), result.build().iterator());
}
@@ -219,6 +171,154 @@ final class LatLonPointDistanceQuery extends Query {
};
}
+
+ /**
+ * Create a visitor that collects documents matching the range.
+ */
+ private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result) {
+ return new IntersectVisitor() {
+
+ DocIdSetBuilder.BulkAdder adder;
+
+ @Override
+ public void grow(int count) {
+ adder = result.grow(count);
+ }
+
+ @Override
+ public void visit(int docID) {
+ adder.add(docID);
+ }
+
+ @Override
+ public void visit(int docID, byte[] packedValue) {
+ // bounding box check
+ if (StringHelper.compare(Integer.BYTES, packedValue, 0, maxLat, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, packedValue, 0, minLat, 0) < 0) {
+ // latitude out of bounding box range
+ return;
+ }
+
+ if ((StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, maxLon, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, minLon, 0) < 0)
+ && StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, minLon2, 0) < 0) {
+ // longitude out of bounding box range
+ return;
+ }
+
+ int docLatitude = NumericUtils.sortableBytesToInt(packedValue, 0);
+ int docLongitude = NumericUtils.sortableBytesToInt(packedValue, Integer.BYTES);
+ if (distancePredicate.test(docLatitude, docLongitude)) {
+ adder.add(docID);
+ }
+ }
+
+ // algorithm: we create a bounding box (two bounding boxes if we cross the dateline).
+ // 1. check our bounding box(es) first. if the subtree is entirely outside of those, bail.
+ // 2. check if the subtree is disjoint. it may cross the bounding box but not intersect with circle
+ // 3. see if the subtree is fully contained. if the subtree is enormous along the x axis, wrapping half way around the world, etc: then this can't work, just go to step 4.
+ // 4. recurse naively (subtrees crossing over circle edge)
+ @Override
+ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+ if (StringHelper.compare(Integer.BYTES, minPackedValue, 0, maxLat, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, maxPackedValue, 0, minLat, 0) < 0) {
+ // latitude out of bounding box range
+ return Relation.CELL_OUTSIDE_QUERY;
+ }
+
+ if ((StringHelper.compare(Integer.BYTES, minPackedValue, Integer.BYTES, maxLon, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, maxPackedValue, Integer.BYTES, minLon, 0) < 0)
+ && StringHelper.compare(Integer.BYTES, maxPackedValue, Integer.BYTES, minLon2, 0) < 0) {
+ // longitude out of bounding box range
+ return Relation.CELL_OUTSIDE_QUERY;
+ }
+
+ double latMin = decodeLatitude(minPackedValue, 0);
+ double lonMin = decodeLongitude(minPackedValue, Integer.BYTES);
+ double latMax = decodeLatitude(maxPackedValue, 0);
+ double lonMax = decodeLongitude(maxPackedValue, Integer.BYTES);
+
+ return GeoUtils.relate(latMin, latMax, lonMin, lonMax, latitude, longitude, sortKey, axisLat);
+ }
+ };
+ }
+
+ /**
+ * Create a visitor that clears documents that do NOT match the range.
+ */
+ private IntersectVisitor getInverseIntersectVisitor(FixedBitSet result, int[] cost) {
+ return new IntersectVisitor() {
+
+ @Override
+ public void visit(int docID) {
+ result.clear(docID);
+ cost[0]--;
+ }
+
+ @Override
+ public void visit(int docID, byte[] packedValue) {
+ // bounding box check
+ if (StringHelper.compare(Integer.BYTES, packedValue, 0, maxLat, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, packedValue, 0, minLat, 0) < 0) {
+ // latitude out of bounding box range
+ result.clear(docID);
+ cost[0]--;
+ return;
+ }
+
+ if ((StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, maxLon, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, minLon, 0) < 0)
+ && StringHelper.compare(Integer.BYTES, packedValue, Integer.BYTES, minLon2, 0) < 0) {
+ // longitude out of bounding box range
+ result.clear(docID);
+ cost[0]--;
+ return;
+ }
+
+ int docLatitude = NumericUtils.sortableBytesToInt(packedValue, 0);
+ int docLongitude = NumericUtils.sortableBytesToInt(packedValue, Integer.BYTES);
+ if (!distancePredicate.test(docLatitude, docLongitude)) {
+ result.clear(docID);
+ cost[0]--;
+ }
+ }
+
+ @Override
+ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+ if (StringHelper.compare(Integer.BYTES, minPackedValue, 0, maxLat, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, maxPackedValue, 0, minLat, 0) < 0) {
+ // latitude out of bounding box range
+ return Relation.CELL_INSIDE_QUERY;
+ }
+
+ if ((StringHelper.compare(Integer.BYTES, minPackedValue, Integer.BYTES, maxLon, 0) > 0 ||
+ StringHelper.compare(Integer.BYTES, maxPackedValue, Integer.BYTES, minLon, 0) < 0)
+ && StringHelper.compare(Integer.BYTES, maxPackedValue, Integer.BYTES, minLon2, 0) < 0) {
+ // latitude out of bounding box range
+ return Relation.CELL_INSIDE_QUERY;
+ }
+
+ double latMin = decodeLatitude(minPackedValue, 0);
+ double lonMin = decodeLongitude(minPackedValue, Integer.BYTES);
+ double latMax = decodeLatitude(maxPackedValue, 0);
+ double lonMax = decodeLongitude(maxPackedValue, Integer.BYTES);
+
+ Relation relation = GeoUtils.relate(latMin, latMax, lonMin, lonMax, latitude, longitude, sortKey, axisLat);
+ switch (relation) {
+ case CELL_INSIDE_QUERY:
+ // all points match, skip this subtree
+ return Relation.CELL_OUTSIDE_QUERY;
+ case CELL_OUTSIDE_QUERY:
+ // none of the points match, clear all documents
+ return Relation.CELL_INSIDE_QUERY;
+ default:
+ return relation;
+ }
+ }
+
+ };
+ }
+
};
}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
index 39c7a045832..f0e96124e21 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
@@ -16,11 +16,18 @@
*/
package org.apache.lucene.search;
+import java.io.IOException;
+
import org.apache.lucene.document.Document;
import org.apache.lucene.document.LatLonPoint;
import org.apache.lucene.geo.BaseGeoPointTestCase;
-import org.apache.lucene.geo.Polygon;
import org.apache.lucene.geo.GeoEncodingUtils;
+import org.apache.lucene.geo.Polygon;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.bkd.BKDWriter;
public class TestLatLonPointQueries extends BaseGeoPointTestCase {
@@ -53,4 +60,32 @@ public class TestLatLonPointQueries extends BaseGeoPointTestCase {
protected double quantizeLon(double lonRaw) {
return GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(lonRaw));
}
+
+ public void testDistanceQueryWithInvertedIntersection() throws IOException {
+ final int numMatchingDocs = atLeast(10 * BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE);
+
+ try (Directory dir = newDirectory()) {
+
+ try (IndexWriter w = new IndexWriter(dir, newIndexWriterConfig())) {
+ for (int i = 0; i < numMatchingDocs; ++i) {
+ Document doc = new Document();
+ addPointToDoc("field", doc, 18.313694, -65.227444);
+ w.addDocument(doc);
+ }
+
+ // Add a handful of docs that don't match
+ for (int i = 0; i < 11; ++i) {
+ Document doc = new Document();
+ addPointToDoc("field", doc, 10, -65.227444);
+ w.addDocument(doc);
+ }
+ w.forceMerge(1);
+ }
+
+ try (IndexReader r = DirectoryReader.open(dir)) {
+ IndexSearcher searcher = newSearcher(r);
+ assertEquals(numMatchingDocs, searcher.count(newDistanceQuery("field", 18, -65, 50_000)));
+ }
+ }
+ }
}
From bd5c09b1eeb61123f3c799fa6428f2202e6d9356 Mon Sep 17 00:00:00 2001
From: yonik
Date: Mon, 7 Aug 2017 12:29:28 -0400
Subject: [PATCH 02/26] SOLR-10939: add point support to join query
---
solr/CHANGES.txt | 3 +
.../apache/solr/search/JoinQParserPlugin.java | 30 +++
.../search/join/GraphPointsCollector.java | 122 ++++++++++
.../apache/solr/search/join/GraphQuery.java | 35 ++-
.../solr/search/join/GraphTermsCollector.java | 215 ++++--------------
.../src/test/org/apache/solr/TestJoin.java | 13 +-
.../solr/search/facet/TestJsonFacets.java | 6 +-
.../solr/search/join/GraphQueryTest.java | 31 +--
8 files changed, 252 insertions(+), 203 deletions(-)
create mode 100644 solr/core/src/java/org/apache/solr/search/join/GraphPointsCollector.java
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 85d78023962..d39a7f67185 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -343,6 +343,9 @@ New Features
* SOLR-10845: Add support for PointFields to {!graphTerms} query that is internally
used by some graph traversal streaming expressions. (yonik)
+* SOLR-10939: Add support for PointsFields to {!join} query. Joined fields should
+ also have docValues enabled. (yonik)
+
Bug Fixes
----------------------
* SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.
diff --git a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
index e4e92cfec62..7afdb5ffccc 100644
--- a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
@@ -52,7 +52,9 @@ import org.apache.solr.handler.component.ResponseBuilder;
import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.schema.SchemaField;
import org.apache.solr.schema.TrieField;
+import org.apache.solr.search.join.GraphPointsCollector;
import org.apache.solr.search.join.ScoreJoinQParserPlugin;
import org.apache.solr.util.RTimer;
import org.apache.solr.util.RefCounted;
@@ -281,6 +283,7 @@ class JoinQuery extends Query {
}
+ // most of these statistics are only used for the enum method
int fromSetSize; // number of docs in the fromSet (that match the from query)
long resultListDocs; // total number of docs collected
int fromTermCount;
@@ -295,6 +298,33 @@ class JoinQuery extends Query {
public DocSet getDocSet() throws IOException {
+ SchemaField fromSchemaField = fromSearcher.getSchema().getField(fromField);
+ SchemaField toSchemaField = toSearcher.getSchema().getField(toField);
+
+ boolean usePoints = false;
+ if (toSchemaField.getType().isPointField()) {
+ if (!fromSchemaField.hasDocValues()) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "join from field " + fromSchemaField + " should have docValues to join with points field " + toSchemaField);
+ }
+ usePoints = true;
+ }
+
+ if (!usePoints) {
+ return getDocSetEnumerate();
+ }
+
+ // point fields
+ GraphPointsCollector collector = new GraphPointsCollector(fromSchemaField, null, null);
+ fromSearcher.search(q, collector);
+ Query resultQ = collector.getResultQuery(toSchemaField, false);
+ // don't cache the resulting docSet... the query may be very large. Better to cache the results of the join query itself
+ DocSet result = resultQ==null ? DocSet.EMPTY : toSearcher.getDocSetNC(resultQ, null);
+ return result;
+ }
+
+
+
+ public DocSet getDocSetEnumerate() throws IOException {
FixedBitSet resultBits = null;
// minimum docFreq to use the cache
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphPointsCollector.java b/solr/core/src/java/org/apache/solr/search/join/GraphPointsCollector.java
new file mode 100644
index 00000000000..290136a13d1
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphPointsCollector.java
@@ -0,0 +1,122 @@
+/*
+ * 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.search.join;
+
+import java.io.IOException;
+
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.solr.schema.NumberType;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocSet;
+import org.apache.solr.util.LongIterator;
+import org.apache.solr.util.LongSet;
+
+/** @lucene.internal */
+public class GraphPointsCollector extends GraphEdgeCollector {
+ final LongSet set = new LongSet(256);
+
+ SortedNumericDocValues values = null;
+
+ public GraphPointsCollector(SchemaField collectField, DocSet skipSet, DocSet leafNodes) {
+ super(collectField, skipSet, leafNodes);
+ }
+
+ @Override
+ public void doSetNextReader(LeafReaderContext context) throws IOException {
+ super.doSetNextReader(context);
+ values = DocValues.getSortedNumeric(context.reader(), collectField.getName());
+ }
+
+ @Override
+ void addEdgeIdsToResult(int doc) throws IOException {
+ // set the doc to pull the edges ids for.
+ int valuesDoc = values.docID();
+ if (valuesDoc < doc) {
+ valuesDoc = values.advance(doc);
+ }
+ if (valuesDoc == doc) {
+ int count = values.docValueCount();
+ for (int i = 0; i < count; i++) {
+ long v = values.nextValue();
+ set.add(v);
+ }
+ }
+ }
+
+ @Override
+ public Query getResultQuery(SchemaField matchField, boolean useAutomaton) {
+ if (set.cardinality() == 0) return null;
+
+ Query q = null;
+
+ // How we interpret the longs collected depends on the field we collect from (single valued can be diff from multi valued)
+ // The basic type of the from & to field must match though (int/long/float/double)
+ NumberType ntype = collectField.getType().getNumberType();
+ boolean multiValued = collectField.multiValued();
+
+ if (ntype == NumberType.LONG || ntype == NumberType.DATE) {
+ long[] vals = new long[set.cardinality()];
+ int i = 0;
+ for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
+ long bits = iter.next();
+ long v = bits;
+ vals[i++] = v;
+ }
+ q = LongPoint.newSetQuery(matchField.getName(), vals);
+ } else if (ntype == NumberType.INTEGER) {
+ int[] vals = new int[set.cardinality()];
+ int i = 0;
+ for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
+ long bits = iter.next();
+ int v = (int)bits;
+ vals[i++] = v;
+ }
+ q = IntPoint.newSetQuery(matchField.getName(), vals);
+ } else if (ntype == NumberType.DOUBLE) {
+ double[] vals = new double[set.cardinality()];
+ int i = 0;
+ for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
+ long bits = iter.next();
+ double v = multiValued ? NumericUtils.sortableLongToDouble(bits) : Double.longBitsToDouble(bits);
+ vals[i++] = v;
+ }
+ q = DoublePoint.newSetQuery(matchField.getName(), vals);
+ } else if (ntype == NumberType.FLOAT) {
+ float[] vals = new float[set.cardinality()];
+ int i = 0;
+ for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
+ long bits = iter.next();
+ float v = multiValued ? NumericUtils.sortableIntToFloat((int) bits) : Float.intBitsToFloat((int) bits);
+ vals[i++] = v;
+ }
+ q = FloatPoint.newSetQuery(matchField.getName(), vals);
+ }
+
+ return q;
+ }
+
+
+}
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
index 7e52f0c9edb..f63a6644205 100644
--- a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java
@@ -25,6 +25,7 @@ import java.util.TreeSet;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DocIdSet;
@@ -133,15 +134,15 @@ public class GraphQuery extends Query {
private int currentDepth = -1;
private Filter filter;
private DocSet resultSet;
- SchemaField fromSchemaField;
- SchemaField toSchemaField;
+ SchemaField collectSchemaField; // the field to collect values from
+ SchemaField matchSchemaField; // the field to match those values
public GraphQueryWeight(SolrIndexSearcher searcher, float boost) {
// Grab the searcher so we can run additional searches.
super(null);
this.fromSearcher = searcher;
- this.fromSchemaField = searcher.getSchema().getField(fromField);
- this.toSchemaField = searcher.getSchema().getField(toField);
+ this.matchSchemaField = searcher.getSchema().getField(fromField);
+ this.collectSchemaField = searcher.getSchema().getField(toField);
}
GraphQuery getGraphQuery() {
@@ -196,13 +197,25 @@ public class GraphQuery extends Query {
} else {
// when we're not at the max depth level, we need to collect edges
// Create the graph result collector for this level
- GraphEdgeCollector graphResultCollector = toSchemaField.getType().isPointField()
- ? new GraphPointsCollector(this, capacity, resultBits, leafNodes)
- : new GraphTermsCollector(this, capacity, resultBits, leafNodes);
+ GraphEdgeCollector graphResultCollector = collectSchemaField.getType().isPointField()
+ ? new GraphPointsCollector(collectSchemaField, new BitDocSet(resultBits), leafNodes)
+ : new GraphTermsCollector(collectSchemaField, new BitDocSet(resultBits), leafNodes);
+
+ fromSet = new BitDocSet(new FixedBitSet(capacity));
+ graphResultCollector.setCollectDocs(fromSet.getBits());
fromSearcher.search(frontierQuery, graphResultCollector);
- fromSet = graphResultCollector.getDocSet();
- frontierQuery = graphResultCollector.getFrontierQuery();
+
+ frontierQuery = graphResultCollector.getResultQuery(matchSchemaField, isUseAutn());
+ // If there is a filter to be used while crawling the graph, add that.
+ if (frontierQuery != null && getTraversalFilter() != null) {
+ BooleanQuery.Builder builder = new BooleanQuery.Builder();
+ builder.add(frontierQuery, BooleanClause.Occur.MUST);
+ builder.add(getTraversalFilter(), BooleanClause.Occur.MUST);
+ frontierQuery = builder.build();
+ }
+
+
}
if (currentDepth == 0 && !returnRoot) {
// grab a copy of the root bits but only if we need it.
@@ -230,9 +243,9 @@ public class GraphQuery extends Query {
}
private DocSet resolveLeafNodes() throws IOException {
- String field = toSchemaField.getName();
+ String field = collectSchemaField.getName();
BooleanQuery.Builder leafNodeQuery = new BooleanQuery.Builder();
- Query edgeQuery = toSchemaField.hasDocValues() ? new DocValuesFieldExistsQuery(field) : new WildcardQuery(new Term(field, "*"));
+ Query edgeQuery = collectSchemaField.hasDocValues() ? new DocValuesFieldExistsQuery(field) : new WildcardQuery(new Term(field, "*"));
leafNodeQuery.add(edgeQuery, Occur.MUST_NOT);
DocSet leafNodes = fromSearcher.getDocSet(leafNodeQuery.build());
return leafNodes;
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java b/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java
index f32c83b9c99..174db3c691f 100644
--- a/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java
@@ -21,36 +21,23 @@ import java.util.ArrayList;
import java.util.List;
import java.util.TreeSet;
-import org.apache.lucene.document.DoublePoint;
-import org.apache.lucene.document.FloatPoint;
-import org.apache.lucene.document.IntPoint;
-import org.apache.lucene.document.LongPoint;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.AutomatonQuery;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.TermInSetQuery;
import org.apache.lucene.util.BitSet;
-import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.DaciukMihovAutomatonBuilder;
-import org.apache.solr.schema.NumberType;
import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.BitDocSet;
import org.apache.solr.search.DocSet;
-import org.apache.solr.util.LongIterator;
-import org.apache.solr.util.LongSet;
/**
* A graph hit collector. This accumulates the edges for a given graph traversal.
@@ -59,49 +46,51 @@ import org.apache.solr.util.LongSet;
* @lucene.internal
*/
abstract class GraphEdgeCollector extends SimpleCollector implements Collector {
-
- GraphQuery.GraphQueryWeight weight;
-
- // the result set that is being collected.
- Bits currentResult;
+ // For graph traversal, the result set that has already been visited and thus can be skipped for during value collection.
+ DocSet skipSet;
// known leaf nodes
DocSet leafNodes;
- // number of hits discovered at this level.
- int numHits=0;
- BitSet bits;
- final int maxDoc;
- int base;
- int baseInParent;
- // if we care to track this.
- boolean hasCycles = false;
- GraphEdgeCollector(GraphQuery.GraphQueryWeight weight, int maxDoc, Bits currentResult, DocSet leafNodes) {
- this.weight = weight;
- this.maxDoc = maxDoc;
- this.currentResult = currentResult;
+ int numHits=0; // number of documents visited
+ BitSet bits; // if not null, used to collect documents visited
+
+ int base;
+
+ SchemaField collectField;
+
+ // skipSet and leafNodes may be null
+ GraphEdgeCollector(SchemaField collectField, DocSet skipSet, DocSet leafNodes) {
+ this.collectField = collectField;
+ this.skipSet = skipSet;
this.leafNodes = leafNodes;
- if (bits==null) {
- // create a bitset at the start that will hold the graph traversal result set
- bits = new FixedBitSet(maxDoc);
- }
}
+
+ // Set to use to collect docs being visited
+ // TODO: this should be replaced with a more general delegating collector
+ public void setCollectDocs(FixedBitSet target) {
+ this.bits = target;
+ }
+
+ // the number of docs visited
+ public int getNumHits() { return numHits; }
- public void collect(int doc) throws IOException {
- doc += base;
- if (currentResult.get(doc)) {
- // cycle detected / already been here.
- // knowing if your graph had a cycle might be useful and it's lightweight to implement here.
- hasCycles = true;
+ public void collect(int segDoc) throws IOException {
+ int doc = segDoc + base;
+ if (skipSet != null && skipSet.exists(doc)) {
+ // when skipSet == all nodes visited so far, then this represents a cycle and we can
+ // keep track of that here in the future if we need to.
return;
}
- // collect the docs
- addDocToResult(doc);
- // Optimization to not look up edges for a document that is a leaf node
+
+ if (bits != null) bits.set(doc);
+ // increment the hit count so we know how many docs we traversed this time.
+ numHits++;
+
+ // Optimization to not look up edges for a document that is a leaf node (i.e. has no outgoing edges)
if (leafNodes == null || !leafNodes.exists(doc)) {
- addEdgeIdsToResult(doc-base);
+ addEdgeIdsToResult(segDoc);
}
// Note: tracking links in for each result would be a huge memory hog... so not implementing at this time.
-
}
abstract void addEdgeIdsToResult(int doc) throws IOException;
@@ -113,37 +102,13 @@ abstract class GraphEdgeCollector extends SimpleCollector implements Collector {
numHits++;
}
- public BitDocSet getDocSet() {
- if (bits == null) {
- // TODO: this shouldn't happen
- bits = new FixedBitSet(maxDoc);
- }
- return new BitDocSet((FixedBitSet)bits,numHits);
- }
-
@Override
public void doSetNextReader(LeafReaderContext context) throws IOException {
base = context.docBase;
- baseInParent = context.docBaseInParent;
}
- protected abstract Query getResultQuery();
+ public abstract Query getResultQuery(SchemaField matchField, boolean useAutomaton);
- public Query getFrontierQuery() {
- Query q = getResultQuery();
- if (q == null) return null;
-
- // If there is a filter to be used while crawling the graph, add that.
- if (weight.getGraphQuery().getTraversalFilter() != null) {
- BooleanQuery.Builder builder = new BooleanQuery.Builder();
- builder.add(q, BooleanClause.Occur.MUST);
- builder.add(weight.getGraphQuery().getTraversalFilter(), BooleanClause.Occur.MUST);
- q = builder.build();
- }
-
- return q;
- }
-
@Override
public boolean needsScores() {
return false;
@@ -157,8 +122,8 @@ class GraphTermsCollector extends GraphEdgeCollector {
private SortedSetDocValues docTermOrds;
- GraphTermsCollector(GraphQuery.GraphQueryWeight weight, int maxDoc, Bits currentResult, DocSet leafNodes) {
- super(weight, maxDoc, currentResult, leafNodes);
+ GraphTermsCollector(SchemaField collectField, DocSet skipSet, DocSet leafNodes) {
+ super(collectField, skipSet, leafNodes);
this.collectorTerms = new BytesRefHash();
}
@@ -166,7 +131,7 @@ class GraphTermsCollector extends GraphEdgeCollector {
public void doSetNextReader(LeafReaderContext context) throws IOException {
super.doSetNextReader(context);
// Grab the updated doc values.
- docTermOrds = DocValues.getSortedSet(context.reader(), weight.getGraphQuery().getToField());
+ docTermOrds = DocValues.getSortedSet(context.reader(), collectField.getName());
}
@Override
@@ -187,7 +152,7 @@ class GraphTermsCollector extends GraphEdgeCollector {
}
@Override
- protected Query getResultQuery() {
+ public Query getResultQuery(SchemaField matchField, boolean useAutomaton) {
if (collectorTerms == null || collectorTerms.size() == 0) {
// return null if there are no terms (edges) to traverse.
return null;
@@ -195,12 +160,11 @@ class GraphTermsCollector extends GraphEdgeCollector {
// Create a query
Query q = null;
- GraphQuery gq = weight.getGraphQuery();
// TODO: see if we should dynamically select this based on the frontier size.
- if (gq.isUseAutn()) {
+ if (useAutomaton) {
// build an automaton based query for the frontier.
Automaton autn = buildAutomaton(collectorTerms);
- AutomatonQuery autnQuery = new AutomatonQuery(new Term(gq.getFromField()), autn);
+ AutomatonQuery autnQuery = new AutomatonQuery(new Term(matchField.getName()), autn);
q = autnQuery;
} else {
List termList = new ArrayList<>(collectorTerms.size());
@@ -209,7 +173,7 @@ class GraphTermsCollector extends GraphEdgeCollector {
collectorTerms.get(i, ref);
termList.add(ref);
}
- q = new TermInSetQuery(gq.getFromField(), termList);
+ q = new TermInSetQuery(matchField.getName(), termList);
}
return q;
@@ -232,98 +196,3 @@ class GraphTermsCollector extends GraphEdgeCollector {
}
-class GraphPointsCollector extends GraphEdgeCollector {
- final LongSet set = new LongSet(256);
-
- SortedNumericDocValues values = null;
-
- GraphPointsCollector(GraphQuery.GraphQueryWeight weight, int maxDoc, Bits currentResult, DocSet leafNodes) {
- super(weight, maxDoc, currentResult, leafNodes);
- }
-
- @Override
- public void doSetNextReader(LeafReaderContext context) throws IOException {
- super.doSetNextReader(context);
- values = DocValues.getSortedNumeric(context.reader(), weight.getGraphQuery().getToField());
- }
-
- @Override
- void addEdgeIdsToResult(int doc) throws IOException {
- // set the doc to pull the edges ids for.
- int valuesDoc = values.docID();
- if (valuesDoc < doc) {
- valuesDoc = values.advance(doc);
- }
- if (valuesDoc == doc) {
- int count = values.docValueCount();
- for (int i = 0; i < count; i++) {
- long v = values.nextValue();
- set.add(v);
- }
- }
- }
-
- @Override
- protected Query getResultQuery() {
- if (set.cardinality() == 0) return null;
-
- Query q = null;
- SchemaField sfield = weight.fromSchemaField;
- NumberType ntype = sfield.getType().getNumberType();
- boolean multiValued = sfield.multiValued();
-
- if (ntype == NumberType.LONG || ntype == NumberType.DATE) {
- long[] vals = new long[set.cardinality()];
- int i = 0;
- for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
- long bits = iter.next();
- long v = bits;
- vals[i++] = v;
- }
- q = LongPoint.newSetQuery(sfield.getName(), vals);
- } else if (ntype == NumberType.INTEGER) {
- int[] vals = new int[set.cardinality()];
- int i = 0;
- for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
- long bits = iter.next();
- int v = (int)bits;
- vals[i++] = v;
- }
- q = IntPoint.newSetQuery(sfield.getName(), vals);
- } else if (ntype == NumberType.DOUBLE) {
- double[] vals = new double[set.cardinality()];
- int i = 0;
- for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
- long bits = iter.next();
- double v = multiValued ? NumericUtils.sortableLongToDouble(bits) : Double.longBitsToDouble(bits);
- vals[i++] = v;
- }
- q = DoublePoint.newSetQuery(sfield.getName(), vals);
- } else if (ntype == NumberType.FLOAT) {
- float[] vals = new float[set.cardinality()];
- int i = 0;
- for (LongIterator iter = set.iterator(); iter.hasNext(); ) {
- long bits = iter.next();
- float v = multiValued ? NumericUtils.sortableIntToFloat((int) bits) : Float.intBitsToFloat((int) bits);
- vals[i++] = v;
- }
- q = FloatPoint.newSetQuery(sfield.getName(), vals);
- }
-
- return q;
- }
-
-
- /** Build an automaton to represent the frontier query */
- private Automaton buildAutomaton(BytesRefHash termBytesHash) {
- // need top pass a sorted set of terms to the autn builder (maybe a better way to avoid this?)
- final TreeSet terms = new TreeSet();
- for (int i = 0 ; i < termBytesHash.size(); i++) {
- BytesRef ref = new BytesRef();
- termBytesHash.get(i, ref);
- terms.add(ref);
- }
- final Automaton a = DaciukMihovAutomatonBuilder.build(terms);
- return a;
- }
-}
diff --git a/solr/core/src/test/org/apache/solr/TestJoin.java b/solr/core/src/test/org/apache/solr/TestJoin.java
index 0b99b11d0f0..67a3741e979 100644
--- a/solr/core/src/test/org/apache/solr/TestJoin.java
+++ b/solr/core/src/test/org/apache/solr/TestJoin.java
@@ -16,7 +16,6 @@
*/
package org.apache.solr;
-import org.apache.solr.SolrTestCaseJ4.SuppressPointFields;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.noggit.JSONUtil;
import org.noggit.ObjectBuilder;
@@ -37,7 +36,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
-@SuppressPointFields(bugUrl="https://issues.apache.org/jira/browse/SOLR-10939")
public class TestJoin extends SolrTestCaseJ4 {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -45,6 +43,12 @@ public class TestJoin extends SolrTestCaseJ4 {
@BeforeClass
public static void beforeTests() throws Exception {
System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
+
+ if (System.getProperty("solr.tests.IntegerFieldType").contains("Point")) { // all points change at the same time
+ // point fields need docvalues
+ System.setProperty("solr.tests.numeric.dv", "true");
+ }
+
initCore("solrconfig.xml","schema12.xml");
}
@@ -181,12 +185,15 @@ public class TestJoin extends SolrTestCaseJ4 {
for (int qiter=0; qiter
*/
- public void testQureyJoinBooksAndPages() throws Exception {
+ public void testQueryJoinBooksAndPages() throws Exception {
final Client client = Client.localClient();
@@ -1854,8 +1854,8 @@ public class TestJsonFacets extends SolrTestCaseHS {
// the domains we'll be testing, initially setup for block join
final String toChildren = "join: { from:'id', to:'book_id_s' }";
final String toParents = "join: { from:'book_id_s', to:'id' }";
- final String toBogusChildren = "join: { from:'id', to:'does_not_exist' }";
- final String toBogusParents = "join: { from:'book_id_s', to:'does_not_exist' }";
+ final String toBogusChildren = "join: { from:'id', to:'does_not_exist_s' }";
+ final String toBogusParents = "join: { from:'book_id_s', to:'does_not_exist_s' }";
client.testJQ(params(p, "q", "*:*"
, "json.facet", "{ " +
diff --git a/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java b/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java
index 22e532c06a4..e1cfc815c0e 100644
--- a/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java
+++ b/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java
@@ -32,13 +32,18 @@ public class GraphQueryTest extends SolrTestCaseJ4 {
@Test
public void testGraph() throws Exception {
// normal strings
- doGraph( params("node_id","node_s", "edge_id","edge_ss") );
+ doGraph( params("node_id","node_s", "edge_id","edge_ss") );
+ doGraph( params("node_id","node_ss", "edge_id","edge_ss") );
- // point based fields with docvalues
- doGraph( params("node_id","node_ip", "edge_id","edge_ips") );
- doGraph( params("node_id","node_lp", "edge_id","edge_lps") );
- doGraph( params("node_id","node_fp", "edge_id","edge_fps") );
- doGraph( params("node_id","node_dp", "edge_id","edge_dps") );
+ // point based fields with docvalues (single and multi-valued for the node field)
+ doGraph( params("node_id","node_ip", "edge_id","edge_ips") );
+ doGraph( params("node_id","node_ips", "edge_id","edge_ips") );
+ doGraph( params("node_id","node_lp", "edge_id","edge_lps") );
+ doGraph( params("node_id","node_lps", "edge_id","edge_lps") );
+ doGraph( params("node_id","node_fp", "edge_id","edge_fps") );
+ doGraph( params("node_id","node_fps", "edge_id","edge_fps") );
+ doGraph( params("node_id","node_dp", "edge_id","edge_dps") );
+ doGraph( params("node_id","node_dps", "edge_id","edge_dps") );
}
public void doGraph(SolrParams p) throws Exception {
@@ -46,10 +51,10 @@ public class GraphQueryTest extends SolrTestCaseJ4 {
String edge_id = p.get("edge_id");
// NOTE: from/to fields are reversed from {!join}... values are looked up in the "toField" and then matched on the "fromField"
- // 1->2->(3,9)->(4,5)->7
- // 8->(1,2)->...
- assertU(adoc("id", "doc_1", node_id, "1", edge_id, "2", "text", "foo", "title", "foo10" ));
- assertU(adoc("id", "doc_2", node_id, "2", edge_id, "3", "text", "foo" ));
+ // 1->-2->(3,9)->(4,5)->7
+ // 8->(1,-2)->...
+ assertU(adoc("id", "doc_1", node_id, "1", edge_id, "-2", "text", "foo", "title", "foo10" ));
+ assertU(adoc("id", "doc_2", node_id, "-2", edge_id, "3", "text", "foo" ));
assertU(commit());
assertU(adoc("id", "doc_3", node_id, "3", edge_id, "4", edge_id, "5"));
assertU(adoc("id", "doc_4", node_id, "4" ));
@@ -57,12 +62,12 @@ public class GraphQueryTest extends SolrTestCaseJ4 {
assertU(adoc("id", "doc_5", node_id, "5", edge_id, "7" ));
assertU(adoc("id", "doc_6", node_id, "6", edge_id, "3" ));
assertU(adoc("id", "doc_7", node_id, "7", edge_id, "1" ));
- assertU(adoc("id", "doc_8", node_id, "8", edge_id, "1", edge_id, "2" ));
+ assertU(adoc("id", "doc_8", node_id, "8", edge_id, "1", edge_id, "-2" ));
assertU(adoc("id", "doc_9", node_id, "9"));
assertU(commit());
// update docs so they're in a new segment.
- assertU(adoc("id", "doc_1", node_id, "1", edge_id, "2", "text", "foo"));
- assertU(adoc("id", "doc_2", node_id, "2", edge_id, "3", edge_id, "9", "text", "foo11"));
+ assertU(adoc("id", "doc_1", node_id, "1", edge_id, "-2", "text", "foo"));
+ assertU(adoc("id", "doc_2", node_id, "-2", edge_id, "3", edge_id, "9", "text", "foo11"));
assertU(commit());
// a graph for testing traversal filter 10 - 11 -> (12 | 13)
assertU(adoc("id", "doc_10", node_id, "10", edge_id, "11", "title", "foo"));
From 53db72c5985fd6d0027b6888683973ae764c2f85 Mon Sep 17 00:00:00 2001
From: Erick Erickson
Date: Mon, 7 Aug 2017 09:57:27 -0700
Subject: [PATCH 03/26] SOLR-11198: fix test failures
---
.../test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java | 10 ----------
.../apache/solr/common/cloud/ZkMaintenanceUtils.java | 5 -----
2 files changed, 15 deletions(-)
diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java
index 9152ce43f89..38c73a4f206 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java
@@ -21,7 +21,6 @@ import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
-import java.lang.invoke.MethodHandles;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.nio.file.FileVisitResult;
@@ -41,13 +40,9 @@ import org.apache.zookeeper.data.Stat;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(1)
@@ -451,11 +446,6 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
assertEquals("Copy should have succeeded.", 0, res);
Path locEmpty = Paths.get(tmp2.toAbsolutePath().toString(), "stopwords", "emptyfile");
- log.info("EOE Checking file at (var1) " + locEmpty.toAbsolutePath().toString()); // TODO: remove me EOE
- log.info("EOE Checking file at (var2) " + locEmpty.toFile().getAbsolutePath()); // TODO: remove me EOE
- log.info("EOE Checking file exists: " + Boolean.toString(locEmpty.toFile().exists()));
- log.info("EOE Checking isFile: " + Boolean.toString(locEmpty.toFile().isFile()));
- log.info("EOE Checking isDirectory: " + Boolean.toString(locEmpty.toFile().isDirectory())); //TODO: remove me EOE to here.
assertTrue("Empty files should NOT be copied down as directories", locEmpty.toFile().isFile());
}
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java
index 28c37fa8943..6843480ab8f 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java
@@ -314,12 +314,7 @@ public class ZkMaintenanceUtils {
// If we didn't copy data down, then we also didn't create the file. But we still need a marker on the local
// disk so create an empty file.
if (copyDataDown(zkClient, zkPath, file.toFile()) == 0) {
- log.info("EOE Creating file at (var 1)" + file.toAbsolutePath().toString()); // TODO remove me EOE
- log.info("EOE Creating file at (var2) " + file.toFile().getAbsolutePath()); // TODO remove me EOE
Files.createFile(file);
- log.info("EOE Creating file exists: " + Boolean.toString(file.toFile().exists()));
- log.info("EOE Creating isFile: " + Boolean.toString(file.toFile().isFile()));
- log.info("EOE Creating isDirectory: " + Boolean.toString(file.toFile().isDirectory())); //TODO: remove me EOE to here.
}
} else {
Files.createDirectories(file); // Make parent dir.
From 80530c14a3e50f78d182859ca69d4519576f9f4b Mon Sep 17 00:00:00 2001
From: Cassandra Targett
Date: Mon, 7 Aug 2017 12:08:23 -0500
Subject: [PATCH 04/26] SOLR-10821: resolve TODOs; copy edits & cleanups;
reorder section flow
---
.../src/solrcloud-autoscaling-api.adoc | 57 ++++----
.../src/solrcloud-autoscaling-overview.adoc | 22 +--
...rcloud-autoscaling-policy-preferences.adoc | 138 +++++++++---------
.../src/solrcloud-autoscaling.adoc | 3 +-
4 files changed, 114 insertions(+), 106 deletions(-)
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
index 0dc875a2a18..52b5598a0e8 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-api.adoc
@@ -1,4 +1,4 @@
-= SolrCloud Autoscaling API
+= Autoscaling API
:page-shortname: solrcloud-autoscaling-api
:page-permalink: solrcloud-autoscaling-api.html
:page-toclevels: 2
@@ -20,7 +20,7 @@
// specific language governing permissions and limitations
// under the License.
-The Autoscaling API can be used to manage autoscaling policies and preferences, and to get diagnostics on the state of the cluster.
+The Autoscaling API is used to manage autoscaling policies and preferences, and to get diagnostics on the state of the cluster.
== Read API
@@ -56,7 +56,7 @@ The output will contain cluster preferences, cluster policy and collection speci
== Diagnostics API
-The diagnostics API shows the violations, if any, of all conditions in the cluster or collection-specific policy. It is available at the `/admin/autoscaling/diagnostics` path.
+The diagnostics API shows the violations, if any, of all conditions in the cluster and, if applicable, the collection-specific policy. It is available at the `/admin/autoscaling/diagnostics` path.
This API does not take any parameters.
@@ -150,15 +150,17 @@ The Write API is available at the same `/admin/autoscaling` and `/v2/cluster/aut
The payload of the POST request is a JSON message with commands to set and remove components. Multiple commands can be specified together in the payload. The commands are executed in the order specified and the changes are atomic, i.e., either all succeed or none.
-=== set-cluster-preferences: Create and Modify Cluster Preferences
+=== Create and Modify Cluster Preferences
-The cluster preferences are specified as a list of sort preferences. Multiple sorting preferences can be specified and they are applied in order.
+Cluster preferences are specified as a list of sort preferences. Multiple sorting preferences can be specified and they are applied in order.
+
+They are defined using the `set-cluster-preferences` command.
Each preference is a JSON map having the following syntax:
-`{'': '', 'precision' : ''}`
+`{'':'', 'precision':''}`
-You can see the __TODO__ section to know more about the allowed values for the `sort_order`, `sort_param` and `precision` parameters.
+See the section <> for details about the allowed values for the `sort_order`, `sort_param` and `precision` parameters.
Changing the cluster preferences after the cluster is already built doesn't automatically reconfigure the cluster. However, all future cluster management operations will use the changed preferences.
@@ -167,9 +169,9 @@ Changing the cluster preferences after the cluster is already built doesn't auto
[source,json]
----
{
- "set-cluster-preferences" : [
- {"minimize": "cores"}
- ]
+"set-cluster-preferences" : [
+ {"minimize": "cores"}
+ ]
}
----
@@ -221,17 +223,21 @@ We can remove all cluster preferences by setting preferences to an empty list.
}
----
-=== set-cluster-policy: Create and Modify Cluster Policies
+=== Create and Modify Cluster Policies
-You can see the __TODO__ section to know more about the allowed values for each condition in the policy.
+Cluster policies are set using the `set-cluster-policy` command.
+
+Like `set-cluster-preferences`, the policy definition is a JSON map defining the desired attributes and values.
+
+Refer to the <> section for details of the allowed values for each condition in the policy.
*Input*:
[source,json]
----
{
- "set-cluster-policy": [
- {"replica": "<2", "shard": "#EACH", "node": "#ANY"}
- ]
+"set-cluster-policy": [
+ {"replica": "<2", "shard": "#EACH", "node": "#ANY"}
+ ]
}
----
@@ -249,6 +255,7 @@ Output:
----
We can remove all cluster policy conditions by setting policy to an empty list.
+
[source,json]
----
{
@@ -258,21 +265,21 @@ We can remove all cluster policy conditions by setting policy to an empty list.
Changing the cluster policy after the cluster is already built doesn't automatically reconfigure the cluster. However, all future cluster management operations will use the changed cluster policy.
-=== set-policy: Create and Modify Collection-Specific Policy
+=== Create and Modify Collection-Specific Policy
-This command accepts a map of policy name to the list of conditions for that policy. Multiple named policies can be specified together. A named policy that does not exist already is created and if the named policy accepts already then it is replaced.
+The `set-policy` command accepts a map of policy names to the list of conditions for that policy. Multiple named policies can be specified together. A named policy that does not exist already is created and if the named policy accepts already then it is replaced.
-You can see the __TODO__ section to know more about the allowed values for each condition in the policy.
+Refer to the <> section for details of the allowed values for each condition in the policy.
*Input*
[source,json]
----
{
- "set-policy": {
- "policy1": [
- {"replica": "1", "shard": "#EACH", "port": "8983"}
- ]
+"set-policy": {
+ "policy1": [
+ {"replica": "1", "shard": "#EACH", "port": "8983"}
+ ]
}
}
----
@@ -293,9 +300,9 @@ You can see the __TODO__ section to know more about the allowed values for each
Changing the policy after the collection is already built doesn't automatically reconfigure the collection. However, all future cluster management operations will use the changed policy.
-=== remove-policy: Remove a Collection-Specific Policy
+=== Remove a Collection-Specific Policy
-This command accepts a policy name to be removed from Solr. The policy being removed must not be attached to any collection otherwise the command will fail.
+The `remove-policy` command accepts a policy name to be removed from Solr. The policy being removed must not be attached to any collection otherwise the command will fail.
*Input*
[source,json]
@@ -316,4 +323,4 @@ This command accepts a policy name to be removed from Solr. The policy being rem
}
----
-If you attempt to remove a policy that is being used by a collection then this command will fail to delete the policy until the collection itself is deleted.
+If you attempt to remove a policy that is being used by a collection, this command will fail to delete the policy until the collection itself is deleted.
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc
index 9bb12e5e6f3..3facb65d921 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-overview.adoc
@@ -1,4 +1,4 @@
-= Overview of Autoscaling in SolrCloud
+= Overview of SolrCloud Autoscaling
:page-shortname: solrcloud-autoscaling-overview
:page-permalink: solrcloud-autoscaling-overview.html
:page-toclevels: 1
@@ -20,40 +20,40 @@
// specific language governing permissions and limitations
// under the License.
-Autoscaling in Solr aims to provide good defaults such that the cluster remains balanced and stable in the face of various events such as a node joining the cluster or leaving the cluster. This is achieved by satisfying a set of rules and sorting preferences that help Solr select the target of cluster management operations.
+Autoscaling in Solr aims to provide good defaults so a SolrCloud cluster remains balanced and stable in the face of various cluster change events. This balance is achieved by satisfying a set of rules and sorting preferences to select the target of cluster management operations.
== Cluster Preferences
Cluster preferences, as the name suggests, apply to all cluster management operations regardless of which collection they affect.
-A preference is a set of conditions that help Solr select nodes that either maximize or minimize given metrics. For example, a preference `{minimize : cores}` will help Solr select nodes such that the number of cores on each node is minimized. We write cluster preference in a way that reduces the overall load on the system. You can add more than one preferences to break ties.
+A preference is a set of conditions that help Solr select nodes that either maximize or minimize given metrics. For example, a preference such as `{minimize:cores}` will help Solr select nodes such that the number of cores on each node is minimized. We write cluster preferences in a way that reduces the overall load on the system. You can add more than one preferences to break ties.
The default cluster preferences consist of the above example (`{minimize : cores}`) which is to minimize the number of cores on all nodes.
-You can learn more about preferences in the __TODO__ section.
+You can learn more about preferences in the <> section.
== Cluster Policy
-A cluster policy is a set of conditions that a node, shard, or collection must satisfy before it can be chosen as the target of a cluster management operation. These conditions are applied across the cluster regardless of the collection being managed. For example, the condition `{"cores":"<10", "node":"#ANY"}` means that any node must have less than ten Solr cores in total regardless of which collection they belong to.
+A cluster policy is a set of conditions that a node, shard, or collection must satisfy before it can be chosen as the target of a cluster management operation. These conditions are applied across the cluster regardless of the collection being managed. For example, the condition `{"cores":"<10", "node":"#ANY"}` means that any node must have less than 10 Solr cores in total regardless of which collection they belong to.
-There are many metrics on which the condition can be based e.g., system load average, heap usage, free disk space etc. The full list of supported metrics can be found at __TODO__ section.
+There are many metrics on which the condition can be based, e.g., system load average, heap usage, free disk space, etc. The full list of supported metrics can be found in the section describing <>.
-When a node, shard or collection does not satisfy the policy, we call it a *violation*. Solr ensures that cluster management operations minimize the number of violations. The cluster management operations are either invoked manually by us. In future, these cluster management operations may be invoked automatically in response to cluster events such as node being added or lost.
+When a node, shard, or collection does not satisfy the policy, we call it a *violation*. Solr ensures that cluster management operations minimize the number of violations. Cluster management operations are currently invoked manually. In the future, these cluster management operations may be invoked automatically in response to cluster events such as a node being added or lost.
== Collection-Specific Policies
-Sometimes a collection may need conditions in addition to those specified in the cluster policy. In such cases, we can create named policies that can be used for specific collections. Firstly, we can use the `set-policy` API to create a new policy and then specify the `policy=` parameter to the CREATE command of the Collection API.
+A collection may need conditions in addition to those specified in the cluster policy. In such cases, we can create named policies that can be used for specific collections. Firstly, we can use the `set-policy` API to create a new policy and then specify the `policy=` parameter to the CREATE command of the Collection API.
`/admin/collections?action=CREATE&name=coll1&numShards=1&replicationFactor=2&policy=policy1`
The above create collection command will associate a policy named `policy1` with the collection named `coll1`. Only a single policy may be associated with a collection.
-Note that the collection-specific policy is applied *in addition* to the cluster policy, i.e., it is not an override but an augmentation. Therefore the collection will follow all conditions laid out in the cluster preferences, cluster policy, and the policy named `policy1`.
+Note that the collection-specific policy is applied *in addition to* the cluster policy, i.e., it is not an override but an augmentation. Therefore the collection will follow all conditions laid out in the cluster preferences, cluster policy, and the policy named `policy1`.
-You can learn more about collection specific policies in the __TODO__ section.
+You can learn more about collection-specific policies in the section <>.
== Autoscaling APIs
The autoscaling APIs available at `/admin/autoscaling` can be used to read and modify each of the components discussed above.
-You can learn more about these APIs in the __TODO__ section.
+You can learn more about these APIs in the section <>.
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
index f8bbb9232f3..9fafd69ce6a 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-policy-preferences.adoc
@@ -1,4 +1,4 @@
-= SolrCloud Autoscaling Policy and Preferences
+= Autoscaling Policy and Preferences
:page-shortname: solrcloud-autoscaling-policy-preferences
:page-permalink: solrcloud-autoscaling-policy-preferences.html
:page-toclevels: 2
@@ -20,71 +20,77 @@
// specific language governing permissions and limitations
// under the License.
-The autoscaling policy and preferences are a set of rules and sorting preferences that help Solr select the target of cluster management operations such that the overall load on the cluster is balanced.
+The autoscaling policy and preferences are a set of rules and sorting preferences that help Solr select the target of cluster management operations so the overall load on the cluster remains balanced.
-== Cluster preferences specification
+== Cluster Preferences Specification
-A preference is a hint to Solr on how to sort nodes based on their utilization. The default cluster preference is to sort by the total number of Solr cores (or replicas) hosted by the node. Therefore, by default, when selecting a node to add a replica, Solr can apply the preferences and choose the node with the least number of cores.
+A preference is a hint to Solr on how to sort nodes based on their utilization. The default cluster preference is to sort by the total number of Solr cores (or replicas) hosted by a node. Therefore, by default, when selecting a node to add a replica, Solr can apply the preferences and choose the node with the least number of cores.
-More than one preferences can be added to break ties. For example, we may choose to use free disk space to break ties if number of cores on two nodes are the same so that the node with the higher free disk space can be chosen as the target of the cluster operation.
+More than one preferences can be added to break ties. For example, we may choose to use free disk space to break ties if the number of cores on two nodes are the same so the node with the higher free disk space can be chosen as the target of the cluster operation.
Each preference is of the following form:
+
[source,json]
-----
-{"": "", "precision" : ""}
-----
+{"":"", "precision":""}
`sort_order`::
-The value can be either `maximize` or `minimize`. `minimize` sorts the nodes with least value as the least loaded. e.g `{"minimize" : "cores"}` sorts the nodes with the least number of cores as the least loaded node. `{"maximize" : "freedisk"}` sorts the nodes with maximum free disk space as the least loaded node. The objective of the system is to make every node the least loaded. So, e.g. in case of a `MOVEREPLICA` operation, it usually targets the _most loaded_ node and takes load off of it. In a sort of more loaded to less loaded, minimize is akin to sort in descending order and maximize is akin to sorting in ascending order. This is a required parameter.
+The value can be either `maximize` or `minimize`. `minimize` sorts the nodes with least value as the least loaded. For example, `{"minimize":"cores"}` sorts the nodes with the least number of cores as the least loaded node. A sort order such as `{"maximize":"freedisk"}` sorts the nodes with maximum free disk space as the least loaded node.
++
+The objective of the system is to make every node the least loaded. So, in case of a `MOVEREPLICA` operation, it usually targets the _most loaded_ node and takes load off of it. In a sort of more loaded to less loaded, `minimize` is akin to sort in descending order and `maximize` is akin to sorting in ascending order.
++
+This is a required parameter.
`sort_param`::
-One and only one of the following supported parameter must be specified:
-1. `cores`: The number of total Solr cores on a node
-2. `freedisk`: The amount of free disk space for Solr's data home directory. This is always in gigabytes.
-3. `sysLoadAvg`: The system load average on a node as reported by the Metrics API under the key `solr.jvm/os.systemLoadAverage`. This is always a double value between 0 and 1 and the higher the value, the more loaded the node is.
-4. `heapUsage`: The heap usage of a node as reported by the Metrics API under the key `solr.jvm/memory.heap.usage`. This is always a double value between 0 and 1 and the higher the value, the more loaded the node is.
+One and only one of the following supported parameters must be specified:
+
+. `cores`: The number of total Solr cores on a node.
+. `freedisk`: The amount of free disk space for Solr's data home directory. This is always in gigabytes.
+. `sysLoadAvg`: The system load average on a node as reported by the Metrics API under the key `solr.jvm/os.systemLoadAverage`. This is always a double value between 0 and 1 and the higher the value, the more loaded the node is.
+. `heapUsage`: The heap usage of a node as reported by the Metrics API under the key `solr.jvm/memory.heap.usage`. This is always a double value between 0 and 1 and the higher the value, the more loaded the node is.
`precision`::
-Precision tells the system the minimum (absolute) difference between 2 values to treat them as distinct values. For example, a precision of 10 for `freedisk` means that two nodes whose free disk space is within 10GB of each other should be treated as equal for the purpose of sorting. This helps create ties without which, specifying multiple preferences is not useful. This is an optional parameter whose value must be a positive integer. The maximum value of precision must be less than the maximum value of the `sort_value`, if any.
+Precision tells the system the minimum (absolute) difference between 2 values to treat them as distinct values.
++
+For example, a precision of 10 for `freedisk` means that two nodes whose free disk space is within 10GB of each other should be treated as equal for the purpose of sorting. This helps create ties without which specifying multiple preferences is not useful. This is an optional parameter whose value must be a positive integer. The maximum value of `precision` must be less than the maximum value of the `sort_value`, if any.
-See the `set-cluster-preferences` API section for details on how to manage cluster preferences.
+See the section <> for details on how to manage cluster preferences.
=== Examples of Cluster Preferences
-The following is the default cluster preferences. This is applied automatically by Solr when no explicit cluster preferences have been set using the Autoscaling API.
-[source,json]
-----
-[{"minimize":"cores"}]
-----
+==== Default Preferences
+The following shows the default cluster preferences. This is applied automatically by Solr when no explicit cluster preferences have been set using the <>.
+
+[source,json]
+[{"minimize":"cores"}]
+
+==== Minimize Cores; Maximize Free Disk
+In this example, we want to minimize the number of Solr cores and in case of a tie, maximize the amount of free disk space on each node.
-In this example, we want to minimize the number of solr cores and in case of tie, maximize the amount of free disk space on each node.
[source,json]
-----
[
{"minimize" : "cores"},
{"maximize" : "freedisk"}
]
-----
+==== Add Precision to Free Disk; Minimize System Load
In this example, we add a precision to the `freedisk` parameter so that nodes with free disk space within 10GB of each other are considered equal. In such a case, the tie is broken by minimizing `sysLoadAvg`.
+
[source,json]
-----
[
{"minimize" : "cores"},
{"maximize" : "freedisk", "precision" : 10},
{"minimize" : "sysLoadAvg"}
]
-----
-== Policy specification
+== Policy Specification
-A policy is a hard rule to be satisfied by each node. If a node does not satisfy the rule then it is called a `violation`. Solr ensures that the number of violations are minimized while invoking any cluster management operations.
+A policy is a hard rule to be satisfied by each node. If a node does not satisfy the rule then it is called a *violation*. Solr ensures that the number of violations are minimized while invoking any cluster management operations.
-=== Policy attributes
+=== Policy Attributes
A policy can have the following attributes:
`cores`::
-This is a special attribute that applies to the entire cluster. It can only be used along with the `node` attribute and no other. This parameter is optional.
+This is a special attribute that applies to the entire cluster. It can only be used along with the `node` attribute and no other. This attribute is optional.
`collection`::
The name of the collection to which the policy rule should apply. If omitted, the rule applies to all collections. This attribute is optional.
@@ -98,7 +104,7 @@ The number of replicas that must exist to satisfy the rule. This must be a posit
`strict`::
An optional boolean value. The default is `true`. If true, the rule must be satisfied. If false, Solr tries to satisfy the rule on a best effort basis but if no node can satisfy the rule then any node may be chosen.
-One and only one of the following attribute can be specified in addition to the above attributes:
+One and only one of the following attributes can be specified in addition to the above attributes:
`node`::
The name of the node to which the rule should apply. The default value is `#ANY` which means that any node in the cluster may satisfy the rule.
@@ -121,11 +127,11 @@ The heap usage of the node as reported by the Metrics API under the key `solr.jv
`nodeRole`::
The role of the node. The only supported value currently is `overseer`.
-`ip_1 , ip_2, ip_3, ip_4`:
+`ip_1 , ip_2, ip_3, ip_4`::
The least significant to most significant segments of IP address. For example, for an IP address `192.168.1.2`, `ip_1 = 2`, `ip_2 = 1`, `ip_3 = 168`, `ip_4 = 192`.
-`sysprop.`:
-The system property set on the node on startup.
+`sysprop.`::
+Any arbitrary system property set on the node on startup.
=== Policy Operators
@@ -136,74 +142,68 @@ Each attribute in the policy may specify one of the following operators along wi
* `!`: Not
* None means equal
-=== Examples of policy rules
+=== Examples of Policy Rules
-`Example 1`::
-Do not place more than one replica of the same shard on the same node
+==== Limit Replica Placement
+Do not place more than one replica of the same shard on the same node:
[source,json]
-----
{"replica": "<2", "shard": "#EACH", "node": "#ANY"}
-----
-`Example 2`::
+==== Limit Cores per Node
Do not place more than 10 cores in any node. This rule can only be added to the cluster policy because it mentions the `cores` attribute that is only applicable cluster-wide.
+
[source,json]
-----
{"cores": "<10", "node": "#ANY"}
-----
-`Example 3`::
+==== Place Replicas Based on Port
Place exactly 1 replica of each shard of collection `xyz` on a node running on port `8983`
+
[source,json]
-----
{"replica": 1, "shard": "#EACH", "collection": "xyz", "port": "8983"}
-----
-`Example 4`::
-Place all replicas on a node with system property `availability_zone=us-east-1a`. Note that we have to write this rule in the negative sense i.e. *0* replicas must be on nodes *not* having the sysprop `availability_zone=us-east-1a`
+==== Place Replicas Based on a System Property
+Place all replicas on a node with system property `availability_zone=us-east-1a`. Note that we have to write this rule in the negative sense i.e., *0* replicas must be on nodes *not* having the system property `availability_zone=us-east-1a`
+
[source,json]
-----
{"replica": 0, "sysprop.availability_zone": "!us-east-1a"}
-----
-`Example 5`::
+==== Place Replicas Based on Node Role
Do not place any replica on a node which has the overseer role. Note that the role is added by the `addRole` collection API. It is *not* automatically the node which is currently the overseer.
+
[source,json]
-----
{"replica": 0, "nodeRole": "overseer"}
-----
-`Example 6`::
+==== Place Replicas Based on Free Disk
Place all replicas in nodes with freedisk more than 500GB. Here again, we have to write the rule in the negative sense.
+
[source,json]
-----
{"replica": 0, "freedisk": "<500"}
-----
-`Example 7`::
+==== Try to Place Replicas Based on Free Disk
Place all replicas in nodes with freedisk more than 500GB when possible. Here we use the strict keyword to signal that this rule is to be honored on a best effort basis.
+
[source,json]
-----
{"replica": 0, "freedisk": "<500", "strict" : false}
-----
-
-== Cluster Policy vs Collection-specific Policy
+== Defining Collection-Specific Policies
By default, the cluster policy, if it exists, is used automatically for all collections in the cluster. However, we can create named policies which can be attached to a collection at the time of its creation by specifying the policy name along with a `policy` parameter.
-When a collection-specific policy is used, the rules in that policy are appended to the rules in the cluster policy and the combination of both are used. Therefore, it is recommended that you do not add rules to collection-specific policy that conflict with the ones in the cluster policy. Doing so will disqualify all nodes in the cluster from matching all criteria and make the policy useless. Also, if `maxShardsPerNode` is specified during the time of collection creation then both `maxShardsPerNode` and the policy rules must be satisfied.
+When a collection-specific policy is used, the rules in that policy are *appended* to the rules in the cluster policy and the combination of both are used. Therefore, it is recommended that you do not add rules to collection-specific policy that conflict with the ones in the cluster policy. Doing so will disqualify all nodes in the cluster from matching all criteria and make the policy useless.
-Some attributes such as `cores` can only be used in the cluster policy.
+It is possible to override conditions specified in the cluster policy using collection-specific policy. For example, if a clause `{replica:'<3', node:'#ANY'}` is present in the cluster policy and the collection-specific policy has a clause `{replica:'<4', node:'#ANY'}`, the cluster policy is ignored in favor of the collection policy.
-The policy is used by Collection APIs such as:
+Also, if `maxShardsPerNode` is specified during the time of collection creation, then both `maxShardsPerNode` and the policy rules must be satisfied.
-* create
-* createshard
-* addreplica
-* restore
-* splitshard
+Some attributes such as `cores` can only be used in the cluster policy. See the section above on policy attributes for details.
-In future, the policy and preferences will be used by the Autoscaling framework to automatically change the cluster in response to events such as a node being added or lost.
+The policy is used by these <> commands:
+* CREATE
+* CREATESHARD
+* ADDREPLICA
+* RESTORE
+* SPLITSHARD
+
+In the future, the policy and preferences will be used by the Autoscaling framework to automatically change the cluster in response to events such as a node being added or lost.
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling.adoc
index d9d20176bb5..457120b2f20 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling.adoc
@@ -1,7 +1,7 @@
= SolrCloud Autoscaling
:page-shortname: solrcloud-autoscaling
:page-permalink: solrcloud-autoscaling.html
-:page-children: solrcloud-autoscaling-overview, solrcloud-autoscaling-api, solrcloud-autoscaling-policy-preferences
+:page-children: solrcloud-autoscaling-overview, solrcloud-autoscaling-policy-preferences, solrcloud-autoscaling-api
// 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
@@ -19,6 +19,7 @@
// specific language governing permissions and limitations
// under the License.
+[.lead]
The goal of autoscaling is to make SolrCloud cluster management easier by providing a way for changes to the cluster to be more automatic and more intelligent.
Autoscaling includes an API to manage cluster-wide and collection-specific policies and preferences and a rules syntax to define the guidelines for your cluster. Future Solr releases will include features to utilize the policies and preferences so they perform actions automatically when the rules are violated.
From 432516a1fbf70866fdff8fbb8fb4a3c10d9b8979 Mon Sep 17 00:00:00 2001
From: Erick Erickson
Date: Mon, 7 Aug 2017 12:28:30 -0700
Subject: [PATCH 05/26] Minor documentation fix for CDCR and managed schema
---
.../src/cross-data-center-replication-cdcr.adoc | 7 ++++---
1 file changed, 4 insertions(+), 3 deletions(-)
diff --git a/solr/solr-ref-guide/src/cross-data-center-replication-cdcr.adoc b/solr/solr-ref-guide/src/cross-data-center-replication-cdcr.adoc
index 77332d3b3da..15d5ac8581a 100644
--- a/solr/solr-ref-guide/src/cross-data-center-replication-cdcr.adoc
+++ b/solr/solr-ref-guide/src/cross-data-center-replication-cdcr.adoc
@@ -157,10 +157,11 @@ The CDC Replicator is a background thread that is responsible for replicating up
The current design of CDCR has some limitations. CDCR will continue to evolve over time and many of these limitations will be addressed. Among them are:
-* CDCR is unlikely to be satisfactory for bulk-load situations where the update rate is high, especially if the bandwidth between the Source and target clusters is restricted. In this scenario, the initial bulk load should be performed, the Source and target data centers synchronized and CDCR be utilized for incremental updates.
-* CDCR is currently only active-passive; data is pushed from the Source cluster to the target cluster. There is active work being done in this area in the 6x code line to remove this limitation.
-* CDCR works most robustly with the same number of shards in the Source and target collection. The shards in the two collections may have different numbers of replicas.
+* CDCR is unlikely to be satisfactory for bulk-load situations where the update rate is high, especially if the bandwidth between the Source and Target clusters is restricted. In this scenario, the initial bulk load should be performed, the Source and Target data centers synchronized and CDCR be utilized for incremental updates.
+* CDCR is currently only active-passive; data is pushed from the Source cluster to the Target cluster. There is active work being done in this area in the 6x code line to remove this limitation.
+* CDCR works most robustly with the same number of shards in the Source and Target collection. The shards in the two collections may have different numbers of replicas.
* Running CDCR with the indexes on HDFS is not currently supported, see the https://issues.apache.org/jira/browse/SOLR-9861[Solr CDCR over HDFS] JIRA issue.
+* Configuration files (solrconfig.xml, schema etc.) are not automatically synchronized between the Source and Target clusters. This means that when the Source schema or solrconfig files are changed, those changes must be replicated manually to the Target cluster. This includes adding fields by the <> or <> as well as hand editing those files.
== CDCR Configuration
From bf168ad37e4326be28950ede8f958b6c3f1330fa Mon Sep 17 00:00:00 2001
From: Erick Erickson
Date: Mon, 7 Aug 2017 14:16:21 -0700
Subject: [PATCH 06/26] SOLR-11177: CoreContainer.load needs to send lazily
loaded core descriptors to the proper list rather than send them all to the
transient lists.
---
solr/CHANGES.txt | 3 +++
solr/core/src/java/org/apache/solr/core/CoreContainer.java | 3 +--
2 files changed, 4 insertions(+), 2 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d39a7f67185..3adc0a2eb0c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -772,6 +772,9 @@ when using one of Exact*StatsCache (Mikhail Khludnev)
* SOLR-11198: downconfig downloads empty file as folder (Erick Erickson)
+* SOLR-11177: CoreContainer.load needs to send lazily loaded core descriptors to the proper list rather than send
+ them all to the transient lists. (Erick Erickson) (note, not in 7.0, is in 7.1)
+
Optimizations
----------------------
* SOLR-10634: JSON Facet API: When a field/terms facet will retrieve all buckets (i.e. limit:-1)
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 0b789f8fba4..6013b284ca5 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -608,7 +608,7 @@ public class CoreContainer {
for (final CoreDescriptor cd : cds) {
if (cd.isTransient() || !cd.isLoadOnStartup()) {
- solrCores.getTransientCacheHandler().addTransientDescriptor(cd.getName(), cd);
+ solrCores.addCoreDescriptor(cd);
} else if (asyncSolrCoreLoad) {
solrCores.markCoreAsLoading(cd);
}
@@ -845,7 +845,6 @@ public class CoreContainer {
core.close();
throw new IllegalStateException("This CoreContainer has been closed");
}
- solrCores.addCoreDescriptor(cd);
SolrCore old = solrCores.putCore(cd, core);
/*
* set both the name of the descriptor and the name of the
From 0dca964a5d9d2d845c9031529630a5455177981b Mon Sep 17 00:00:00 2001
From: Anshum Gupta
Date: Mon, 7 Aug 2017 15:45:38 -0700
Subject: [PATCH 07/26] SOLR-11126: Reduce logging to debug, and remove the
call to updateLiveNodes on every call
---
.../handler/admin/HealthCheckHandler.java | 20 +++++++------------
1 file changed, 7 insertions(+), 13 deletions(-)
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java
index 03c7bd47d1c..093524d2642 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java
@@ -65,7 +65,7 @@ public class HealthCheckHandler extends RequestHandlerBase {
@Override
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
- log.info("Invoked HealthCheckHandler on [{}]", coreContainer.getZkController().getNodeName());
+ log.debug("Invoked HealthCheckHandler on [{}]", coreContainer.getZkController().getNodeName());
CoreContainer cores = getCoreContainer();
if(cores == null) {
@@ -73,6 +73,7 @@ public class HealthCheckHandler extends RequestHandlerBase {
return;
}
if(!cores.isZooKeeperAware()) {
+ //TODO: Support standalone instances
rsp.setException(new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Health check is only available when running in SolrCloud mode"));
return;
}
@@ -85,19 +86,12 @@ public class HealthCheckHandler extends RequestHandlerBase {
return;
}
- try {
- zkStateReader.updateLiveNodes();
-
- // Set status to true if this node is in live_nodes
- if (clusterState.getLiveNodes().contains(cores.getZkController().getNodeName())) {
- rsp.add(STATUS, OK);
- } else {
- rsp.add(STATUS, FAILURE);
- rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Host Unavailable: Not in live nodes as per zk"));
- }
- } catch (KeeperException e) {
+ // Set status to true if this node is in live_nodes
+ if (clusterState.getLiveNodes().contains(cores.getZkController().getNodeName())) {
+ rsp.add(STATUS, OK);
+ } else {
rsp.add(STATUS, FAILURE);
- rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Host Unavailable: Not connected to zk"));
+ rsp.setException(new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, "Host Unavailable: Not in live nodes as per zk"));
}
rsp.setHttpCaching(false);
From b531fbc5fd91d5fabf90a552b809727d68fd1c9f Mon Sep 17 00:00:00 2001
From: Mike McCandless
Date: Mon, 7 Aug 2017 18:53:40 -0400
Subject: [PATCH 08/26] LUCENE-7919: remove useless notify
---
.../org/apache/lucene/index/DocumentsWriterPerThreadPool.java | 2 --
1 file changed, 2 deletions(-)
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
index de5b3fe88dc..102628fd1de 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
@@ -212,8 +212,6 @@ final class DocumentsWriterPerThreadPool {
state.unlock();
synchronized (this) {
freeList.add(state);
- // In case any thread is waiting, wake one of them up since we just released a thread state:
- notify();
}
}
From a0ad20f5e6caedc50b8a4030ab4ac9e19095e731 Mon Sep 17 00:00:00 2001
From: Anshum Gupta
Date: Mon, 7 Aug 2017 16:04:05 -0700
Subject: [PATCH 09/26] SOLR-11126: Remove unused import from
HealthCheckHandler
---
.../java/org/apache/solr/handler/admin/HealthCheckHandler.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java
index 093524d2642..7b07a1ec9a1 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/HealthCheckHandler.java
@@ -27,7 +27,6 @@ import org.apache.solr.core.CoreContainer;
import org.apache.solr.handler.RequestHandlerBase;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.response.SolrQueryResponse;
-import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
From a4db6ce3e681d96fd05f6814818b3270ca527821 Mon Sep 17 00:00:00 2001
From: Robert Muir
Date: Mon, 7 Aug 2017 20:47:30 -0400
Subject: [PATCH 10/26] LUCENE-7916: Remove use of deprecated
UScript.CODE_LIMIT in ICUTokenizer
---
lucene/CHANGES.txt | 5 +++++
.../org/apache/lucene/analysis/icu/ICUFoldingFilter.java | 2 ++
.../analysis/icu/segmentation/CompositeBreakIterator.java | 4 +++-
.../analysis/icu/segmentation/DefaultICUTokenizerConfig.java | 4 ++++
.../analysis/icu/segmentation/ICUTokenizerFactory.java | 3 +--
5 files changed, 15 insertions(+), 3 deletions(-)
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d6083b91317..5bfc6df9879 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -30,6 +30,11 @@ Bug Fixes
functions (Operations.isFinite and Operations.topsortState) to prevent
large automaton to overflow the stack (Robert Muir, Adrien Grand, Jim Ferenczi)
+* LUCENE-7916: Prevent ArrayIndexOutOfBoundsException if ICUTokenizer is used
+ with a different ICU JAR version than it is compiled against. Note, this is
+ not recommended, lucene-analyzers-icu contains binary data structures
+ specific to ICU/Unicode versions it is built against. (Chris Koenig, Robert Muir)
+
======================= Lucene 7.0.0 =======================
New Features
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUFoldingFilter.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUFoldingFilter.java
index 3fad465965a..0895b47438a 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUFoldingFilter.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUFoldingFilter.java
@@ -61,6 +61,8 @@ import com.ibm.icu.text.Normalizer2;
*
*/
public final class ICUFoldingFilter extends ICUNormalizer2Filter {
+ // TODO: if the wrong version of the ICU jar is used, loading these data files may give a strange error.
+ // maybe add an explicit check? http://icu-project.org/apiref/icu4j/com/ibm/icu/util/VersionInfo.html
private static final Normalizer2 normalizer = Normalizer2.getInstance(
ICUFoldingFilter.class.getResourceAsStream("utr30.nrm"),
"utr30", Normalizer2.Mode.COMPOSE);
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/CompositeBreakIterator.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/CompositeBreakIterator.java
index e7b5f76bd45..096eada2de3 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/CompositeBreakIterator.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/CompositeBreakIterator.java
@@ -17,6 +17,8 @@
package org.apache.lucene.analysis.icu.segmentation;
+import com.ibm.icu.lang.UCharacter;
+import com.ibm.icu.lang.UProperty;
import com.ibm.icu.lang.UScript;
import com.ibm.icu.text.BreakIterator;
@@ -38,7 +40,7 @@ import com.ibm.icu.text.BreakIterator;
*/
final class CompositeBreakIterator {
private final ICUTokenizerConfig config;
- private final BreakIteratorWrapper wordBreakers[] = new BreakIteratorWrapper[UScript.CODE_LIMIT];
+ private final BreakIteratorWrapper wordBreakers[] = new BreakIteratorWrapper[1 + UCharacter.getIntPropertyMaxValue(UProperty.SCRIPT)];
private BreakIteratorWrapper rbbi;
private final ScriptIterator scriptIterator;
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java
index 3cd62c8b2fc..50a6b4c71d8 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/DefaultICUTokenizerConfig.java
@@ -60,6 +60,10 @@ public class DefaultICUTokenizerConfig extends ICUTokenizerConfig {
// we keep the cjk breaking separate, thats because it cannot be customized (because dictionary
// is only triggered when kind = WORD, but kind = LINE by default and we have no non-evil way to change it)
private static final BreakIterator cjkBreakIterator = BreakIterator.getWordInstance(ULocale.ROOT);
+
+ // TODO: if the wrong version of the ICU jar is used, loading these data files may give a strange error.
+ // maybe add an explicit check? http://icu-project.org/apiref/icu4j/com/ibm/icu/util/VersionInfo.html
+
// the same as ROOT, except no dictionary segmentation for cjk
private static final BreakIterator defaultBreakIterator =
readBreakIterator("Default.brk");
diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
index 974e719880e..4d29b0c36bc 100644
--- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
+++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/segmentation/ICUTokenizerFactory.java
@@ -33,7 +33,6 @@ import org.apache.lucene.util.IOUtils;
import com.ibm.icu.lang.UCharacter;
import com.ibm.icu.lang.UProperty;
-import com.ibm.icu.lang.UScript;
import com.ibm.icu.text.BreakIterator;
import com.ibm.icu.text.RuleBasedBreakIterator;
@@ -108,7 +107,7 @@ public class ICUTokenizerFactory extends TokenizerFactory implements ResourceLoa
if (tailored.isEmpty()) {
config = new DefaultICUTokenizerConfig(cjkAsWords, myanmarAsWords);
} else {
- final BreakIterator breakers[] = new BreakIterator[UScript.CODE_LIMIT];
+ final BreakIterator breakers[] = new BreakIterator[1 + UCharacter.getIntPropertyMaxValue(UProperty.SCRIPT)];
for (Map.Entry entry : tailored.entrySet()) {
int code = entry.getKey();
String resourcePath = entry.getValue();
From 2a8930cf838b323eeadba240eb7141ec1f14ca6d Mon Sep 17 00:00:00 2001
From: Jim Ferenczi
Date: Tue, 8 Aug 2017 14:25:23 +0200
Subject: [PATCH 11/26] LUCENE-7914: Fix TestSuggestField#testRealisticKeys:
trim big titles to make sure that they can pass the max recursion level in
Operations#topsortState.
---
.../lucene/search/suggest/document/TestSuggestField.java | 8 +++++---
1 file changed, 5 insertions(+), 3 deletions(-)
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index a797ca5f1be..a6659e082d5 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -719,13 +719,15 @@ public class TestSuggestField extends LuceneTestCase {
for (int i = 0; i < num; i++) {
Document document = lineFileDocs.nextDoc();
String title = document.getField("title").stringValue();
+ int maxLen = Math.min(title.length(), 500);
+ String prefix = title.substring(0, maxLen);
int weight = random().nextInt(Integer.MAX_VALUE);
- Integer prevWeight = mappings.get(title);
+ Integer prevWeight = mappings.get(prefix);
if (prevWeight == null || prevWeight < weight) {
- mappings.put(title, weight);
+ mappings.put(prefix, weight);
}
Document doc = new Document();
- doc.add(new SuggestField("suggest_field", title, weight));
+ doc.add(new SuggestField("suggest_field", prefix, weight));
iw.addDocument(doc);
if (rarely()) {
From ea85543aced4fbdc7f4a82e6ea67e74998c898c9 Mon Sep 17 00:00:00 2001
From: Joel Bernstein
Date: Tue, 8 Aug 2017 14:53:45 -0400
Subject: [PATCH 12/26] SOLR-11212: Allow the predict StreamEvaluator to work
on arrays as well as a single numeric parameter
---
.../solrj/io/eval/PredictEvaluator.java | 20 +++++++++++++++----
.../solrj/io/stream/StreamExpressionTest.java | 4 +++-
2 files changed, 19 insertions(+), 5 deletions(-)
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java
index af8a7f02006..ed9034a9b22 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PredictEvaluator.java
@@ -19,6 +19,8 @@ package org.apache.solr.client.solrj.io.eval;
import java.io.IOException;
import java.util.Locale;
+import java.util.List;
+import java.util.ArrayList;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@@ -38,17 +40,27 @@ public class PredictEvaluator extends ComplexEvaluator implements Expressible {
if(2 != subEvaluators.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two values (regression result and a number) but found %d",expression,subEvaluators.size()));
}
-
}
- public Number evaluate(Tuple tuple) throws IOException {
+ public Object evaluate(Tuple tuple) throws IOException {
StreamEvaluator r = subEvaluators.get(0);
StreamEvaluator d = subEvaluators.get(1);
RegressionEvaluator.RegressionTuple rt= (RegressionEvaluator.RegressionTuple)r.evaluate(tuple);
- Number n = (Number)d.evaluate(tuple);
- return rt.predict(n.doubleValue());
+
+ Object o = d.evaluate(tuple);
+ if(o instanceof Number) {
+ Number n = (Number)o;
+ return rt.predict(n.doubleValue());
+ } else {
+ List list = (List)o;
+ List predications = new ArrayList();
+ for(Number n : list) {
+ predications.add(rt.predict(n.doubleValue()));
+ }
+ return predications;
+ }
}
@Override
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 699a5629d1d..93e52871d6a 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -6270,7 +6270,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
String expr1 = "search("+COLLECTIONORALIAS+", q=\"col_s:a\", fl=\"price_f, order_i\", sort=\"order_i asc\")";
String expr2 = "search("+COLLECTIONORALIAS+", q=\"col_s:b\", fl=\"price_f, order_i\", sort=\"order_i asc\")";
- String cexpr = "let(a="+expr1+", b="+expr2+", c=col(a, price_f), d=col(b, price_f), e=regress(c, d), tuple(regress=e, p=predict(e, 300)))";
+ String cexpr = "let(a="+expr1+", b="+expr2+", c=col(a, price_f), d=col(b, price_f), e=regress(c, d), tuple(regress=e, p=predict(e, 300), pl=predict(e, c)))";
ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", cexpr);
@@ -6293,6 +6293,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(rSquare == 1.0D);
double prediction = tuple.getDouble("p");
assertTrue(prediction == 600.0D);
+ List predictions = (List)tuple.get("pl");
+ assertList(predictions, 200.0, 400.0, 600.0, 200.0, 400.0, 800.0, 1200.0);
}
From 7ed0a40eaab2510cfbc4dba5353e55806a3b0c02 Mon Sep 17 00:00:00 2001
From: Varun Thacker
Date: Tue, 8 Aug 2017 14:52:57 -0700
Subject: [PATCH 13/26] SOLR-11199: Support OR queries in the
PayloadScoreParser and a sum function
---
.../queries/payloads/SumPayloadFunction.java | 55 +++++++++++++++++++
solr/CHANGES.txt | 5 ++
.../search/PayloadScoreQParserPlugin.java | 11 +++-
.../org/apache/solr/util/PayloadUtils.java | 20 +++++--
.../search/TestPayloadScoreQParserPlugin.java | 10 +++-
solr/solr-ref-guide/src/other-parsers.adoc | 6 +-
6 files changed, 98 insertions(+), 9 deletions(-)
create mode 100644 lucene/queries/src/java/org/apache/lucene/queries/payloads/SumPayloadFunction.java
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/payloads/SumPayloadFunction.java b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SumPayloadFunction.java
new file mode 100644
index 00000000000..29e7206097b
--- /dev/null
+++ b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SumPayloadFunction.java
@@ -0,0 +1,55 @@
+/*
+ * 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.lucene.queries.payloads;
+
+/**
+ * Calculate the final score as the sum of scores of all payloads seen.
+ *
+ * Is thread safe and completely reusable.
+ *
+ **/
+public class SumPayloadFunction extends PayloadFunction {
+
+ @Override
+ public float currentScore(int docId, String field, int start, int end, int numPayloadsSeen, float currentScore, float currentPayloadScore) {
+ return currentPayloadScore + currentScore;
+ }
+
+ @Override
+ public float docScore(int docId, String field, int numPayloadsSeen, float payloadScore) {
+ return numPayloadsSeen > 0 ? payloadScore : 1;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + this.getClass().hashCode();
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ return true;
+ }
+}
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3adc0a2eb0c..16da34a25ae 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -65,6 +65,11 @@ New Features
* SOLR-11126: Node level health check handler (Anshum Gupta)
+* SOLR-11199: Payloads supports an "operator" param. Supported operators are 'or', "phrase" ( default ).
+ A new "sum" function is also added. Example :
+ {!payload_score f=payload_field func=sum operator=or}A B C" (Varun Thacker)
+
+
Bug Fixes
----------------------
diff --git a/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java
index 4098e0928d5..7042cda920f 100644
--- a/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/PayloadScoreQParserPlugin.java
@@ -37,11 +37,12 @@ import org.apache.solr.util.PayloadUtils;
* Other parameters:
* f, the field (required)
* func, payload function (min, max, or average; required)
- * includeSpanScore, multiple payload function result by similarity score or not (default: false)
+ * includeSpanScore, multiply payload function result by similarity score or not (default: false)
* Example: {!payload_score f=weighted_terms_dpf}Foo Bar creates a SpanNearQuery with "Foo" followed by "Bar"
*/
public class PayloadScoreQParserPlugin extends QParserPlugin {
public static final String NAME = "payload_score";
+ public static final String DEFAULT_OPERATOR = "phrase";
@Override
public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
@@ -51,6 +52,10 @@ public class PayloadScoreQParserPlugin extends QParserPlugin {
String field = localParams.get(QueryParsing.F);
String value = localParams.get(QueryParsing.V);
String func = localParams.get("func");
+ String operator = localParams.get("operator", DEFAULT_OPERATOR);
+ if (!(operator.equalsIgnoreCase(DEFAULT_OPERATOR) || operator.equalsIgnoreCase("or"))) {
+ throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Supported operators are : or , phrase");
+ }
boolean includeSpanScore = localParams.getBool("includeSpanScore", false);
if (field == null) {
@@ -63,9 +68,9 @@ public class PayloadScoreQParserPlugin extends QParserPlugin {
FieldType ft = req.getCore().getLatestSchema().getFieldType(field);
Analyzer analyzer = ft.getQueryAnalyzer();
- SpanQuery query = null;
+ SpanQuery query;
try {
- query = PayloadUtils.createSpanQuery(field, value, analyzer);
+ query = PayloadUtils.createSpanQuery(field, value, analyzer, operator);
} catch (IOException e) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,e);
}
diff --git a/solr/core/src/java/org/apache/solr/util/PayloadUtils.java b/solr/core/src/java/org/apache/solr/util/PayloadUtils.java
index 6fe8a6199ae..2de730746ec 100644
--- a/solr/core/src/java/org/apache/solr/util/PayloadUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/PayloadUtils.java
@@ -33,12 +33,15 @@ import org.apache.lucene.queries.payloads.AveragePayloadFunction;
import org.apache.lucene.queries.payloads.MaxPayloadFunction;
import org.apache.lucene.queries.payloads.MinPayloadFunction;
import org.apache.lucene.queries.payloads.PayloadFunction;
+import org.apache.lucene.queries.payloads.SumPayloadFunction;
import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.analysis.TokenizerChain;
import org.apache.solr.schema.FieldType;
+import org.apache.solr.search.PayloadScoreQParserPlugin;
public class PayloadUtils {
public static String getPayloadEncoder(FieldType fieldType) {
@@ -95,15 +98,22 @@ public class PayloadUtils {
if ("average".equals(func)) {
payloadFunction = new AveragePayloadFunction();
}
-
+ if ("sum".equals(func)) {
+ payloadFunction = new SumPayloadFunction();
+ }
return payloadFunction;
}
+ public static SpanQuery createSpanQuery(String field, String value, Analyzer analyzer) throws IOException {
+ return createSpanQuery(field, value, analyzer, PayloadScoreQParserPlugin.DEFAULT_OPERATOR);
+ }
+
+
/**
* The generated SpanQuery will be either a SpanTermQuery or an ordered, zero slop SpanNearQuery, depending
* on how many tokens are emitted.
*/
- public static SpanQuery createSpanQuery(String field, String value, Analyzer analyzer) throws IOException {
+ public static SpanQuery createSpanQuery(String field, String value, Analyzer analyzer, String operator) throws IOException {
// adapted this from QueryBuilder.createSpanQuery (which isn't currently public) and added reset(), end(), and close() calls
List terms = new ArrayList<>();
try (TokenStream in = analyzer.tokenStream(field, value)) {
@@ -121,9 +131,11 @@ public class PayloadUtils {
query = null;
} else if (terms.size() == 1) {
query = terms.get(0);
+ } else if (operator != null && operator.equalsIgnoreCase("or")) {
+ query = new SpanOrQuery(terms.toArray(new SpanTermQuery[terms.size()]));
} else {
- query = new SpanNearQuery(terms.toArray(new SpanTermQuery[terms.size()]), 0, true);
- }
+ query = new SpanNearQuery(terms.toArray(new SpanTermQuery[terms.size()]), 0, true);
+ }
return query;
}
}
diff --git a/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java
index 8ac09bb6530..9c9c50e0d43 100644
--- a/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestPayloadScoreQParserPlugin.java
@@ -35,7 +35,6 @@ public class TestPayloadScoreQParserPlugin extends SolrTestCaseJ4 {
@Test
public void test() {
- clearIndex();
assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=B func=min}"), "//float[@name='score']='2.0'");
assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=mult func=min}"), "//float[@name='score']='50.0'");
@@ -47,6 +46,15 @@ public class TestPayloadScoreQParserPlugin extends SolrTestCaseJ4 {
assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=average}B C"), "//float[@name='score']='2.5'");
assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=max}A B C"), "//float[@name='score']='3.0'");
+ assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=sum}A B C"), "//float[@name='score']='6.0'");
+ assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=sum operator=or}A C"), "//float[@name='score']='4.0'");
+ assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=sum operator=or}A"), "//float[@name='score']='1.0'");
+ assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=sum operator=or}foo"), "//result[@numFound='0']");
+
+ assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=max operator=or}A C"), "//float[@name='score']='3.0'");
+ assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=min operator=or}A x"), "//float[@name='score']='1.0'");
+ assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf func=average operator=or}A C"), "//float[@name='score']='2.0'");
+
// TODO: fix this includeSpanScore test to be less brittle - score result is score of "A" (via BM25) multipled by 1.0 (payload value)
assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=A func=min}"), "//float[@name='score']='1.0'");
assertQ(req("fl","*,score", "q", "{!payload_score f=vals_dpf v=A func=min includeSpanScore=true}"), "//float[@name='score']='0.2876821'");
diff --git a/solr/solr-ref-guide/src/other-parsers.adoc b/solr/solr-ref-guide/src/other-parsers.adoc
index db484193775..e54910ae279 100644
--- a/solr/solr-ref-guide/src/other-parsers.adoc
+++ b/solr/solr-ref-guide/src/other-parsers.adoc
@@ -657,7 +657,11 @@ This parser accepts the following parameters:
The field to use (required).
`func`::
-Payload function: min, max, average (required).
+Payload function: min, max, average, sum (required).
+
+`operator`::
+Search operator: or , phrase ( default ) (optional). This defines if the search query should be an OR
+query or a phrase query
`includeSpanScore`::
If `true`, multiples computed payload factor by the score of the original query. If `false`, the default, the computed payload factor is the score.
From 68bda0be421ce18811e03b229781fd6152fcc04a Mon Sep 17 00:00:00 2001
From: Cao Manh Dat
Date: Wed, 9 Aug 2017 10:26:19 +0700
Subject: [PATCH 14/26] SOLR-10126: PeerSyncReplicationTest is a flakey test.
---
.../java/org/apache/solr/update/PeerSync.java | 8 +++++++
.../solr/cloud/PeerSyncReplicationTest.java | 21 ++++++++++++-------
2 files changed, 22 insertions(+), 7 deletions(-)
diff --git a/solr/core/src/java/org/apache/solr/update/PeerSync.java b/solr/core/src/java/org/apache/solr/update/PeerSync.java
index 7371a943cdc..426e0f78bf3 100644
--- a/solr/core/src/java/org/apache/solr/update/PeerSync.java
+++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java
@@ -266,6 +266,14 @@ public class PeerSync implements SolrMetricProducer {
requestVersions(replica);
}
+ try {
+ // waiting a little bit, there are a chance that an update is sending from leader,
+ // so it will present in the response, but not in our recent updates (SOLR-10126)
+ Thread.sleep(300);
+ } catch (InterruptedException e) {
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+ }
+
try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) {
ourUpdates = recentUpdates.getVersions(nUpdates);
}
diff --git a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
index 0859eb5e615..8d71f1abef1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
@@ -37,7 +37,6 @@ import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.lucene.util.LuceneTestCase.Slow;
-import org.apache.lucene.util.LuceneTestCase.BadApple;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.request.UpdateRequest;
@@ -49,6 +48,7 @@ import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.metrics.SolrMetricManager;
import org.apache.solr.util.TimeOut;
import org.junit.Test;
import org.slf4j.Logger;
@@ -63,7 +63,6 @@ import static java.util.concurrent.TimeUnit.SECONDS;
* This test is modeled after SyncSliceTest
*/
@Slow
-@BadApple(bugUrl = "https://issues.apache.org/jira/browse/SOLR-10126")
public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -181,13 +180,21 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
assertEquals(nodePeerSynced, shardToLeaderJetty.get("shard1"));
// assert metrics
- MetricRegistry registry = nodePeerSynced.jetty.getCoreContainer().getMetricManager().registry("solr.core.collection1");
+ SolrMetricManager manager = nodePeerSynced.jetty.getCoreContainer().getMetricManager();
+ MetricRegistry registry = null;
+ for (String name : manager.registryNames()) {
+ if (name.startsWith("solr.core.collection1")) {
+ registry = manager.registry(name);
+ break;
+ }
+ }
+ assertNotNull(registry);
Map metrics = registry.getMetrics();
- assertTrue("REPLICATION.time present", metrics.containsKey("REPLICATION.time"));
- assertTrue("REPLICATION.errors present", metrics.containsKey("REPLICATION.errors"));
- Timer timer = (Timer)metrics.get("REPLICATION.time");
+ assertTrue("REPLICATION.peerSync.time present", metrics.containsKey("REPLICATION.peerSync.time"));
+ assertTrue("REPLICATION.peerSync.errors present", metrics.containsKey("REPLICATION.peerSync.errors"));
+ Timer timer = (Timer)metrics.get("REPLICATION.peerSync.time");
assertEquals(1L, timer.getCount());
- Counter counter = (Counter)metrics.get("REPLICATION.errors");
+ Counter counter = (Counter)metrics.get("REPLICATION.peerSync.errors");
assertEquals(0L, counter.getCount());
success = true;
} finally {
From 8e2dab7315739a0f5194600ee524f6a2ea616af6 Mon Sep 17 00:00:00 2001
From: Christine Poerschke
Date: Mon, 7 Aug 2017 16:23:48 +0100
Subject: [PATCH 15/26] SOLR-11090: Add Replica.getProperty accessor.
---
solr/CHANGES.txt | 2 +
.../solr/cloud/CollectionsAPISolrJTest.java | 6 +-
.../solr/cloud/ReplicaPropertiesBase.java | 8 +-
.../apache/solr/cloud/TestCollectionAPI.java | 120 +++++++++---------
.../solr/cloud/TestReplicaProperties.java | 4 +-
.../org/apache/solr/common/cloud/Replica.java | 11 ++
.../solr/common/cloud/ZkStateReader.java | 1 +
7 files changed, 83 insertions(+), 69 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 16da34a25ae..9e401c26352 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -132,6 +132,8 @@ Other Changes
* SOLR-11187: contrib/ltr TestModelManagerPersistence improvements. (Yuki Yano via Christine Poerschke)
+* SOLR-11090: Add Replica.getProperty accessor. (Christine Poerschke)
+
================== 7.0.0 ==================
Versions of Major Components
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 99e4fda3832..d6d492c779e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -369,14 +369,14 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
assertEquals(0, response.getStatus());
waitForState("Expecting property 'preferredleader' to appear on replica " + replica.getName(), collection,
- (n, c) -> "true".equals(c.getReplica(replica.getName()).getStr("property.preferredleader")));
+ (n, c) -> "true".equals(c.getReplica(replica.getName()).getProperty("preferredleader")));
response = CollectionAdminRequest.deleteReplicaProperty(collection, "shard1", replica.getName(), "property.preferredleader")
.process(cluster.getSolrClient());
assertEquals(0, response.getStatus());
waitForState("Expecting property 'preferredleader' to be removed from replica " + replica.getName(), collection,
- (n, c) -> c.getReplica(replica.getName()).getStr("property.preferredleader") == null);
+ (n, c) -> c.getReplica(replica.getName()).getProperty("preferredleader") == null);
}
@@ -396,7 +396,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
for (Slice slice : c) {
int count = 0;
for (Replica replica : slice) {
- if ("true".equals(replica.getStr("property.preferredleader")))
+ if ("true".equals(replica.getProperty("preferredleader")))
count += 1;
}
if (count != 1)
diff --git a/solr/core/src/test/org/apache/solr/cloud/ReplicaPropertiesBase.java b/solr/core/src/test/org/apache/solr/cloud/ReplicaPropertiesBase.java
index 0cb3f8f87dd..a3fbb32d466 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ReplicaPropertiesBase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ReplicaPropertiesBase.java
@@ -62,7 +62,7 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
if (replica == null) {
fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
}
- if (StringUtils.isBlank(replica.getStr(property))) return;
+ if (StringUtils.isBlank(replica.getProperty(property))) return;
Thread.sleep(100);
}
fail("Property " + property + " not set correctly for collection/replica pair: " +
@@ -88,11 +88,11 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
if (replica == null) {
fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
}
- if (StringUtils.equals(val, replica.getStr(property))) return;
+ if (StringUtils.equals(val, replica.getProperty(property))) return;
Thread.sleep(100);
}
- fail("Property '" + property + "' with value " + replica.getStr(property) +
+ fail("Property '" + property + "' with value " + replica.getProperty(property) +
" not set correctly for collection/replica pair: " + collectionName + "/" + replicaName + " property map is " +
replica.getProperties().toString() + ".");
@@ -131,7 +131,7 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
int propCount = 0;
for (Replica replica : slice.getReplicas()) {
uniqueNodes.add(replica.getNodeName());
- String propVal = replica.getStr(property);
+ String propVal = replica.getProperty(property);
if (StringUtils.isNotBlank(propVal)) {
++propCount;
if (counts.containsKey(replica.getNodeName()) == false) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
index 037a3e6806e..cf4111e6a85 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
@@ -401,8 +401,8 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
client.request(request);
// The above should have set exactly one preferredleader...
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader", "true");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "preferredleader", "true");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
@@ -412,8 +412,8 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property", "preferredLeader",
"property.value", "true");
// The preferred leader property for shard1 should have switched to the other replica.
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
@@ -424,9 +424,9 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property.value", "true");
// Now we should have a preferred leader in both shards...
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
@@ -437,11 +437,11 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property.value", "true");
// Now we should have three preferred leaders.
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader", "true");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME1, c2_s1_r1, "preferredleader", "true");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toString(),
@@ -452,10 +452,10 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
// Now we should have two preferred leaders.
// But first we have to wait for the overseer to finish the action
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
// Try adding an arbitrary property to one that has the leader property
doPropertyAction(client,
@@ -466,11 +466,11 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property", "testprop",
"property.value", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "true");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "testprop", "true");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
@@ -480,12 +480,12 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property", "prop",
"property.value", "silly");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "testprop", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "prop", "silly");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.ADDREPLICAPROP.toLower(),
@@ -496,12 +496,12 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property.value", "nonsense",
SHARD_UNIQUE, "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "nonsense");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "testprop", "nonsense");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "prop", "silly");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
doPropertyAction(client,
@@ -513,12 +513,12 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property.value", "true",
SHARD_UNIQUE, "false");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "testprop", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "prop", "silly");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
@@ -527,12 +527,12 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"replica", c1_s1_r1,
"property", "property.testprop");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.testprop");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "testprop");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "prop", "silly");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
try {
doPropertyAction(client,
@@ -549,12 +549,12 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
se.getMessage().contains("with the shardUnique parameter set to something other than 'true'"));
}
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
- verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.testprop");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "property.preferredLeader");
- verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "property.preferredLeader");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "preferredleader", "true");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "preferredleader", "true");
+ verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "testprop");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "prop", "silly");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME, "preferredLeader");
+ verifyUniquePropertyWithinCollection(client, COLLECTION_NAME1, "preferredLeader");
Map origProps = getProps(client, COLLECTION_NAME, c1_s1_r1,
"state", "core", "node_name", "base_url");
@@ -592,10 +592,10 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"property.value", "base_url_bad");
// The above should be on new proeprties.
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.state", "state_bad");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.core", "core_bad");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.node_name", "node_name_bad");
- verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.base_url", "base_url_bad");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "state", "state_bad");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "core", "core_bad");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "node_name", "node_name_bad");
+ verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "base_url", "base_url_bad");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
@@ -630,10 +630,10 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, ent.getKey(), ent.getValue());
}
- verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.state");
- verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.core");
- verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.node_name");
- verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.base_url");
+ verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "state");
+ verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "core");
+ verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "node_name");
+ verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "base_url");
}
}
@@ -776,7 +776,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
Replica replica = docCollection.getReplica(replicaName);
Map propMap = new HashMap<>();
for (String prop : props) {
- propMap.put(prop, replica.getStr(prop));
+ propMap.put(prop, replica.getProperty(prop));
}
return propMap;
}
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java b/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java
index 9a9af9722c6..f654e8ffc6d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestReplicaProperties.java
@@ -103,7 +103,7 @@ public class TestReplicaProperties extends ReplicaPropertiesBase {
"collection", COLLECTION_NAME,
"property", "preferredLeader");
- verifyUniqueAcrossCollection(client, COLLECTION_NAME, "property.preferredleader");
+ verifyUniqueAcrossCollection(client, COLLECTION_NAME, "preferredleader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
@@ -170,7 +170,7 @@ public class TestReplicaProperties extends ReplicaPropertiesBase {
"shardUnique", "true");
verifyPropertyVal(client, COLLECTION_NAME,
- c1_s1_r1, "property.bogus1", "true");
+ c1_s1_r1, "bogus1", "true");
verifyPropertyVal(client, COLLECTION_NAME,
c1_s1_r2, "property.bogus1", "whatever");
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index 78283933669..b8ca240fa3b 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -159,6 +159,17 @@ public class Replica extends ZkNodeProps {
return this.type;
}
+ public String getProperty(String propertyName) {
+ final String propertyKey;
+ if (!propertyName.startsWith(ZkStateReader.PROPERTY_PROP_PREFIX)) {
+ propertyKey = ZkStateReader.PROPERTY_PROP_PREFIX+propertyName;
+ } else {
+ propertyKey = propertyName;
+ }
+ final String propertyValue = getStr(propertyKey);
+ return propertyValue;
+ }
+
@Override
public String toString() {
return name + ':' + JSONUtil.toJSON(propMap, -1); // small enough, keep it on one line (i.e. no indent)
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 0a7d76fa1f0..11061de1a8a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -83,6 +83,7 @@ public class ZkStateReader implements Closeable {
public static final String NUM_SHARDS_PROP = "numShards";
public static final String LEADER_PROP = "leader";
public static final String PROPERTY_PROP = "property";
+ public static final String PROPERTY_PROP_PREFIX = "property.";
public static final String PROPERTY_VALUE_PROP = "property.value";
public static final String MAX_AT_ONCE_PROP = "maxAtOnce";
public static final String MAX_WAIT_SECONDS_PROP = "maxWaitSeconds";
From b091934f9e98568b848d0584a1145c8e514cbd21 Mon Sep 17 00:00:00 2001
From: Shalin Shekhar Mangar
Date: Wed, 9 Aug 2017 16:16:53 +0530
Subject: [PATCH 16/26] Create znode upfront and fix chroot handling in
delegation token feature
---
.../DelegationTokenKerberosFilter.java | 42 +++-
.../solr/security/HadoopAuthFilter.java | 42 +++-
.../solr/security/HadoopAuthPlugin.java | 1 +
.../hadoop/TestZkAclsWithHadoopAuth.java | 216 ++++++++++++++++++
.../solr/cloud/MiniSolrCloudCluster.java | 2 +
5 files changed, 293 insertions(+), 10 deletions(-)
create mode 100644 solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
diff --git a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
index 007e0bdc5d4..ce3544c0658 100644
--- a/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/DelegationTokenKerberosFilter.java
@@ -46,6 +46,8 @@ import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkACLProvider;
import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.ACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -65,8 +67,12 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider"))) {
SolrZkClient zkClient =
(SolrZkClient)conf.getServletContext().getAttribute(KerberosPlugin.DELEGATION_TOKEN_ZK_CLIENT);
- conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
- getCuratorClient(zkClient));
+ try {
+ conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
+ getCuratorClient(zkClient));
+ } catch (InterruptedException | KeeperException e) {
+ throw new ServletException(e);
+ }
}
super.init(conf);
}
@@ -147,7 +153,7 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
newAuthHandler.setAuthHandler(authHandler);
}
- protected CuratorFramework getCuratorClient(SolrZkClient zkClient) {
+ protected CuratorFramework getCuratorClient(SolrZkClient zkClient) throws InterruptedException, KeeperException {
// should we try to build a RetryPolicy off of the ZkController?
RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
if (zkClient == null) {
@@ -161,6 +167,17 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient);
final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561.
+ // Create /security znode upfront. Without this, the curator framework creates this directory path
+ // without the appropriate ACL configuration. This issue is possibly related to HADOOP-11973
+ try {
+ zkClient.makePath(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, CreateMode.PERSISTENT, true);
+
+ } catch (KeeperException ex) {
+ if (ex.code() != KeeperException.Code.NODEEXISTS) {
+ throw ex;
+ }
+ }
+
curatorFramework = CuratorFrameworkFactory.builder()
.namespace(zkNamespace)
.connectString(zkConnectionString)
@@ -178,12 +195,15 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
* Convert Solr Zk Credentials/ACLs to Curator versions
*/
protected static class SolrZkToCuratorCredentialsACLs {
+ private final String zkChroot;
private final ACLProvider aclProvider;
private final List authInfos;
public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) {
this.aclProvider = createACLProvider(zkClient);
this.authInfos = createAuthInfo(zkClient);
+ String zkHost = zkClient.getZkServerAddress();
+ this.zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): null;
}
public ACLProvider getACLProvider() { return aclProvider; }
@@ -199,8 +219,20 @@ public class DelegationTokenKerberosFilter extends DelegationTokenAuthentication
@Override
public List getAclForPath(String path) {
- List acls = zkACLProvider.getACLsToAdd(path);
- return acls;
+ List acls = null;
+
+ // The logic in SecurityAwareZkACLProvider does not work when
+ // the Solr zkPath is chrooted (e.g. /solr instead of /). This
+ // due to the fact that the getACLsToAdd(..) callback provides
+ // an absolute path (instead of relative path to the chroot) and
+ // the string comparison in SecurityAwareZkACLProvider fails.
+ if (zkACLProvider instanceof SecurityAwareZkACLProvider && zkChroot != null) {
+ acls = zkACLProvider.getACLsToAdd(path.replace(zkChroot, ""));
+ } else {
+ acls = zkACLProvider.getACLsToAdd(path);
+ }
+
+ return acls;
}
};
}
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
index fb35e722281..205becc8835 100644
--- a/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthFilter.java
@@ -43,6 +43,8 @@ import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkACLProvider;
import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.ACL;
/**
@@ -62,8 +64,12 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
if (conf != null && "zookeeper".equals(conf.getInitParameter("signer.secret.provider"))) {
SolrZkClient zkClient =
(SolrZkClient)conf.getServletContext().getAttribute(DELEGATION_TOKEN_ZK_CLIENT);
- conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
- getCuratorClient(zkClient));
+ try {
+ conf.getServletContext().setAttribute("signer.secret.provider.zookeeper.curator.client",
+ getCuratorClient(zkClient));
+ } catch (KeeperException | InterruptedException e) {
+ throw new ServletException(e);
+ }
}
super.init(conf);
}
@@ -125,7 +131,7 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
newAuthHandler.setAuthHandler(authHandler);
}
- protected CuratorFramework getCuratorClient(SolrZkClient zkClient) {
+ protected CuratorFramework getCuratorClient(SolrZkClient zkClient) throws KeeperException, InterruptedException {
// should we try to build a RetryPolicy off of the ZkController?
RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
if (zkClient == null) {
@@ -139,6 +145,17 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
SolrZkToCuratorCredentialsACLs curatorToSolrZk = new SolrZkToCuratorCredentialsACLs(zkClient);
final int connectionTimeoutMs = 30000; // this value is currently hard coded, see SOLR-7561.
+ // Create /security znode upfront. Without this, the curator framework creates this directory path
+ // without the appropriate ACL configuration. This issue is possibly related to HADOOP-11973
+ try {
+ zkClient.makePath(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH, CreateMode.PERSISTENT, true);
+
+ } catch (KeeperException ex) {
+ if (ex.code() != KeeperException.Code.NODEEXISTS) {
+ throw ex;
+ }
+ }
+
curatorFramework = CuratorFrameworkFactory.builder()
.namespace(zkNamespace)
.connectString(zkConnectionString)
@@ -156,12 +173,15 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
* Convert Solr Zk Credentials/ACLs to Curator versions
*/
protected static class SolrZkToCuratorCredentialsACLs {
+ private final String zkChroot;
private final ACLProvider aclProvider;
private final List authInfos;
public SolrZkToCuratorCredentialsACLs(SolrZkClient zkClient) {
this.aclProvider = createACLProvider(zkClient);
this.authInfos = createAuthInfo(zkClient);
+ String zkHost = zkClient.getZkServerAddress();
+ this.zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): null;
}
public ACLProvider getACLProvider() { return aclProvider; }
@@ -177,8 +197,20 @@ public class HadoopAuthFilter extends DelegationTokenAuthenticationFilter {
@Override
public List getAclForPath(String path) {
- List acls = zkACLProvider.getACLsToAdd(path);
- return acls;
+ List acls = null;
+
+ // The logic in SecurityAwareZkACLProvider does not work when
+ // the Solr zkPath is chrooted (e.g. /solr instead of /). This
+ // due to the fact that the getACLsToAdd(..) callback provides
+ // an absolute path (instead of relative path to the chroot) and
+ // the string comparison in SecurityAwareZkACLProvider fails.
+ if (zkACLProvider instanceof SecurityAwareZkACLProvider && zkChroot != null) {
+ acls = zkACLProvider.getACLsToAdd(path.replace(zkChroot, ""));
+ } else {
+ acls = zkACLProvider.getACLsToAdd(path);
+ }
+
+ return acls;
}
};
}
diff --git a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
index fa59d38bec5..28352f34ff9 100644
--- a/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/HadoopAuthPlugin.java
@@ -142,6 +142,7 @@ public class HadoopAuthPlugin extends AuthenticationPlugin {
authFilter.init(conf);
} catch (ServletException e) {
+ log.error("Error initializing " + getClass().getSimpleName(), e);
throw new SolrException(ErrorCode.SERVER_ERROR, "Error initializing " + getClass().getName() + ": "+e);
}
}
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
new file mode 100644
index 00000000000..ed734991c06
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
@@ -0,0 +1,216 @@
+/*
+ * 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.security.hadoop;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.Constants;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.SecurityAwareZkACLProvider;
+import org.apache.solr.common.cloud.ZkCredentialsProvider;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.ServerCnxn;
+import org.apache.zookeeper.server.auth.AuthenticationProvider;
+import org.apache.zookeeper.server.auth.ProviderRegistry;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestZkAclsWithHadoopAuth extends SolrCloudTestCase {
+ protected static final int NUM_SERVERS = 1;
+ protected static final int NUM_SHARDS = 1;
+ protected static final int REPLICATION_FACTOR = 1;
+
+ @BeforeClass
+ public static void setupClass() throws Exception {
+ assumeFalse("Hadoop does not work on Windows", Constants.WINDOWS);
+ assumeFalse("FIXME: SOLR-8182: This test fails under Java 9", Constants.JRE_IS_MINIMUM_JAVA9);
+
+ System.setProperty("zookeeper.authProvider.1", DummyZKAuthProvider.class.getName());
+ System.setProperty("zkCredentialsProvider", DummyZkCredentialsProvider.class.getName());
+ System.setProperty("zkACLProvider", DummyZkAclProvider.class.getName());
+
+ ProviderRegistry.initialize();
+
+ configureCluster(NUM_SERVERS)// nodes
+ .withSolrXml(MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML)
+ .withSecurityJson(TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegation.json"))
+ .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+ .configure();
+ }
+
+ @AfterClass
+ public static void tearDownClass() {
+ System.clearProperty("zookeeper.authProvider.1");
+ System.clearProperty("zkCredentialsProvider");
+ System.clearProperty("zkACLProvider");
+ }
+
+ @Test
+ public void testZkAcls() throws Exception {
+ ZooKeeper keeper = null;
+ try {
+ keeper = new ZooKeeper(cluster.getZkServer().getZkAddress(), (int) TimeUnit.MINUTES.toMillis(1), new Watcher() {
+ @Override
+ public void process(WatchedEvent arg0) {
+ // Do nothing
+ }
+ });
+
+ keeper.addAuthInfo("dummyauth", "solr".getBytes(StandardCharsets.UTF_8));
+
+ // Test well known paths.
+ checkNonSecurityACLs(keeper, "/solr.xml");
+ checkSecurityACLs(keeper, "/security/token");
+ checkSecurityACLs(keeper, "/security");
+
+ // Now test all ZK tree.
+ String zkHost = cluster.getSolrClient().getZkHost();
+ String zkChroot = zkHost.contains("/")? zkHost.substring(zkHost.indexOf("/")): null;
+ walkZkTree(keeper, zkChroot, "/");
+
+ } finally {
+ if (keeper != null) {
+ keeper.close();
+ }
+ }
+ }
+
+ private void walkZkTree (ZooKeeper keeper, String zkChroot, String path) throws Exception {
+ if (isSecurityZNode(zkChroot, path)) {
+ checkSecurityACLs(keeper, path);
+ } else {
+ checkNonSecurityACLs(keeper, path);
+ }
+
+ List children = keeper.getChildren(path, false);
+ for (String child : children) {
+ String subpath = path.endsWith("/") ? path + child : path + "/" + child;
+ walkZkTree(keeper, zkChroot, subpath);
+ }
+ }
+
+ private boolean isSecurityZNode(String zkChroot, String path) {
+ String temp = path;
+ if (zkChroot != null) {
+ temp = path.replace(zkChroot, "");
+ }
+ return !ZkStateReader.SOLR_SECURITY_CONF_PATH.equals(path) &&
+ temp.startsWith(SecurityAwareZkACLProvider.SECURITY_ZNODE_PATH);
+ }
+
+ private void checkSecurityACLs(ZooKeeper keeper, String path) throws Exception {
+ List acls = keeper.getACL(path, new Stat());
+ String message = String.format(Locale.ROOT, "Path %s ACLs found %s", path, acls);
+ assertEquals(message, 1, acls.size());
+ assertTrue(message, acls.contains(new ACL(ZooDefs.Perms.ALL, new Id("dummyauth", "solr"))));
+ }
+
+ private void checkNonSecurityACLs(ZooKeeper keeper, String path) throws Exception {
+ List acls = keeper.getACL(path, new Stat());
+ String message = String.format(Locale.ROOT, "Path %s ACLs found %s", path, acls);
+ assertEquals(message, 2, acls.size());
+ assertTrue(message, acls.contains(new ACL(ZooDefs.Perms.ALL, new Id("dummyauth", "solr"))));
+ assertTrue(message, acls.contains(new ACL(ZooDefs.Perms.READ, new Id("world", "anyone"))));
+ }
+
+ public static class DummyZKAuthProvider implements AuthenticationProvider {
+ public static final String zkSuperUser = "zookeeper";
+ public static final Collection validUsers = Arrays.asList(zkSuperUser, "solr", "foo");
+
+ @Override
+ public String getScheme() {
+ return "dummyauth";
+ }
+
+ @Override
+ public Code handleAuthentication(ServerCnxn arg0, byte[] arg1) {
+ String userName = new String(arg1, StandardCharsets.UTF_8);
+
+ if (validUsers.contains(userName)) {
+ if (zkSuperUser.equals(userName)) {
+ arg0.addAuthInfo(new Id("super", ""));
+ }
+ arg0.addAuthInfo(new Id(getScheme(), userName));
+ return KeeperException.Code.OK;
+ }
+
+ return KeeperException.Code.AUTHFAILED;
+ }
+
+ @Override
+ public boolean isAuthenticated() {
+ return true;
+ }
+
+ @Override
+ public boolean isValid(String arg0) {
+ return (arg0 != null) && validUsers.contains(arg0);
+ }
+
+ @Override
+ public boolean matches(String arg0, String arg1) {
+ return arg0.equals(arg1);
+ }
+ }
+
+ public static class DummyZkCredentialsProvider implements ZkCredentialsProvider {
+ public static final Collection solrCreds =
+ Arrays.asList(new ZkCredentials("dummyauth", "solr".getBytes(StandardCharsets.UTF_8)));
+
+ @Override
+ public Collection getCredentials() {
+ return solrCreds;
+ }
+ }
+
+ public static class DummyZkAclProvider extends SecurityAwareZkACLProvider {
+
+ @Override
+ protected List createNonSecurityACLsToAdd() {
+ List result = new ArrayList<>(2);
+ result.add(new ACL(ZooDefs.Perms.ALL, new Id("dummyauth", "solr")));
+ result.add(new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE));
+
+ return result;
+ }
+
+ @Override
+ protected List createSecurityACLsToAdd() {
+ List ret = new ArrayList();
+ ret.add(new ACL(ZooDefs.Perms.ALL, new Id("dummyauth", "solr")));
+ return ret;
+ }
+ }
+
+}
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index 06052819f6b..7f4f0cb4e8d 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -86,6 +86,8 @@ public class MiniSolrCloudCluster {
" 10000\n" +
" ${distribUpdateConnTimeout:45000}\n" +
" ${distribUpdateSoTimeout:340000}\n" +
+ " ${zkCredentialsProvider:org.apache.solr.common.cloud.DefaultZkCredentialsProvider} \n" +
+ " ${zkACLProvider:org.apache.solr.common.cloud.DefaultZkACLProvider} \n" +
" \n" +
" \n" +
" \n" +
From 5a36775d6517cbb36429981ccf4eb923dc1c7b33 Mon Sep 17 00:00:00 2001
From: Dawid Weiss
Date: Wed, 9 Aug 2017 14:40:43 +0200
Subject: [PATCH 17/26] LUCENE-7923: Removed FST.Arc.node field (unused).
---
lucene/CHANGES.txt | 2 ++
.../src/java/org/apache/lucene/util/fst/FST.java | 13 -------------
.../src/java/org/apache/lucene/util/fst/Util.java | 4 +---
3 files changed, 3 insertions(+), 16 deletions(-)
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 5bfc6df9879..8edd0bdba5e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -188,6 +188,8 @@ Optimizations
Other
+* LUCENE-7923: Removed FST.Arc.node field (unused). (Dawid Weiss)
+
* LUCENE-7328: Remove LegacyNumericEncoding from GeoPointField. (Nick Knize)
* LUCENE-7360: Remove Explanation.toHtml() (Alan Woodward)
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index 31ee2b7bde8..82305fed354 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -161,10 +161,6 @@ public final class FST implements Accountable {
public int label;
public T output;
- // From node (ord or address); currently only used when
- // building an FST w/ willPackFST=true:
- long node;
-
/** To node (ord or address) */
public long target;
@@ -193,7 +189,6 @@ public final class FST implements Accountable {
/** Returns this */
public Arc copyFrom(Arc other) {
- node = other.node;
label = other.label;
target = other.target;
flags = other.flags;
@@ -224,7 +219,6 @@ public final class FST implements Accountable {
@Override
public String toString() {
StringBuilder b = new StringBuilder();
- b.append("node=" + node);
b.append(" target=" + target);
b.append(" label=0x" + Integer.toHexString(label));
if (flag(BIT_FINAL_ARC)) {
@@ -770,7 +764,6 @@ public final class FST implements Accountable {
return arc;
} else {
in.setPosition(follow.target);
- arc.node = follow.target;
final byte b = in.readByte();
if (b == ARCS_AS_FIXED_ARRAY) {
// array: jump straight to end
@@ -842,7 +835,6 @@ public final class FST implements Accountable {
if (follow.target <= 0) {
arc.flags |= BIT_LAST_ARC;
} else {
- arc.node = follow.target;
// NOTE: nextArc is a node (not an address!) in this case:
arc.nextArc = follow.target;
}
@@ -860,7 +852,6 @@ public final class FST implements Accountable {
//System.out.println(" readFirstRealTargtArc address="
//+ address);
//System.out.println(" flags=" + arc.flags);
- arc.node = node;
if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
//System.out.println(" fixedArray");
@@ -1035,7 +1026,6 @@ public final class FST implements Accountable {
assert cachedArc.label == result.label;
assert cachedArc.nextArc == result.nextArc;
assert cachedArc.nextFinalOutput.equals(result.nextFinalOutput);
- assert cachedArc.node == result.node;
assert cachedArc.numArcs == result.numArcs;
assert cachedArc.output.equals(result.output);
assert cachedArc.posArcsStart == result.posArcsStart;
@@ -1066,7 +1056,6 @@ public final class FST implements Accountable {
arc.flags = 0;
// NOTE: nextArc is a node (not an address!) in this case:
arc.nextArc = follow.target;
- arc.node = follow.target;
}
arc.output = follow.nextFinalOutput;
arc.label = END_LABEL;
@@ -1098,8 +1087,6 @@ public final class FST implements Accountable {
in.setPosition(follow.target);
- arc.node = follow.target;
-
// System.out.println("fta label=" + (char) labelToMatch);
if (in.readByte() == ARCS_AS_FIXED_ARRAY) {
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/Util.java b/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
index 2f83dd18f0c..ba2ff742912 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
@@ -620,8 +620,7 @@ public final class Util {
*
*
* Note: larger FSTs (a few thousand nodes) won't even
- * render, don't bother. If the FST is > 2.1 GB in size
- * then this method will throw strange exceptions.
+ * render, don't bother.
*
* @param sameRank
* If true, the resulting dot file will try
@@ -945,7 +944,6 @@ public final class Util {
arc.flags = 0;
// NOTE: nextArc is a node (not an address!) in this case:
arc.nextArc = follow.target;
- arc.node = follow.target;
}
arc.output = follow.nextFinalOutput;
arc.label = FST.END_LABEL;
From d4b4782943f79787d0931b24b839e9cc99e81c20 Mon Sep 17 00:00:00 2001
From: Andrzej Bialecki
Date: Wed, 9 Aug 2017 16:07:35 +0200
Subject: [PATCH 18/26] SOLR-11061: Add a spins metric for data directory
paths.
---
solr/CHANGES.txt | 2 ++
.../org/apache/solr/core/CoreContainer.java | 18 ++++++++++++++++++
.../java/org/apache/solr/core/SolrCore.java | 9 +++++++++
.../metrics/SolrMetricsIntegrationTest.java | 12 ++++++++++++
4 files changed, 41 insertions(+)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9e401c26352..6962e2f2263 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -134,6 +134,8 @@ Other Changes
* SOLR-11090: Add Replica.getProperty accessor. (Christine Poerschke)
+* SOLR-11061: Add a spins metric for data directory paths. (ab)
+
================== 7.0.0 ==================
Versions of Major Components
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 6013b284ca5..bf24db86548 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -568,12 +568,30 @@ public class CoreContainer {
true, "usableSpace", SolrInfoBean.Category.CONTAINER.toString(), "fs");
metricManager.registerGauge(null, registryName, () -> dataHome.toAbsolutePath().toString(),
true, "path", SolrInfoBean.Category.CONTAINER.toString(), "fs");
+ metricManager.registerGauge(null, registryName, () -> {
+ try {
+ return org.apache.lucene.util.IOUtils.spins(dataHome.toAbsolutePath());
+ } catch (IOException e) {
+ // default to spinning
+ return true;
+ }
+ },
+ true, "spins", SolrInfoBean.Category.CONTAINER.toString(), "fs");
metricManager.registerGauge(null, registryName, () -> cfg.getCoreRootDirectory().toFile().getTotalSpace(),
true, "totalSpace", SolrInfoBean.Category.CONTAINER.toString(), "fs", "coreRoot");
metricManager.registerGauge(null, registryName, () -> cfg.getCoreRootDirectory().toFile().getUsableSpace(),
true, "usableSpace", SolrInfoBean.Category.CONTAINER.toString(), "fs", "coreRoot");
metricManager.registerGauge(null, registryName, () -> cfg.getCoreRootDirectory().toAbsolutePath().toString(),
true, "path", SolrInfoBean.Category.CONTAINER.toString(), "fs", "coreRoot");
+ metricManager.registerGauge(null, registryName, () -> {
+ try {
+ return org.apache.lucene.util.IOUtils.spins(cfg.getCoreRootDirectory().toAbsolutePath());
+ } catch (IOException e) {
+ // default to spinning
+ return true;
+ }
+ },
+ true, "spins", SolrInfoBean.Category.CONTAINER.toString(), "fs", "coreRoot");
// add version information
metricManager.registerGauge(null, registryName, () -> this.getClass().getPackage().getSpecificationVersion(),
true, "specification", SolrInfoBean.Category.CONTAINER.toString(), "version");
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index cc6a9c28e43..e8f1358fab0 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1162,6 +1162,15 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
File dataDirFile = dataDirPath.toFile();
manager.registerGauge(this, registry, () -> dataDirFile.getTotalSpace(), true, "totalSpace", Category.CORE.toString(), "fs");
manager.registerGauge(this, registry, () -> dataDirFile.getUsableSpace(), true, "usableSpace", Category.CORE.toString(), "fs");
+ manager.registerGauge(this, registry, () -> dataDirPath.toAbsolutePath().toString(), true, "fs", "dataDir");
+ manager.registerGauge(this, registry, () -> {
+ try {
+ return org.apache.lucene.util.IOUtils.spins(dataDirPath.toAbsolutePath());
+ } catch (IOException e) {
+ // default to spinning
+ return true;
+ }
+ }, true, "spins", Category.CORE.toString(), "fs", "dataDir");
}
private void checkVersionFieldExistsInSchema(IndexSchema schema, CoreDescriptor coreDescriptor) {
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
index 1a8eda80045..1184402cd91 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -27,6 +27,7 @@ import com.codahale.metrics.Metric;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import org.apache.commons.io.FileUtils;
+import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.TestUtil;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.core.CoreContainer;
@@ -166,12 +167,23 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
assertTrue(metrics.containsKey("CONTAINER.fs.totalSpace"));
assertTrue(metrics.containsKey("CONTAINER.fs.usableSpace"));
assertTrue(metrics.containsKey("CONTAINER.fs.path"));
+ assertTrue(metrics.containsKey("CONTAINER.fs.spins"));
assertTrue(metrics.containsKey("CONTAINER.fs.coreRoot.totalSpace"));
assertTrue(metrics.containsKey("CONTAINER.fs.coreRoot.usableSpace"));
assertTrue(metrics.containsKey("CONTAINER.fs.coreRoot.path"));
+ assertTrue(metrics.containsKey("CONTAINER.fs.coreRoot.spins"));
assertTrue(metrics.containsKey("CONTAINER.version.specification"));
assertTrue(metrics.containsKey("CONTAINER.version.implementation"));
Gauge> g = (Gauge>)metrics.get("CONTAINER.fs.path");
assertEquals(g.getValue(), cc.getResourceLoader().getInstancePath().toAbsolutePath().toString());
+ boolean spins = IOUtils.spins(cc.getCoreRootDirectory());
+ g = (Gauge>)metrics.get("CONTAINER.fs.coreRoot.spins");
+ g = (Gauge>)metrics.get("CONTAINER.fs.coreRoot.spins");
+ assertEquals(spins, g.getValue());
+ if (cc.getConfig().getSolrDataHome() != null) {
+ spins = IOUtils.spins(cc.getConfig().getSolrDataHome());
+ g = (Gauge>)metrics.get("CONTAINER.fs.spins");
+ assertEquals(spins, g.getValue());
+ }
}
}
From 915b36564fcb728f467949775a4c18b274a933a7 Mon Sep 17 00:00:00 2001
From: Andrzej Bialecki
Date: Wed, 9 Aug 2017 17:14:58 +0200
Subject: [PATCH 19/26] SOLR-11061: Fix incorrect metric path.
---
solr/core/src/java/org/apache/solr/core/SolrCore.java | 4 ++--
.../org/apache/solr/metrics/SolrMetricsIntegrationTest.java | 5 +++--
2 files changed, 5 insertions(+), 4 deletions(-)
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index e8f1358fab0..641d1a1bb62 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1162,7 +1162,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
File dataDirFile = dataDirPath.toFile();
manager.registerGauge(this, registry, () -> dataDirFile.getTotalSpace(), true, "totalSpace", Category.CORE.toString(), "fs");
manager.registerGauge(this, registry, () -> dataDirFile.getUsableSpace(), true, "usableSpace", Category.CORE.toString(), "fs");
- manager.registerGauge(this, registry, () -> dataDirPath.toAbsolutePath().toString(), true, "fs", "dataDir");
+ manager.registerGauge(this, registry, () -> dataDirPath.toAbsolutePath().toString(), true, "path", Category.CORE.toString(), "fs");
manager.registerGauge(this, registry, () -> {
try {
return org.apache.lucene.util.IOUtils.spins(dataDirPath.toAbsolutePath());
@@ -1170,7 +1170,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
// default to spinning
return true;
}
- }, true, "spins", Category.CORE.toString(), "fs", "dataDir");
+ }, true, "spins", Category.CORE.toString(), "fs");
}
private void checkVersionFieldExistsInSchema(IndexSchema schema, CoreDescriptor coreDescriptor) {
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
index 1184402cd91..055109e0fb5 100644
--- a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -178,11 +178,12 @@ public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
assertEquals(g.getValue(), cc.getResourceLoader().getInstancePath().toAbsolutePath().toString());
boolean spins = IOUtils.spins(cc.getCoreRootDirectory());
g = (Gauge>)metrics.get("CONTAINER.fs.coreRoot.spins");
- g = (Gauge>)metrics.get("CONTAINER.fs.coreRoot.spins");
assertEquals(spins, g.getValue());
+ g = (Gauge>)metrics.get("CONTAINER.fs.spins");
if (cc.getConfig().getSolrDataHome() != null) {
spins = IOUtils.spins(cc.getConfig().getSolrDataHome());
- g = (Gauge>)metrics.get("CONTAINER.fs.spins");
+ assertEquals(spins, g.getValue());
+ } else {
assertEquals(spins, g.getValue());
}
}
From 0250368751fba54472cd7eddbbbf1ea5e0217a78 Mon Sep 17 00:00:00 2001
From: Shalin Shekhar Mangar
Date: Wed, 9 Aug 2017 23:20:37 +0530
Subject: [PATCH 20/26] Ignoring flakey test
---
.../apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java | 2 ++
1 file changed, 2 insertions(+)
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
index ed734991c06..a597a2fe945 100644
--- a/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestZkAclsWithHadoopAuth.java
@@ -44,8 +44,10 @@ import org.apache.zookeeper.server.auth.AuthenticationProvider;
import org.apache.zookeeper.server.auth.ProviderRegistry;
import org.junit.AfterClass;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Test;
+@Ignore
public class TestZkAclsWithHadoopAuth extends SolrCloudTestCase {
protected static final int NUM_SERVERS = 1;
protected static final int NUM_SHARDS = 1;
From e7062b6f91c161965aec0cef5a9ae68280f306a4 Mon Sep 17 00:00:00 2001
From: Varun Thacker
Date: Wed, 9 Aug 2017 12:15:01 -0700
Subject: [PATCH 21/26] SOLR-11190: GraphQuery also supports string fields
which are indexed=false and docValues=true
---
solr/CHANGES.txt | 3 ++
.../solr/search/join/GraphQueryParser.java | 30 +++++++++++++++++++
.../solr/search/join/GraphTermsCollector.java | 5 +++-
.../solr/collection1/conf/schema_latest.xml | 4 +++
.../solr/search/join/GraphQueryTest.java | 24 ++++++++++++++-
solr/solr-ref-guide/src/other-parsers.adoc | 4 +++
6 files changed, 68 insertions(+), 2 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6962e2f2263..ba8222afe0b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -88,6 +88,9 @@ Bug Fixes
may not have a registered searcher. This causes spikes in response times when adding a replica
in busy clusters. (Ludovic Boutros, Timothy Potter, shalin)
+* SOLR-11190: GraphQuery also supports string fields which are indexed=false and docValues=true. Please refer to the
+ Javadocs for DocValuesTermsQuery for it's performance characteristics. (Karthik Ramachandran, Varun Thacker)
+
Optimizations
----------------------
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphQueryParser.java b/solr/core/src/java/org/apache/solr/search/join/GraphQueryParser.java
index 9c9b85234fc..3e2d938162e 100644
--- a/solr/core/src/java/org/apache/solr/search/join/GraphQueryParser.java
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphQueryParser.java
@@ -19,6 +19,7 @@ package org.apache.solr.search.join;
import org.apache.lucene.search.Query;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.schema.StrField;
import org.apache.solr.search.QParser;
import org.apache.solr.search.QueryParsing;
import org.apache.solr.search.SyntaxError;
@@ -45,6 +46,9 @@ public class GraphQueryParser extends QParser {
String fromField = localParams.get("from", "node_id");
String toField = localParams.get("to", "edge_ids");
+ validateFields(fromField);
+ validateFields(toField);
+
// only documents that do not have values in the edge id fields.
boolean onlyLeafNodes = localParams.getBool("returnOnlyLeaf", false);
// choose if you want to return documents that match the initial query or not.
@@ -65,5 +69,31 @@ public class GraphQueryParser extends QParser {
// return the parsed graph query.
return gq;
}
+
+ public void validateFields(String field) throws SyntaxError {
+
+ if (req.getSchema().getField(field) == null) {
+ throw new SyntaxError("field " + field + " not defined in schema");
+ }
+
+ if (req.getSchema().getField(field).getType().isPointField()) {
+ if (req.getSchema().getField(field).hasDocValues()) {
+ return;
+ } else {
+ throw new SyntaxError("point field " + field + " must have docValues=true");
+ }
+ }
+
+ if (req.getSchema().getField(field).getType() instanceof StrField) {
+ if ((req.getSchema().getField(field).hasDocValues() || req.getSchema().getField(field).indexed())) {
+ return;
+ } else {
+ throw new SyntaxError("string field " + field + " must have indexed=true or docValues=true");
+ }
+ }
+
+ throw new SyntaxError("FieldType for field=" + field + " not supported");
+
+ }
}
diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java b/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java
index 174db3c691f..07cec7d8d43 100644
--- a/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java
+++ b/solr/core/src/java/org/apache/solr/search/join/GraphTermsCollector.java
@@ -27,6 +27,7 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.AutomatonQuery;
import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.DocValuesTermsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.TermInSetQuery;
@@ -173,7 +174,9 @@ class GraphTermsCollector extends GraphEdgeCollector {
collectorTerms.get(i, ref);
termList.add(ref);
}
- q = new TermInSetQuery(matchField.getName(), termList);
+ q = (matchField.hasDocValues() && !matchField.indexed())
+ ? new DocValuesTermsQuery(matchField.getName(), termList)
+ : new TermInSetQuery(matchField.getName(), termList);
}
return q;
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
index 1135d20485f..889d171e393 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
@@ -239,6 +239,10 @@
+
+
+
+
diff --git a/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java b/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java
index e1cfc815c0e..4b550e4a844 100644
--- a/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java
+++ b/solr/core/src/test/org/apache/solr/search/join/GraphQueryTest.java
@@ -17,6 +17,7 @@
package org.apache.solr.search.join;
import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.SolrParams;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -25,7 +26,6 @@ public class GraphQueryTest extends SolrTestCaseJ4 {
@BeforeClass
public static void beforeTests() throws Exception {
-
initCore("solrconfig.xml","schema_latest.xml");
}
@@ -44,6 +44,9 @@ public class GraphQueryTest extends SolrTestCaseJ4 {
doGraph( params("node_id","node_fps", "edge_id","edge_fps") );
doGraph( params("node_id","node_dp", "edge_id","edge_dps") );
doGraph( params("node_id","node_dps", "edge_id","edge_dps") );
+
+ // string with indexed=false and docValues=true
+ doGraph( params("node_id","node_sdN", "edge_id","edge_sdsN") );
}
public void doGraph(SolrParams p) throws Exception {
@@ -118,4 +121,23 @@ public class GraphQueryTest extends SolrTestCaseJ4 {
);
}
+ @Test
+ public void testGraphQueryParserValidation() throws Exception {
+ // from schema field existence
+ doGraphQuery( params("node_id","node_nothere", "edge_id","edge_ss",
+ "message", "field node_nothere not defined in schema", "errorCode", String.valueOf(SolrException.ErrorCode.BAD_REQUEST.code)) );
+
+ // to schema field existence
+ doGraphQuery( params("node_id","node_s", "edge_id","edge_notthere",
+ "message", "field node_nothere not defined in schema", "errorCode", String.valueOf(SolrException.ErrorCode.BAD_REQUEST.code)) );
+ }
+
+ public void doGraphQuery(SolrParams p) throws Exception {
+ String message = p.get("message");
+ int errorCode = p.getInt("errorCode", SolrException.ErrorCode.UNKNOWN.code);
+
+ assertQEx(message , req(p, "q","{!graph from=${node_id} to=${edge_id} returnRoot=false maxDepth=1}id:doc_1")
+ , errorCode
+ );
+ }
}
diff --git a/solr/solr-ref-guide/src/other-parsers.adoc b/solr/solr-ref-guide/src/other-parsers.adoc
index e54910ae279..72ea05c82e8 100644
--- a/solr/solr-ref-guide/src/other-parsers.adoc
+++ b/solr/solr-ref-guide/src/other-parsers.adoc
@@ -307,6 +307,10 @@ The `graph` query parser does a breadth first, cyclic aware, graph traversal of
The graph is built according to linkages between documents based on the terms found in `from` and `to` fields that you specify as part of the query.
+The supported fieldTypes are point fields with docValues enabled or string fields with indexed=true or docValues=true.
+For string fields which are indexed=false and docValues=true please refer to the javadocs for `DocValuesTermsQuery`
+for it's performance characteristics so indexed=true will perform better for most use-cases.
+
=== Graph Query Parameters
`to`::
From 4fcd8a806f8641786b455e0e92ceaf4481a0180d Mon Sep 17 00:00:00 2001
From: Tomas Fernandez Lobbe
Date: Wed, 9 Aug 2017 16:45:03 -0700
Subject: [PATCH 22/26] SOLR-11071: Improve TestIntervalFacets.testRandom
---
solr/CHANGES.txt | 2 +
.../conf/schema-docValuesFaceting.xml | 9 +-
.../solr/request/TestIntervalFaceting.java | 211 ++++++++++++++----
3 files changed, 179 insertions(+), 43 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ba8222afe0b..5cfd0ea41cf 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -139,6 +139,8 @@ Other Changes
* SOLR-11061: Add a spins metric for data directory paths. (ab)
+* SOLR-11071: Improve TestIntervalFacets.testRandom (Tomás Fernández Löbbe)
+
================== 7.0.0 ==================
Versions of Major Components
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
index a8eed081d49..a18f230d3ce 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesFaceting.xml
@@ -28,7 +28,7 @@
-
+
+
+
+ id
@@ -78,6 +81,8 @@
+
+
@@ -85,5 +90,7 @@
+
+
diff --git a/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java b/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java
index 0421e03c96f..93575d2e0c2 100644
--- a/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java
+++ b/solr/core/src/test/org/apache/solr/request/TestIntervalFaceting.java
@@ -16,6 +16,13 @@
*/
package org.apache.solr.request;
+import java.lang.invoke.MethodHandles;
+import java.text.SimpleDateFormat;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Set;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrClient;
@@ -28,7 +35,10 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.request.IntervalFacets.FacetInterval;
import org.apache.solr.request.IntervalFacets.IntervalCompareResult;
import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.NumberType;
import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.StrField;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SyntaxError;
import org.apache.solr.util.RefCounted;
@@ -37,13 +47,13 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.lang.invoke.MethodHandles;
-import java.util.Arrays;
-
public class TestIntervalFaceting extends SolrTestCaseJ4 {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
+
+ private final static long DATE_START_TIME_RANDOM_TEST = 1499797224224L;
+ private final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", Locale.ROOT);
+
@BeforeClass
public static void beforeTests() throws Exception {
// we need DVs on point fields to compute stats & facets
@@ -245,13 +255,14 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
@Slow
public void testRandom() throws Exception {
// All field values will be a number between 0 and cardinality
- int cardinality = 100000;
+ int cardinality = 10000;
// Fields to use for interval faceting
String[] fields = new String[]{
- "test_s_dv", "test_i_dv", "test_l_dv", "test_f_dv", "test_d_dv",
- "test_ss_dv", "test_is_dv", "test_fs_dv", "test_ls_dv", "test_ds_dv", "test_s", "test_i",
- "test_l", "test_f", "test_d", "test_ss", "test_is", "test_fs", "test_ls", "test_ds",
- "test_i_p", "test_is_p", "test_l_p", "test_ls_p", "test_f_p", "test_fs_p", "test_d_p", "test_ds_p"};
+ "test_s_dv", "test_i_dv", "test_l_dv", "test_f_dv", "test_d_dv", "test_dt_dv",
+ "test_ss_dv", "test_is_dv", "test_fs_dv", "test_ls_dv", "test_ds_dv", "test_dts_dv", "test_s", "test_i",
+ "test_l", "test_f", "test_d", "test_dt", "test_ss", "test_is", "test_fs", "test_ls", "test_ds", "test_dts",
+ "test_i_p", "test_is_p", "test_l_p", "test_ls_p", "test_f_p", "test_fs_p", "test_d_p", "test_ds_p", "test_dts_p"
+ };
for (int i = 0; i < atLeast(500); i++) {
if (random().nextInt(50) == 0) {
//have some empty docs
@@ -263,30 +274,34 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
//delete some docs
assertU(delI(String.valueOf(i - 1)));
}
- String[] docFields = new String[(random().nextInt(5)) * 10 + 12];
+ String[] docFields = new String[(random().nextInt(5)) * 12 + 14];
docFields[0] = "id";
- docFields[1] = String.valueOf(i);
+ docFields[1] = String.valueOf(i * (random().nextBoolean()?1:-1)); // in the queries we do positive and negative
docFields[2] = "test_s";
- docFields[3] = String.valueOf(random().nextInt(cardinality));
+ docFields[3] = String.valueOf(randomInt(cardinality));
docFields[4] = "test_i";
- docFields[5] = String.valueOf(random().nextInt(cardinality));
+ docFields[5] = String.valueOf(randomInt(cardinality));
docFields[6] = "test_l";
- docFields[7] = String.valueOf(random().nextInt(cardinality));
+ docFields[7] = String.valueOf(randomLong(cardinality));
docFields[8] = "test_f";
- docFields[9] = String.valueOf(random().nextFloat() * cardinality);
+ docFields[9] = String.valueOf(randomFloat(cardinality));
docFields[10] = "test_d";
- docFields[11] = String.valueOf(random().nextDouble() * cardinality);
- for (int j = 12; j < docFields.length; ) {
+ docFields[11] = String.valueOf(raondomDouble(cardinality));
+ docFields[12] = "test_dt";
+ docFields[13] = dateFormat.format(new Date(randomMs(cardinality)));
+ for (int j = 14; j < docFields.length; ) {
docFields[j++] = "test_ss";
- docFields[j++] = String.valueOf(random().nextInt(cardinality));
+ docFields[j++] = String.valueOf(randomInt(cardinality));
docFields[j++] = "test_is";
- docFields[j++] = String.valueOf(random().nextInt(cardinality));
+ docFields[j++] = String.valueOf(randomInt(cardinality));
docFields[j++] = "test_ls";
- docFields[j++] = String.valueOf(random().nextInt(cardinality));
+ docFields[j++] = String.valueOf(randomLong(cardinality));
docFields[j++] = "test_fs";
- docFields[j++] = String.valueOf(random().nextFloat() * cardinality);
+ docFields[j++] = String.valueOf(randomFloat(cardinality));
docFields[j++] = "test_ds";
- docFields[j++] = String.valueOf(random().nextDouble() * cardinality);
+ docFields[j++] = String.valueOf(raondomDouble(cardinality));
+ docFields[j++] = "test_dts";
+ docFields[j++] = dateFormat.format(new Date(randomMs(cardinality)));
}
assertU(adoc(docFields));
if (random().nextInt(50) == 0) {
@@ -295,12 +310,64 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
}
assertU(commit());
- for (int i = 0; i < atLeast(100); i++) {
+ for (int i = 0; i < atLeast(10000); i++) {
doTestQuery(cardinality, fields);
}
}
+ long randomMs(int cardinality) {
+ return DATE_START_TIME_RANDOM_TEST + random().nextInt(cardinality) * 1000 * (random().nextBoolean()?1:-1);
+ }
+
+ double raondomDouble(int cardinality) {
+ if (rarely()) {
+ int num = random().nextInt(4);
+ if (num == 0) return Double.NEGATIVE_INFINITY;
+ if (num == 1) return Double.POSITIVE_INFINITY;
+ if (num == 2) return Double.MIN_VALUE;
+ if (num == 3) return Double.MAX_VALUE;
+ }
+ Double d = Double.NaN;
+ while (d.isNaN()) {
+ d = random().nextDouble();
+ }
+ return d * cardinality * (random().nextBoolean()?1:-1);
+ }
+
+ float randomFloat(int cardinality) {
+ if (rarely()) {
+ int num = random().nextInt(4);
+ if (num == 0) return Float.NEGATIVE_INFINITY;
+ if (num == 1) return Float.POSITIVE_INFINITY;
+ if (num == 2) return Float.MIN_VALUE;
+ if (num == 3) return Float.MAX_VALUE;
+ }
+ Float f = Float.NaN;
+ while (f.isNaN()) {
+ f = random().nextFloat();
+ }
+ return f * cardinality * (random().nextBoolean()?1:-1);
+ }
+
+ int randomInt(int cardinality) {
+ if (rarely()) {
+ int num = random().nextInt(2);
+ if (num == 0) return Integer.MAX_VALUE;
+ if (num == 1) return Integer.MIN_VALUE;
+ }
+ return random().nextInt(cardinality) * (random().nextBoolean()?1:-1);
+ }
+
+ long randomLong(int cardinality) {
+ if (rarely()) {
+ int num = random().nextInt(2);
+ if (num == 0) return Long.MAX_VALUE;
+ if (num == 1) return Long.MIN_VALUE;
+ }
+ return randomInt(cardinality);
+ }
+
/**
* Executes one query using interval faceting and compares with the same query using
* facet query with the same range
@@ -309,18 +376,22 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
private void doTestQuery(int cardinality, String[] fields) throws Exception {
String[] startOptions = new String[]{"(", "["};
String[] endOptions = new String[]{")", "]"};
- // the query should match some documents in most cases
- Integer[] qRange = getRandomRange(cardinality, "id");
ModifiableSolrParams params = new ModifiableSolrParams();
- params.set("q", "id:[" + qRange[0] + " TO " + qRange[1] + "]");
+ if (rarely()) {
+ params.set("q", "*:*");
+ } else {
+ // the query should match some documents in most cases
+ String[] qRange = getRandomRange(cardinality, "id");
+ params.set("q", "id:[" + qRange[0] + " TO " + qRange[1] + "]");
+ }
params.set("facet", "true");
- String field = fields[random().nextInt(fields.length)]; //choose from any of the fields
+ String field = pickRandom(fields); //choose from any of the fields
params.set("facet.interval", field);
// number of intervals
for (int i = 0; i < 1 + random().nextInt(20); i++) {
- Integer[] interval = getRandomRange(cardinality, field);
- String open = startOptions[interval[0] % 2];
- String close = endOptions[interval[1] % 2];
+ String[] interval = getRandomRange(cardinality, field);
+ String open = pickRandom(startOptions);
+ String close = pickRandom(endOptions);
params.add("f." + field + ".facet.interval.set", open + interval[0] + "," + interval[1] + close);
params.add("facet.query", field + ":" + open.replace('(', '{') + interval[0] + " TO " + interval[1] + close.replace(')', '}'));
}
@@ -331,10 +402,11 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
NamedList