This commit is contained in:
Karl Wright 2017-08-30 10:40:56 -04:00
commit 54d760fdbf
10 changed files with 1278 additions and 182 deletions

View File

@ -20,6 +20,9 @@ New Features
* LUCENE-7621: Add CoveringQuery, a query whose required number of matching
clauses can be defined per document. (Adrien Grand)
* LUCENE-7927: Add LongValueFacetCounts, to compute facet counts for individual
numeric values (Mike McCandless)
Optimizations
* LUCENE-7905: Optimize how OrdinalMap (used by
@ -84,6 +87,9 @@ New Features
* LUCENE-7838: Knn classifier based on fuzzified term queries (Tommaso Teofili)
* LUCENE-7855: Added advanced options of the Wikipedia tokenizer to its factory.
(Juan Pedro via Adrien Grand)
API Changes
* LUCENE-2605: Classic QueryParser no longer splits on whitespace by default.
@ -175,6 +181,9 @@ Bug Fixes
functions (Operations.isFinite and Operations.topsortState) to prevent
large automaton to overflow the stack (Robert Muir, Adrien Grand, Jim Ferenczi)
* LUCENE-7864: IndexMergeTool is not using intermediate hard links (even
if possible). (Dawid Weiss)
Improvements
* LUCENE-7489: Better storage of sparse doc-values fields with the default
@ -186,6 +195,8 @@ Improvements
* LUCENE-7901: Original Highlighter now eagerly throws an exception if you
provide components that are null. (Jason Gerlowski, David Smiley)
* LUCENE-7841: Normalize ґ to г in Ukrainian analyzer. (Andriy Rysin via Dawid Weiss)
Optimizations
* LUCENE-7416: BooleanQuery optimizes queries that have queries that occur both
@ -210,6 +221,10 @@ Optimizations
* LUCENE-7874: DisjunctionMaxQuery rewrites to a BooleanQuery when tiebreaker is set to 1. (Jim Ferenczi)
* LUCENE-7828: Speed up range queries on range fields by improving how we
compute the relation between the query and inner nodes of the BKD tree.
(Adrien Grand)
Other
* LUCENE-7923: Removed FST.Arc.node field (unused). (Dawid Weiss)
@ -247,27 +262,7 @@ Other
* LUCENE-7773: Remove unused/deprecated token types from StandardTokenizer.
(Ahmet Arslan via Steve Rowe)
======================= Lucene 6.7.0 =======================
New Features
* LUCENE-7855: Added advanced options of the Wikipedia tokenizer to its factory.
(Juan Pedro via Adrien Grand)
Bug Fixes
* LUCENE-7864: IndexMergeTool is not using intermediate hard links (even
if possible). (Dawid Weiss)
* LUCENE-7869: Changed MemoryIndex to sort 1d points. In case of 1d points, the PointInSetQuery.MergePointVisitor expects
that these points are visited in ascending order. The memory index doesn't do this and this can result in document
with multiple points that should match to not match. (Martijn van Groningen)
* LUCENE-7878: Fix query builder to keep the SHOULD clause that wraps multi-word synonyms. (Jim Ferenczi)
Other
* LUCENE-7800: Remove code that potentially rethrows checked exceptions
* LUCENE-7800: Remove code that potentially rethrows checked exceptions
from methods that don't declare them ("sneaky throw" hack). (Robert Muir,
Uwe Schindler, Dawid Weiss)
@ -275,15 +270,15 @@ Other
that are trivially replaced by LeafReader.terms() and MultiFields.getTerms()
(David Smiley)
Improvements
======================= Lucene 6.6.1 =======================
* LUCENE-7841: Normalize ґ to г in Ukrainian analyzer. (Andriy Rysin via Dawid Weiss)
Bug Fixes
Optimizations
* LUCENE-7869: Changed MemoryIndex to sort 1d points. In case of 1d points, the PointInSetQuery.MergePointVisitor expects
that these points are visited in ascending order. The memory index doesn't do this and this can result in document
with multiple points that should match to not match. (Martijn van Groningen)
* LUCENE-7828: Speed up range queries on range fields by improving how we
compute the relation between the query and inner nodes of the BKD tree.
(Adrien Grand)
* LUCENE-7878: Fix query builder to keep the SHOULD clause that wraps multi-word synonyms. (Jim Ferenczi)
======================= Lucene 6.6.0 =======================

View File

@ -0,0 +1,499 @@
/*
* 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.facet;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.facet.FacetResult;
import org.apache.lucene.facet.Facets;
import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
import org.apache.lucene.facet.FacetsCollector;
import org.apache.lucene.facet.LabelAndValue;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.search.ConjunctionDISI;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.LongValues;
import org.apache.lucene.search.LongValuesSource;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.PriorityQueue;
/** {@link Facets} implementation that computes counts for
* all uniqute long values, more efficiently counting small values (0-1023) using an int array,
* and switching to a <code>HashMap</code> for values above 1023.
* Retrieve all facet counts, in value order, with {@link #getAllChildrenSortByValue},
* or get the topN values sorted by count with {@link #getTopChildrenSortByCount}.
*
* @lucene.experimental */
public class LongValueFacetCounts extends Facets {
/** Used for all values that are < 1K. */
private final int[] counts = new int[1024];
/** Used for all values that are >= 1K. */
private final HashTable hashCounts = new HashTable();
private final String field;
/** Total number of values counted, which is the subset of hits that had a value for this field. */
private int totCount;
/** Create {@code LongValueFacetCounts}, using either single-valued {@link
* NumericDocValues} or multi-valued {@link SortedNumericDocValues} from the
* specified field. */
public LongValueFacetCounts(String field, FacetsCollector hits, boolean multiValued) throws IOException {
this(field, null, hits, multiValued);
}
/** Create {@code LongValueFacetCounts}, using the provided
* {@link org.apache.lucene.queries.function.ValueSource}. If hits is
* null then all facets are counted. */
public LongValueFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits) throws IOException {
this(field, valueSource, hits, false);
}
/** Create {@code LongValueFacetCounts}, using the provided
* {@link org.apache.lucene.queries.function.ValueSource}.
* random access (implement {@link org.apache.lucene.search.DocIdSet#bits}). */
public LongValueFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits,
boolean multiValued) throws IOException {
this.field = field;
if (valueSource == null) {
if (multiValued) {
countMultiValued(field, hits.getMatchingDocs());
} else {
count(field, hits.getMatchingDocs());
}
} else {
// value source is always single valued
if (multiValued) {
throw new IllegalArgumentException("can only compute multi-valued facets directly from doc values (when valueSource is null)");
}
count(valueSource, hits.getMatchingDocs());
}
}
/** Counts all facet values for this reader. This produces the same result as computing
* facets on a {@link org.apache.lucene.search.MatchAllDocsQuery}, but is more efficient. */
public LongValueFacetCounts(String field, IndexReader reader, boolean multiValued) throws IOException {
this.field = field;
if (multiValued) {
countAllMultiValued(reader, field);
} else {
countAll(reader, field);
}
}
/** Counts all facet values for the provided {@link LongValuesSource}. This produces the same result as computing
* facets on a {@link org.apache.lucene.search.MatchAllDocsQuery}, but is more efficient. */
public LongValueFacetCounts(String field, LongValuesSource valueSource, IndexReader reader) throws IOException {
this.field = field;
countAll(valueSource, field, reader);
}
private void count(LongValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
for (MatchingDocs hits : matchingDocs) {
LongValues fv = valueSource.getValues(hits.context, null);
// NOTE: this is not as efficient as working directly with the doc values APIs in the sparse case
// because we are doing a linear scan across all hits, but this API is more flexible since a
// LongValuesSource can compute interesting values at query time
DocIdSetIterator docs = hits.bits.iterator();
for (int doc = docs.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS;) {
// Skip missing docs:
if (fv.advanceExact(doc)) {
increment(fv.longValue());
totCount++;
}
doc = docs.nextDoc();
}
}
}
private void count(String field, List<MatchingDocs> matchingDocs) throws IOException {
for (MatchingDocs hits : matchingDocs) {
NumericDocValues fv = hits.context.reader().getNumericDocValues(field);
if (fv == null) {
continue;
}
countOneSegment(fv, hits);
}
}
private void countOneSegment(NumericDocValues values, MatchingDocs hits) throws IOException {
DocIdSetIterator it = ConjunctionDISI.intersectIterators(
Arrays.asList(hits.bits.iterator(), values));
for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
increment(values.longValue());
totCount++;
}
}
/** Counts directly from SortedNumericDocValues. */
private void countMultiValued(String field, List<MatchingDocs> matchingDocs) throws IOException {
for (MatchingDocs hits : matchingDocs) {
SortedNumericDocValues values = hits.context.reader().getSortedNumericDocValues(field);
if (values == null) {
// this field has no doc values for this segment
continue;
}
NumericDocValues singleValues = DocValues.unwrapSingleton(values);
if (singleValues != null) {
countOneSegment(singleValues, hits);
} else {
DocIdSetIterator it = ConjunctionDISI.intersectIterators(
Arrays.asList(hits.bits.iterator(), values));
for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
int limit = values.docValueCount();
totCount += limit;
for (int i = 0; i < limit; i++) {
increment(values.nextValue());
}
}
}
}
}
/** Optimized version that directly counts all doc values. */
private void countAll(IndexReader reader, String field) throws IOException {
for (LeafReaderContext context : reader.leaves()) {
NumericDocValues values = context.reader().getNumericDocValues(field);
if (values == null) {
// this field has no doc values for this segment
continue;
}
countAllOneSegment(values);
}
}
private void countAllOneSegment(NumericDocValues values) throws IOException {
int doc;
while ((doc = values.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
totCount++;
increment(values.longValue());
}
}
private void countAll(LongValuesSource valueSource, String field, IndexReader reader) throws IOException {
for (LeafReaderContext context : reader.leaves()) {
LongValues fv = valueSource.getValues(context, null);
int maxDoc = context.reader().maxDoc();
for (int doc = 0; doc < maxDoc; doc++) {
// Skip missing docs:
if (fv.advanceExact(doc)) {
increment(fv.longValue());
totCount++;
}
}
}
}
private void countAllMultiValued(IndexReader reader, String field) throws IOException {
for (LeafReaderContext context : reader.leaves()) {
SortedNumericDocValues values = context.reader().getSortedNumericDocValues(field);
if (values == null) {
// this field has no doc values for this segment
continue;
}
NumericDocValues singleValues = DocValues.unwrapSingleton(values);
if (singleValues != null) {
countAllOneSegment(singleValues);
} else {
int doc;
while ((doc = values.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
int limit = values.docValueCount();
totCount += limit;
for (int i = 0; i < limit; i++) {
increment(values.nextValue());
}
}
}
}
}
private void increment(long value) {
/*
if (value >= 0 && value < counts.length) {
counts[(int) value]++;
} else {
hashCounts.add(value, 1);
}
*/
hashCounts.add(value, 1);
}
@Override
public FacetResult getTopChildren(int topN, String dim, String... path) {
if (dim.equals(field) == false) {
throw new IllegalArgumentException("invalid dim \"" + dim + "\"; should be \"" + field + "\"");
}
if (path.length != 0) {
throw new IllegalArgumentException("path.length should be 0");
}
return getTopChildrenSortByCount(topN);
}
/** Reusable hash entry to hold long facet value and int count. */
private static class Entry {
int count;
long value;
}
/** Returns the specified top number of facets, sorted by count. */
public FacetResult getTopChildrenSortByCount(int topN) {
PriorityQueue<Entry> pq = new PriorityQueue<Entry>(Math.min(topN, counts.length + hashCounts.size)) {
@Override
protected boolean lessThan(Entry a, Entry b) {
// sort by count descending, breaking ties by value ascending:
return a.count < b.count || (a.count == b.count && a.value > b.value);
}
};
int childCount = 0;
Entry e = null;
for (int i = 0; i < counts.length; i++) {
if (counts[i] != 0) {
childCount++;
if (e == null) {
e = new Entry();
}
e.value = i;
e.count = counts[i];
e = pq.insertWithOverflow(e);
}
}
if (hashCounts.size != 0) {
childCount += hashCounts.size;
for (int i = 0; i < hashCounts.values.length; i++) {
int count = hashCounts.counts[i];
if (count != 0) {
if (e == null) {
e = new Entry();
}
e.value = hashCounts.values[i];
e.count = count;
e = pq.insertWithOverflow(e);
}
}
}
LabelAndValue[] results = new LabelAndValue[pq.size()];
while (pq.size() != 0) {
Entry entry = pq.pop();
results[pq.size()] = new LabelAndValue(Long.toString(entry.value), entry.count);
}
return new FacetResult(field, new String[0], totCount, results, childCount);
}
/** Returns all unique values seen, sorted by value. */
public FacetResult getAllChildrenSortByValue() {
List<LabelAndValue> labelValues = new ArrayList<>();
// compact & sort hash table's arrays by value
int upto = 0;
for (int i = 0; i < hashCounts.values.length; i++) {
if (hashCounts.counts[i] != 0) {
hashCounts.counts[upto] = hashCounts.counts[i];
hashCounts.values[upto] = hashCounts.values[i];
upto++;
}
}
// zero fill all remaining counts so if we are called again we don't mistake these as real values
Arrays.fill(hashCounts.counts, upto, hashCounts.counts.length, 0);
assert upto == hashCounts.size : "upto=" + upto + " hashCounts.size=" + hashCounts.size;
new InPlaceMergeSorter() {
@Override
public int compare(int i, int j) {
return Long.compare(hashCounts.values[i], hashCounts.values[j]);
}
@Override
public void swap(int i, int j) {
int x = hashCounts.counts[i];
hashCounts.counts[i] = hashCounts.counts[j];
hashCounts.counts[j] = x;
long y = hashCounts.values[j];
hashCounts.values[j] = hashCounts.values[i];
hashCounts.values[i] = y;
}
}.sort(0, upto);
boolean countsAdded = false;
for (int i = 0; i < upto; i++) {
/*
if (countsAdded == false && hashCounts.values[i] >= counts.length) {
countsAdded = true;
appendCounts(labelValues);
}
*/
labelValues.add(new LabelAndValue(Long.toString(hashCounts.values[i]),
hashCounts.counts[i]));
}
/*
if (countsAdded == false) {
appendCounts(labelValues);
}
*/
return new FacetResult(field, new String[0], totCount, labelValues.toArray(new LabelAndValue[0]), labelValues.size());
}
private void appendCounts(List<LabelAndValue> labelValues) {
for (int i = 0; i < counts.length; i++) {
if (counts[i] != 0) {
labelValues.add(new LabelAndValue(Long.toString(i), counts[i]));
}
}
}
@Override
public Number getSpecificValue(String dim, String... path) throws IOException {
// TODO: should we impl this?
throw new UnsupportedOperationException();
}
@Override
public List<FacetResult> getAllDims(int topN) throws IOException {
return Collections.singletonList(getTopChildren(topN, field));
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("LongValueFacetCounts totCount=");
b.append(totCount);
b.append(":\n");
for (int i = 0; i < counts.length; i++) {
if (counts[i] != 0) {
b.append(" ");
b.append(i);
b.append(" -> count=");
b.append(counts[i]);
b.append('\n');
}
}
if (hashCounts.size != 0) {
for (int i = 0; i < hashCounts.values.length; i++) {
if (hashCounts.counts[i] != 0) {
b.append(" ");
b.append(hashCounts.values[i]);
b.append(" -> count=");
b.append(hashCounts.counts[i]);
b.append('\n');
}
}
}
return b.toString();
}
/** Native typed hash table. */
static class HashTable {
static final float LOAD_FACTOR = 0.7f;
long[] values; // values identifying a value
int[] counts;
int mask;
int size;
int threshold;
HashTable() {
int capacity = 64; // must be a power of 2
values = new long[capacity];
counts = new int[capacity];
mask = capacity - 1;
size = 0;
threshold = (int) (capacity * LOAD_FACTOR);
}
private int hash(long v) {
int h = (int) (v ^ (v >>> 32));
h = (31 * h) & mask; // * 31 to try to use the whole table, even if values are dense
return h;
}
void add(long value, int inc) {
if (size >= threshold) {
rehash();
}
final int h = hash(value);
for (int slot = h;; slot = (slot + 1) & mask) {
if (counts[slot] == 0) {
values[slot] = value;
++size;
} else if (values[slot] != value) {
continue;
}
counts[slot] += inc;
break;
}
}
private void rehash() {
final long[] oldValues = values;
final int[] oldCounts = counts;
final int newCapacity = values.length * 2;
values = new long[newCapacity];
counts = new int[newCapacity];
mask = newCapacity - 1;
threshold = (int) (LOAD_FACTOR * newCapacity);
size = 0;
for (int i = 0; i < oldValues.length; ++i) {
if (oldCounts[i] > 0) {
add(oldValues[i], oldCounts[i]);
}
}
}
}
}

View File

@ -51,16 +51,18 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
this(field, LongValuesSource.fromLongField(field), hits, ranges);
}
/** Create {@code RangeFacetCounts}, using the provided
/** Create {@code LongRangeFacetCounts}, using the provided
* {@link ValueSource}. */
public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
this(field, valueSource, hits, null, ranges);
}
/** Create {@code RangeFacetCounts}, using the provided
/** Create {@code LongRangeFacetCounts}, using the provided
* {@link ValueSource}, and using the provided Filter as
* a fastmatch: only documents passing the filter are
* checked for the matching ranges. The filter must be
* checked for the matching ranges, which is helpful when
* the provided {@link LongValuesSource} is costly per-document,
* such as a geo distance. The filter must be
* random access (implement {@link DocIdSet#bits}). */
public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
super(field, ranges, fastMatchQuery);
@ -121,7 +123,7 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
missingCount += x;
//System.out.println("totCount " + totCount + " missingCount " + counter.missingCount);
//System.out.println("totCount " + totCount + " x " + x + " missingCount " + missingCount);
totCount -= missingCount;
}
}

View File

@ -78,7 +78,7 @@ abstract class RangeFacetCounts extends Facets {
@Override
public List<FacetResult> getAllDims(int topN) throws IOException {
return Collections.singletonList(getTopChildren(topN, null));
return Collections.singletonList(getTopChildren(topN, field));
}
@Override

View File

@ -65,6 +65,9 @@ class CharBlockArray implements Appendable, Serializable, CharSequence {
}
private void addBlock() {
if (blockSize * (long) (blocks.size() + 1) > Integer.MAX_VALUE) {
throw new IllegalStateException("cannot store more than 2 GB in CharBlockArray");
}
this.current = new Block(this.blockSize);
this.blocks.add(this.current);
}

View File

@ -0,0 +1,543 @@
/*
* 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.facet;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.IntPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.facet.FacetResult;
import org.apache.lucene.facet.Facets;
import org.apache.lucene.facet.FacetsCollector;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LongValuesSource;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
/** Tests long value facets. */
public class TestLongValueFacetCounts extends LuceneTestCase {
public void testBasic() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);
for (long l = 0; l < 100; l++) {
Document doc = new Document();
doc.add(new NumericDocValuesField("field", l % 5));
w.addDocument(doc);
}
// Also add Long.MAX_VALUE
Document doc = new Document();
doc.add(new NumericDocValuesField("field", Long.MAX_VALUE));
w.addDocument(doc);
IndexReader r = w.getReader();
w.close();
FacetsCollector fc = new FacetsCollector();
IndexSearcher s = newSearcher(r);
s.search(new MatchAllDocsQuery(), fc);
LongValueFacetCounts facets = new LongValueFacetCounts("field", fc, false);
FacetResult result = facets.getAllChildrenSortByValue();
assertEquals("dim=field path=[] value=101 childCount=6\n 0 (20)\n 1 (20)\n 2 (20)\n 3 (20)\n " +
"4 (20)\n 9223372036854775807 (1)\n",
result.toString());
r.close();
d.close();
}
public void testOnlyBigLongs() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);
for (long l = 0; l < 3; l++) {
Document doc = new Document();
doc.add(new NumericDocValuesField("field", Long.MAX_VALUE - l));
w.addDocument(doc);
}
IndexReader r = w.getReader();
w.close();
FacetsCollector fc = new FacetsCollector();
IndexSearcher s = newSearcher(r);
s.search(new MatchAllDocsQuery(), fc);
LongValueFacetCounts facets = new LongValueFacetCounts("field", fc, false);
FacetResult result = facets.getAllChildrenSortByValue();
assertEquals("dim=field path=[] value=3 childCount=3\n 9223372036854775805 (1)\n " +
"9223372036854775806 (1)\n 9223372036854775807 (1)\n",
result.toString());
r.close();
d.close();
}
public void testGetAllDims() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);
for (long l = 0; l < 100; l++) {
Document doc = new Document();
doc.add(new NumericDocValuesField("field", l % 5));
w.addDocument(doc);
}
// Also add Long.MAX_VALUE
Document doc = new Document();
doc.add(new NumericDocValuesField("field", Long.MAX_VALUE));
w.addDocument(doc);
IndexReader r = w.getReader();
w.close();
FacetsCollector fc = new FacetsCollector();
IndexSearcher s = newSearcher(r);
s.search(new MatchAllDocsQuery(), fc);
Facets facets = new LongValueFacetCounts("field", fc, false);
List<FacetResult> result = facets.getAllDims(10);
assertEquals(1, result.size());
assertEquals("dim=field path=[] value=101 childCount=6\n 0 (20)\n 1 (20)\n 2 (20)\n " +
"3 (20)\n 4 (20)\n 9223372036854775807 (1)\n",
result.get(0).toString());
r.close();
d.close();
}
public void testRandom() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
int valueCount = atLeast(1000);
double missingChance = random().nextDouble();
long maxValue;
if (random().nextBoolean()) {
maxValue = random().nextLong() & Long.MAX_VALUE;
} else {
maxValue = random().nextInt(1000);
}
if (VERBOSE) {
System.out.println("TEST: valueCount=" + valueCount + " valueRange=-" + maxValue +
"-" + maxValue + " missingChance=" + missingChance);
}
Long[] values = new Long[valueCount];
int missingCount = 0;
for (int i = 0; i < valueCount; i++) {
Document doc = new Document();
doc.add(new IntPoint("id", i));
if (random().nextDouble() > missingChance) {
long value = TestUtil.nextLong(random(), -maxValue, maxValue);
doc.add(new NumericDocValuesField("field", value));
values[i] = value;
} else {
missingCount++;
}
w.addDocument(doc);
}
IndexReader r = w.getReader();
w.close();
IndexSearcher s = newSearcher(r);
int iters = atLeast(100);
for (int iter = 0; iter < iters; iter++) {
FacetsCollector fc = new FacetsCollector();
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter);
System.out.println(" test all docs");
}
// all docs
Map<Long, Integer> expected = new HashMap<>();
int expectedChildCount = 0;
for (int i = 0; i < valueCount; i++) {
if (values[i] != null) {
Integer curCount = expected.get(values[i]);
if (curCount == null) {
curCount = 0;
expectedChildCount++;
}
expected.put(values[i], curCount + 1);
}
}
List<Map.Entry<Long, Integer>> expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
LongValueFacetCounts facetCounts;
if (random().nextBoolean()) {
s.search(new MatchAllDocsQuery(), fc);
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println(" use value source");
}
facetCounts = new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), fc);
} else {
if (VERBOSE) {
System.out.println(" use doc values");
}
facetCounts = new LongValueFacetCounts("field", fc, false);
}
} else {
// optimized count all:
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println(" count all value source");
}
facetCounts = new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), r);
} else {
if (VERBOSE) {
System.out.println(" count all doc values");
}
facetCounts = new LongValueFacetCounts("field", r, false);
}
}
FacetResult actual = facetCounts.getAllChildrenSortByValue();
assertSame("all docs, sort facets by value", expectedCounts, expectedChildCount,
valueCount - missingCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
int topN;
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
if (VERBOSE) {
System.out.println(" topN=" + topN);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("all docs, sort facets by count", expectedCounts, expectedChildCount, valueCount - missingCount, actual, topN);
// subset of docs
int minId = random().nextInt(valueCount);
int maxId = random().nextInt(valueCount);
if (minId > maxId) {
int tmp = minId;
minId = maxId;
maxId = tmp;
}
if (VERBOSE) {
System.out.println(" test id range " + minId + "-" + maxId);
}
fc = new FacetsCollector();
s.search(IntPoint.newRangeQuery("id", minId, maxId), fc);
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println(" use doc values");
}
facetCounts = new LongValueFacetCounts("field", fc, false);
} else {
if (VERBOSE) {
System.out.println(" use value source");
}
facetCounts = new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), fc);
}
expected = new HashMap<>();
expectedChildCount = 0;
int totCount = 0;
for (int i = minId; i <= maxId; i++) {
if (values[i] != null) {
totCount++;
Integer curCount = expected.get(values[i]);
if (curCount == null) {
expectedChildCount++;
curCount = 0;
}
expected.put(values[i], curCount + 1);
}
}
expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
actual = facetCounts.getAllChildrenSortByValue();
assertSame("id " + minId + "-" + maxId + ", sort facets by value", expectedCounts,
expectedChildCount, totCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("id " + minId + "-" + maxId + ", sort facets by count", expectedCounts, expectedChildCount, totCount, actual, topN);
}
r.close();
dir.close();
}
public void testRandomMultiValued() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
int valueCount = atLeast(1000);
double missingChance = random().nextDouble();
// sometimes exercise codec optimizations when a claimed multi valued field is in fact single valued:
boolean allSingleValued = rarely();
long maxValue;
if (random().nextBoolean()) {
maxValue = random().nextLong() & Long.MAX_VALUE;
} else {
maxValue = random().nextInt(1000);
}
if (VERBOSE) {
System.out.println("TEST: valueCount=" + valueCount + " valueRange=-" + maxValue +
"-" + maxValue + " missingChance=" + missingChance + " allSingleValued=" + allSingleValued);
}
long[][] values = new long[valueCount][];
int missingCount = 0;
for (int i = 0; i < valueCount; i++) {
Document doc = new Document();
doc.add(new IntPoint("id", i));
if (random().nextDouble() > missingChance) {
if (allSingleValued) {
values[i] = new long[1];
} else {
values[i] = new long[TestUtil.nextInt(random(), 1, 5)];
}
for (int j = 0; j < values[i].length; j++) {
long value = TestUtil.nextLong(random(), -maxValue, maxValue);
values[i][j] = value;
doc.add(new SortedNumericDocValuesField("field", value));
}
if (VERBOSE) {
System.out.println(" doc=" + i + " values=" + Arrays.toString(values[i]));
}
} else {
missingCount++;
if (VERBOSE) {
System.out.println(" doc=" + i + " missing values");
}
}
w.addDocument(doc);
}
IndexReader r = w.getReader();
w.close();
IndexSearcher s = newSearcher(r);
int iters = atLeast(100);
for (int iter = 0; iter < iters; iter++) {
FacetsCollector fc = new FacetsCollector();
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter);
System.out.println(" test all docs");
}
// all docs
Map<Long, Integer> expected = new HashMap<>();
int expectedChildCount = 0;
int expectedTotalCount = 0;
for (int i = 0; i < valueCount; i++) {
if (values[i] != null) {
for (long value : values[i]) {
Integer curCount = expected.get(value);
if (curCount == null) {
curCount = 0;
expectedChildCount++;
}
expected.put(value, curCount + 1);
expectedTotalCount++;
}
}
}
List<Map.Entry<Long, Integer>> expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
LongValueFacetCounts facetCounts;
if (random().nextBoolean()) {
s.search(new MatchAllDocsQuery(), fc);
if (VERBOSE) {
System.out.println(" use doc values");
}
facetCounts = new LongValueFacetCounts("field", fc, true);
} else {
// optimized count all:
if (VERBOSE) {
System.out.println(" count all doc values");
}
facetCounts = new LongValueFacetCounts("field", r, true);
}
FacetResult actual = facetCounts.getAllChildrenSortByValue();
assertSame("all docs, sort facets by value", expectedCounts, expectedChildCount,
expectedTotalCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
int topN;
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
if (VERBOSE) {
System.out.println(" topN=" + topN);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("all docs, sort facets by count", expectedCounts, expectedChildCount, expectedTotalCount, actual, topN);
// subset of docs
int minId = random().nextInt(valueCount);
int maxId = random().nextInt(valueCount);
if (minId > maxId) {
int tmp = minId;
minId = maxId;
maxId = tmp;
}
if (VERBOSE) {
System.out.println(" test id range " + minId + "-" + maxId);
}
fc = new FacetsCollector();
s.search(IntPoint.newRangeQuery("id", minId, maxId), fc);
// cannot use value source here because we are multi valued
facetCounts = new LongValueFacetCounts("field", fc, true);
expected = new HashMap<>();
expectedChildCount = 0;
int totCount = 0;
for (int i = minId; i <= maxId; i++) {
if (values[i] != null) {
for (long value : values[i]) {
totCount++;
Integer curCount = expected.get(value);
if (curCount == null) {
expectedChildCount++;
curCount = 0;
}
expected.put(value, curCount + 1);
}
}
}
expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
actual = facetCounts.getAllChildrenSortByValue();
assertSame("id " + minId + "-" + maxId + ", sort facets by value", expectedCounts,
expectedChildCount, totCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("id " + minId + "-" + maxId + ", sort facets by count", expectedCounts, expectedChildCount, totCount, actual, topN);
}
r.close();
dir.close();
}
private static void assertSame(String desc, List<Map.Entry<Long, Integer>> expectedCounts,
int expectedChildCount, int expectedTotalCount, FacetResult actual, int topN) {
int expectedTopN = Math.min(topN, expectedCounts.size());
if (VERBOSE) {
System.out.println(" expected topN=" + expectedTopN);
for (int i = 0; i < expectedTopN; i++) {
System.out.println(" " + i + ": value=" + expectedCounts.get(i).getKey() + " count=" + expectedCounts.get(i).getValue());
}
System.out.println(" actual topN=" + actual.labelValues.length);
for (int i = 0; i < actual.labelValues.length; i++) {
System.out.println(" " + i + ": value=" + actual.labelValues[i].label + " count=" + actual.labelValues[i].value);
}
}
assertEquals(desc + ": topN", expectedTopN, actual.labelValues.length);
assertEquals(desc + ": childCount", expectedChildCount, actual.childCount);
assertEquals(desc + ": totCount", expectedTotalCount, actual.value.intValue());
assertTrue(actual.labelValues.length <= topN);
for (int i = 0; i < expectedTopN; i++) {
assertEquals(desc + ": label[" + i + "]", Long.toString(expectedCounts.get(i).getKey()), actual.labelValues[i].label);
assertEquals(desc + ": counts[" + i + "]", expectedCounts.get(i).getValue().intValue(), actual.labelValues[i].value.intValue());
}
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.facet.range;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
@ -28,8 +29,8 @@ import org.apache.lucene.document.DoublePoint;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.facet.DrillDownQuery;
import org.apache.lucene.facet.DrillSideways;
import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
import org.apache.lucene.facet.DrillSideways;
import org.apache.lucene.facet.FacetField;
import org.apache.lucene.facet.FacetResult;
import org.apache.lucene.facet.FacetTestCase;
@ -99,6 +100,44 @@ public class TestRangeFacetCounts extends FacetTestCase {
d.close();
}
public void testLongGetAllDims() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);
Document doc = new Document();
NumericDocValuesField field = new NumericDocValuesField("field", 0L);
doc.add(field);
for(long l=0;l<100;l++) {
field.setLongValue(l);
w.addDocument(doc);
}
// Also add Long.MAX_VALUE
field.setLongValue(Long.MAX_VALUE);
w.addDocument(doc);
IndexReader r = w.getReader();
w.close();
FacetsCollector fc = new FacetsCollector();
IndexSearcher s = newSearcher(r);
s.search(new MatchAllDocsQuery(), fc);
Facets facets = new LongRangeFacetCounts("field", fc,
new LongRange("less than 10", 0L, true, 10L, false),
new LongRange("less than or equal to 10", 0L, true, 10L, true),
new LongRange("over 90", 90L, false, 100L, false),
new LongRange("90 or above", 90L, true, 100L, false),
new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true));
List<FacetResult> result = facets.getAllDims(10);
assertEquals(1, result.size());
assertEquals("dim=field path=[] value=22 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (1)\n",
result.get(0).toString());
r.close();
d.close();
}
public void testUselessRange() {
expectThrows(IllegalArgumentException.class, () -> {
new LongRange("useless", 7, true, 6, true);

View File

@ -0,0 +1,46 @@
/*
* 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.facet.taxonomy.writercache;
import org.apache.lucene.util.LuceneTestCase.Monster;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@Monster("uses lots of space and takes a few minutes")
public class Test2GBCharBlockArray extends LuceneTestCase {
public void test2GBChars() throws Exception {
int blockSize = 32768;
CharBlockArray array = new CharBlockArray(blockSize);
int size = TestUtil.nextInt(random(), 20000, 40000);
char[] chars = new char[size];
int count = 0;
while (true) {
count++;
try {
array.append(chars, 0, size);
} catch (IllegalStateException ise) {
assertTrue(count * (long) size + blockSize > Integer.MAX_VALUE);
break;
}
assertFalse("appended " + (count * (long) size - Integer.MAX_VALUE) + " characters beyond Integer.MAX_VALUE!",
count * (long) size > Integer.MAX_VALUE);
}
}
}

View File

@ -27,11 +27,9 @@ import java.nio.file.Path;
import org.apache.lucene.facet.FacetTestCase;
import org.junit.Test;
public class TestCharBlockArray extends FacetTestCase {
@Test public void testArray() throws Exception {
public void testArray() throws Exception {
CharBlockArray array = new CharBlockArray();
StringBuilder builder = new StringBuilder();

View File

@ -325,6 +325,14 @@ Upgrading from Solr 6.x
detect Java 9 correctly and setup Garbage Collector logging. If the configuration file contains
logging options that are no longer supported with Java 9, startup will fail.
* SOLR-10307: If starting Jetty without the Solr start script, you must now pass keystore and truststore
passwords via the env variables SOLR_SSL_KEY_STORE_PASSWORD and SOLR_SSL_TRUST_STORE_PASSWORD rather
than system properties.
* SOLR-10379: ManagedSynonymFilterFactory has been deprecated in favor of ManagedSynonymGraphFilterFactory.
* SOLR-10503: CurrencyField has been deprecated in favor of new CurrencyFieldType.
New Features
----------------------
* SOLR-9857, SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer. (ab)
@ -406,6 +414,59 @@ New Features
* SOLR-11173 TermsComponent support for Points fields. (yonik)
* SOLR-10849: MoreLikeThisComponent should expose setMaxDocFreqPct (maxDoc
frequency percentage). (Dawid Weiss)
* SOLR-10307: Allow Passing SSL passwords through environment variables. (Mano Kovacs, Michael Suzuki via Mark Miller)
* SOLR-10379: Add ManagedSynonymGraphFilterFactory, deprecate ManagedSynonymFilterFactory. (Steve Rowe)
* SOLR-10479: Adds support for HttpShardHandlerFactory.loadBalancerRequests(MinimumAbsolute|MaximumFraction)
configuration. (Ramsey Haddad, Daniel Collins, Christine Poerschke)
* SOLR-3702: concat(...) function query (Andrey Kudryavtsev via Mikhail Khludnev)
* SOLR-10767: Add movingAvg Stream Evaluator (Joel Bernstein)
* SOLR-10813: Add arraySort Stream Evaluator (Joel Bernstein)
* SOLR-10696: Add cumulative probability function (Joel Bernstein)
* SOLR-10765: Add anova Stream Evaluator (Joel Bernstein)
* SOLR-10754: Add hist Stream Evaluator (Joel Bernstein)
* SOLR-10753: Add array Stream Evaluator (Joel Bernstein)
* SOLR-10747: Allow /stream handler to execute Stream Evaluators directly (Joel Bernstein)
* SOLR-10743: Add sequence StreamEvaluator (Joel Bernstein)
* SOLR-10684: Add finddelay Stream Evaluator (Joel Bernstein)
* SOLR-10731: Add knn Streaming Expression (Joel Bernstein)
* SOLR-10724: Add describe Stream Evaluator (Joel Bernstein)
* SOLR-10693: Add copyOfRange Stream Evaluator (Joel Bernstein)
* SOLR-10623: Add sql Streaming Expression (Joel Bernstein)
* SOLR-10661: Add copyOf Stream Evaluator (Joel Bernstein)
* SOLR-10663: Add distance Stream Evaluator (Joel Bernstein)
* SOLR-10664: Add scale Stream Evaluator (Joel Bernstein)
* SOLR-10666: Add rank transformation Stream Evaluator (Joel Bernstein)
* SOLR-10662: Add length Stream Evaluator (Joel Bernstein)
* SOLR-10660: Add reverse Stream Evaluator (Joel Bernstein)
* SOLR-9910: Add solr/solr.cmd parameter to append jetty parameters to the start script.
(Mano Kovacs via Mark Miller)
Bug Fixes
----------------------
* SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.
@ -504,6 +565,49 @@ Bug Fixes
* SOLR-8689: Fix bin/solr.cmd so it can run properly on Java 9 (Uwe Schindler, hossman)
* SOLR-10723 JSON Facet API: resize() implemented incorrectly for CountSlotAcc, HllAgg.NumericAcc
resulting in exceptions when using a hashing faceting method and sorting by hll(numeric_field).
(yonik)
* SOLR-10719: Creating a core.properties fails if the parent of core.properties is a symlinked dierctory
(Erick Erickson)
* SOLR-10360: Solr HDFS snapshot export fails due to FileNotFoundException error when using MR1 instead of
yarn. (Hrishikesh via Mark Miller)
* SOLR-10137: Ensure that ConfigSets created via API are mutable. (Hrishikesh via Mark Miller)
* SOLR-10829: Fixed IndexSchema to enforce that uniqueKey can not be Points based for correctness (hossman)
* SOLR-10836: The query parsers igain, significantTerms, and tlogit (used by streaming expressions by
the same name) might throw a NullPointerException if the referenced field had no indexed data in some
shards. The fix included an optimization to use Solr's cached AtomicReader instead of re-calculating.
(David Smiley)
* SOLR-10715: /v2/ should not be an alias for /v2/collections (Cao Manh Dat)
* SOLR-10835: Add support for point fields in Export Handler (Tomás Fernández Löbbe)
* SOLR-10704: REPLACENODE may cause data loss when replicationFactor is 1. (ab, shalin)
* SOLR-10833: Point numeric fields should throw SolrException(BAD_REQUEST) for malformed numbers in queries.
Trie numeric fields should throw SolrException(BAD_REQUEST) for malformed docValues range queries.
(hossman, Tomás Fernández Löbbe)
* SOLR-10832: Fixed VersionInfo.getMaxVersionFromIndex when using PointsField with indexed="true" (hossman)
* SOLR-10763: Admin UI replication tab sometimes empty when failed replications (janhoy, Bojan Vitnik)
* SOLR-10824: fix NPE ExactSharedStatsCache, fixing maxdocs skew for terms which are absent at one of shards
when using one of Exact*StatsCache (Mikhail Khludnev)
* SOLR-10963: Fix example json in MultipleAdditiveTreesModel javadocs.
(Stefan Langenmaier via Christine Poerschke)
* SOLR-10914: RecoveryStrategy's sendPrepRecoveryCmd can get stuck for 5 minutes if leader is unloaded. (shalin)
* SOLR-11198: downconfig downloads empty file as folder (Erick Erickson)
Optimizations
----------------------
@ -527,6 +631,13 @@ Optimizations
* SOLR-11070: Make docValues range queries behave the same as Trie/Point fields for Double/Float Infinity cases
(Tomás Fernández Löbbe, Andrey Kudryavtsev)
* SOLR-10634: JSON Facet API: When a field/terms facet will retrieve all buckets (i.e. limit:-1)
and there are no nested facets, aggregations are computed in the first collection phase
so that the second phase which would normally involve calculating the domain for the bucket
can be skipped entirely, leading to large performance improvements. (yonik)
* SOLR-10722: Speed up Solr's use of the UnifiedHighlighter be re-using FieldInfos. (David Smiley)
Other Changes
----------------------
* SOLR-10236: Removed FieldType.getNumericType(). Use getNumberType() instead. (Tomás Fernández Löbbe)
@ -715,155 +826,15 @@ Other Changes
* SOLR-11183: V2 APIs are now available at /api endpoint. (Ishan Chattopadhyaya)
================== 6.7.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
Versions of Major Components
---------------------
Apache Tika 1.13
Carrot2 3.15.0
Velocity 1.7 and Velocity Tools 2.0
Apache UIMA 2.3.1
Apache ZooKeeper 3.4.10
Jetty 9.3.14.v20161028
Detailed Change List
----------------------
Upgrade Notes
----------------------
* SOLR-10307: If starting Jetty without the Solr start script, you must now pass keystore and truststore
passwords via the env variables SOLR_SSL_KEY_STORE_PASSWORD and SOLR_SSL_TRUST_STORE_PASSWORD rather
than system properties.
* SOLR-10379: ManagedSynonymFilterFactory has been deprecated in favor of ManagedSynonymGraphFilterFactory.
* SOLR-10503: CurrencyField has been deprecated in favor of new CurrencyFieldType.
New Features
----------------------
* SOLR-10849: MoreLikeThisComponent should expose setMaxDocFreqPct (maxDoc
frequency percentage). (Dawid Weiss)
* SOLR-10307: Allow Passing SSL passwords through environment variables. (Mano Kovacs, Michael Suzuki via Mark Miller)
* SOLR-10379: Add ManagedSynonymGraphFilterFactory, deprecate ManagedSynonymFilterFactory. (Steve Rowe)
* SOLR-10479: Adds support for HttpShardHandlerFactory.loadBalancerRequests(MinimumAbsolute|MaximumFraction)
configuration. (Ramsey Haddad, Daniel Collins, Christine Poerschke)
* SOLR-3702: concat(...) function query (Andrey Kudryavtsev via Mikhail Khludnev)
* SOLR-10767: Add movingAvg Stream Evaluator (Joel Bernstein)
* SOLR-10813: Add arraySort Stream Evaluator (Joel Bernstein)
* SOLR-10696: Add cumulative probability function (Joel Bernstein)
* SOLR-10765: Add anova Stream Evaluator (Joel Bernstein)
* SOLR-10754: Add hist Stream Evaluator (Joel Bernstein)
* SOLR-10753: Add array Stream Evaluator (Joel Bernstein)
* SOLR-10747: Allow /stream handler to execute Stream Evaluators directly (Joel Bernstein)
* SOLR-10743: Add sequence StreamEvaluator (Joel Bernstein)
* SOLR-10684: Add finddelay Stream Evaluator (Joel Bernstein)
* SOLR-10731: Add knn Streaming Expression (Joel Bernstein)
* SOLR-10724: Add describe Stream Evaluator (Joel Bernstein)
* SOLR-10693: Add copyOfRange Stream Evaluator (Joel Bernstein)
* SOLR-10623: Add sql Streaming Expression (Joel Bernstein)
* SOLR-10661: Add copyOf Stream Evaluator (Joel Bernstein)
* SOLR-10663: Add distance Stream Evaluator (Joel Bernstein)
* SOLR-10664: Add scale Stream Evaluator (Joel Bernstein)
* SOLR-10666: Add rank transformation Stream Evaluator (Joel Bernstein)
* SOLR-10662: Add length Stream Evaluator (Joel Bernstein)
* SOLR-10660: Add reverse Stream Evaluator (Joel Bernstein)
* SOLR-9910: Add solr/solr.cmd parameter to append jetty parameters to the start script.
(Mano Kovacs via Mark Miller)
Bug Fixes
----------------------
* SOLR-10723 JSON Facet API: resize() implemented incorrectly for CountSlotAcc, HllAgg.NumericAcc
resulting in exceptions when using a hashing faceting method and sorting by hll(numeric_field).
(yonik)
* SOLR-10719: Creating a core.properties fails if the parent of core.properties is a symlinked dierctory
(Erick Erickson)
* SOLR-10360: Solr HDFS snapshot export fails due to FileNotFoundException error when using MR1 instead of
yarn. (Hrishikesh via Mark Miller)
* SOLR-10137: Ensure that ConfigSets created via API are mutable. (Hrishikesh via Mark Miller)
* SOLR-10829: Fixed IndexSchema to enforce that uniqueKey can not be Points based for correctness (hossman)
* SOLR-10836: The query parsers igain, significantTerms, and tlogit (used by streaming expressions by
the same name) might throw a NullPointerException if the referenced field had no indexed data in some
shards. The fix included an optimization to use Solr's cached AtomicReader instead of re-calculating.
(David Smiley)
* SOLR-10715: /v2/ should not be an alias for /v2/collections (Cao Manh Dat)
* SOLR-10835: Add support for point fields in Export Handler (Tomás Fernández Löbbe)
* SOLR-10704: REPLACENODE may cause data loss when replicationFactor is 1. (ab, shalin)
* SOLR-10833: Point numeric fields should throw SolrException(BAD_REQUEST) for malformed numbers in queries.
Trie numeric fields should throw SolrException(BAD_REQUEST) for malformed docValues range queries.
(hossman, Tomás Fernández Löbbe)
* SOLR-10832: Fixed VersionInfo.getMaxVersionFromIndex when using PointsField with indexed="true" (hossman)
* SOLR-10763: Admin UI replication tab sometimes empty when failed replications (janhoy, Bojan Vitnik)
* SOLR-10824: fix NPE ExactSharedStatsCache, fixing maxdocs skew for terms which are absent at one of shards
when using one of Exact*StatsCache (Mikhail Khludnev)
* SOLR-10963: Fix example json in MultipleAdditiveTreesModel javadocs.
(Stefan Langenmaier via Christine Poerschke)
* SOLR-10914: RecoveryStrategy's sendPrepRecoveryCmd can get stuck for 5 minutes if leader is unloaded. (shalin)
* SOLR-11198: downconfig downloads empty file as folder (Erick Erickson)
Optimizations
----------------------
* SOLR-10634: JSON Facet API: When a field/terms facet will retrieve all buckets (i.e. limit:-1)
and there are no nested facets, aggregations are computed in the first collection phase
so that the second phase which would normally involve calculating the domain for the bucket
can be skipped entirely, leading to large performance improvements. (yonik)
* SOLR-10722: Speed up Solr's use of the UnifiedHighlighter be re-using FieldInfos. (David Smiley)
Other Changes
----------------------
* SOLR-10617: JDBCStream accepts columns of type TIME, DATE & TIMESTAMP as well as CLOBs and decimal
numeric types (James Dyer)
* SOLR-10400: Replace (instanceof TrieFooField || instanceof FooPointField) constructs with
FieldType.getNumberType() or SchemaField.getSortField() where appropriate. (hossman, Steve Rowe)
* SOLR-10438: Assign explicit useDocValuesAsStored values to all points field types in
* SOLR-10438: Assign explicit useDocValuesAsStored values to all points field types in
schema-point.xml/TestPointFields. (hossman, Steve Rowe)
* LUCENE-7705: Allow CharTokenizer-derived tokenizers and KeywordTokenizer to configure the max token length.
(Amrit Sarkar via Erick Erickson)
@ -885,9 +856,9 @@ Other Changes
* SOLR-10761: Switch trie numeric/date fields to points in data-driven-enabled example and test schemas.
(Steve Rowe)
* SOLR-10851: SolrClients should clarify expectations for solrServerUrl parameter (Jason Gerlowski
* SOLR-10851: SolrClients should clarify expectations for solrServerUrl parameter (Jason Gerlowski
via Tomás Fernández Löbbe)
* SOLR-10891: BBoxField should support point-based number sub-fields. (Steve Rowe)
* SOLR-10834: Fixed tests and test configs to stop using numeric uniqueKey fields (hossman)
@ -915,7 +886,7 @@ Bug Fixes
* SOLR-10908: CloudSolrStream.toExpression incorrectly handles fq clauses (Rohit Singh via 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)
them all to the transient lists. (Erick Erickson)
* SOLR-11122: Creating a core should write a core.properties file first and clean up on failure
(Erick Erickson)