mirror of https://github.com/apache/lucene.git
Merge branch 'apache-https-master' into jira/solr-8593
This commit is contained in:
commit
bda84d8442
|
@ -85,6 +85,14 @@ Improvements
|
|||
* LUCENE-7524: Added more detailed explanation of how IDF is computed in
|
||||
ClassicSimilarity and BM25Similarity. (Adrien Grand)
|
||||
|
||||
* LUCENE-7526: Enhanced UnifiedHighlighter's passage relevancy for queries with
|
||||
wildcards and sometimes just terms. Added shouldPreferPassageRelevancyOverSpeed()
|
||||
which can be overridden to return false to eek out more speed in some cases.
|
||||
(Timothy M. Rodriguez, David Smiley)
|
||||
|
||||
* LUCENE-7537: Index time sorting now supports multi-valued sorts
|
||||
using selectors (MIN, MAX, etc.) (Jim Ferenczi via Mike McCandless)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-7546: Fixed references to benchmark wikipedia data and the Jenkins line-docs file
|
||||
|
@ -92,6 +100,11 @@ Other
|
|||
|
||||
* LUCENE-7534: fix smokeTestRelease.py to run on Cygwin (Mikhail Khludnev)
|
||||
|
||||
Build
|
||||
|
||||
* LUCENE-7387: fix defaultCodec in build.xml to account for the line ending (hossman)
|
||||
|
||||
|
||||
======================= Lucene 6.3.0 =======================
|
||||
|
||||
API Changes
|
||||
|
|
|
@ -34,7 +34,7 @@ content.source=org.apache.lucene.benchmark.byTask.feeds.EnwikiContentSource
|
|||
docs.file=temp/enwiki-20070527-pages-articles.xml.bz2
|
||||
|
||||
query.maker=org.apache.lucene.benchmark.byTask.feeds.FileBasedQueryMaker
|
||||
file.query.maker.file=conf/query-phrases.txt
|
||||
file.query.maker.file=conf/query-terms.txt
|
||||
log.queries=false
|
||||
log.step.SearchTravRetHighlight=-1
|
||||
|
||||
|
@ -55,7 +55,7 @@ highlighter=HlImpl:NONE:SH_A:UH_A:PH_P:UH_P:UH_PV
|
|||
|
||||
{ "Warm" SearchTravRetHighlight > : 1000
|
||||
|
||||
{ "HL" SearchTravRetHighlight > : 500
|
||||
{ "HL" SearchTravRetHighlight > : 2000
|
||||
|
||||
CloseReader
|
||||
|
||||
|
|
|
@ -54,7 +54,7 @@ highlighter=HlImpl:NONE:SH_V:FVH_V:UH_V
|
|||
|
||||
{ "Warm" SearchTravRetHighlight > : 1000
|
||||
|
||||
{ "HL" SearchTravRetHighlight > : 500
|
||||
{ "HL" SearchTravRetHighlight > : 2000
|
||||
|
||||
CloseReader
|
||||
|
||||
|
|
|
@ -213,6 +213,8 @@
|
|||
<filterchain>
|
||||
<!-- private static Codec defaultCodec = LOADER . lookup ( "LuceneXXX" ) ; -->
|
||||
<containsregex pattern="^.*defaultCodec\s*=\s*LOADER\s*\.\s*lookup\s*\(\s*"([^"]+)"\s*\)\s*;.*$" replace="\1"/>
|
||||
<fixcrlf eol="unix" eof="remove" />
|
||||
<deletecharacters chars="\n"/>
|
||||
</filterchain>
|
||||
</loadfile>
|
||||
|
||||
|
|
|
@ -33,9 +33,14 @@ import org.apache.lucene.index.IndexFileNames;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedNumericSelector;
|
||||
import org.apache.lucene.search.SortedNumericSortField;
|
||||
import org.apache.lucene.search.SortedSetSelector;
|
||||
import org.apache.lucene.search.SortedSetSortField;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
|
@ -64,6 +69,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
|
|||
final static BytesRef SI_SORT = new BytesRef(" sort ");
|
||||
final static BytesRef SI_SORT_FIELD = new BytesRef(" field ");
|
||||
final static BytesRef SI_SORT_TYPE = new BytesRef(" type ");
|
||||
final static BytesRef SI_SELECTOR_TYPE = new BytesRef(" selector ");
|
||||
final static BytesRef SI_SORT_REVERSE = new BytesRef(" reverse ");
|
||||
final static BytesRef SI_SORT_MISSING = new BytesRef(" missing ");
|
||||
|
||||
|
@ -158,6 +164,8 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
|
|||
final String typeAsString = readString(SI_SORT_TYPE.length, scratch);
|
||||
|
||||
final SortField.Type type;
|
||||
SortedSetSelector.Type selectorSet = null;
|
||||
SortedNumericSelector.Type selectorNumeric = null;
|
||||
switch (typeAsString) {
|
||||
case "string":
|
||||
type = SortField.Type.STRING;
|
||||
|
@ -174,6 +182,26 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
|
|||
case "float":
|
||||
type = SortField.Type.FLOAT;
|
||||
break;
|
||||
case "multi_valued_string":
|
||||
type = SortField.Type.STRING;
|
||||
selectorSet = readSetSelector(input, scratch);
|
||||
break;
|
||||
case "multi_valued_long":
|
||||
type = SortField.Type.LONG;
|
||||
selectorNumeric = readNumericSelector(input, scratch);
|
||||
break;
|
||||
case "multi_valued_int":
|
||||
type = SortField.Type.INT;
|
||||
selectorNumeric = readNumericSelector(input, scratch);
|
||||
break;
|
||||
case "multi_valued_double":
|
||||
type = SortField.Type.DOUBLE;
|
||||
selectorNumeric = readNumericSelector(input, scratch);
|
||||
break;
|
||||
case "multi_valued_float":
|
||||
type = SortField.Type.FLOAT;
|
||||
selectorNumeric = readNumericSelector(input, scratch);
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("unable to parse sort type string: " + typeAsString, input);
|
||||
}
|
||||
|
@ -245,7 +273,13 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
|
|||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
if (selectorSet != null) {
|
||||
sortField[i] = new SortedSetSortField(field, reverse);
|
||||
} else if (selectorNumeric != null) {
|
||||
sortField[i] = new SortedNumericSortField(field, type, reverse);
|
||||
} else {
|
||||
sortField[i] = new SortField(field, type, reverse);
|
||||
}
|
||||
if (missingValue != null) {
|
||||
sortField[i].setMissingValue(missingValue);
|
||||
}
|
||||
|
@ -266,6 +300,38 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
|
|||
return new String(scratch.bytes(), offset, scratch.length()-offset, StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
private SortedSetSelector.Type readSetSelector(IndexInput input, BytesRefBuilder scratch) throws IOException {
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch.get(), SI_SELECTOR_TYPE);
|
||||
final String selectorAsString = readString(SI_SELECTOR_TYPE.length, scratch);
|
||||
switch (selectorAsString) {
|
||||
case "min":
|
||||
return SortedSetSelector.Type.MIN;
|
||||
case "middle_min":
|
||||
return SortedSetSelector.Type.MIDDLE_MIN;
|
||||
case "middle_max":
|
||||
return SortedSetSelector.Type.MIDDLE_MAX;
|
||||
case "max":
|
||||
return SortedSetSelector.Type.MAX;
|
||||
default:
|
||||
throw new CorruptIndexException("unable to parse SortedSetSelector type: " + selectorAsString, input);
|
||||
}
|
||||
}
|
||||
|
||||
private SortedNumericSelector.Type readNumericSelector(IndexInput input, BytesRefBuilder scratch) throws IOException {
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch.get(), SI_SELECTOR_TYPE);
|
||||
final String selectorAsString = readString(SI_SELECTOR_TYPE.length, scratch);
|
||||
switch (selectorAsString) {
|
||||
case "min":
|
||||
return SortedNumericSelector.Type.MIN;
|
||||
case "max":
|
||||
return SortedNumericSelector.Type.MAX;
|
||||
default:
|
||||
throw new CorruptIndexException("unable to parse SortedNumericSelector type: " + selectorAsString, input);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
|
||||
|
||||
|
@ -352,29 +418,93 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
|
|||
SimpleTextUtil.writeNewline(output);
|
||||
|
||||
SimpleTextUtil.write(output, SI_SORT_TYPE);
|
||||
final String sortType;
|
||||
switch (sortField.getType()) {
|
||||
final String sortTypeString;
|
||||
final SortField.Type sortType;
|
||||
final boolean multiValued;
|
||||
if (sortField instanceof SortedSetSortField) {
|
||||
sortType = SortField.Type.STRING;
|
||||
multiValued = true;
|
||||
} else if (sortField instanceof SortedNumericSortField) {
|
||||
sortType = ((SortedNumericSortField) sortField).getNumericType();
|
||||
multiValued = true;
|
||||
} else {
|
||||
sortType = sortField.getType();
|
||||
multiValued = false;
|
||||
}
|
||||
switch (sortType) {
|
||||
case STRING:
|
||||
sortType = "string";
|
||||
if (multiValued) {
|
||||
sortTypeString = "multi_valued_string";
|
||||
} else {
|
||||
sortTypeString = "string";
|
||||
}
|
||||
break;
|
||||
case LONG:
|
||||
sortType = "long";
|
||||
if (multiValued) {
|
||||
sortTypeString = "multi_valued_long";
|
||||
} else {
|
||||
sortTypeString = "long";
|
||||
}
|
||||
break;
|
||||
case INT:
|
||||
sortType = "int";
|
||||
if (multiValued) {
|
||||
sortTypeString = "multi_valued_int";
|
||||
} else {
|
||||
sortTypeString = "int";
|
||||
}
|
||||
break;
|
||||
case DOUBLE:
|
||||
sortType = "double";
|
||||
if (multiValued) {
|
||||
sortTypeString = "multi_valued_double";
|
||||
} else {
|
||||
sortTypeString = "double";
|
||||
}
|
||||
break;
|
||||
case FLOAT:
|
||||
sortType = "float";
|
||||
if (multiValued) {
|
||||
sortTypeString = "multi_valued_float";
|
||||
} else {
|
||||
sortTypeString = "float";
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
|
||||
}
|
||||
SimpleTextUtil.write(output, sortType, scratch);
|
||||
SimpleTextUtil.write(output, sortTypeString, scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
|
||||
if (sortField instanceof SortedSetSortField) {
|
||||
SortedSetSelector.Type selector = ((SortedSetSortField) sortField).getSelector();
|
||||
final String selectorString;
|
||||
if (selector == SortedSetSelector.Type.MIN) {
|
||||
selectorString = "min";
|
||||
} else if (selector == SortedSetSelector.Type.MIDDLE_MIN) {
|
||||
selectorString = "middle_min";
|
||||
} else if (selector == SortedSetSelector.Type.MIDDLE_MAX) {
|
||||
selectorString = "middle_max";
|
||||
} else if (selector == SortedSetSelector.Type.MAX) {
|
||||
selectorString = "max";
|
||||
} else {
|
||||
throw new IllegalStateException("Unexpected SortedSetSelector type selector: " + selector);
|
||||
}
|
||||
SimpleTextUtil.write(output, SI_SELECTOR_TYPE);
|
||||
SimpleTextUtil.write(output, selectorString, scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
} else if (sortField instanceof SortedNumericSortField) {
|
||||
SortedNumericSelector.Type selector = ((SortedNumericSortField) sortField).getSelector();
|
||||
final String selectorString;
|
||||
if (selector == SortedNumericSelector.Type.MIN) {
|
||||
selectorString = "min";
|
||||
} else if (selector == SortedNumericSelector.Type.MAX) {
|
||||
selectorString = "max";
|
||||
} else {
|
||||
throw new IllegalStateException("Unexpected SortedNumericSelector type selector: " + selector);
|
||||
}
|
||||
SimpleTextUtil.write(output, SI_SELECTOR_TYPE);
|
||||
SimpleTextUtil.write(output, selectorString, scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
}
|
||||
|
||||
SimpleTextUtil.write(output, SI_SORT_REVERSE);
|
||||
SimpleTextUtil.write(output, Boolean.toString(sortField.getReverse()), scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
|
|
|
@ -29,6 +29,10 @@ import org.apache.lucene.index.SegmentInfo; // javadocs
|
|||
import org.apache.lucene.index.SegmentInfos; // javadocs
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedNumericSelector;
|
||||
import org.apache.lucene.search.SortedNumericSortField;
|
||||
import org.apache.lucene.search.SortedSetSelector;
|
||||
import org.apache.lucene.search.SortedSetSortField;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.DataOutput; // javadocs
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -110,6 +114,8 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
|
|||
String fieldName = input.readString();
|
||||
int sortTypeID = input.readVInt();
|
||||
SortField.Type sortType;
|
||||
SortedSetSelector.Type sortedSetSelector = null;
|
||||
SortedNumericSelector.Type sortedNumericSelector = null;
|
||||
switch(sortTypeID) {
|
||||
case 0:
|
||||
sortType = SortField.Type.STRING;
|
||||
|
@ -126,6 +132,43 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
|
|||
case 4:
|
||||
sortType = SortField.Type.FLOAT;
|
||||
break;
|
||||
case 5:
|
||||
sortType = SortField.Type.STRING;
|
||||
byte selector = input.readByte();
|
||||
if (selector == 0) {
|
||||
sortedSetSelector = SortedSetSelector.Type.MIN;
|
||||
} else if (selector == 1) {
|
||||
sortedSetSelector = SortedSetSelector.Type.MAX;
|
||||
} else if (selector == 2) {
|
||||
sortedSetSelector = SortedSetSelector.Type.MIDDLE_MIN;
|
||||
} else if (selector == 3) {
|
||||
sortedSetSelector = SortedSetSelector.Type.MIDDLE_MAX;
|
||||
} else {
|
||||
throw new CorruptIndexException("invalid index SortedSetSelector ID: " + selector, input);
|
||||
}
|
||||
break;
|
||||
case 6:
|
||||
byte type = input.readByte();
|
||||
if (type == 0) {
|
||||
sortType = SortField.Type.LONG;
|
||||
} else if (type == 1) {
|
||||
sortType = SortField.Type.INT;
|
||||
} else if (type == 2) {
|
||||
sortType = SortField.Type.DOUBLE;
|
||||
} else if (type == 3) {
|
||||
sortType = SortField.Type.FLOAT;
|
||||
} else {
|
||||
throw new CorruptIndexException("invalid index SortedNumericSortField type ID: " + type, input);
|
||||
}
|
||||
byte numericSelector = input.readByte();
|
||||
if (numericSelector == 0) {
|
||||
sortedNumericSelector = SortedNumericSelector.Type.MIN;
|
||||
} else if (numericSelector == 1) {
|
||||
sortedNumericSelector = SortedNumericSelector.Type.MAX;
|
||||
} else {
|
||||
throw new CorruptIndexException("invalid index SortedNumericSelector ID: " + numericSelector, input);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new CorruptIndexException("invalid index sort field type ID: " + sortTypeID, input);
|
||||
}
|
||||
|
@ -139,7 +182,13 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
|
|||
throw new CorruptIndexException("invalid index sort reverse: " + b, input);
|
||||
}
|
||||
|
||||
if (sortedSetSelector != null) {
|
||||
sortFields[i] = new SortedSetSortField(fieldName, reverse, sortedSetSelector);
|
||||
} else if (sortedNumericSelector != null) {
|
||||
sortFields[i] = new SortedNumericSortField(fieldName, sortType, reverse, sortedNumericSelector);
|
||||
} else {
|
||||
sortFields[i] = new SortField(fieldName, sortType, reverse);
|
||||
}
|
||||
|
||||
Object missingValue;
|
||||
b = input.readByte();
|
||||
|
@ -245,6 +294,7 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
|
|||
output.writeVInt(numSortFields);
|
||||
for (int i = 0; i < numSortFields; ++i) {
|
||||
SortField sortField = indexSort.getSort()[i];
|
||||
SortField.Type sortType = sortField.getType();
|
||||
output.writeString(sortField.getField());
|
||||
int sortTypeID;
|
||||
switch (sortField.getType()) {
|
||||
|
@ -263,10 +313,55 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
|
|||
case FLOAT:
|
||||
sortTypeID = 4;
|
||||
break;
|
||||
case CUSTOM:
|
||||
if (sortField instanceof SortedSetSortField) {
|
||||
sortTypeID = 5;
|
||||
sortType = SortField.Type.STRING;
|
||||
} else if (sortField instanceof SortedNumericSortField) {
|
||||
sortTypeID = 6;
|
||||
sortType = ((SortedNumericSortField) sortField).getNumericType();
|
||||
} else {
|
||||
throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
|
||||
}
|
||||
output.writeVInt(sortTypeID);
|
||||
if (sortTypeID == 5) {
|
||||
SortedSetSortField ssf = (SortedSetSortField) sortField;
|
||||
if (ssf.getSelector() == SortedSetSelector.Type.MIN) {
|
||||
output.writeByte((byte) 0);
|
||||
} else if (ssf.getSelector() == SortedSetSelector.Type.MAX) {
|
||||
output.writeByte((byte) 1);
|
||||
} else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) {
|
||||
output.writeByte((byte) 2);
|
||||
} else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) {
|
||||
output.writeByte((byte) 3);
|
||||
} else {
|
||||
throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector());
|
||||
}
|
||||
} else if (sortTypeID == 6) {
|
||||
SortedNumericSortField snsf = (SortedNumericSortField) sortField;
|
||||
if (snsf.getNumericType() == SortField.Type.LONG) {
|
||||
output.writeByte((byte) 0);
|
||||
} else if (snsf.getNumericType() == SortField.Type.INT) {
|
||||
output.writeByte((byte) 1);
|
||||
} else if (snsf.getNumericType() == SortField.Type.DOUBLE) {
|
||||
output.writeByte((byte) 2);
|
||||
} else if (snsf.getNumericType() == SortField.Type.FLOAT) {
|
||||
output.writeByte((byte) 3);
|
||||
} else {
|
||||
throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType());
|
||||
}
|
||||
if (snsf.getSelector() == SortedNumericSelector.Type.MIN) {
|
||||
output.writeByte((byte) 0);
|
||||
} else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) {
|
||||
output.writeByte((byte) 1);
|
||||
} else {
|
||||
throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector());
|
||||
}
|
||||
}
|
||||
output.writeByte((byte) (sortField.getReverse() ? 0 : 1));
|
||||
|
||||
// write missing value
|
||||
|
@ -274,7 +369,7 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
|
|||
if (missingValue == null) {
|
||||
output.writeByte((byte) 0);
|
||||
} else {
|
||||
switch(sortField.getType()) {
|
||||
switch(sortType) {
|
||||
case STRING:
|
||||
if (missingValue == SortField.STRING_LAST) {
|
||||
output.writeByte((byte) 1);
|
||||
|
@ -314,5 +409,6 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat {
|
|||
public final static String SI_EXTENSION = "si";
|
||||
static final String CODEC_NAME = "Lucene62SegmentInfo";
|
||||
static final int VERSION_START = 0;
|
||||
static final int VERSION_CURRENT = VERSION_START;
|
||||
static final int VERSION_MULTI_VALUED_SORT = 1;
|
||||
static final int VERSION_CURRENT = VERSION_MULTI_VALUED_SORT;
|
||||
}
|
||||
|
|
|
@ -468,7 +468,8 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
*/
|
||||
public IndexWriterConfig setIndexSort(Sort sort) {
|
||||
for(SortField sortField : sort.getSort()) {
|
||||
if (ALLOWED_INDEX_SORT_TYPES.contains(sortField.getType()) == false) {
|
||||
final SortField.Type sortType = Sorter.getSortFieldType(sortField);
|
||||
if (ALLOWED_INDEX_SORT_TYPES.contains(sortType) == false) {
|
||||
throw new IllegalArgumentException("invalid SortField type: must be one of " + ALLOWED_INDEX_SORT_TYPES + " but got: " + sortField);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -141,33 +141,25 @@ final class MultiSorter {
|
|||
private static ComparableProvider[] getComparableProviders(List<CodecReader> readers, SortField sortField) throws IOException {
|
||||
|
||||
ComparableProvider[] providers = new ComparableProvider[readers.size()];
|
||||
final int reverseMul = sortField.getReverse() ? -1 : 1;
|
||||
final SortField.Type sortType = Sorter.getSortFieldType(sortField);
|
||||
|
||||
switch(sortField.getType()) {
|
||||
switch(sortType) {
|
||||
|
||||
case STRING:
|
||||
{
|
||||
// this uses the efficient segment-local ordinal map:
|
||||
final SortedDocValues[] values = new SortedDocValues[readers.size()];
|
||||
for(int i=0;i<readers.size();i++) {
|
||||
SortedDocValues v = readers.get(i).getSortedDocValues(sortField.getField());
|
||||
if (v == null) {
|
||||
v = DocValues.emptySorted();
|
||||
}
|
||||
values[i] = v;
|
||||
final SortedDocValues sorted = Sorter.getOrWrapSorted(readers.get(i), sortField);
|
||||
values[i] = sorted;
|
||||
}
|
||||
MultiDocValues.OrdinalMap ordinalMap = MultiDocValues.OrdinalMap.build(null, values, PackedInts.DEFAULT);
|
||||
final int missingOrd;
|
||||
if (sortField.getMissingValue() == SortField.STRING_LAST) {
|
||||
missingOrd = Integer.MAX_VALUE;
|
||||
missingOrd = sortField.getReverse() ? Integer.MIN_VALUE : Integer.MAX_VALUE;
|
||||
} else {
|
||||
missingOrd = Integer.MIN_VALUE;
|
||||
}
|
||||
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
missingOrd = sortField.getReverse() ? Integer.MAX_VALUE : Integer.MIN_VALUE;
|
||||
}
|
||||
|
||||
for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
|
||||
|
@ -205,13 +197,6 @@ final class MultiSorter {
|
|||
|
||||
case LONG:
|
||||
{
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
final Long missingValue;
|
||||
if (sortField.getMissingValue() != null) {
|
||||
missingValue = (Long) sortField.getMissingValue();
|
||||
|
@ -220,7 +205,7 @@ final class MultiSorter {
|
|||
}
|
||||
|
||||
for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
|
||||
final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
|
||||
final NumericDocValues values = Sorter.getOrWrapNumeric(readers.get(readerIndex), sortField);
|
||||
|
||||
providers[readerIndex] = new ComparableProvider() {
|
||||
// used only by assert:
|
||||
|
@ -243,7 +228,7 @@ final class MultiSorter {
|
|||
if (readerDocID == docID) {
|
||||
return reverseMul * values.longValue();
|
||||
} else {
|
||||
return missingValue;
|
||||
return reverseMul * missingValue;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -253,13 +238,6 @@ final class MultiSorter {
|
|||
|
||||
case INT:
|
||||
{
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
final Integer missingValue;
|
||||
if (sortField.getMissingValue() != null) {
|
||||
missingValue = (Integer) sortField.getMissingValue();
|
||||
|
@ -268,7 +246,7 @@ final class MultiSorter {
|
|||
}
|
||||
|
||||
for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
|
||||
final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
|
||||
final NumericDocValues values = Sorter.getOrWrapNumeric(readers.get(readerIndex), sortField);
|
||||
|
||||
providers[readerIndex] = new ComparableProvider() {
|
||||
// used only by assert:
|
||||
|
@ -291,7 +269,7 @@ final class MultiSorter {
|
|||
if (readerDocID == docID) {
|
||||
return reverseMul * (int) values.longValue();
|
||||
} else {
|
||||
return missingValue;
|
||||
return reverseMul * missingValue;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -301,13 +279,6 @@ final class MultiSorter {
|
|||
|
||||
case DOUBLE:
|
||||
{
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
final Double missingValue;
|
||||
if (sortField.getMissingValue() != null) {
|
||||
missingValue = (Double) sortField.getMissingValue();
|
||||
|
@ -316,7 +287,7 @@ final class MultiSorter {
|
|||
}
|
||||
|
||||
for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
|
||||
final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
|
||||
final NumericDocValues values = Sorter.getOrWrapNumeric(readers.get(readerIndex), sortField);
|
||||
|
||||
providers[readerIndex] = new ComparableProvider() {
|
||||
// used only by assert:
|
||||
|
@ -339,7 +310,7 @@ final class MultiSorter {
|
|||
if (readerDocID == docID) {
|
||||
return reverseMul * Double.longBitsToDouble(values.longValue());
|
||||
} else {
|
||||
return missingValue;
|
||||
return reverseMul * missingValue;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -349,13 +320,6 @@ final class MultiSorter {
|
|||
|
||||
case FLOAT:
|
||||
{
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
final Float missingValue;
|
||||
if (sortField.getMissingValue() != null) {
|
||||
missingValue = (Float) sortField.getMissingValue();
|
||||
|
@ -364,7 +328,7 @@ final class MultiSorter {
|
|||
}
|
||||
|
||||
for(int readerIndex=0;readerIndex<readers.size();readerIndex++) {
|
||||
final NumericDocValues values = DocValues.getNumeric(readers.get(readerIndex), sortField.getField());
|
||||
final NumericDocValues values = Sorter.getOrWrapNumeric(readers.get(readerIndex), sortField);
|
||||
|
||||
providers[readerIndex] = new ComparableProvider() {
|
||||
// used only by assert:
|
||||
|
@ -387,7 +351,7 @@ final class MultiSorter {
|
|||
if (readerDocID == docID) {
|
||||
return reverseMul * Float.intBitsToFloat((int) values.longValue());
|
||||
} else {
|
||||
return missingValue;
|
||||
return reverseMul * missingValue;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
|
|
@ -25,6 +25,10 @@ import org.apache.lucene.search.FieldComparator;
|
|||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedNumericSelector;
|
||||
import org.apache.lucene.search.SortedNumericSortField;
|
||||
import org.apache.lucene.search.SortedSetSelector;
|
||||
import org.apache.lucene.search.SortedSetSortField;
|
||||
import org.apache.lucene.util.TimSorter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedLongValues;
|
||||
|
@ -200,18 +204,55 @@ final class Sorter {
|
|||
};
|
||||
}
|
||||
|
||||
/** Returns the native sort type for {@link SortedSetSortField} and {@link SortedNumericSortField},
|
||||
* {@link SortField#getType()} otherwise */
|
||||
static SortField.Type getSortFieldType(SortField sortField) {
|
||||
if (sortField instanceof SortedSetSortField) {
|
||||
return SortField.Type.STRING;
|
||||
} else if (sortField instanceof SortedNumericSortField) {
|
||||
return ((SortedNumericSortField) sortField).getNumericType();
|
||||
} else {
|
||||
return sortField.getType();
|
||||
}
|
||||
}
|
||||
|
||||
/** Wraps a {@link SortedNumericDocValues} as a single-valued view if the field is an instance of {@link SortedNumericSortField},
|
||||
* returns {@link NumericDocValues} for the field otherwise. */
|
||||
static NumericDocValues getOrWrapNumeric(LeafReader reader, SortField sortField) throws IOException {
|
||||
if (sortField instanceof SortedNumericSortField) {
|
||||
SortedNumericSortField sf = (SortedNumericSortField) sortField;
|
||||
return SortedNumericSelector.wrap(DocValues.getSortedNumeric(reader, sf.getField()), sf.getSelector(), sf.getNumericType());
|
||||
} else {
|
||||
return DocValues.getNumeric(reader, sortField.getField());
|
||||
}
|
||||
}
|
||||
|
||||
/** Wraps a {@link SortedSetDocValues} as a single-valued view if the field is an instance of {@link SortedSetSortField},
|
||||
* returns {@link SortedDocValues} for the field otherwise. */
|
||||
static SortedDocValues getOrWrapSorted(LeafReader reader, SortField sortField) throws IOException {
|
||||
if (sortField instanceof SortedSetSortField) {
|
||||
SortedSetSortField sf = (SortedSetSortField) sortField;
|
||||
return SortedSetSelector.wrap(DocValues.getSortedSet(reader, sf.getField()), sf.getSelector());
|
||||
} else {
|
||||
return DocValues.getSorted(reader, sortField.getField());
|
||||
}
|
||||
}
|
||||
|
||||
/** We cannot use the {@link FieldComparator} API because that API requires that you send it docIDs in order. Note that this API
|
||||
* allocates arrays[maxDoc] to hold the native values needed for comparison, but 1) they are transient (only alive while sorting this one
|
||||
* segment), and 2) in the typical index sorting case, they are only used to sort newly flushed segments, which will be smaller than
|
||||
* merged segments. */
|
||||
private static DocComparator getDocComparator(LeafReader reader, SortField sortField) throws IOException {
|
||||
|
||||
int maxDoc = reader.maxDoc();
|
||||
final int maxDoc = reader.maxDoc();
|
||||
final int reverseMul = sortField.getReverse() ? -1 : 1;
|
||||
final SortField.Type sortType = getSortFieldType(sortField);
|
||||
|
||||
switch(sortField.getType()) {
|
||||
switch(sortType) {
|
||||
|
||||
case STRING:
|
||||
{
|
||||
final SortedDocValues sorted = getOrWrapSorted(reader, sortField);
|
||||
final int missingOrd;
|
||||
if (sortField.getMissingValue() == SortField.STRING_LAST) {
|
||||
missingOrd = Integer.MAX_VALUE;
|
||||
|
@ -221,19 +262,11 @@ final class Sorter {
|
|||
|
||||
final int[] ords = new int[reader.maxDoc()];
|
||||
Arrays.fill(ords, missingOrd);
|
||||
SortedDocValues sorted = DocValues.getSorted(reader, sortField.getField());
|
||||
int docID;
|
||||
while ((docID = sorted.nextDoc()) != NO_MORE_DOCS) {
|
||||
ords[docID] = sorted.ordValue();
|
||||
}
|
||||
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
return new DocComparator() {
|
||||
@Override
|
||||
public int compare(int docID1, int docID2) {
|
||||
|
@ -244,9 +277,8 @@ final class Sorter {
|
|||
|
||||
case LONG:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
long[] values = new long[maxDoc];
|
||||
NumericDocValues dvs = DocValues.getNumeric(reader, sortField.getField());
|
||||
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Long) sortField.getMissingValue());
|
||||
}
|
||||
|
@ -258,13 +290,6 @@ final class Sorter {
|
|||
values[docID] = dvs.longValue();
|
||||
}
|
||||
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
return new DocComparator() {
|
||||
@Override
|
||||
public int compare(int docID1, int docID2) {
|
||||
|
@ -275,9 +300,8 @@ final class Sorter {
|
|||
|
||||
case INT:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
int[] values = new int[maxDoc];
|
||||
NumericDocValues dvs = DocValues.getNumeric(reader, sortField.getField());
|
||||
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Integer) sortField.getMissingValue());
|
||||
}
|
||||
|
@ -290,13 +314,6 @@ final class Sorter {
|
|||
values[docID] = (int) dvs.longValue();
|
||||
}
|
||||
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
return new DocComparator() {
|
||||
@Override
|
||||
public int compare(int docID1, int docID2) {
|
||||
|
@ -307,9 +324,8 @@ final class Sorter {
|
|||
|
||||
case DOUBLE:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
double[] values = new double[maxDoc];
|
||||
NumericDocValues dvs = DocValues.getNumeric(reader, sortField.getField());
|
||||
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Double) sortField.getMissingValue());
|
||||
}
|
||||
|
@ -321,13 +337,6 @@ final class Sorter {
|
|||
values[docID] = Double.longBitsToDouble(dvs.longValue());
|
||||
}
|
||||
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
return new DocComparator() {
|
||||
@Override
|
||||
public int compare(int docID1, int docID2) {
|
||||
|
@ -338,9 +347,8 @@ final class Sorter {
|
|||
|
||||
case FLOAT:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
float[] values = new float[maxDoc];
|
||||
NumericDocValues dvs = DocValues.getNumeric(reader, sortField.getField());
|
||||
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Float) sortField.getMissingValue());
|
||||
}
|
||||
|
@ -352,13 +360,6 @@ final class Sorter {
|
|||
values[docID] = Float.intBitsToFloat((int) dvs.longValue());
|
||||
}
|
||||
|
||||
final int reverseMul;
|
||||
if (sortField.getReverse()) {
|
||||
reverseMul = -1;
|
||||
} else {
|
||||
reverseMul = 1;
|
||||
}
|
||||
|
||||
return new DocComparator() {
|
||||
@Override
|
||||
public int compare(int docID1, int docID2) {
|
||||
|
@ -386,7 +387,6 @@ final class Sorter {
|
|||
*/
|
||||
DocMap sort(LeafReader reader) throws IOException {
|
||||
SortField fields[] = sort.getSort();
|
||||
final int reverseMul[] = new int[fields.length];
|
||||
final DocComparator comparators[] = new DocComparator[fields.length];
|
||||
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
|
|
|
@ -83,6 +83,11 @@ public class SortedNumericSortField extends SortField {
|
|||
this.type = type;
|
||||
}
|
||||
|
||||
/** Returns the numeric type in use for this sort */
|
||||
public SortField.Type getNumericType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
/** Returns the selector in use for this sort */
|
||||
public SortedNumericSelector.Type getSelector() {
|
||||
return selector;
|
||||
|
|
|
@ -61,6 +61,8 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedNumericSortField;
|
||||
import org.apache.lucene.search.SortedSetSortField;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TermStatistics;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
|
@ -115,6 +117,49 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicMultiValuedString() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
Sort indexSort = new Sort(new SortedSetSortField("foo", false));
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("zzz")));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("aaa")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("zzz")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("bcg")));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("mmm")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("pppp")));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1l, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2l, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3l, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingStringFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -152,6 +197,51 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedStringFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedSetSortField("foo", false);
|
||||
sortField.setMissingValue(SortField.STRING_FIRST);
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("zzz")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("zzza")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("zzzd")));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("mmm")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("nnnn")));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1l, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2l, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3l, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingStringLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -189,6 +279,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedStringLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedSetSortField("foo", false);
|
||||
sortField.setMissingValue(SortField.STRING_LAST);
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("zzz")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("zzzd")));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("mmm")));
|
||||
doc.add(new SortedSetDocValuesField("foo", new BytesRef("ppp")));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1l, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2l, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3l, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicLong() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -226,6 +360,48 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicMultiValuedLong() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
Sort indexSort = new Sort(new SortedNumericSortField("foo", SortField.Type.LONG));
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 18));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 35));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", -1));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 7));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 22));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingLongFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -263,6 +439,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedLongFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.LONG);
|
||||
sortField.setMissingValue(Long.valueOf(Long.MIN_VALUE));
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 18));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 27));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 7));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 24));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingLongLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -300,6 +520,51 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedLongLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.LONG);
|
||||
sortField.setMissingValue(Long.valueOf(Long.MAX_VALUE));
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 18));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 65));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 7));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 34));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 74));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicInt() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -337,6 +602,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicMultiValuedInt() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
Sort indexSort = new Sort(new SortedNumericSortField("foo", SortField.Type.INT));
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 18));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 34));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", -1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 34));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 7));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 22));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 27));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingIntFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -373,6 +682,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedIntFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.INT);
|
||||
sortField.setMissingValue(Integer.valueOf(Integer.MIN_VALUE));
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 18));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 187667));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 7));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 34));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingIntLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -410,6 +763,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedIntLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.INT);
|
||||
sortField.setMissingValue(Integer.valueOf(Integer.MAX_VALUE));
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 18));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 6372));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 7));
|
||||
doc.add(new SortedNumericDocValuesField("foo", 8));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicDouble() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -447,6 +844,49 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicMultiValuedDouble() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
Sort indexSort = new Sort(new SortedNumericSortField("foo", SortField.Type.DOUBLE));
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(7.54)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(27.0)));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(-1.0)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(0.0)));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(7.0)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(7.67)));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingDoubleFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -483,6 +923,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedDoubleFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.DOUBLE);
|
||||
sortField.setMissingValue(Double.NEGATIVE_INFINITY);
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(18.0)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(18.76)));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(7.0)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(70.0)));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingDoubleLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -520,6 +1004,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedDoubleLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.DOUBLE);
|
||||
sortField.setMissingValue(Double.POSITIVE_INFINITY);
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(18.0)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(8262.0)));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(7.0)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.doubleToSortableLong(7.87)));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicFloat() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -557,6 +1085,48 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testBasicMultiValuedFloat() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
Sort indexSort = new Sort(new SortedNumericSortField("foo", SortField.Type.FLOAT));
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(18.0f)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(29.0f)));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(-1.0f)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(34.0f)));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(7.0f)));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingFloatFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -593,6 +1163,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedFloatFirst() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.FLOAT);
|
||||
sortField.setMissingValue(Float.NEGATIVE_INFINITY);
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(18.0f)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(726.0f)));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(7.0f)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(18.0f)));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingFloatLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
|
@ -630,6 +1244,50 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMissingMultiValuedFloatLast() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
SortField sortField = new SortedNumericSortField("foo", SortField.Type.FLOAT);
|
||||
sortField.setMissingValue(Float.POSITIVE_INFINITY);
|
||||
Sort indexSort = new Sort(sortField);
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 2));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(726.0f)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(18.0f)));
|
||||
w.addDocument(doc);
|
||||
// so we get more than one segment, so that forceMerge actually does merge, since we only get a sorted segment by merging:
|
||||
w.commit();
|
||||
|
||||
// missing
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 3));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new NumericDocValuesField("id", 1));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(12.67f)));
|
||||
doc.add(new SortedNumericDocValuesField("foo", NumericUtils.floatToSortableInt(7.0f)));
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
|
||||
DirectoryReader r = DirectoryReader.open(w);
|
||||
LeafReader leaf = getOnlyLeafReader(r);
|
||||
assertEquals(3, leaf.maxDoc());
|
||||
NumericDocValues values = leaf.getNumericDocValues("id");
|
||||
assertEquals(0, values.nextDoc());
|
||||
assertEquals(1, values.longValue());
|
||||
assertEquals(1, values.nextDoc());
|
||||
assertEquals(2, values.longValue());
|
||||
assertEquals(2, values.nextDoc());
|
||||
assertEquals(3, values.longValue());
|
||||
r.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testRandom1() throws IOException {
|
||||
boolean withDeletes = random().nextBoolean();
|
||||
Directory dir = newDirectory();
|
||||
|
@ -703,6 +1361,58 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
public void testMultiValuedRandom1() throws IOException {
|
||||
boolean withDeletes = random().nextBoolean();
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
|
||||
Sort indexSort = new Sort(new SortedNumericSortField("foo", SortField.Type.LONG));
|
||||
iwc.setIndexSort(indexSort);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
final int numDocs = atLeast(1000);
|
||||
final FixedBitSet deleted = new FixedBitSet(numDocs);
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
Document doc = new Document();
|
||||
int num = random().nextInt(10);
|
||||
for (int j = 0; j < num; j++) {
|
||||
doc.add(new SortedNumericDocValuesField("foo", random().nextInt(2000)));
|
||||
}
|
||||
doc.add(new StringField("id", Integer.toString(i), Store.YES));
|
||||
doc.add(new NumericDocValuesField("id", i));
|
||||
w.addDocument(doc);
|
||||
if (random().nextInt(5) == 0) {
|
||||
w.getReader().close();
|
||||
} else if (random().nextInt(30) == 0) {
|
||||
w.forceMerge(2);
|
||||
} else if (random().nextInt(4) == 0) {
|
||||
final int id = TestUtil.nextInt(random(), 0, i);
|
||||
deleted.set(id);
|
||||
w.deleteDocuments(new Term("id", Integer.toString(id)));
|
||||
}
|
||||
}
|
||||
|
||||
DirectoryReader reader = w.getReader();
|
||||
// Now check that the index is consistent
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
for (int i = 0; i < numDocs; ++i) {
|
||||
TermQuery termQuery = new TermQuery(new Term("id", Integer.toString(i)));
|
||||
final TopDocs topDocs = searcher.search(termQuery, 1);
|
||||
if (deleted.get(i)) {
|
||||
assertEquals(0, topDocs.totalHits);
|
||||
} else {
|
||||
assertEquals(1, topDocs.totalHits);
|
||||
NumericDocValues values = MultiDocValues.getNumericValues(reader, "id");
|
||||
assertEquals(topDocs.scoreDocs[0].doc, values.advance(topDocs.scoreDocs[0].doc));
|
||||
assertEquals(i, values.longValue());
|
||||
Document document = reader.document(topDocs.scoreDocs[0].doc);
|
||||
assertEquals(Integer.toString(i), document.get("id"));
|
||||
}
|
||||
}
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
static class UpdateRunnable implements Runnable {
|
||||
|
||||
private final int numDocs;
|
||||
|
@ -1105,10 +1815,10 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
doc.add(norms);
|
||||
doc.add(new BinaryDocValuesField("binary", new BytesRef(Integer.toString(id))));
|
||||
doc.add(new SortedDocValuesField("sorted", new BytesRef(Integer.toString(id))));
|
||||
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Integer.toString(id))));
|
||||
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Integer.toString(id + 1))));
|
||||
doc.add(new SortedNumericDocValuesField("sorted_numeric", id));
|
||||
doc.add(new SortedNumericDocValuesField("sorted_numeric", id + 1));
|
||||
doc.add(new SortedSetDocValuesField("multi_valued_string", new BytesRef(Integer.toString(id))));
|
||||
doc.add(new SortedSetDocValuesField("multi_valued_string", new BytesRef(Integer.toString(id + 1))));
|
||||
doc.add(new SortedNumericDocValuesField("multi_valued_numeric", id));
|
||||
doc.add(new SortedNumericDocValuesField("multi_valued_numeric", id + 1));
|
||||
doc.add(new Field("term_vectors", Integer.toString(id), TERM_VECTORS_TYPE));
|
||||
byte[] bytes = new byte[4];
|
||||
NumericUtils.intToSortableBytes(id, bytes, 0);
|
||||
|
@ -1179,10 +1889,16 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
private static final class RandomDoc {
|
||||
public final int id;
|
||||
public final int intValue;
|
||||
public final int[] intValues;
|
||||
public final long longValue;
|
||||
public final long[] longValues;
|
||||
public final float floatValue;
|
||||
public final float[] floatValues;
|
||||
public final double doubleValue;
|
||||
public final double[] doubleValues;
|
||||
public final byte[] bytesValue;
|
||||
public final byte[][] bytesValues;
|
||||
|
||||
|
||||
public RandomDoc(int id) {
|
||||
this.id = id;
|
||||
|
@ -1192,16 +1908,28 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
doubleValue = random().nextDouble();
|
||||
bytesValue = new byte[TestUtil.nextInt(random(), 1, 50)];
|
||||
random().nextBytes(bytesValue);
|
||||
|
||||
int numValues = random().nextInt(10);
|
||||
intValues = new int[numValues];
|
||||
longValues = new long[numValues];
|
||||
floatValues = new float[numValues];
|
||||
doubleValues = new double[numValues];
|
||||
bytesValues = new byte[numValues][];
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
intValues[i] = random().nextInt();
|
||||
longValues[i] = random().nextLong();
|
||||
floatValues[i] = random().nextFloat();
|
||||
doubleValues[i] = random().nextDouble();
|
||||
bytesValues[i] = new byte[TestUtil.nextInt(random(), 1, 50)];
|
||||
random().nextBytes(bytesValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static Sort randomSort() {
|
||||
int numFields = TestUtil.nextInt(random(), 1, 3);
|
||||
SortField[] sortFields = new SortField[numFields];
|
||||
for(int i=0;i<numFields-1;i++) {
|
||||
private static SortField randomIndexSortField() {
|
||||
boolean reversed = random().nextBoolean();
|
||||
SortField sortField;
|
||||
switch(random().nextInt(5)) {
|
||||
switch(random().nextInt(10)) {
|
||||
case 0:
|
||||
sortField = new SortField("int", SortField.Type.INT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
|
@ -1209,32 +1937,73 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
}
|
||||
break;
|
||||
case 1:
|
||||
sortField = new SortedNumericSortField("multi_valued_int", SortField.Type.INT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextInt());
|
||||
}
|
||||
break;
|
||||
case 2:
|
||||
sortField = new SortField("long", SortField.Type.LONG, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextLong());
|
||||
}
|
||||
break;
|
||||
case 2:
|
||||
case 3:
|
||||
sortField = new SortedNumericSortField("multi_valued_long", SortField.Type.LONG, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextLong());
|
||||
}
|
||||
break;
|
||||
case 4:
|
||||
sortField = new SortField("float", SortField.Type.FLOAT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextFloat());
|
||||
}
|
||||
break;
|
||||
case 3:
|
||||
case 5:
|
||||
sortField = new SortedNumericSortField("multi_valued_float", SortField.Type.FLOAT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextFloat());
|
||||
}
|
||||
break;
|
||||
case 6:
|
||||
sortField = new SortField("double", SortField.Type.DOUBLE, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextDouble());
|
||||
}
|
||||
break;
|
||||
case 4:
|
||||
case 7:
|
||||
sortField = new SortedNumericSortField("multi_valued_double", SortField.Type.DOUBLE, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextDouble());
|
||||
}
|
||||
break;
|
||||
case 8:
|
||||
sortField = new SortField("bytes", SortField.Type.STRING, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(SortField.STRING_LAST);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError();
|
||||
case 9:
|
||||
sortField = new SortedSetSortField("multi_valued_bytes", reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(SortField.STRING_LAST);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
sortField = null;
|
||||
fail();
|
||||
}
|
||||
return sortField;
|
||||
}
|
||||
|
||||
|
||||
private static Sort randomSort() {
|
||||
// at least 2
|
||||
int numFields = TestUtil.nextInt(random(), 2, 4);
|
||||
SortField[] sortFields = new SortField[numFields];
|
||||
for(int i=0;i<numFields-1;i++) {
|
||||
SortField sortField = randomIndexSortField();
|
||||
sortFields[i] = sortField;
|
||||
}
|
||||
|
||||
|
@ -1294,6 +2063,27 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
doc.add(new DoubleDocValuesField("double", docValues.doubleValue));
|
||||
doc.add(new FloatDocValuesField("float", docValues.floatValue));
|
||||
doc.add(new SortedDocValuesField("bytes", new BytesRef(docValues.bytesValue)));
|
||||
|
||||
for (int value : docValues.intValues) {
|
||||
doc.add(new SortedNumericDocValuesField("multi_valued_int", value));
|
||||
}
|
||||
|
||||
for (long value : docValues.longValues) {
|
||||
doc.add(new SortedNumericDocValuesField("multi_valued_long", value));
|
||||
}
|
||||
|
||||
for (float value : docValues.floatValues) {
|
||||
doc.add(new SortedNumericDocValuesField("multi_valued_float", NumericUtils.floatToSortableInt(value)));
|
||||
}
|
||||
|
||||
for (double value : docValues.doubleValues) {
|
||||
doc.add(new SortedNumericDocValuesField("multi_valued_double", NumericUtils.doubleToSortableLong(value)));
|
||||
}
|
||||
|
||||
for (byte[] value : docValues.bytesValues) {
|
||||
doc.add(new SortedSetDocValuesField("multi_valued_bytes", new BytesRef(value)));
|
||||
}
|
||||
|
||||
w1.addDocument(doc);
|
||||
w2.addDocument(doc);
|
||||
if (random().nextDouble() < deleteChance) {
|
||||
|
|
|
@ -17,181 +17,154 @@
|
|||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.FilteringTokenFilter;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
|
||||
/**
|
||||
* Uses an {@link Analyzer} on content to get offsets. It may use a {@link MemoryIndex} too.
|
||||
* Provides a base class for analysis based offset strategies to extend from.
|
||||
* Requires an Analyzer and provides an override-able method for altering how
|
||||
* the TokenStream is created.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class AnalysisOffsetStrategy extends FieldOffsetStrategy {
|
||||
public abstract class AnalysisOffsetStrategy extends FieldOffsetStrategy {
|
||||
|
||||
//TODO: Consider splitting this highlighter into a MemoryIndexFieldHighlighter and a TokenStreamFieldHighlighter
|
||||
private static final BytesRef[] ZERO_LEN_BYTES_REF_ARRAY = new BytesRef[0];
|
||||
private final Analyzer analyzer;
|
||||
private final MemoryIndex memoryIndex;
|
||||
private final LeafReader leafReader;
|
||||
private final CharacterRunAutomaton preMemIndexFilterAutomaton;
|
||||
protected final Analyzer analyzer;
|
||||
|
||||
public AnalysisOffsetStrategy(String field, BytesRef[] extractedTerms, PhraseHelper phraseHelper,
|
||||
CharacterRunAutomaton[] automata, Analyzer analyzer,
|
||||
Function<Query, Collection<Query>> multiTermQueryRewrite) {
|
||||
super(field, extractedTerms, phraseHelper, automata);
|
||||
public AnalysisOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer analyzer) {
|
||||
super(field, queryTerms, phraseHelper, automata);
|
||||
this.analyzer = analyzer;
|
||||
// Automata (Wildcards / MultiTermQuery):
|
||||
this.automata = automata;
|
||||
|
||||
if (terms.length > 0 && !strictPhrases.hasPositionSensitivity()) {
|
||||
this.automata = convertTermsToAutomata(terms, automata);
|
||||
// clear the terms array now that we've moved them to be expressed as automata
|
||||
terms = ZERO_LEN_BYTES_REF_ARRAY;
|
||||
if (analyzer.getOffsetGap(field) != 1) { // note: 1 is the default. It is RARELY changed.
|
||||
throw new IllegalArgumentException(
|
||||
"offset gap of the provided analyzer should be 1 (field " + field + ")");
|
||||
}
|
||||
|
||||
if (terms.length > 0 || strictPhrases.willRewrite()) { //needs MemoryIndex
|
||||
// init MemoryIndex
|
||||
boolean storePayloads = strictPhrases.hasPositionSensitivity(); // might be needed
|
||||
memoryIndex = new MemoryIndex(true, storePayloads);//true==store offsets
|
||||
leafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader();
|
||||
// preFilter for MemoryIndex
|
||||
preMemIndexFilterAutomaton = buildCombinedAutomaton(field, terms, this.automata, strictPhrases,
|
||||
multiTermQueryRewrite);
|
||||
} else {
|
||||
memoryIndex = null;
|
||||
leafReader = null;
|
||||
preMemIndexFilterAutomaton = null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
public final UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.ANALYSIS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
// note: don't need LimitTokenOffsetFilter since content is already truncated to maxLength
|
||||
TokenStream tokenStream = tokenStream(content);
|
||||
|
||||
if (memoryIndex != null) { // also handles automata.length > 0
|
||||
// We use a MemoryIndex and index the tokenStream so that later we have the PostingsEnum with offsets.
|
||||
|
||||
// note: An *alternative* strategy is to get PostingsEnums without offsets from the main index
|
||||
// and then marry this up with a fake PostingsEnum backed by a TokenStream (which has the offsets) and
|
||||
// can use that to filter applicable tokens? It would have the advantage of being able to exit
|
||||
// early and save some re-analysis. This would be an additional method/offset-source approach
|
||||
// since it's still useful to highlight without any index (so we build MemoryIndex).
|
||||
|
||||
// note: probably unwise to re-use TermsEnum on reset mem index so we don't. But we do re-use the
|
||||
// leaf reader, which is a bit more top level than in the guts.
|
||||
memoryIndex.reset();
|
||||
|
||||
// Filter the tokenStream to applicable terms
|
||||
if (preMemIndexFilterAutomaton != null) {
|
||||
tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
|
||||
}
|
||||
memoryIndex.addField(field, tokenStream);//note: calls tokenStream.reset() & close()
|
||||
tokenStream = null; // it's consumed; done.
|
||||
docId = 0;
|
||||
|
||||
if (automata.length > 0) {
|
||||
Terms foundTerms = leafReader.terms(field);
|
||||
if (foundTerms == null) {
|
||||
return Collections.emptyList(); //No offsets for this field.
|
||||
}
|
||||
// Un-invert for the automata. Much more compact than a CachingTokenStream
|
||||
tokenStream = MultiTermHighlighting.uninvertAndFilterTerms(foundTerms, 0, automata, content.length());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return createOffsetsEnums(leafReader, docId, tokenStream);
|
||||
}
|
||||
|
||||
protected TokenStream tokenStream(String content) throws IOException {
|
||||
return MultiValueTokenStream.wrap(field, analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR);
|
||||
// If there is no splitChar in content then we needn't wrap:
|
||||
int splitCharIdx = content.indexOf(UnifiedHighlighter.MULTIVAL_SEP_CHAR);
|
||||
if (splitCharIdx == -1) {
|
||||
return analyzer.tokenStream(field, content);
|
||||
}
|
||||
|
||||
private static CharacterRunAutomaton[] convertTermsToAutomata(BytesRef[] terms, CharacterRunAutomaton[] automata) {
|
||||
CharacterRunAutomaton[] newAutomata = new CharacterRunAutomaton[terms.length + automata.length];
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
newAutomata[i] = MultiTermHighlighting.makeStringMatchAutomata(terms[i]);
|
||||
}
|
||||
// Append existing automata (that which is used for MTQs)
|
||||
System.arraycopy(automata, 0, newAutomata, terms.length, automata.length);
|
||||
return newAutomata;
|
||||
}
|
||||
TokenStream subTokenStream = analyzer.tokenStream(field, content.substring(0, splitCharIdx));
|
||||
|
||||
private static FilteringTokenFilter newKeepWordFilter(final TokenStream tokenStream,
|
||||
final CharacterRunAutomaton charRunAutomaton) {
|
||||
// it'd be nice to use KeepWordFilter but it demands a CharArraySet. TODO File JIRA? Need a new interface?
|
||||
return new FilteringTokenFilter(tokenStream) {
|
||||
final CharTermAttribute charAtt = addAttribute(CharTermAttribute.class);
|
||||
|
||||
@Override
|
||||
protected boolean accept() throws IOException {
|
||||
return charRunAutomaton.run(charAtt.buffer(), 0, charAtt.length());
|
||||
return new MultiValueTokenStream(subTokenStream, field, analyzer, content, UnifiedHighlighter.MULTIVAL_SEP_CHAR, splitCharIdx);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Build one {@link CharacterRunAutomaton} matching any term the query might match.
|
||||
* Wraps an {@link Analyzer} and string text that represents multiple values delimited by a specified character. This
|
||||
* exposes a TokenStream that matches what would get indexed considering the
|
||||
* {@link Analyzer#getPositionIncrementGap(String)}. Currently this assumes {@link Analyzer#getOffsetGap(String)} is
|
||||
* 1; an exception will be thrown if it isn't.
|
||||
* <br />
|
||||
* It would be more orthogonal for this to be an Analyzer since we're wrapping an Analyzer but doing so seems like
|
||||
* more work. The underlying components see a Reader not a String -- and the String is easy to
|
||||
* split up without redundant buffering.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
private static CharacterRunAutomaton buildCombinedAutomaton(String field, BytesRef[] terms,
|
||||
CharacterRunAutomaton[] automata,
|
||||
PhraseHelper strictPhrases,
|
||||
Function<Query, Collection<Query>> multiTermQueryRewrite) {
|
||||
List<CharacterRunAutomaton> allAutomata = new ArrayList<>();
|
||||
if (terms.length > 0) {
|
||||
allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(terms))));
|
||||
}
|
||||
Collections.addAll(allAutomata, automata);
|
||||
for (SpanQuery spanQuery : strictPhrases.getSpanQueries()) {
|
||||
Collections.addAll(allAutomata,
|
||||
MultiTermHighlighting.extractAutomata(spanQuery, field, true, multiTermQueryRewrite));//true==lookInSpan
|
||||
private static final class MultiValueTokenStream extends TokenFilter {
|
||||
|
||||
private final String fieldName;
|
||||
private final Analyzer indexAnalyzer;
|
||||
private final String content;
|
||||
private final char splitChar;
|
||||
|
||||
private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
|
||||
private int startValIdx = 0;
|
||||
private int endValIdx;
|
||||
private int remainingPosInc = 0;
|
||||
|
||||
private MultiValueTokenStream(TokenStream subTokenStream, String fieldName, Analyzer indexAnalyzer,
|
||||
String content, char splitChar, int splitCharIdx) {
|
||||
super(subTokenStream); // subTokenStream is already initialized to operate on the first value
|
||||
this.fieldName = fieldName;
|
||||
this.indexAnalyzer = indexAnalyzer;
|
||||
this.content = content;
|
||||
this.splitChar = splitChar;
|
||||
this.endValIdx = splitCharIdx;
|
||||
}
|
||||
|
||||
if (allAutomata.size() == 1) {
|
||||
return allAutomata.get(0);
|
||||
}
|
||||
//TODO it'd be nice if we could get at the underlying Automaton in CharacterRunAutomaton so that we
|
||||
// could union them all. But it's not exposed, and note TermRangeQuery isn't modelled as an Automaton
|
||||
// by MultiTermHighlighting.
|
||||
|
||||
// Return an aggregate CharacterRunAutomaton of others
|
||||
return new CharacterRunAutomaton(Automata.makeEmpty()) {// the makeEmpty() is bogus; won't be used
|
||||
@Override
|
||||
public boolean run(char[] chars, int offset, int length) {
|
||||
for (int i = 0; i < allAutomata.size(); i++) {// don't use foreach to avoid Iterator allocation
|
||||
if (allAutomata.get(i).run(chars, offset, length)) {
|
||||
public void reset() throws IOException {
|
||||
if (startValIdx != 0) {
|
||||
throw new IllegalStateException("This TokenStream wasn't developed to be re-used.");
|
||||
// ... although we could if a need for it arises.
|
||||
}
|
||||
super.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
while (true) {
|
||||
|
||||
if (input.incrementToken()) {
|
||||
// Position tracking:
|
||||
if (remainingPosInc > 0) {//usually true first token of additional values (not first val)
|
||||
posIncAtt.setPositionIncrement(remainingPosInc + posIncAtt.getPositionIncrement());
|
||||
remainingPosInc = 0;//reset
|
||||
}
|
||||
// Offset tracking:
|
||||
offsetAtt.setOffset(
|
||||
startValIdx + offsetAtt.startOffset(),
|
||||
startValIdx + offsetAtt.endOffset()
|
||||
);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
if (endValIdx == content.length()) {//no more
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
input.end(); // might adjust position increment
|
||||
remainingPosInc += posIncAtt.getPositionIncrement();
|
||||
input.close();
|
||||
remainingPosInc += indexAnalyzer.getPositionIncrementGap(fieldName);
|
||||
|
||||
// Get new tokenStream based on next segment divided by the splitChar
|
||||
startValIdx = endValIdx + 1;
|
||||
endValIdx = content.indexOf(splitChar, startValIdx);
|
||||
if (endValIdx == -1) {//EOF
|
||||
endValIdx = content.length();
|
||||
}
|
||||
TokenStream tokenStream = indexAnalyzer.tokenStream(fieldName, content.substring(startValIdx, endValIdx));
|
||||
if (tokenStream != input) {// (input is defined in TokenFilter set in the constructor)
|
||||
// This is a grand trick we do -- knowing that the analyzer's re-use strategy is going to produce the
|
||||
// very same tokenStream instance and thus have the same AttributeSource as this wrapping TokenStream
|
||||
// since we used it as our input in the constructor.
|
||||
// Were this not the case, we'd have to copy every attribute of interest since we can't alter the
|
||||
// AttributeSource of this wrapping TokenStream post-construction (it's all private/final).
|
||||
// If this is a problem, we could do that instead; maybe with a custom CharTermAttribute that allows
|
||||
// us to easily set the char[] reference without literally copying char by char.
|
||||
throw new IllegalStateException("Require TokenStream re-use. Unsupported re-use strategy?: " +
|
||||
indexAnalyzer.getReuseStrategy());
|
||||
}
|
||||
tokenStream.reset();
|
||||
} // while loop to increment token of this new value
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
super.end();
|
||||
// Offset tracking:
|
||||
offsetAtt.setOffset(
|
||||
startValIdx + offsetAtt.startOffset(),
|
||||
startValIdx + offsetAtt.endOffset());
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,145 @@
|
|||
/*
|
||||
* 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.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/**
|
||||
* Provides a view over several underlying PostingsEnums for the iteration of offsets on the current document only.
|
||||
* It's not general purpose; the position returned is always -1 and it doesn't iterate the documents.
|
||||
*/
|
||||
final class CompositeOffsetsPostingsEnum extends PostingsEnum {
|
||||
|
||||
private final int docId;
|
||||
private final int freq;
|
||||
private final PriorityQueue<BoundsCheckingPostingsEnum> queue;
|
||||
private boolean firstPositionConsumed = false;
|
||||
|
||||
/**
|
||||
* This class is used to ensure we don't over iterate the underlying
|
||||
* postings enum by keeping track of the position relative to the
|
||||
* frequency.
|
||||
* Ideally this would've been an implementation of a PostingsEnum
|
||||
* but it would have to delegate most methods and it seemed easier
|
||||
* to just wrap the tweaked method.
|
||||
*/
|
||||
private static final class BoundsCheckingPostingsEnum {
|
||||
|
||||
private final PostingsEnum postingsEnum;
|
||||
private int remainingPositions;
|
||||
|
||||
BoundsCheckingPostingsEnum(PostingsEnum postingsEnum) throws IOException {
|
||||
this.postingsEnum = postingsEnum;
|
||||
this.remainingPositions = postingsEnum.freq();
|
||||
nextPosition();
|
||||
}
|
||||
|
||||
/** Advances to the next position and returns true, or returns false if it can't. */
|
||||
private boolean nextPosition() throws IOException {
|
||||
if (remainingPositions-- > 0) {
|
||||
postingsEnum.nextPosition(); // ignore the actual position; we don't care.
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** The provided {@link PostingsEnum}s must all be positioned to the same document, and must have offsets. */
|
||||
CompositeOffsetsPostingsEnum(List<PostingsEnum> postingsEnums) throws IOException {
|
||||
queue = new PriorityQueue<BoundsCheckingPostingsEnum>(postingsEnums.size()) {
|
||||
@Override
|
||||
protected boolean lessThan(BoundsCheckingPostingsEnum a, BoundsCheckingPostingsEnum b) {
|
||||
try {
|
||||
return a.postingsEnum.startOffset() < b.postingsEnum.startOffset();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
int freqAdd = 0;
|
||||
for (PostingsEnum postingsEnum : postingsEnums) {
|
||||
queue.add(new BoundsCheckingPostingsEnum(postingsEnum));
|
||||
freqAdd += postingsEnum.freq();
|
||||
}
|
||||
freq = freqAdd;
|
||||
this.docId = queue.top().postingsEnum.docID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return freq;
|
||||
}
|
||||
|
||||
/** Advances to the next position. Always returns -1; the caller is assumed not to care for the highlighter. */
|
||||
@Override
|
||||
public int nextPosition() throws IOException {
|
||||
if (!firstPositionConsumed) {
|
||||
firstPositionConsumed = true;
|
||||
} else if (queue.size() == 0) {
|
||||
throw new IllegalStateException("nextPosition called too many times");
|
||||
} else if (queue.top().nextPosition()) { // advance head
|
||||
queue.updateTop(); //the new position may be behind another postingsEnum in the queue
|
||||
} else {
|
||||
queue.pop(); //this postingsEnum is consumed; get rid of it. Another will take it's place.
|
||||
}
|
||||
assert queue.size() > 0;
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startOffset() throws IOException {
|
||||
return queue.top().postingsEnum.startOffset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endOffset() throws IOException {
|
||||
return queue.top().postingsEnum.endOffset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() throws IOException {
|
||||
return queue.top().postingsEnum.getPayload();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return docId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 1L; //at most 1 doc is returned
|
||||
}
|
||||
}
|
|
@ -14,16 +14,14 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
|
@ -31,6 +29,7 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.spans.Spans;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
/**
|
||||
|
@ -42,14 +41,14 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
|||
public abstract class FieldOffsetStrategy {
|
||||
|
||||
protected final String field;
|
||||
protected BytesRef[] terms; // Query: free-standing terms
|
||||
protected PhraseHelper strictPhrases; // Query: position-sensitive information TODO: rename
|
||||
protected CharacterRunAutomaton[] automata; // Query: free-standing wildcards (multi-term query)
|
||||
protected final PhraseHelper phraseHelper; // Query: position-sensitive information TODO: rename
|
||||
protected final BytesRef[] terms; // Query: free-standing terms
|
||||
protected final CharacterRunAutomaton[] automata; // Query: free-standing wildcards (multi-term query)
|
||||
|
||||
public FieldOffsetStrategy(String field, BytesRef[] queryTerms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
|
||||
this.field = field;
|
||||
this.terms = queryTerms;
|
||||
this.strictPhrases = phraseHelper;
|
||||
this.phraseHelper = phraseHelper;
|
||||
this.automata = automata;
|
||||
}
|
||||
|
||||
|
@ -65,58 +64,90 @@ public abstract class FieldOffsetStrategy {
|
|||
*/
|
||||
public abstract List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException;
|
||||
|
||||
protected List<OffsetsEnum> createOffsetsEnums(LeafReader leafReader, int doc, TokenStream tokenStream) throws IOException {
|
||||
List<OffsetsEnum> offsetsEnums = createOffsetsEnumsFromReader(leafReader, doc);
|
||||
if (automata.length > 0) {
|
||||
offsetsEnums.add(createOffsetsEnumFromTokenStream(doc, tokenStream));
|
||||
}
|
||||
return offsetsEnums;
|
||||
protected List<OffsetsEnum> createOffsetsEnumsFromReader(LeafReader leafReader, int doc) throws IOException {
|
||||
final Terms termsIndex = leafReader.terms(field);
|
||||
if (termsIndex == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
protected List<OffsetsEnum> createOffsetsEnumsFromReader(LeafReader atomicReader, int doc) throws IOException {
|
||||
// For strict positions, get a Map of term to Spans:
|
||||
// note: ScriptPhraseHelper.NONE does the right thing for these method calls
|
||||
final Map<BytesRef, Spans> strictPhrasesTermToSpans =
|
||||
strictPhrases.getTermToSpans(atomicReader, doc);
|
||||
phraseHelper.getTermToSpans(leafReader, doc);
|
||||
// Usually simply wraps terms in a List; but if willRewrite() then can be expanded
|
||||
final List<BytesRef> sourceTerms =
|
||||
strictPhrases.expandTermsIfRewrite(terms, strictPhrasesTermToSpans);
|
||||
phraseHelper.expandTermsIfRewrite(terms, strictPhrasesTermToSpans);
|
||||
|
||||
final List<OffsetsEnum> offsetsEnums = new ArrayList<>(sourceTerms.size() + 1);
|
||||
final List<OffsetsEnum> offsetsEnums = new ArrayList<>(sourceTerms.size() + automata.length);
|
||||
|
||||
Terms termsIndex = atomicReader == null || sourceTerms.isEmpty() ? null : atomicReader.terms(field);
|
||||
if (termsIndex != null) {
|
||||
// Handle sourceTerms:
|
||||
if (!sourceTerms.isEmpty()) {
|
||||
TermsEnum termsEnum = termsIndex.iterator();//does not return null
|
||||
for (BytesRef term : sourceTerms) {
|
||||
if (!termsEnum.seekExact(term)) {
|
||||
continue; // term not found
|
||||
}
|
||||
if (termsEnum.seekExact(term)) {
|
||||
PostingsEnum postingsEnum = termsEnum.postings(null, PostingsEnum.OFFSETS);
|
||||
|
||||
if (postingsEnum == null) {
|
||||
// no offsets or positions available
|
||||
throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
|
||||
}
|
||||
if (doc != postingsEnum.advance(doc)) { // now it's positioned, although may be exhausted
|
||||
continue;
|
||||
}
|
||||
postingsEnum = strictPhrases.filterPostings(term, postingsEnum, strictPhrasesTermToSpans.get(term));
|
||||
if (postingsEnum == null) {
|
||||
continue;// completely filtered out
|
||||
}
|
||||
|
||||
if (doc == postingsEnum.advance(doc)) { // now it's positioned, although may be exhausted
|
||||
postingsEnum = phraseHelper.filterPostings(term, postingsEnum, strictPhrasesTermToSpans.get(term));
|
||||
if (postingsEnum != null) {
|
||||
offsetsEnums.add(new OffsetsEnum(term, postingsEnum));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Handle automata
|
||||
if (automata.length > 0) {
|
||||
offsetsEnums.addAll(createAutomataOffsetsFromTerms(termsIndex, doc));
|
||||
}
|
||||
|
||||
return offsetsEnums;
|
||||
}
|
||||
|
||||
protected OffsetsEnum createOffsetsEnumFromTokenStream(int doc, TokenStream tokenStream) throws IOException {
|
||||
// if there are automata (MTQ), we have to initialize the "fake" enum wrapping them.
|
||||
assert tokenStream != null;
|
||||
// TODO Opt: we sometimes evaluate the automata twice when this TS isn't the original; can we avoid?
|
||||
PostingsEnum mtqPostingsEnum = MultiTermHighlighting.getDocsEnum(tokenStream, automata);
|
||||
assert mtqPostingsEnum instanceof Closeable; // FYI we propagate close() later.
|
||||
mtqPostingsEnum.advance(doc);
|
||||
return new OffsetsEnum(null, mtqPostingsEnum);
|
||||
protected List<OffsetsEnum> createAutomataOffsetsFromTerms(Terms termsIndex, int doc) throws IOException {
|
||||
List<List<PostingsEnum>> automataPostings = new ArrayList<>(automata.length);
|
||||
for (int i = 0; i < automata.length; i++) {
|
||||
automataPostings.add(new ArrayList<>());
|
||||
}
|
||||
|
||||
TermsEnum termsEnum = termsIndex.iterator();
|
||||
BytesRef term;
|
||||
CharsRefBuilder refBuilder = new CharsRefBuilder();
|
||||
while ((term = termsEnum.next()) != null) {
|
||||
for (int i = 0; i < automata.length; i++) {
|
||||
CharacterRunAutomaton automaton = automata[i];
|
||||
refBuilder.copyUTF8Bytes(term);
|
||||
if (automaton.run(refBuilder.chars(), 0, refBuilder.length())) {
|
||||
PostingsEnum postings = termsEnum.postings(null, PostingsEnum.OFFSETS);
|
||||
if (doc == postings.advance(doc)) {
|
||||
automataPostings.get(i).add(postings);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
List<OffsetsEnum> offsetsEnums = new ArrayList<>(automata.length); //will be at most this long
|
||||
for (int i = 0; i < automata.length; i++) {
|
||||
CharacterRunAutomaton automaton = automata[i];
|
||||
List<PostingsEnum> postingsEnums = automataPostings.get(i);
|
||||
int size = postingsEnums.size();
|
||||
if (size > 0) { //only add if we have offsets
|
||||
BytesRef wildcardTerm = new BytesRef(automaton.toString());
|
||||
if (size == 1) { //don't wrap in a composite if there's only one OffsetsEnum
|
||||
offsetsEnums.add(new OffsetsEnum(wildcardTerm, postingsEnums.get(0)));
|
||||
} else {
|
||||
offsetsEnums.add(new OffsetsEnum(wildcardTerm, new CompositeOffsetsPostingsEnum(postingsEnums)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return offsetsEnums;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,129 @@
|
|||
/*
|
||||
* 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.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.FilteringTokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
|
||||
/**
|
||||
* Uses an {@link Analyzer} on content to get offsets and then populates a {@link MemoryIndex}.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class MemoryIndexOffsetStrategy extends AnalysisOffsetStrategy {
|
||||
|
||||
private final MemoryIndex memoryIndex;
|
||||
private final LeafReader leafReader;
|
||||
private final CharacterRunAutomaton preMemIndexFilterAutomaton;
|
||||
|
||||
public MemoryIndexOffsetStrategy(String field, BytesRef[] extractedTerms, PhraseHelper phraseHelper,
|
||||
CharacterRunAutomaton[] automata, Analyzer analyzer,
|
||||
Function<Query, Collection<Query>> multiTermQueryRewrite) {
|
||||
super(field, extractedTerms, phraseHelper, automata, analyzer);
|
||||
boolean storePayloads = phraseHelper.hasPositionSensitivity(); // might be needed
|
||||
memoryIndex = new MemoryIndex(true, storePayloads);//true==store offsets
|
||||
leafReader = (LeafReader) memoryIndex.createSearcher().getIndexReader(); // appears to be re-usable
|
||||
// preFilter for MemoryIndex
|
||||
preMemIndexFilterAutomaton = buildCombinedAutomaton(field, terms, this.automata, phraseHelper, multiTermQueryRewrite);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build one {@link CharacterRunAutomaton} matching any term the query might match.
|
||||
*/
|
||||
private static CharacterRunAutomaton buildCombinedAutomaton(String field, BytesRef[] terms,
|
||||
CharacterRunAutomaton[] automata,
|
||||
PhraseHelper strictPhrases,
|
||||
Function<Query, Collection<Query>> multiTermQueryRewrite) {
|
||||
List<CharacterRunAutomaton> allAutomata = new ArrayList<>();
|
||||
if (terms.length > 0) {
|
||||
allAutomata.add(new CharacterRunAutomaton(Automata.makeStringUnion(Arrays.asList(terms))));
|
||||
}
|
||||
Collections.addAll(allAutomata, automata);
|
||||
for (SpanQuery spanQuery : strictPhrases.getSpanQueries()) {
|
||||
Collections.addAll(allAutomata,
|
||||
MultiTermHighlighting.extractAutomata(spanQuery, field, true, multiTermQueryRewrite));//true==lookInSpan
|
||||
}
|
||||
|
||||
if (allAutomata.size() == 1) {
|
||||
return allAutomata.get(0);
|
||||
}
|
||||
//TODO it'd be nice if we could get at the underlying Automaton in CharacterRunAutomaton so that we
|
||||
// could union them all. But it's not exposed, and note TermRangeQuery isn't modelled as an Automaton
|
||||
// by MultiTermHighlighting.
|
||||
|
||||
// Return an aggregate CharacterRunAutomaton of others
|
||||
return new CharacterRunAutomaton(Automata.makeEmpty()) {// the makeEmpty() is bogus; won't be used
|
||||
@Override
|
||||
public boolean run(char[] chars, int offset, int length) {
|
||||
for (int i = 0; i < allAutomata.size(); i++) {// don't use foreach to avoid Iterator allocation
|
||||
if (allAutomata.get(i).run(chars, offset, length)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
// note: don't need LimitTokenOffsetFilter since content is already truncated to maxLength
|
||||
TokenStream tokenStream = tokenStream(content);
|
||||
|
||||
// Filter the tokenStream to applicable terms
|
||||
tokenStream = newKeepWordFilter(tokenStream, preMemIndexFilterAutomaton);
|
||||
memoryIndex.reset();
|
||||
memoryIndex.addField(field, tokenStream);//note: calls tokenStream.reset() & close()
|
||||
docId = 0;
|
||||
|
||||
return createOffsetsEnumsFromReader(leafReader, docId);
|
||||
}
|
||||
|
||||
|
||||
private static FilteringTokenFilter newKeepWordFilter(final TokenStream tokenStream,
|
||||
final CharacterRunAutomaton charRunAutomaton) {
|
||||
// it'd be nice to use KeepWordFilter but it demands a CharArraySet. TODO File JIRA? Need a new interface?
|
||||
return new FilteringTokenFilter(tokenStream) {
|
||||
final CharTermAttribute charAtt = addAttribute(CharTermAttribute.class);
|
||||
|
||||
@Override
|
||||
protected boolean accept() throws IOException {
|
||||
return charRunAutomaton.run(charAtt.buffer(), 0, charAtt.length());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -16,8 +16,6 @@
|
|||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -25,15 +23,7 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.index.FilterLeafReader;
|
||||
import org.apache.lucene.index.FilteredTermsEnum;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.AutomatonQuery;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
|
@ -48,9 +38,7 @@ import org.apache.lucene.search.spans.SpanNearQuery;
|
|||
import org.apache.lucene.search.spans.SpanNotQuery;
|
||||
import org.apache.lucene.search.spans.SpanOrQuery;
|
||||
import org.apache.lucene.search.spans.SpanPositionCheckQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
import org.apache.lucene.util.automaton.Automaton;
|
||||
|
@ -210,182 +198,4 @@ class MultiTermHighlighting {
|
|||
return list.toArray(new CharacterRunAutomaton[list.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a "fake" DocsAndPositionsEnum over the tokenstream, returning offsets where {@code matchers}
|
||||
* matches tokens.
|
||||
* <p>
|
||||
* This is solely used internally by PostingsHighlighter: <b>DO NOT USE THIS METHOD!</b>
|
||||
*/
|
||||
public static PostingsEnum getDocsEnum(final TokenStream ts, final CharacterRunAutomaton[] matchers) throws IOException {
|
||||
return new TokenStreamPostingsEnum(ts, matchers);
|
||||
}
|
||||
|
||||
// TODO: we could use CachingWrapperFilter, (or consume twice) to allow us to have a true freq()
|
||||
// but this would have a performance cost for likely little gain in the user experience, it
|
||||
// would only serve to make this method less bogus.
|
||||
// instead, we always return freq() = Integer.MAX_VALUE and let the highlighter terminate based on offset...
|
||||
// TODO: DWS perhaps instead OffsetsEnum could become abstract and this would be an impl?
|
||||
private static class TokenStreamPostingsEnum extends PostingsEnum implements Closeable {
|
||||
TokenStream stream; // becomes null when closed
|
||||
final CharacterRunAutomaton[] matchers;
|
||||
final CharTermAttribute charTermAtt;
|
||||
final OffsetAttribute offsetAtt;
|
||||
|
||||
int currentDoc = -1;
|
||||
int currentMatch = -1;
|
||||
int currentStartOffset = -1;
|
||||
|
||||
int currentEndOffset = -1;
|
||||
|
||||
final BytesRef matchDescriptions[];
|
||||
|
||||
TokenStreamPostingsEnum(TokenStream ts, CharacterRunAutomaton[] matchers) throws IOException {
|
||||
this.stream = ts;
|
||||
this.matchers = matchers;
|
||||
matchDescriptions = new BytesRef[matchers.length];
|
||||
charTermAtt = ts.addAttribute(CharTermAttribute.class);
|
||||
offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
||||
ts.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextPosition() throws IOException {
|
||||
if (stream != null) {
|
||||
while (stream.incrementToken()) {
|
||||
for (int i = 0; i < matchers.length; i++) {
|
||||
if (matchers[i].run(charTermAtt.buffer(), 0, charTermAtt.length())) {
|
||||
currentStartOffset = offsetAtt.startOffset();
|
||||
currentEndOffset = offsetAtt.endOffset();
|
||||
currentMatch = i;
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
stream.end();
|
||||
close();
|
||||
}
|
||||
// exhausted
|
||||
currentStartOffset = currentEndOffset = Integer.MAX_VALUE;
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return Integer.MAX_VALUE; // lie
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startOffset() throws IOException {
|
||||
assert currentStartOffset >= 0;
|
||||
return currentStartOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endOffset() throws IOException {
|
||||
assert currentEndOffset >= 0;
|
||||
return currentEndOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() throws IOException {
|
||||
if (matchDescriptions[currentMatch] == null) {
|
||||
matchDescriptions[currentMatch] = new BytesRef(matchers[currentMatch].toString());
|
||||
}
|
||||
return matchDescriptions[currentMatch];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return currentDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return currentDoc = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (stream != null) {
|
||||
stream.close();
|
||||
stream = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a TokenStream un-inverted from the provided Terms, but filtered based on the automata. The
|
||||
* Terms must have exactly one doc count (e.g. term vector or MemoryIndex).
|
||||
*/
|
||||
//TODO: Alternatively, produce a list of OffsetsEnums from the Terms that match the automata.
|
||||
public static TokenStream uninvertAndFilterTerms(Terms termsIndex,
|
||||
int doc,
|
||||
final CharacterRunAutomaton[] automata,
|
||||
int offsetLength)
|
||||
throws IOException {
|
||||
assert automata.length > 0;
|
||||
//Note: if automata were plain Automaton (not CharacterRunAutomaton), we might instead use
|
||||
// TermsEnum.intersect(compiledAutomaton). But probably won't help due to O(N) TV impl so whatever.
|
||||
FilterLeafReader.FilterTerms filteredTermsIndex = new FilterLeafReader.FilterTerms(termsIndex) {
|
||||
@Override
|
||||
public TermsEnum iterator() throws IOException {
|
||||
return new FilteredTermsEnum(super.iterator(), false) {//false == no seek
|
||||
CharsRefBuilder tempCharsRefBuilder = new CharsRefBuilder();//reuse only for UTF8->UTF16 call
|
||||
|
||||
@Override
|
||||
protected AcceptStatus accept(BytesRef termBytesRef) throws IOException {
|
||||
//Grab the term (in same way as BytesRef.utf8ToString() but we don't want a String obj)
|
||||
tempCharsRefBuilder.grow(termBytesRef.length);
|
||||
final int charLen = UnicodeUtil.UTF8toUTF16(termBytesRef, tempCharsRefBuilder.chars());
|
||||
for (CharacterRunAutomaton runAutomaton : automata) {
|
||||
if (runAutomaton.run(tempCharsRefBuilder.chars(), 0, charLen)) {
|
||||
return AcceptStatus.YES;
|
||||
}
|
||||
}
|
||||
return AcceptStatus.NO;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() throws IOException {
|
||||
return -1; // unknown
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() throws IOException {
|
||||
return -1; // unknown
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumDocFreq() throws IOException {
|
||||
return -1; // unknown
|
||||
}
|
||||
};
|
||||
float loadFactor = 1f / 64f;
|
||||
return new TokenStreamFromTermVector(filteredTermsIndex, doc, offsetLength, loadFactor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a simple automata that matches the specified term.
|
||||
*/
|
||||
public static CharacterRunAutomaton makeStringMatchAutomata(BytesRef term) {
|
||||
String termString = term.utf8ToString();
|
||||
return new CharacterRunAutomaton(Automata.makeString(termString)) {
|
||||
@Override
|
||||
public String toString() {
|
||||
return termString;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,148 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
|
||||
/**
|
||||
* Wraps an {@link Analyzer} and string text that represents multiple values delimited by a specified character. This
|
||||
* exposes a TokenStream that matches what would get indexed considering the
|
||||
* {@link Analyzer#getPositionIncrementGap(String)}. Currently this assumes {@link Analyzer#getOffsetGap(String)} is
|
||||
* 1; an exception will be thrown if it isn't.
|
||||
* <br />
|
||||
* It would be more orthogonal for this to be an Analyzer since we're wrapping an Analyzer but doing so seems like
|
||||
* more work. The underlying components see a Reader not a String -- and the String is easy to
|
||||
* split up without redundant buffering.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class MultiValueTokenStream extends TokenFilter {
|
||||
|
||||
private final String fieldName;
|
||||
private final Analyzer indexAnalyzer;
|
||||
private final String content;
|
||||
private final char splitChar;
|
||||
|
||||
private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
|
||||
private int startValIdx = 0;
|
||||
private int endValIdx;
|
||||
private int remainingPosInc = 0;
|
||||
|
||||
/** note: The caller must remember to close the TokenStream eventually. */
|
||||
static TokenStream wrap(String fieldName, Analyzer indexAnalyzer, String content, char splitChar)
|
||||
throws IOException {
|
||||
if (indexAnalyzer.getOffsetGap(fieldName) != 1) { // note: 1 is the default. It is RARELY changed.
|
||||
throw new IllegalArgumentException(
|
||||
"offset gap of the provided analyzer should be 1 (field " + fieldName + ")");
|
||||
}
|
||||
// If there is no splitChar in content then we needn't wrap:
|
||||
int splitCharIdx = content.indexOf(splitChar);
|
||||
if (splitCharIdx == -1) {
|
||||
return indexAnalyzer.tokenStream(fieldName, content);
|
||||
}
|
||||
|
||||
TokenStream subTokenStream = indexAnalyzer.tokenStream(fieldName, content.substring(0, splitCharIdx));
|
||||
|
||||
return new MultiValueTokenStream(subTokenStream, fieldName, indexAnalyzer, content, splitChar, splitCharIdx);
|
||||
}
|
||||
|
||||
private MultiValueTokenStream(TokenStream subTokenStream, String fieldName, Analyzer indexAnalyzer,
|
||||
String content, char splitChar, int splitCharIdx) {
|
||||
super(subTokenStream); // subTokenStream is already initialized to operate on the first value
|
||||
this.fieldName = fieldName;
|
||||
this.indexAnalyzer = indexAnalyzer;
|
||||
this.content = content;
|
||||
this.splitChar = splitChar;
|
||||
this.endValIdx = splitCharIdx;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
if (startValIdx != 0) {
|
||||
throw new IllegalStateException("This TokenStream wasn't developed to be re-used.");
|
||||
// ... although we could if a need for it arises.
|
||||
}
|
||||
super.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
while (true) {
|
||||
|
||||
if (input.incrementToken()) {
|
||||
// Position tracking:
|
||||
if (remainingPosInc > 0) {//usually true first token of additional values (not first val)
|
||||
posIncAtt.setPositionIncrement(remainingPosInc + posIncAtt.getPositionIncrement());
|
||||
remainingPosInc = 0;//reset
|
||||
}
|
||||
// Offset tracking:
|
||||
offsetAtt.setOffset(
|
||||
startValIdx + offsetAtt.startOffset(),
|
||||
startValIdx + offsetAtt.endOffset()
|
||||
);
|
||||
return true;
|
||||
}
|
||||
|
||||
if (endValIdx == content.length()) {//no more
|
||||
return false;
|
||||
}
|
||||
|
||||
input.end(); // might adjust position increment
|
||||
remainingPosInc += posIncAtt.getPositionIncrement();
|
||||
input.close();
|
||||
remainingPosInc += indexAnalyzer.getPositionIncrementGap(fieldName);
|
||||
|
||||
// Get new tokenStream based on next segment divided by the splitChar
|
||||
startValIdx = endValIdx + 1;
|
||||
endValIdx = content.indexOf(splitChar, startValIdx);
|
||||
if (endValIdx == -1) {//EOF
|
||||
endValIdx = content.length();
|
||||
}
|
||||
TokenStream tokenStream = indexAnalyzer.tokenStream(fieldName, content.substring(startValIdx, endValIdx));
|
||||
if (tokenStream != input) {// (input is defined in TokenFilter set in the constructor)
|
||||
// This is a grand trick we do -- knowing that the analyzer's re-use strategy is going to produce the
|
||||
// very same tokenStream instance and thus have the same AttributeSource as this wrapping TokenStream
|
||||
// since we used it as our input in the constructor.
|
||||
// Were this not the case, we'd have to copy every attribute of interest since we can't alter the
|
||||
// AttributeSource of this wrapping TokenStream post-construction (it's all private/final).
|
||||
// If this is a problem, we could do that instead; maybe with a custom CharTermAttribute that allows
|
||||
// us to easily set the char[] reference without literally copying char by char.
|
||||
throw new IllegalStateException("Require TokenStream re-use. Unsupported re-use strategy?: " +
|
||||
indexAnalyzer.getReuseStrategy());
|
||||
}
|
||||
tokenStream.reset();
|
||||
} // while loop to increment token of this new value
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
super.end();
|
||||
// Offset tracking:
|
||||
offsetAtt.setOffset(
|
||||
startValIdx + offsetAtt.startOffset(),
|
||||
startValIdx + offsetAtt.endOffset());
|
||||
}
|
||||
|
||||
}
|
|
@ -76,6 +76,7 @@ public class OffsetsEnum implements Comparable<OffsetsEnum>, Closeable {
|
|||
}
|
||||
|
||||
void nextPosition() throws IOException {
|
||||
assert hasMorePositions();
|
||||
pos++;
|
||||
postingsEnum.nextPosition();
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ public final class Passage {
|
|||
BytesRef matchTerms[] = new BytesRef[8];
|
||||
int numMatches = 0;
|
||||
|
||||
void addMatch(int startOffset, int endOffset, BytesRef term) {
|
||||
public void addMatch(int startOffset, int endOffset, BytesRef term) {
|
||||
assert startOffset >= this.startOffset && startOffset <= this.endOffset;
|
||||
if (numMatches == matchStarts.length) {
|
||||
int newLength = ArrayUtil.oversize(numMatches+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||
|
|
|
@ -266,7 +266,7 @@ public class PhraseHelper {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns terms as a List, but expanded to any terms in strictPhrases' keySet if present. That can only
|
||||
* Returns terms as a List, but expanded to any terms in phraseHelper' keySet if present. That can only
|
||||
* happen if willRewrite() is true.
|
||||
*/
|
||||
List<BytesRef> expandTermsIfRewrite(BytesRef[] terms, Map<BytesRef, Spans> strictPhrasesTermToSpans) {
|
||||
|
|
|
@ -41,7 +41,7 @@ public class PostingsOffsetStrategy extends FieldOffsetStrategy {
|
|||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
LeafReader leafReader;
|
||||
final LeafReader leafReader;
|
||||
if (reader instanceof LeafReader) {
|
||||
leafReader = (LeafReader) reader;
|
||||
} else {
|
||||
|
@ -54,6 +54,7 @@ public class PostingsOffsetStrategy extends FieldOffsetStrategy {
|
|||
return createOffsetsEnumsFromReader(leafReader, docId);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.POSTINGS;
|
||||
|
|
|
@ -20,7 +20,6 @@ import java.io.IOException;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
@ -58,14 +57,11 @@ public class PostingsWithTermVectorsOffsetStrategy extends FieldOffsetStrategy {
|
|||
}
|
||||
leafReader = new TermVectorFilteredLeafReader(leafReader, docTerms);
|
||||
|
||||
TokenStream tokenStream = automata.length > 0 ? MultiTermHighlighting
|
||||
.uninvertAndFilterTerms(leafReader.terms(field), docId, this.automata, content.length()) : null;
|
||||
|
||||
return createOffsetsEnums(leafReader, docId, tokenStream);
|
||||
return createOffsetsEnumsFromReader(leafReader, docId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnifiedHighlighter.OffsetSource getOffsetSource() {
|
||||
return UnifiedHighlighter.OffsetSource.POSTINGS;
|
||||
return UnifiedHighlighter.OffsetSource.POSTINGS_WITH_TERM_VECTORS;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@ import java.io.IOException;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -51,18 +50,10 @@ public class TermVectorOffsetStrategy extends FieldOffsetStrategy {
|
|||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
LeafReader leafReader = null;
|
||||
if ((terms.length > 0) || strictPhrases.willRewrite()) {
|
||||
leafReader = new TermVectorLeafReader(field, tvTerms);
|
||||
LeafReader leafReader = new TermVectorLeafReader(field, tvTerms);
|
||||
docId = 0;
|
||||
}
|
||||
|
||||
TokenStream tokenStream = null;
|
||||
if (automata.length > 0) {
|
||||
tokenStream = MultiTermHighlighting.uninvertAndFilterTerms(tvTerms, 0, automata, content.length());
|
||||
}
|
||||
|
||||
return createOffsetsEnums(leafReader, docId, tokenStream);
|
||||
return createOffsetsEnumsFromReader(leafReader, docId);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,395 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.search.uhighlight;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefArray;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
/**
|
||||
* TokenStream created from a term vector field. The term vector requires positions and/or offsets (either). If you
|
||||
* want payloads add PayloadAttributeImpl (as you would normally) but don't assume the attribute is already added just
|
||||
* because you know the term vector has payloads, since the first call to incrementToken() will observe if you asked
|
||||
* for them and if not then won't get them. This TokenStream supports an efficient {@link #reset()}, so there's
|
||||
* no need to wrap with a caching impl.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class TokenStreamFromTermVector extends TokenStream {
|
||||
// note: differs from similar class in the standard highlighter. This one is optimized for sparse cases.
|
||||
|
||||
/**
|
||||
* content length divided by distinct positions; an average of dense text.
|
||||
*/
|
||||
private static final double AVG_CHARS_PER_POSITION = 6;
|
||||
|
||||
private static final int INSERTION_SORT_THRESHOLD = 16;
|
||||
|
||||
private final Terms vector;
|
||||
|
||||
private final int filteredDocId;
|
||||
|
||||
private final CharTermAttribute termAttribute;
|
||||
|
||||
private final PositionIncrementAttribute positionIncrementAttribute;
|
||||
|
||||
private final int offsetLength;
|
||||
|
||||
private final float loadFactor;
|
||||
|
||||
private OffsetAttribute offsetAttribute;//maybe null
|
||||
|
||||
private PayloadAttribute payloadAttribute;//maybe null
|
||||
|
||||
private CharsRefBuilder termCharsBuilder;//term data here
|
||||
|
||||
private BytesRefArray payloadsBytesRefArray;//only used when payloadAttribute is non-null
|
||||
private BytesRefBuilder spareBytesRefBuilder;//only used when payloadAttribute is non-null
|
||||
|
||||
private TokenLL firstToken = null; // the head of a linked-list
|
||||
|
||||
private TokenLL incrementToken = null;
|
||||
|
||||
private boolean initialized = false;//lazy
|
||||
|
||||
public TokenStreamFromTermVector(Terms vector, int offsetLength) throws IOException {
|
||||
this(vector, 0, offsetLength, 1f);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
*
|
||||
* @param vector Terms that contains the data for
|
||||
* creating the TokenStream. Must have positions and/or offsets.
|
||||
* @param filteredDocId The docID we will process.
|
||||
* @param offsetLength Supply the character length of the text being uninverted, or a lower value if you don't want
|
||||
* to invert text beyond an offset (in so doing this will act as a filter). If you don't
|
||||
* know the length, pass -1. In conjunction with {@code loadFactor}, it's used to
|
||||
* determine how many buckets to create during uninversion.
|
||||
* It's also used to filter out tokens with a start offset exceeding this value.
|
||||
* @param loadFactor The percent of tokens from the original terms (by position count) that are
|
||||
* expected to be inverted. If they are filtered (e.g.
|
||||
* {@link org.apache.lucene.index.FilterLeafReader.FilterTerms})
|
||||
* then consider using less than 1.0 to avoid wasting space.
|
||||
* 1.0 means all, 1/64th would suggest 1/64th of all tokens coming from vector.
|
||||
*/
|
||||
TokenStreamFromTermVector(Terms vector, int filteredDocId, int offsetLength, float loadFactor) throws IOException {
|
||||
super();
|
||||
this.filteredDocId = filteredDocId;
|
||||
this.offsetLength = offsetLength == Integer.MAX_VALUE ? -1 : offsetLength;
|
||||
if (loadFactor <= 0f || loadFactor > 1f) {
|
||||
throw new IllegalArgumentException("loadFactor should be > 0 and <= 1");
|
||||
}
|
||||
this.loadFactor = loadFactor;
|
||||
assert !hasAttribute(PayloadAttribute.class) : "AttributeFactory shouldn't have payloads *yet*";
|
||||
if (!vector.hasPositions() && !vector.hasOffsets()) {
|
||||
throw new IllegalArgumentException("The term vector needs positions and/or offsets.");
|
||||
}
|
||||
assert vector.hasFreqs();
|
||||
this.vector = vector;
|
||||
termAttribute = addAttribute(CharTermAttribute.class);
|
||||
positionIncrementAttribute = addAttribute(PositionIncrementAttribute.class);
|
||||
}
|
||||
|
||||
public Terms getTermVectorTerms() {
|
||||
return vector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
incrementToken = null;
|
||||
super.reset();
|
||||
}
|
||||
|
||||
//We delay initialization because we can see which attributes the consumer wants, particularly payloads
|
||||
private void init() throws IOException {
|
||||
assert !initialized;
|
||||
int dpEnumFlags = 0;
|
||||
if (vector.hasOffsets()) {
|
||||
offsetAttribute = addAttribute(OffsetAttribute.class);
|
||||
dpEnumFlags |= PostingsEnum.OFFSETS;
|
||||
}
|
||||
if (vector.hasPayloads() && hasAttribute(PayloadAttribute.class)) {
|
||||
payloadAttribute = getAttribute(PayloadAttribute.class);
|
||||
payloadsBytesRefArray = new BytesRefArray(Counter.newCounter());
|
||||
spareBytesRefBuilder = new BytesRefBuilder();
|
||||
dpEnumFlags |= PostingsEnum.PAYLOADS;
|
||||
}
|
||||
|
||||
// We put term data here
|
||||
termCharsBuilder = new CharsRefBuilder();
|
||||
termCharsBuilder.grow(initTotalTermCharLen());
|
||||
|
||||
// Step 1: iterate termsEnum and create a token, placing into a bucketed array (given a load factor)
|
||||
|
||||
final TokenLL[] tokenBuckets = initTokenBucketsArray();
|
||||
final double OFFSET_TO_BUCKET_IDX = loadFactor / AVG_CHARS_PER_POSITION;
|
||||
final double POSITION_TO_BUCKET_IDX = loadFactor;
|
||||
|
||||
final TermsEnum termsEnum = vector.iterator();
|
||||
BytesRef termBytesRef;
|
||||
PostingsEnum dpEnum = null;
|
||||
final CharsRefBuilder tempCharsRefBuilder = new CharsRefBuilder();//only for UTF8->UTF16 call
|
||||
|
||||
TERM_LOOP:
|
||||
while ((termBytesRef = termsEnum.next()) != null) {
|
||||
//Grab the term (in same way as BytesRef.utf8ToString() but we don't want a String obj)
|
||||
// note: if term vectors supported seek by ord then we might just keep an int and seek by ord on-demand
|
||||
tempCharsRefBuilder.grow(termBytesRef.length);
|
||||
final int termCharsLen = UnicodeUtil.UTF8toUTF16(termBytesRef, tempCharsRefBuilder.chars());
|
||||
final int termCharsOff = termCharsBuilder.length();
|
||||
termCharsBuilder.append(tempCharsRefBuilder.chars(), 0, termCharsLen);
|
||||
dpEnum = termsEnum.postings(dpEnum, dpEnumFlags);
|
||||
assert dpEnum != null; // presumably checked by TokenSources.hasPositions earlier
|
||||
int currentDocId = dpEnum.advance(filteredDocId);
|
||||
if (currentDocId != filteredDocId) {
|
||||
continue; //Not expected
|
||||
}
|
||||
final int freq = dpEnum.freq();
|
||||
for (int j = 0; j < freq; j++) {
|
||||
TokenLL token = new TokenLL();
|
||||
token.position = dpEnum.nextPosition(); // can be -1 if not in the TV
|
||||
token.termCharsOff = termCharsOff;
|
||||
token.termCharsLen = (short) Math.min(termCharsLen, Short.MAX_VALUE);
|
||||
// copy offset (if it's there) and compute bucketIdx
|
||||
int bucketIdx;
|
||||
if (offsetAttribute != null) {
|
||||
token.startOffset = dpEnum.startOffset();
|
||||
if (offsetLength >= 0 && token.startOffset > offsetLength) {
|
||||
continue TERM_LOOP;//filter this token out; exceeds threshold
|
||||
}
|
||||
token.endOffsetInc = (short) Math.min(dpEnum.endOffset() - token.startOffset, Short.MAX_VALUE);
|
||||
bucketIdx = (int) (token.startOffset * OFFSET_TO_BUCKET_IDX);
|
||||
} else {
|
||||
bucketIdx = (int) (token.position * POSITION_TO_BUCKET_IDX);
|
||||
}
|
||||
if (bucketIdx >= tokenBuckets.length) {
|
||||
bucketIdx = tokenBuckets.length - 1;
|
||||
}
|
||||
|
||||
if (payloadAttribute != null) {
|
||||
final BytesRef payload = dpEnum.getPayload();
|
||||
token.payloadIndex = payload == null ? -1 : payloadsBytesRefArray.append(payload);
|
||||
}
|
||||
|
||||
//Add token to the head of the bucket linked list
|
||||
token.next = tokenBuckets[bucketIdx];
|
||||
tokenBuckets[bucketIdx] = token;
|
||||
}
|
||||
}
|
||||
|
||||
// Step 2: Link all Tokens into a linked-list and sort all tokens at the same position
|
||||
|
||||
firstToken = initLinkAndSortTokens(tokenBuckets);
|
||||
|
||||
// If the term vector didn't have positions, synthesize them
|
||||
if (!vector.hasPositions() && firstToken != null) {
|
||||
TokenLL prevToken = firstToken;
|
||||
prevToken.position = 0;
|
||||
for (TokenLL token = prevToken.next; token != null; prevToken = token, token = token.next) {
|
||||
if (prevToken.startOffset == token.startOffset) {
|
||||
token.position = prevToken.position;
|
||||
} else {
|
||||
token.position = prevToken.position + 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
private static TokenLL initLinkAndSortTokens(TokenLL[] tokenBuckets) {
|
||||
TokenLL firstToken = null;
|
||||
List<TokenLL> scratchTokenArray = new ArrayList<>(); // declare here for re-use. TODO use native array
|
||||
TokenLL prevToken = null;
|
||||
for (TokenLL tokenHead : tokenBuckets) {
|
||||
if (tokenHead == null) {
|
||||
continue;
|
||||
}
|
||||
//sort tokens at this position and link them; return the first
|
||||
TokenLL tokenTail;
|
||||
// just one token
|
||||
if (tokenHead.next == null) {
|
||||
tokenTail = tokenHead;
|
||||
} else {
|
||||
// add the linked list to a temporary array
|
||||
for (TokenLL cur = tokenHead; cur != null; cur = cur.next) {
|
||||
scratchTokenArray.add(cur);
|
||||
}
|
||||
// sort; and set tokenHead & tokenTail
|
||||
if (scratchTokenArray.size() < INSERTION_SORT_THRESHOLD) {
|
||||
// insertion sort by creating a linked list (leave scratchTokenArray alone)
|
||||
tokenHead = tokenTail = scratchTokenArray.get(0);
|
||||
tokenHead.next = null;
|
||||
for (int i = 1; i < scratchTokenArray.size(); i++) {
|
||||
TokenLL insertToken = scratchTokenArray.get(i);
|
||||
if (insertToken.compareTo(tokenHead) <= 0) {
|
||||
// takes the place of tokenHead
|
||||
insertToken.next = tokenHead;
|
||||
tokenHead = insertToken;
|
||||
} else {
|
||||
// goes somewhere after tokenHead
|
||||
for (TokenLL prev = tokenHead; true; prev = prev.next) {
|
||||
if (prev.next == null || insertToken.compareTo(prev.next) <= 0) {
|
||||
if (prev.next == null) {
|
||||
tokenTail = insertToken;
|
||||
}
|
||||
insertToken.next = prev.next;
|
||||
prev.next = insertToken;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Collections.sort(scratchTokenArray);
|
||||
// take back out and create a linked list
|
||||
TokenLL prev = tokenHead = scratchTokenArray.get(0);
|
||||
for (int i = 1; i < scratchTokenArray.size(); i++) {
|
||||
prev.next = scratchTokenArray.get(i);
|
||||
prev = prev.next;
|
||||
}
|
||||
tokenTail = prev;
|
||||
tokenTail.next = null;
|
||||
}
|
||||
scratchTokenArray.clear();//too bad ArrayList nulls it out; we don't actually need that
|
||||
}
|
||||
|
||||
//link to previous
|
||||
if (prevToken != null) {
|
||||
assert prevToken.next == null;
|
||||
prevToken.next = tokenHead; //concatenate linked-list
|
||||
assert prevToken.compareTo(tokenHead) < 0 : "wrong offset / position ordering expectations";
|
||||
} else {
|
||||
assert firstToken == null;
|
||||
firstToken = tokenHead;
|
||||
}
|
||||
|
||||
prevToken = tokenTail;
|
||||
}
|
||||
return firstToken;
|
||||
}
|
||||
|
||||
private int initTotalTermCharLen() throws IOException {
|
||||
int guessNumTerms;
|
||||
if (vector.size() != -1) {
|
||||
guessNumTerms = (int) vector.size();
|
||||
} else if (offsetLength != -1) {
|
||||
guessNumTerms = (int) (offsetLength * 0.33);//guess 1/3rd
|
||||
} else {
|
||||
return 128;
|
||||
}
|
||||
return Math.max(64, (int) (guessNumTerms * loadFactor * 7.0));//7 is over-estimate of average term len
|
||||
}
|
||||
|
||||
private TokenLL[] initTokenBucketsArray() throws IOException {
|
||||
// Estimate the number of non-empty positions (number of tokens, excluding same-position synonyms).
|
||||
int positionsEstimate;
|
||||
if (offsetLength == -1) { // no clue what the char length is.
|
||||
// Estimate the number of position slots we need from term stats based on Wikipedia.
|
||||
int sumTotalTermFreq = (int) vector.getSumTotalTermFreq();
|
||||
if (sumTotalTermFreq == -1) {//unfortunately term vectors seem to not have this stat
|
||||
int size = (int) vector.size();
|
||||
if (size == -1) {//doesn't happen with term vectors, it seems, but pick a default any way
|
||||
size = 128;
|
||||
}
|
||||
sumTotalTermFreq = (int) (size * 2.4);
|
||||
}
|
||||
positionsEstimate = (int) (sumTotalTermFreq * 1.5);//less than 1 in 10 docs exceed this
|
||||
} else {
|
||||
// guess number of token positions by this factor.
|
||||
positionsEstimate = (int) (offsetLength / AVG_CHARS_PER_POSITION);
|
||||
}
|
||||
// apply the load factor.
|
||||
return new TokenLL[Math.max(1, (int) (positionsEstimate * loadFactor))];
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
int posInc;
|
||||
if (incrementToken == null) {
|
||||
if (!initialized) {
|
||||
init();
|
||||
assert initialized;
|
||||
}
|
||||
incrementToken = firstToken;
|
||||
if (incrementToken == null) {
|
||||
return false;
|
||||
}
|
||||
posInc = incrementToken.position + 1;//first token normally has pos 0; add 1 to get posInc
|
||||
} else if (incrementToken.next != null) {
|
||||
int lastPosition = incrementToken.position;
|
||||
incrementToken = incrementToken.next;
|
||||
posInc = incrementToken.position - lastPosition;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
clearAttributes();
|
||||
termAttribute.copyBuffer(termCharsBuilder.chars(), incrementToken.termCharsOff, incrementToken.termCharsLen);
|
||||
|
||||
positionIncrementAttribute.setPositionIncrement(posInc);
|
||||
if (offsetAttribute != null) {
|
||||
offsetAttribute.setOffset(incrementToken.startOffset, incrementToken.startOffset + incrementToken.endOffsetInc);
|
||||
}
|
||||
if (payloadAttribute != null && incrementToken.payloadIndex >= 0) {
|
||||
payloadAttribute.setPayload(payloadsBytesRefArray.get(spareBytesRefBuilder, incrementToken.payloadIndex));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private static class TokenLL implements Comparable<TokenLL> {
|
||||
// This class should weigh 32 bytes, including object header
|
||||
|
||||
int termCharsOff; // see termCharsBuilder
|
||||
short termCharsLen;
|
||||
|
||||
int position;
|
||||
int startOffset;
|
||||
short endOffsetInc; // add to startOffset to get endOffset
|
||||
int payloadIndex;
|
||||
|
||||
TokenLL next;
|
||||
|
||||
@Override
|
||||
public int compareTo(TokenLL tokenB) {
|
||||
int cmp = Integer.compare(this.position, tokenB.position);
|
||||
if (cmp == 0) {
|
||||
cmp = Integer.compare(this.startOffset, tokenB.startOffset);
|
||||
if (cmp == 0) {
|
||||
cmp = Short.compare(this.endOffsetInc, tokenB.endOffsetInc);
|
||||
}
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,173 @@
|
|||
/*
|
||||
* 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.search.uhighlight;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
|
||||
|
||||
/**
|
||||
* Analyzes the text, producing a single {@link OffsetsEnum} wrapping the {@link TokenStream} filtered to terms
|
||||
* in the query, including wildcards. It can't handle position-sensitive queries (phrases). Passage accuracy suffers
|
||||
* because the freq() is unknown -- it's always {@link Integer#MAX_VALUE} instead.
|
||||
*/
|
||||
public class TokenStreamOffsetStrategy extends AnalysisOffsetStrategy {
|
||||
|
||||
private static final BytesRef[] ZERO_LEN_BYTES_REF_ARRAY = new BytesRef[0];
|
||||
|
||||
public TokenStreamOffsetStrategy(String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Analyzer indexAnalyzer) {
|
||||
super(field, ZERO_LEN_BYTES_REF_ARRAY, phraseHelper, convertTermsToAutomata(terms, automata), indexAnalyzer);
|
||||
assert phraseHelper.hasPositionSensitivity() == false;
|
||||
}
|
||||
|
||||
private static CharacterRunAutomaton[] convertTermsToAutomata(BytesRef[] terms, CharacterRunAutomaton[] automata) {
|
||||
CharacterRunAutomaton[] newAutomata = new CharacterRunAutomaton[terms.length + automata.length];
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
String termString = terms[i].utf8ToString();
|
||||
newAutomata[i] = new CharacterRunAutomaton(Automata.makeString(termString)) {
|
||||
@Override
|
||||
public String toString() {
|
||||
return termString;
|
||||
}
|
||||
};
|
||||
}
|
||||
// Append existing automata (that which is used for MTQs)
|
||||
System.arraycopy(automata, 0, newAutomata, terms.length, automata.length);
|
||||
return newAutomata;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
|
||||
TokenStream tokenStream = tokenStream(content);
|
||||
PostingsEnum mtqPostingsEnum = new TokenStreamPostingsEnum(tokenStream, automata);
|
||||
mtqPostingsEnum.advance(docId);
|
||||
return Collections.singletonList(new OffsetsEnum(null, mtqPostingsEnum));
|
||||
}
|
||||
|
||||
// but this would have a performance cost for likely little gain in the user experience, it
|
||||
// would only serve to make this method less bogus.
|
||||
// instead, we always return freq() = Integer.MAX_VALUE and let the highlighter terminate based on offset...
|
||||
// TODO: DWS perhaps instead OffsetsEnum could become abstract and this would be an impl?
|
||||
private static class TokenStreamPostingsEnum extends PostingsEnum implements Closeable {
|
||||
TokenStream stream; // becomes null when closed
|
||||
final CharacterRunAutomaton[] matchers;
|
||||
final CharTermAttribute charTermAtt;
|
||||
final OffsetAttribute offsetAtt;
|
||||
|
||||
int currentDoc = -1;
|
||||
int currentMatch = -1;
|
||||
int currentStartOffset = -1;
|
||||
|
||||
int currentEndOffset = -1;
|
||||
|
||||
final BytesRef matchDescriptions[];
|
||||
|
||||
TokenStreamPostingsEnum(TokenStream ts, CharacterRunAutomaton[] matchers) throws IOException {
|
||||
this.stream = ts;
|
||||
this.matchers = matchers;
|
||||
matchDescriptions = new BytesRef[matchers.length];
|
||||
charTermAtt = ts.addAttribute(CharTermAttribute.class);
|
||||
offsetAtt = ts.addAttribute(OffsetAttribute.class);
|
||||
ts.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextPosition() throws IOException {
|
||||
if (stream != null) {
|
||||
while (stream.incrementToken()) {
|
||||
for (int i = 0; i < matchers.length; i++) {
|
||||
if (matchers[i].run(charTermAtt.buffer(), 0, charTermAtt.length())) {
|
||||
currentStartOffset = offsetAtt.startOffset();
|
||||
currentEndOffset = offsetAtt.endOffset();
|
||||
currentMatch = i;
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
stream.end();
|
||||
close();
|
||||
}
|
||||
// exhausted
|
||||
currentStartOffset = currentEndOffset = Integer.MAX_VALUE;
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
return Integer.MAX_VALUE; // lie
|
||||
}
|
||||
|
||||
@Override
|
||||
public int startOffset() throws IOException {
|
||||
assert currentStartOffset >= 0;
|
||||
return currentStartOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int endOffset() throws IOException {
|
||||
assert currentEndOffset >= 0;
|
||||
return currentEndOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() throws IOException {
|
||||
if (matchDescriptions[currentMatch] == null) {
|
||||
matchDescriptions[currentMatch] = new BytesRef(matchers[currentMatch].toString());
|
||||
}
|
||||
return matchDescriptions[currentMatch];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return currentDoc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return currentDoc = target;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (stream != null) {
|
||||
stream.close();
|
||||
stream = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -117,6 +117,8 @@ public class UnifiedHighlighter {
|
|||
|
||||
private boolean defaultHighlightPhrasesStrictly = true; // AKA "accuracy" or "query debugging"
|
||||
|
||||
private boolean defaultPassageRelevancyOverSpeed = true; //For analysis, prefer MemoryIndexOffsetStrategy
|
||||
|
||||
// private boolean defaultRequireFieldMatch = true; TODO
|
||||
|
||||
private int maxLength = DEFAULT_MAX_LENGTH;
|
||||
|
@ -213,6 +215,12 @@ public class UnifiedHighlighter {
|
|||
return defaultHighlightPhrasesStrictly;
|
||||
}
|
||||
|
||||
|
||||
protected boolean shouldPreferPassageRelevancyOverSpeed(String field) {
|
||||
return defaultPassageRelevancyOverSpeed;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* The maximum content size to process. Content will be truncated to this size before highlighting. Typically
|
||||
* snippets closer to the beginning of the document better summarize its content.
|
||||
|
@ -716,8 +724,13 @@ public class UnifiedHighlighter {
|
|||
}
|
||||
|
||||
protected FieldHighlighter getFieldHighlighter(String field, Query query, SortedSet<Term> allTerms, int maxPassages) {
|
||||
BytesRef[] terms = filterExtractedTerms(field, allTerms);
|
||||
Set<HighlightFlag> highlightFlags = getFlags(field);
|
||||
PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
|
||||
CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
|
||||
OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
|
||||
return new FieldHighlighter(field,
|
||||
getOffsetStrategy(field, query, allTerms),
|
||||
getOffsetStrategy(offsetSource, field, terms, phraseHelper, automata, highlightFlags),
|
||||
new SplittingBreakIterator(getBreakIterator(field), UnifiedHighlighter.MULTIVAL_SEP_CHAR),
|
||||
getScorer(field),
|
||||
maxPassages,
|
||||
|
@ -725,41 +738,7 @@ public class UnifiedHighlighter {
|
|||
getFormatter(field));
|
||||
}
|
||||
|
||||
protected FieldOffsetStrategy getOffsetStrategy(String field, Query query, SortedSet<Term> allTerms) {
|
||||
EnumSet<HighlightFlag> highlightFlags = getFlags(field);
|
||||
BytesRef[] terms = filterExtractedTerms(field, allTerms);
|
||||
PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
|
||||
CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
|
||||
OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
|
||||
switch (offsetSource) {
|
||||
case ANALYSIS:
|
||||
return new AnalysisOffsetStrategy(field, terms, phraseHelper, automata, getIndexAnalyzer(),
|
||||
this::preMultiTermQueryRewrite);
|
||||
case NONE_NEEDED:
|
||||
return NoOpOffsetStrategy.INSTANCE;
|
||||
case TERM_VECTORS:
|
||||
return new TermVectorOffsetStrategy(field, terms, phraseHelper, automata);
|
||||
case POSTINGS:
|
||||
return new PostingsOffsetStrategy(field, terms, phraseHelper, automata);
|
||||
case POSTINGS_WITH_TERM_VECTORS:
|
||||
return new PostingsWithTermVectorsOffsetStrategy(field, terms, phraseHelper, automata);
|
||||
default:
|
||||
throw new IllegalArgumentException("Unrecognized offset source " + offsetSource);
|
||||
}
|
||||
}
|
||||
|
||||
protected EnumSet<HighlightFlag> getFlags(String field) {
|
||||
EnumSet<HighlightFlag> highlightFlags = EnumSet.noneOf(HighlightFlag.class);
|
||||
if (shouldHandleMultiTermQuery(field)) {
|
||||
highlightFlags.add(HighlightFlag.MULTI_TERM_QUERY);
|
||||
}
|
||||
if (shouldHighlightPhrasesStrictly(field)) {
|
||||
highlightFlags.add(HighlightFlag.PHRASES);
|
||||
}
|
||||
return highlightFlags;
|
||||
}
|
||||
|
||||
protected BytesRef[] filterExtractedTerms(String field, SortedSet<Term> queryTerms) {
|
||||
protected static BytesRef[] filterExtractedTerms(String field, SortedSet<Term> queryTerms) {
|
||||
// TODO consider requireFieldMatch
|
||||
Term floor = new Term(field, "");
|
||||
Term ceiling = new Term(field, UnicodeUtil.BIG_TERM);
|
||||
|
@ -774,7 +753,21 @@ public class UnifiedHighlighter {
|
|||
return terms;
|
||||
}
|
||||
|
||||
protected PhraseHelper getPhraseHelper(String field, Query query, EnumSet<HighlightFlag> highlightFlags) {
|
||||
protected Set<HighlightFlag> getFlags(String field) {
|
||||
Set<HighlightFlag> highlightFlags = EnumSet.noneOf(HighlightFlag.class);
|
||||
if (shouldHandleMultiTermQuery(field)) {
|
||||
highlightFlags.add(HighlightFlag.MULTI_TERM_QUERY);
|
||||
}
|
||||
if (shouldHighlightPhrasesStrictly(field)) {
|
||||
highlightFlags.add(HighlightFlag.PHRASES);
|
||||
}
|
||||
if (shouldPreferPassageRelevancyOverSpeed(field)) {
|
||||
highlightFlags.add(HighlightFlag.PASSAGE_RELEVANCY_OVER_SPEED);
|
||||
}
|
||||
return highlightFlags;
|
||||
}
|
||||
|
||||
protected PhraseHelper getPhraseHelper(String field, Query query, Set<HighlightFlag> highlightFlags) {
|
||||
boolean highlightPhrasesStrictly = highlightFlags.contains(HighlightFlag.PHRASES);
|
||||
boolean handleMultiTermQuery = highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY);
|
||||
return highlightPhrasesStrictly ?
|
||||
|
@ -782,7 +775,7 @@ public class UnifiedHighlighter {
|
|||
PhraseHelper.NONE;
|
||||
}
|
||||
|
||||
protected CharacterRunAutomaton[] getAutomata(String field, Query query, EnumSet<HighlightFlag> highlightFlags) {
|
||||
protected CharacterRunAutomaton[] getAutomata(String field, Query query, Set<HighlightFlag> highlightFlags) {
|
||||
return highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY)
|
||||
? MultiTermHighlighting.extractAutomata(query, field, !highlightFlags.contains(HighlightFlag.PHRASES),
|
||||
this::preMultiTermQueryRewrite)
|
||||
|
@ -790,11 +783,12 @@ public class UnifiedHighlighter {
|
|||
}
|
||||
|
||||
protected OffsetSource getOptimizedOffsetSource(String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
|
||||
OffsetSource offsetSource = getOffsetSource(field);
|
||||
|
||||
if (terms.length == 0 && automata.length == 0 && !phraseHelper.willRewrite()) {
|
||||
return OffsetSource.NONE_NEEDED; //nothing to highlight
|
||||
}
|
||||
|
||||
OffsetSource offsetSource = getOffsetSource(field);
|
||||
switch (offsetSource) {
|
||||
case POSTINGS:
|
||||
if (phraseHelper.willRewrite()) {
|
||||
|
@ -822,6 +816,32 @@ public class UnifiedHighlighter {
|
|||
return offsetSource;
|
||||
}
|
||||
|
||||
protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, String field, BytesRef[] terms,
|
||||
PhraseHelper phraseHelper, CharacterRunAutomaton[] automata,
|
||||
Set<HighlightFlag> highlightFlags) {
|
||||
switch (offsetSource) {
|
||||
case ANALYSIS:
|
||||
if (!phraseHelper.hasPositionSensitivity() &&
|
||||
!highlightFlags.contains(HighlightFlag.PASSAGE_RELEVANCY_OVER_SPEED)) {
|
||||
//skip using a memory index since it's pure term filtering
|
||||
return new TokenStreamOffsetStrategy(field, terms, phraseHelper, automata, getIndexAnalyzer());
|
||||
} else {
|
||||
return new MemoryIndexOffsetStrategy(field, terms, phraseHelper, automata, getIndexAnalyzer(),
|
||||
this::preMultiTermQueryRewrite);
|
||||
}
|
||||
case NONE_NEEDED:
|
||||
return NoOpOffsetStrategy.INSTANCE;
|
||||
case TERM_VECTORS:
|
||||
return new TermVectorOffsetStrategy(field, terms, phraseHelper, automata);
|
||||
case POSTINGS:
|
||||
return new PostingsOffsetStrategy(field, terms, phraseHelper, automata);
|
||||
case POSTINGS_WITH_TERM_VECTORS:
|
||||
return new PostingsWithTermVectorsOffsetStrategy(field, terms, phraseHelper, automata);
|
||||
default:
|
||||
throw new IllegalArgumentException("Unrecognized offset source " + offsetSource);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* When highlighting phrases accurately, we need to know which {@link SpanQuery}'s need to have
|
||||
* {@link Query#rewrite(IndexReader)} called on them. It helps performance to avoid it if it's not needed.
|
||||
|
@ -1041,10 +1061,9 @@ public class UnifiedHighlighter {
|
|||
*/
|
||||
public enum HighlightFlag {
|
||||
PHRASES,
|
||||
MULTI_TERM_QUERY
|
||||
MULTI_TERM_QUERY,
|
||||
PASSAGE_RELEVANCY_OVER_SPEED
|
||||
// TODO: ignoreQueryFields
|
||||
// TODO: useQueryBoosts
|
||||
// TODO: avoidMemoryIndexIfPossible
|
||||
// TODO: preferMemoryIndexForStats
|
||||
}
|
||||
}
|
||||
|
|
|
@ -773,7 +773,40 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
|
|||
ir.close();
|
||||
}
|
||||
|
||||
public void testTokenStreamIsClosed() throws IOException {
|
||||
public void testWithMaxLenAndMultipleWildcardMatches() throws IOException {
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
Field body = new Field("body", "", fieldType);
|
||||
Document doc = new Document();
|
||||
doc.add(body);
|
||||
|
||||
//tests interleaving of multiple wildcard matches with the CompositePostingsEnum
|
||||
//In this case the CompositePostingsEnum will have an underlying PostingsEnum that jumps form pos 1 to 9 for bravo
|
||||
//and a second with position 2 for Bravado
|
||||
body.setStringValue("Alpha Bravo Bravado foo foo foo. Foo foo Alpha Bravo");
|
||||
iw.addDocument(doc);
|
||||
|
||||
IndexReader ir = iw.getReader();
|
||||
iw.close();
|
||||
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
|
||||
highlighter.setMaxLength(32);//a little past first sentence
|
||||
|
||||
BooleanQuery query = new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("body", "alpha")), BooleanClause.Occur.MUST)
|
||||
.add(new PrefixQuery(new Term("body", "bra")), BooleanClause.Occur.MUST)
|
||||
.build();
|
||||
TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
|
||||
String snippets[] = highlighter.highlight("body", query, topDocs, 2);//ask for 2 but we'll only get 1
|
||||
assertArrayEquals(
|
||||
new String[]{"<b>Alpha</b> <b>Bravo</b> <b>Bravado</b> foo foo foo."}, snippets
|
||||
);
|
||||
|
||||
ir.close();
|
||||
}
|
||||
|
||||
public void testTokenStreamIsClosed() throws Exception {
|
||||
// note: test is a derivative of testWithMaxLen()
|
||||
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
|
||||
|
||||
|
@ -828,8 +861,8 @@ public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
|
|||
if (fieldType == UHTestHelper.reanalysisType) {
|
||||
fail("Expecting EXPECTED IOException");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
if (!e.getMessage().equals("EXPECTED")) {
|
||||
} catch (Exception e) {
|
||||
if (!e.getMessage().contains("EXPECTED")) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -50,9 +50,8 @@ public class TestUnifiedHighlighterRanking extends LuceneTestCase {
|
|||
|
||||
Analyzer indexAnalyzer;
|
||||
|
||||
// note: don't choose reanalysis because it doesn't always know the term frequency, which is a statistic used
|
||||
// in passage ranking. Sometimes it does (e.g. when it builds a MemoryIndex) but not necessarily.
|
||||
final FieldType fieldType = UHTestHelper.randomFieldType(random(), UHTestHelper.postingsType, UHTestHelper.tvType);
|
||||
// note: all offset sources, by default, use term freq, so it shouldn't matter which we choose.
|
||||
final FieldType fieldType = UHTestHelper.randomFieldType(random());
|
||||
|
||||
/**
|
||||
* indexes a bunch of gibberish, and then highlights top(n).
|
||||
|
|
|
@ -22,11 +22,13 @@ import java.text.BreakIterator;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -68,6 +70,11 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
|
|||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<OffsetsEnum> createOffsetsEnumsFromReader(LeafReader leafReader, int doc) throws IOException {
|
||||
return super.createOffsetsEnumsFromReader(leafReader, doc);
|
||||
}
|
||||
|
||||
};
|
||||
assertEquals(offsetSource, strategy.getOffsetSource());
|
||||
}
|
||||
|
@ -142,8 +149,8 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected FieldOffsetStrategy getOffsetStrategy(String field, Query query, SortedSet<Term> allTerms) {
|
||||
return super.getOffsetStrategy(field, query, allTerms);
|
||||
protected FieldOffsetStrategy getOffsetStrategy(OffsetSource offsetSource, String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata, Set<HighlightFlag> highlightFlags) {
|
||||
return super.getOffsetStrategy(offsetSource, field, terms, phraseHelper, automata, highlightFlags);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -28,6 +28,8 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedNumericSortField;
|
||||
import org.apache.lucene.search.SortedSetSortField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
|
@ -167,6 +169,78 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
|
|||
return true;
|
||||
}
|
||||
|
||||
private SortField randomIndexSortField() {
|
||||
boolean reversed = random().nextBoolean();
|
||||
SortField sortField;
|
||||
switch(random().nextInt(10)) {
|
||||
case 0:
|
||||
sortField = new SortField(TestUtil.randomSimpleString(random()), SortField.Type.INT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextInt());
|
||||
}
|
||||
break;
|
||||
case 1:
|
||||
sortField = new SortedNumericSortField(TestUtil.randomSimpleString(random()), SortField.Type.INT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextInt());
|
||||
}
|
||||
break;
|
||||
|
||||
case 2:
|
||||
sortField = new SortField(TestUtil.randomSimpleString(random()), SortField.Type.LONG, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextLong());
|
||||
}
|
||||
break;
|
||||
case 3:
|
||||
sortField = new SortedNumericSortField(TestUtil.randomSimpleString(random()), SortField.Type.LONG, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextLong());
|
||||
}
|
||||
break;
|
||||
case 4:
|
||||
sortField = new SortField(TestUtil.randomSimpleString(random()), SortField.Type.FLOAT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextFloat());
|
||||
}
|
||||
break;
|
||||
case 5:
|
||||
sortField = new SortedNumericSortField(TestUtil.randomSimpleString(random()), SortField.Type.FLOAT, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextFloat());
|
||||
}
|
||||
break;
|
||||
case 6:
|
||||
sortField = new SortField(TestUtil.randomSimpleString(random()), SortField.Type.DOUBLE, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextDouble());
|
||||
}
|
||||
break;
|
||||
case 7:
|
||||
sortField = new SortedNumericSortField(TestUtil.randomSimpleString(random()), SortField.Type.DOUBLE, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(random().nextDouble());
|
||||
}
|
||||
break;
|
||||
case 8:
|
||||
sortField = new SortField(TestUtil.randomSimpleString(random()), SortField.Type.STRING, reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(SortField.STRING_LAST);
|
||||
}
|
||||
break;
|
||||
case 9:
|
||||
sortField = new SortedSetSortField(TestUtil.randomSimpleString(random()), reversed);
|
||||
if (random().nextBoolean()) {
|
||||
sortField.setMissingValue(SortField.STRING_LAST);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
sortField = null;
|
||||
fail();
|
||||
}
|
||||
return sortField;
|
||||
}
|
||||
|
||||
/** Test sort */
|
||||
public void testSort() throws IOException {
|
||||
assumeTrue("test requires a codec that can read/write index sort", supportsIndexSort());
|
||||
|
@ -180,22 +254,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
|
|||
final int numSortFields = TestUtil.nextInt(random(), 1, 3);
|
||||
SortField[] sortFields = new SortField[numSortFields];
|
||||
for (int j = 0; j < numSortFields; ++j) {
|
||||
sortFields[j] = new SortField(
|
||||
TestUtil.randomSimpleString(random()),
|
||||
random().nextBoolean() ? SortField.Type.LONG : SortField.Type.STRING,
|
||||
random().nextBoolean());
|
||||
if (random().nextBoolean()) {
|
||||
switch (sortFields[j].getType()) {
|
||||
case LONG:
|
||||
sortFields[j].setMissingValue(random().nextLong());
|
||||
break;
|
||||
case STRING:
|
||||
sortFields[j].setMissingValue(random().nextBoolean() ? SortField.STRING_FIRST : SortField.STRING_LAST);
|
||||
break;
|
||||
default:
|
||||
fail();
|
||||
}
|
||||
}
|
||||
sortFields[j] = randomIndexSortField();
|
||||
}
|
||||
sort = new Sort(sortFields);
|
||||
}
|
||||
|
|
|
@ -116,6 +116,10 @@ New Features
|
|||
* SOLR-9633: Limit memory consumed by FastLRUCache with a new 'maxRamMB' config parameter.
|
||||
(yonik, Michael Sun, shalin)
|
||||
|
||||
* SOLR-9666: SolrJ LukeResponse support dynamic fields (Fengtan via Kevin Risden)
|
||||
|
||||
* SOLR-9077: Streaming expressions should support collection alias (Kevin Risden)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
* SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have
|
||||
|
@ -159,6 +163,8 @@ Bug Fixes
|
|||
* SOLR-9284: The HDFS BlockDirectoryCache should not let it's keysToRelease or names maps grow indefinitely.
|
||||
(Mark Miller, Michael Sun)
|
||||
|
||||
* SOLR-9729: JDBCStream improvements (Kevin Risden)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -115,7 +115,7 @@ public class BlockDirectoryTest extends SolrTestCaseJ4 {
|
|||
Metrics metrics = new Metrics();
|
||||
int blockSize = 8192;
|
||||
int slabSize = blockSize * 32768;
|
||||
long totalMemory = 2 * slabSize;
|
||||
long totalMemory = 1 * slabSize;
|
||||
BlockCache blockCache = new BlockCache(metrics, true, totalMemory, slabSize, blockSize);
|
||||
BlockDirectoryCache cache = new BlockDirectoryCache(blockCache, "/collection1", metrics, true);
|
||||
directory = new BlockDirectory("test", dir, cache, null, true, false);
|
||||
|
@ -267,7 +267,11 @@ public class BlockDirectoryTest extends SolrTestCaseJ4 {
|
|||
|
||||
BlockDirectory d = directory;
|
||||
assertTrue(d.useReadCache("", IOContext.DEFAULT));
|
||||
if (d.getCache() instanceof MapperCache) {
|
||||
assertTrue(d.useWriteCache("", IOContext.DEFAULT));
|
||||
} else {
|
||||
assertFalse(d.useWriteCache("", IOContext.DEFAULT));
|
||||
}
|
||||
assertFalse(d.useWriteCache("", mergeContext));
|
||||
|
||||
d = new BlockDirectory("test", directory, mapperCache, null, true, false);
|
||||
|
@ -277,7 +281,11 @@ public class BlockDirectoryTest extends SolrTestCaseJ4 {
|
|||
|
||||
d = new BlockDirectory("test", directory, mapperCache, null, false, true);
|
||||
assertFalse(d.useReadCache("", IOContext.DEFAULT));
|
||||
if (d.getCache() instanceof MapperCache) {
|
||||
assertTrue(d.useWriteCache("", IOContext.DEFAULT));
|
||||
} else {
|
||||
assertFalse(d.useWriteCache("", IOContext.DEFAULT));
|
||||
}
|
||||
assertFalse(d.useWriteCache("", mergeContext));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,8 +28,8 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.solr.client.solrj.io.stream.CloudSolrStream;
|
||||
import org.apache.solr.client.solrj.io.stream.SolrStream;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Slice;
|
||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||
|
@ -78,12 +78,7 @@ class StatementImpl implements Statement {
|
|||
protected SolrStream constructStream(String sql) throws IOException {
|
||||
try {
|
||||
ZkStateReader zkStateReader = this.connection.getClient().getZkStateReader();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.connection.getCollection());
|
||||
|
||||
if(slices == null) {
|
||||
throw new Exception("Collection not found:"+this.connection.getCollection());
|
||||
}
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(this.connection.getCollection(), zkStateReader, true);
|
||||
|
||||
List<Replica> shuffler = new ArrayList<>();
|
||||
for(Slice slice : slices) {
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
|
|||
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter;
|
||||
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
|
||||
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
|
||||
import org.apache.solr.common.cloud.Aliases;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
|
@ -60,6 +61,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
|
|||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.SolrjNamedThreadFactory;
|
||||
import org.apache.solr.common.util.StrUtils;
|
||||
|
||||
/**
|
||||
* Connects to Zookeeper to pick replicas from a specific collection to send the query to.
|
||||
|
@ -352,37 +354,57 @@ public class CloudSolrStream extends TupleStream implements Expressible {
|
|||
}
|
||||
}
|
||||
|
||||
public static Collection<Slice> getSlicesIgnoreCase(String name, ClusterState clusterState) {
|
||||
for (String coll : clusterState.getCollectionStates().keySet()) {
|
||||
if (coll.equalsIgnoreCase(name)) {
|
||||
DocCollection collection = clusterState.getCollectionOrNull(coll);
|
||||
if (collection != null) return collection.getActiveSlices();
|
||||
public static Collection<Slice> getSlices(String collectionName, ZkStateReader zkStateReader, boolean checkAlias) throws IOException {
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
|
||||
Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
|
||||
|
||||
// Check collection case sensitive
|
||||
if(collectionsMap.containsKey(collectionName)) {
|
||||
return collectionsMap.get(collectionName).getActiveSlices();
|
||||
}
|
||||
|
||||
// Check collection case insensitive
|
||||
for(String collectionMapKey : collectionsMap.keySet()) {
|
||||
if(collectionMapKey.equalsIgnoreCase(collectionName)) {
|
||||
return collectionsMap.get(collectionMapKey).getActiveSlices();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
||||
if(checkAlias) {
|
||||
// check for collection alias
|
||||
Aliases aliases = zkStateReader.getAliases();
|
||||
String alias = aliases.getCollectionAlias(collectionName);
|
||||
if (alias != null) {
|
||||
Collection<Slice> slices = new ArrayList<>();
|
||||
|
||||
List<String> aliasList = StrUtils.splitSmart(alias, ",", true);
|
||||
for (String aliasCollectionName : aliasList) {
|
||||
// Add all active slices for this alias collection
|
||||
slices.addAll(collectionsMap.get(aliasCollectionName).getActiveSlices());
|
||||
}
|
||||
|
||||
return slices;
|
||||
}
|
||||
}
|
||||
|
||||
throw new IOException("Slices not found for " + collectionName);
|
||||
}
|
||||
|
||||
protected void constructStreams() throws IOException {
|
||||
|
||||
try {
|
||||
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
//System.out.println("Connected to zk an got cluster state.");
|
||||
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
|
||||
if (slices == null) slices = getSlicesIgnoreCase(this.collection, clusterState);
|
||||
if (slices == null) {
|
||||
throw new Exception("Collection not found:" + this.collection);
|
||||
}
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, true);
|
||||
|
||||
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
|
||||
mParams.set("distrib", "false"); // We are the aggregator.
|
||||
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
for(Slice slice : slices) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
List<Replica> shuffler = new ArrayList();
|
||||
List<Replica> shuffler = new ArrayList<>();
|
||||
for(Replica replica : replicas) {
|
||||
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName()))
|
||||
shuffler.add(replica);
|
||||
|
|
|
@ -250,17 +250,15 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
|
|||
}
|
||||
|
||||
private List<String> getShardUrls() throws IOException {
|
||||
|
||||
try {
|
||||
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, false);
|
||||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
|
||||
List<String> baseUrls = new ArrayList<>();
|
||||
|
||||
for(Slice slice : slices) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
List<Replica> shuffler = new ArrayList<>();
|
||||
|
|
|
@ -67,7 +67,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
// These are java types that we can directly support as an Object instance. Other supported
|
||||
// types will require some level of conversion (short -> long, etc...)
|
||||
// We'll use a static constructor to load this set.
|
||||
private static HashSet<String> directSupportedTypes = new HashSet<String>();
|
||||
private static final HashSet<String> directSupportedTypes = new HashSet<>();
|
||||
static {
|
||||
directSupportedTypes.add(String.class.getName());
|
||||
directSupportedTypes.add(Double.class.getName());
|
||||
|
@ -107,7 +107,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
|
||||
// Validate there are no unknown parameters - zkHost and alias are namedParameter so we don't need to count it twice
|
||||
if(expression.getParameters().size() != namedParams.size()){
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - unknown operands found",expression));
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - unknown operands found", expression));
|
||||
}
|
||||
|
||||
// All named params we don't care about will be passed to the driver on connection
|
||||
|
@ -124,7 +124,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
connectionUrl = ((StreamExpressionValue)connectionUrlExpression.getParameter()).getValue();
|
||||
}
|
||||
if(null == connectionUrl){
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - connection not found"));
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - connection not found", connectionUrlExpression));
|
||||
}
|
||||
|
||||
// sql, required
|
||||
|
@ -133,16 +133,16 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
sqlQuery = ((StreamExpressionValue)sqlQueryExpression.getParameter()).getValue();
|
||||
}
|
||||
if(null == sqlQuery){
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - sql not found"));
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - sql not found", sqlQueryExpression));
|
||||
}
|
||||
|
||||
// definedSort, required
|
||||
StreamComparator definedSort = null;
|
||||
if(null != sqlQueryExpression && sqlQueryExpression.getParameter() instanceof StreamExpressionValue){
|
||||
if(null != definedSortExpression && definedSortExpression.getParameter() instanceof StreamExpressionValue){
|
||||
definedSort = factory.constructComparator(((StreamExpressionValue)definedSortExpression.getParameter()).getValue(), FieldComparator.class);
|
||||
}
|
||||
if(null == definedSort){
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - sort not found"));
|
||||
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - sort not found", definedSortExpression));
|
||||
}
|
||||
|
||||
// driverClass, optional
|
||||
|
@ -155,7 +155,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
init(connectionUrl, sqlQuery, definedSort, connectionProperties, driverClass);
|
||||
}
|
||||
|
||||
private void init(String connectionUrl, String sqlQuery, StreamComparator definedSort, Properties connectionProperties, String driverClassName) throws IOException {
|
||||
private void init(String connectionUrl, String sqlQuery, StreamComparator definedSort, Properties connectionProperties, String driverClassName) {
|
||||
this.connectionUrl = connectionUrl;
|
||||
this.sqlQuery = sqlQuery;
|
||||
this.definedSort = definedSort;
|
||||
|
@ -188,7 +188,9 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
throw new SQLException("DriverManager.getDriver(url) returned null");
|
||||
}
|
||||
} catch(SQLException e){
|
||||
throw new IOException(String.format(Locale.ROOT, "Failed to determine JDBC driver from connection url '%s'. Usually this means the driver is not loaded - you can have JDBCStream try to load it by providing the 'driverClassName' value", connectionUrl), e);
|
||||
throw new IOException(String.format(Locale.ROOT,
|
||||
"Failed to determine JDBC driver from connection url '%s'. Usually this means the driver is not loaded - " +
|
||||
"you can have JDBCStream try to load it by providing the 'driverClassName' value", connectionUrl), e);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -200,20 +202,23 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
try{
|
||||
statement = connection.createStatement();
|
||||
} catch (SQLException e) {
|
||||
throw new IOException(String.format(Locale.ROOT, "Failed to create a statement from JDBC connection '%s'", connectionUrl), e);
|
||||
throw new IOException(String.format(Locale.ROOT, "Failed to create a statement from JDBC connection '%s'",
|
||||
connectionUrl), e);
|
||||
}
|
||||
|
||||
try{
|
||||
resultSet = statement.executeQuery(sqlQuery);
|
||||
} catch (SQLException e) {
|
||||
throw new IOException(String.format(Locale.ROOT, "Failed to execute sqlQuery '%s' against JDBC connection '%s'.\n"+ e.getMessage(), sqlQuery, connectionUrl), e);
|
||||
throw new IOException(String.format(Locale.ROOT, "Failed to execute sqlQuery '%s' against JDBC connection '%s'.\n"
|
||||
+ e.getMessage(), sqlQuery, connectionUrl), e);
|
||||
}
|
||||
|
||||
try{
|
||||
// using the metadata, build selectors for each column
|
||||
valueSelectors = constructValueSelectors(resultSet.getMetaData());
|
||||
} catch (SQLException e) {
|
||||
throw new IOException(String.format(Locale.ROOT, "Failed to generate value selectors for sqlQuery '%s' against JDBC connection '%s'", sqlQuery, connectionUrl), e);
|
||||
throw new IOException(String.format(Locale.ROOT,
|
||||
"Failed to generate value selectors for sqlQuery '%s' against JDBC connection '%s'", sqlQuery, connectionUrl), e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -221,8 +226,8 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
ResultSetValueSelector[] valueSelectors = new ResultSetValueSelector[metadata.getColumnCount()];
|
||||
|
||||
for(int columnIdx = 0; columnIdx < metadata.getColumnCount(); ++columnIdx){
|
||||
|
||||
final int columnNumber = columnIdx + 1; // cause it starts at 1
|
||||
// Use getColumnLabel instead of getColumnName to make sure fields renamed with AS as picked up properly
|
||||
final String columnName = metadata.getColumnLabel(columnNumber);
|
||||
String className = metadata.getColumnClassName(columnNumber);
|
||||
String typeName = metadata.getColumnTypeName(columnNumber);
|
||||
|
@ -238,8 +243,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
return columnName;
|
||||
}
|
||||
};
|
||||
}
|
||||
else if(Short.class.getName().equals(className)) {
|
||||
} else if(Short.class.getName().equals(className)) {
|
||||
valueSelectors[columnIdx] = new ResultSetValueSelector() {
|
||||
public Object selectValue(ResultSet resultSet) throws SQLException {
|
||||
Short obj = resultSet.getShort(columnNumber);
|
||||
|
@ -250,8 +254,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
return columnName;
|
||||
}
|
||||
};
|
||||
}
|
||||
else if(Integer.class.getName().equals(className)) {
|
||||
} else if(Integer.class.getName().equals(className)) {
|
||||
valueSelectors[columnIdx] = new ResultSetValueSelector() {
|
||||
public Object selectValue(ResultSet resultSet) throws SQLException {
|
||||
Integer obj = resultSet.getInt(columnNumber);
|
||||
|
@ -262,8 +265,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
return columnName;
|
||||
}
|
||||
};
|
||||
}
|
||||
else if(Float.class.getName().equals(className)) {
|
||||
} else if(Float.class.getName().equals(className)) {
|
||||
valueSelectors[columnIdx] = new ResultSetValueSelector() {
|
||||
public Object selectValue(ResultSet resultSet) throws SQLException {
|
||||
Float obj = resultSet.getFloat(columnNumber);
|
||||
|
@ -274,9 +276,10 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
return columnName;
|
||||
}
|
||||
};
|
||||
}
|
||||
else{
|
||||
throw new SQLException(String.format(Locale.ROOT, "Unable to determine the valueSelector for column '%s' (col #%d) of java class '%s' and type '%s'", columnName, columnNumber, className, typeName));
|
||||
} else {
|
||||
throw new SQLException(String.format(Locale.ROOT,
|
||||
"Unable to determine the valueSelector for column '%s' (col #%d) of java class '%s' and type '%s'",
|
||||
columnName, columnNumber, className, typeName));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -305,7 +308,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
public Tuple read() throws IOException {
|
||||
|
||||
try{
|
||||
Map<Object,Object> fields = new HashMap<Object,Object>();
|
||||
Map<Object,Object> fields = new HashMap<>();
|
||||
if(resultSet.next()){
|
||||
// we have a record
|
||||
for(ResultSetValueSelector selector : valueSelectors){
|
||||
|
@ -391,7 +394,7 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
|
||||
@Override
|
||||
public List<TupleStream> children() {
|
||||
return new ArrayList<TupleStream>();
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -404,6 +407,6 @@ public class JDBCStream extends TupleStream implements Expressible {
|
|||
}
|
||||
|
||||
interface ResultSetValueSelector {
|
||||
public String getColumnName();
|
||||
public Object selectValue(ResultSet resultSet) throws SQLException;
|
||||
String getColumnName();
|
||||
Object selectValue(ResultSet resultSet) throws SQLException;
|
||||
}
|
|
@ -257,15 +257,17 @@ public class ParallelStream extends CloudSolrStream implements Expressible {
|
|||
}
|
||||
|
||||
protected void constructStreams() throws IOException {
|
||||
|
||||
try {
|
||||
Object pushStream = ((Expressible) tupleStream).toExpression(streamFactory);
|
||||
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, true);
|
||||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
|
||||
List<Replica> shuffler = new ArrayList();
|
||||
|
||||
List<Replica> shuffler = new ArrayList<>();
|
||||
for(Slice slice : slices) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
for (Replica replica : replicas) {
|
||||
|
|
|
@ -115,8 +115,6 @@ public class SolrStream extends TupleStream {
|
|||
**/
|
||||
|
||||
public void open() throws IOException {
|
||||
|
||||
|
||||
if(cache == null) {
|
||||
client = new HttpSolrClient.Builder(baseUrl).build();
|
||||
} else {
|
||||
|
|
|
@ -332,19 +332,18 @@ public class TextLogitStream extends TupleStream implements Expressible {
|
|||
}
|
||||
|
||||
protected List<String> getShardUrls() throws IOException {
|
||||
|
||||
try {
|
||||
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, false);
|
||||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
|
||||
List baseUrls = new ArrayList();
|
||||
|
||||
List<String> baseUrls = new ArrayList<>();
|
||||
for(Slice slice : slices) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
List<Replica> shuffler = new ArrayList();
|
||||
List<Replica> shuffler = new ArrayList<>();
|
||||
for(Replica replica : replicas) {
|
||||
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
|
||||
shuffler.add(replica);
|
||||
|
@ -359,7 +358,6 @@ public class TextLogitStream extends TupleStream implements Expressible {
|
|||
}
|
||||
|
||||
return baseUrls;
|
||||
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
@ -407,18 +406,21 @@ public class TopicStream extends CloudSolrStream implements Expressible {
|
|||
}
|
||||
|
||||
private void getCheckpoints() throws IOException {
|
||||
this.checkpoints = new HashMap();
|
||||
this.checkpoints = new HashMap<>();
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, false);
|
||||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(collection);
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
|
||||
for(Slice slice : slices) {
|
||||
String sliceName = slice.getName();
|
||||
long checkpoint = 0;
|
||||
long checkpoint;
|
||||
if(initialCheckpoint > -1) {
|
||||
checkpoint = initialCheckpoint;
|
||||
} else {
|
||||
checkpoint = getCheckpoint(slice, clusterState.getLiveNodes());
|
||||
checkpoint = getCheckpoint(slice, liveNodes);
|
||||
}
|
||||
|
||||
this.checkpoints.put(sliceName, checkpoint);
|
||||
|
@ -482,21 +484,19 @@ public class TopicStream extends CloudSolrStream implements Expressible {
|
|||
}
|
||||
|
||||
private void getPersistedCheckpoints() throws IOException {
|
||||
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(checkpointCollection, zkStateReader, false);
|
||||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(checkpointCollection);
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
|
||||
OUTER:
|
||||
for(Slice slice : slices) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
for(Replica replica : replicas) {
|
||||
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())){
|
||||
|
||||
|
||||
HttpSolrClient httpClient = streamContext.getSolrClientCache().getHttpSolrClient(replica.getCoreUrl());
|
||||
try {
|
||||
|
||||
SolrDocument doc = httpClient.getById(id);
|
||||
if(doc != null) {
|
||||
List<String> checkpoints = (List<String>)doc.getFieldValue("checkpoint_ss");
|
||||
|
@ -505,7 +505,7 @@ public class TopicStream extends CloudSolrStream implements Expressible {
|
|||
this.checkpoints.put(pair[0], Long.parseLong(pair[1]));
|
||||
}
|
||||
}
|
||||
}catch (Exception e) {
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
break OUTER;
|
||||
|
@ -515,22 +515,10 @@ public class TopicStream extends CloudSolrStream implements Expressible {
|
|||
}
|
||||
|
||||
protected void constructStreams() throws IOException {
|
||||
|
||||
try {
|
||||
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
//System.out.println("Connected to zk an got cluster state.");
|
||||
Collection<Slice> slices = CloudSolrStream.getSlices(this.collection, zkStateReader, false);
|
||||
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
|
||||
if (slices == null) slices = getSlicesIgnoreCase(this.collection, clusterState);
|
||||
if (slices == null) {
|
||||
throw new Exception("Collection not found:" + this.collection);
|
||||
}
|
||||
|
||||
|
||||
Iterator<String> iterator = params.getParameterNamesIterator();
|
||||
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
|
||||
mParams.set("distrib", "false"); // We are the aggregator.
|
||||
String fl = mParams.get("fl");
|
||||
|
@ -542,12 +530,15 @@ public class TopicStream extends CloudSolrStream implements Expressible {
|
|||
|
||||
Random random = new Random();
|
||||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> liveNodes = clusterState.getLiveNodes();
|
||||
|
||||
for(Slice slice : slices) {
|
||||
ModifiableSolrParams localParams = new ModifiableSolrParams(mParams);
|
||||
long checkpoint = checkpoints.get(slice.getName());
|
||||
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
List<Replica> shuffler = new ArrayList();
|
||||
List<Replica> shuffler = new ArrayList<>();
|
||||
for(Replica replica : replicas) {
|
||||
if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName()))
|
||||
shuffler.add(replica);
|
||||
|
|
|
@ -42,6 +42,7 @@ public class LukeResponse extends SolrResponseBase {
|
|||
boolean tokenized;
|
||||
String analyzer;
|
||||
List<String> fields;
|
||||
List<String> dynamicFields;
|
||||
|
||||
|
||||
public FieldTypeInfo(String name) {
|
||||
|
@ -62,6 +63,10 @@ public class LukeResponse extends SolrResponseBase {
|
|||
return fields;
|
||||
}
|
||||
|
||||
public List<String> getDynamicFields() {
|
||||
return dynamicFields;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
@ -96,6 +101,9 @@ public class LukeResponse extends SolrResponseBase {
|
|||
if ("fields".equals(key) && entry.getValue() != null) {
|
||||
List<String> theFields = (List<String>) entry.getValue();
|
||||
fields = new ArrayList<>(theFields);
|
||||
} else if ("dynamicFields".equals(key) && entry.getValue() != null) {
|
||||
List<String> theDynamicFields = (List<String>) entry.getValue();
|
||||
dynamicFields = new ArrayList<>(theDynamicFields);
|
||||
} else if ("tokenized".equals(key) == true) {
|
||||
tokenized = Boolean.parseBoolean(entry.getValue().toString());
|
||||
} else if ("analyzer".equals(key) == true) {
|
||||
|
@ -194,6 +202,7 @@ public class LukeResponse extends SolrResponseBase {
|
|||
|
||||
private NamedList<Object> indexInfo;
|
||||
private Map<String, FieldInfo> fieldInfo;
|
||||
private Map<String, FieldInfo> dynamicFieldInfo;
|
||||
private Map<String, FieldTypeInfo> fieldTypeInfo;
|
||||
|
||||
@Override
|
||||
|
@ -206,6 +215,8 @@ public class LukeResponse extends SolrResponseBase {
|
|||
|
||||
NamedList<Object> schema = (NamedList<Object>) res.get("schema");
|
||||
NamedList<Object> flds = (NamedList<Object>) res.get("fields");
|
||||
NamedList<Object> dynamicFlds = (NamedList<Object>) res.get("dynamicFields");
|
||||
|
||||
if (flds == null && schema != null ) {
|
||||
flds = (NamedList<Object>) schema.get("fields");
|
||||
}
|
||||
|
@ -218,6 +229,18 @@ public class LukeResponse extends SolrResponseBase {
|
|||
}
|
||||
}
|
||||
|
||||
if (dynamicFlds == null && schema != null) {
|
||||
dynamicFlds = (NamedList<Object>) schema.get("dynamicFields");
|
||||
}
|
||||
if (dynamicFlds != null) {
|
||||
dynamicFieldInfo = new HashMap<>();
|
||||
for (Map.Entry<String, Object> dynamicField : dynamicFlds) {
|
||||
FieldInfo f = new FieldInfo(dynamicField.getKey());
|
||||
f.read((NamedList<Object>) dynamicField.getValue());
|
||||
dynamicFieldInfo.put(dynamicField.getKey(), f);
|
||||
}
|
||||
}
|
||||
|
||||
if( schema != null ) {
|
||||
NamedList<Object> fldTypes = (NamedList<Object>) schema.get("types");
|
||||
if (fldTypes != null) {
|
||||
|
@ -274,5 +297,13 @@ public class LukeResponse extends SolrResponseBase {
|
|||
return fieldInfo.get(f);
|
||||
}
|
||||
|
||||
public Map<String, FieldInfo> getDynamicFieldInfo() {
|
||||
return dynamicFieldInfo;
|
||||
}
|
||||
|
||||
public FieldInfo getDynamicFieldInfo(String f) {
|
||||
return dynamicFieldInfo.get(f);
|
||||
}
|
||||
|
||||
//----------------------------------------------------------------
|
||||
}
|
||||
|
|
|
@ -694,13 +694,14 @@ abstract public class SolrExampleTests extends SolrExampleTestsBase
|
|||
luke.setShowSchema( false );
|
||||
LukeResponse rsp = luke.process( client );
|
||||
assertNull( rsp.getFieldTypeInfo() ); // if you don't ask for it, the schema is null
|
||||
assertNull( rsp.getDynamicFieldInfo() );
|
||||
|
||||
luke.setShowSchema( true );
|
||||
rsp = luke.process( client );
|
||||
assertNotNull( rsp.getFieldTypeInfo() );
|
||||
assertNotNull(rsp.getFieldInfo().get("id").getSchemaFlags());
|
||||
assertTrue(rsp.getFieldInfo().get("id").getSchemaFlags().contains(FieldFlag.INDEXED));
|
||||
|
||||
assertNotNull( rsp.getDynamicFieldInfo() );
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -49,12 +49,10 @@ import org.junit.Test;
|
|||
@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40", "Lucene41", "Lucene42", "Lucene45"})
|
||||
public class JdbcTest extends SolrCloudTestCase {
|
||||
|
||||
private static final String COLLECTION = "collection1";
|
||||
private static final String COLLECTIONORALIAS = "collection1";
|
||||
|
||||
private static final String id = "id";
|
||||
|
||||
private static final int TIMEOUT = 30;
|
||||
|
||||
private static String zkHost;
|
||||
|
||||
@BeforeClass
|
||||
|
@ -63,9 +61,18 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
.addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
|
||||
.configure();
|
||||
|
||||
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient());
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
|
||||
false, true, TIMEOUT);
|
||||
String collection;
|
||||
boolean useAlias = random().nextBoolean();
|
||||
if(useAlias) {
|
||||
collection = COLLECTIONORALIAS + "_collection";
|
||||
CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
|
||||
} else {
|
||||
collection = COLLECTIONORALIAS;
|
||||
}
|
||||
|
||||
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
|
||||
false, true, DEFAULT_TIMEOUT);
|
||||
|
||||
new UpdateRequest()
|
||||
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "testnull_i", null)
|
||||
|
@ -78,7 +85,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8", "testnull_i", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9", "testnull_i", null)
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10", "testnull_i", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), collection);
|
||||
|
||||
zkHost = cluster.getZkServer().getZkAddress();
|
||||
}
|
||||
|
@ -88,9 +95,9 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
Properties props = new Properties();
|
||||
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) {
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=" + COLLECTIONORALIAS, props)) {
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from collection1 order by a_i desc limit 2")) {
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from " + COLLECTIONORALIAS + " order by a_i desc limit 2")) {
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals(14, rs.getLong("a_i"));
|
||||
|
@ -113,7 +120,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
//Test statement reuse
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from collection1 order by a_i asc limit 2")) {
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from " + COLLECTIONORALIAS + " order by a_i asc limit 2")) {
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals(0, rs.getLong("a_i"));
|
||||
|
@ -138,7 +145,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
//Test connection reuse
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from collection1 order by a_i desc limit 2")) {
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from " + COLLECTIONORALIAS + " order by a_i desc limit 2")) {
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals(14, rs.getLong("a_i"));
|
||||
|
@ -154,7 +161,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
//Test statement reuse
|
||||
stmt.setMaxRows(2);
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from collection1 order by a_i asc")) {
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from " + COLLECTIONORALIAS + " order by a_i asc")) {
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals(0, rs.getLong("a_i"));
|
||||
|
@ -169,7 +176,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
//Test simple loop. Since limit is set it will override the statement maxRows.
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from collection1 order by a_i asc LIMIT 100")) {
|
||||
try (ResultSet rs = stmt.executeQuery("select id, a_i, a_s, a_f from " + COLLECTIONORALIAS + " order by a_i asc LIMIT 100")) {
|
||||
int count = 0;
|
||||
while (rs.next()) {
|
||||
++count;
|
||||
|
@ -187,9 +194,9 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
//Test facet aggregation
|
||||
Properties props = new Properties();
|
||||
props.put("aggregationMode", "facet");
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) {
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=" + COLLECTIONORALIAS, props)) {
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from collection1 group by a_s " +
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from " + COLLECTIONORALIAS + " group by a_s " +
|
||||
"order by sum(a_f) desc")) {
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
@ -227,9 +234,9 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
Properties props = new Properties();
|
||||
props.put("aggregationMode", "map_reduce");
|
||||
props.put("numWorkers", "2");
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) {
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=" + COLLECTIONORALIAS, props)) {
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from collection1 group by a_s " +
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from " + COLLECTIONORALIAS + " group by a_s " +
|
||||
"order by sum(a_f) desc")) {
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
@ -265,7 +272,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
//Test params on the url
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost +
|
||||
"?collection=collection1&aggregationMode=map_reduce&numWorkers=2")) {
|
||||
"?collection=" + COLLECTIONORALIAS + "&aggregationMode=map_reduce&numWorkers=2")) {
|
||||
|
||||
Properties p = ((ConnectionImpl) con).getProperties();
|
||||
|
||||
|
@ -273,7 +280,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
assert (p.getProperty("numWorkers").equals("2"));
|
||||
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from collection1 group by a_s " +
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from " + COLLECTIONORALIAS + " group by a_s " +
|
||||
"order by sum(a_f) desc")) {
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
@ -309,7 +316,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
// Test JDBC paramters in URL
|
||||
try (Connection con = DriverManager.getConnection(
|
||||
"jdbc:solr://" + zkHost + "?collection=collection1&username=&password=&testKey1=testValue&testKey2")) {
|
||||
"jdbc:solr://" + zkHost + "?collection=" + COLLECTIONORALIAS + "&username=&password=&testKey1=testValue&testKey2")) {
|
||||
|
||||
Properties p = ((ConnectionImpl) con).getProperties();
|
||||
assertEquals("", p.getProperty("username"));
|
||||
|
@ -318,7 +325,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
assertEquals("", p.getProperty("testKey2"));
|
||||
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from collection1 group by a_s " +
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from " + COLLECTIONORALIAS + " group by a_s " +
|
||||
"order by sum(a_f) desc")) {
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
@ -354,7 +361,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
// Test JDBC paramters in properties
|
||||
Properties providedProperties = new Properties();
|
||||
providedProperties.put("collection", "collection1");
|
||||
providedProperties.put("collection", COLLECTIONORALIAS);
|
||||
providedProperties.put("username", "");
|
||||
providedProperties.put("password", "");
|
||||
providedProperties.put("testKey1", "testValue");
|
||||
|
@ -368,7 +375,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
assert (p.getProperty("testKey2").equals(""));
|
||||
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from collection1 group by a_s " +
|
||||
try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from " + COLLECTIONORALIAS + " group by a_s " +
|
||||
"order by sum(a_f) desc")) {
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
@ -404,9 +411,9 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
//Test error propagation
|
||||
Properties props = new Properties();
|
||||
props.put("aggregationMode", "facet");
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) {
|
||||
try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=" + COLLECTIONORALIAS, props)) {
|
||||
try (Statement stmt = con.createStatement()) {
|
||||
try (ResultSet rs = stmt.executeQuery("select crap from collection1 group by a_s " +
|
||||
try (ResultSet rs = stmt.executeQuery("select crap from " + COLLECTIONORALIAS + " group by a_s " +
|
||||
"order by sum(a_f) desc")) {
|
||||
} catch (Exception e) {
|
||||
String errorMessage = e.getMessage();
|
||||
|
@ -418,7 +425,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
@Test
|
||||
public void testSQLExceptionThrownWhenQueryAndConnUseDiffCollections() throws Exception {
|
||||
String badCollection = COLLECTION + "bad";
|
||||
String badCollection = COLLECTIONORALIAS + "bad";
|
||||
String connectionString = "jdbc:solr://" + zkHost + "?collection=" + badCollection;
|
||||
String sql = "select id, a_i, a_s, a_f from " + badCollection + " order by a_i desc limit 2";
|
||||
|
||||
|
@ -436,7 +443,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
|
||||
@Test
|
||||
public void testDriverMetadata() throws Exception {
|
||||
String collection = COLLECTION;
|
||||
String collection = COLLECTIONORALIAS;
|
||||
|
||||
String connectionString1 = "jdbc:solr://" + zkHost + "?collection=" + collection +
|
||||
"&username=&password=&testKey1=testValue&testKey2";
|
||||
|
|
|
@ -50,7 +50,7 @@ import org.junit.Test;
|
|||
@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
|
||||
public class JDBCStreamTest extends SolrCloudTestCase {
|
||||
|
||||
private static final String COLLECTION = "jdbc";
|
||||
private static final String COLLECTIONORALIAS = "jdbc";
|
||||
|
||||
private static final int TIMEOUT = 30;
|
||||
|
||||
|
@ -62,8 +62,17 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
.addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
|
||||
.configure();
|
||||
|
||||
CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient());
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(),
|
||||
String collection;
|
||||
boolean useAlias = random().nextBoolean();
|
||||
if(useAlias) {
|
||||
collection = COLLECTIONORALIAS + "_collection";
|
||||
CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
|
||||
} else {
|
||||
collection = COLLECTIONORALIAS;
|
||||
}
|
||||
|
||||
CollectionAdminRequest.createCollection(collection, "conf", 2, 1).process(cluster.getSolrClient());
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(),
|
||||
false, true, TIMEOUT);
|
||||
}
|
||||
|
||||
|
@ -99,7 +108,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
public void cleanIndex() throws Exception {
|
||||
new UpdateRequest()
|
||||
.deleteByQuery("*:*")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
}
|
||||
|
||||
@Before
|
||||
|
@ -200,10 +209,10 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
new UpdateRequest()
|
||||
.add(id, "0", "code_s", "GB", "name_s", "Great Britian")
|
||||
.add(id, "1", "code_s", "CA", "name_s", "Canada")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
StreamFactory factory = new StreamFactory()
|
||||
.withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress())
|
||||
.withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
|
||||
.withFunctionName("search", CloudSolrStream.class);
|
||||
|
||||
List<Tuple> tuples;
|
||||
|
@ -211,7 +220,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
// Simple 1
|
||||
TupleStream jdbcStream = new JDBCStream("jdbc:hsqldb:mem:.", "select CODE,COUNTRY_NAME from COUNTRIES order by CODE", new FieldComparator("CODE", ComparatorOrder.ASCENDING));
|
||||
TupleStream selectStream = new SelectStream(jdbcStream, new HashMap<String, String>(){{ put("CODE", "code_s"); put("COUNTRY_NAME", "name_s"); }});
|
||||
TupleStream searchStream = factory.constructStream("search(" + COLLECTION + ", fl=\"code_s,name_s\",q=\"*:*\",sort=\"code_s asc\")");
|
||||
TupleStream searchStream = factory.constructStream("search(" + COLLECTIONORALIAS + ", fl=\"code_s,name_s\",q=\"*:*\",sort=\"code_s asc\")");
|
||||
TupleStream mergeStream = new MergeStream(new FieldComparator("code_s", ComparatorOrder.ASCENDING), new TupleStream[]{selectStream,searchStream});
|
||||
|
||||
tuples = getTuples(mergeStream);
|
||||
|
@ -225,7 +234,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
public void testJDBCSolrInnerJoinExpression() throws Exception{
|
||||
|
||||
StreamFactory factory = new StreamFactory()
|
||||
.withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress())
|
||||
.withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
|
||||
.withFunctionName("search", CloudSolrStream.class)
|
||||
.withFunctionName("select", SelectStream.class)
|
||||
.withFunctionName("innerJoin", InnerJoinStream.class)
|
||||
|
@ -262,7 +271,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
.add(id, "8", "rating_f", "4", "personId_i", "18")
|
||||
.add(id, "9", "rating_f", "4.1", "personId_i", "19")
|
||||
.add(id, "10", "rating_f", "4.8", "personId_i", "20")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
String expression;
|
||||
TupleStream stream;
|
||||
|
@ -272,7 +281,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
expression =
|
||||
"innerJoin("
|
||||
+ " select("
|
||||
+ " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " search(" + COLLECTIONORALIAS + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " personId_i as personId,"
|
||||
+ " rating_f as rating"
|
||||
+ " ),"
|
||||
|
@ -299,7 +308,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
public void testJDBCSolrInnerJoinExpressionWithProperties() throws Exception{
|
||||
|
||||
StreamFactory factory = new StreamFactory()
|
||||
.withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress())
|
||||
.withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
|
||||
.withFunctionName("search", CloudSolrStream.class)
|
||||
.withFunctionName("select", SelectStream.class)
|
||||
.withFunctionName("innerJoin", InnerJoinStream.class)
|
||||
|
@ -336,26 +345,23 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
.add(id, "8", "rating_f", "4", "personId_i", "18")
|
||||
.add(id, "9", "rating_f", "4.1", "personId_i", "19")
|
||||
.add(id, "10", "rating_f", "4.8", "personId_i", "20")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
String expression;
|
||||
TupleStream stream;
|
||||
List<Tuple> tuples;
|
||||
|
||||
// Basic test
|
||||
// the test here is the setting of the property get_column_name=true. In hsqldb if this value is set to true then the use of an
|
||||
// as clause in a select will have no effect. As such even though we have PEOPLE.ID as PERSONID we will still expect the column
|
||||
// name to come out as ID and not PERSONID
|
||||
// Basic test for no alias
|
||||
expression =
|
||||
"innerJoin("
|
||||
+ " select("
|
||||
+ " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " search(" + COLLECTIONORALIAS + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " personId_i as personId,"
|
||||
+ " rating_f as rating"
|
||||
+ " ),"
|
||||
+ " select("
|
||||
+ " jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID as PERSONID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"PERSONID asc\", get_column_name=true),"
|
||||
+ " PERSONID as personId,"
|
||||
+ " jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"ID asc\"),"
|
||||
+ " ID as personId,"
|
||||
+ " NAME as personName,"
|
||||
+ " COUNTRY_NAME as country"
|
||||
+ " ),"
|
||||
|
@ -371,19 +377,16 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
assertOrderOf(tuples, "personName", "Emma","Grace","Hailey","Isabella","Lily","Madison","Mia","Natalie","Olivia","Samantha");
|
||||
assertOrderOf(tuples, "country", "Netherlands","United States","Netherlands","Netherlands","Netherlands","United States","United States","Netherlands","Netherlands","United States");
|
||||
|
||||
// Basic test
|
||||
// the test here is the setting of the property get_column_name=false. In hsqldb if this value is set to false then the use of an
|
||||
// as clause in a select will have effect. As such we have PEOPLE.ID as PERSONID we will still expect the column name to come out
|
||||
// PERSONID and not ID
|
||||
// Basic test for alias
|
||||
expression =
|
||||
"innerJoin("
|
||||
+ " select("
|
||||
+ " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " search(" + COLLECTIONORALIAS + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " personId_i as personId,"
|
||||
+ " rating_f as rating"
|
||||
+ " ),"
|
||||
+ " select("
|
||||
+ " jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID as PERSONID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"PERSONID asc\", get_column_name=false),"
|
||||
+ " jdbc(connection=\"jdbc:hsqldb:mem:.\", sql=\"select PEOPLE.ID as PERSONID, PEOPLE.NAME, COUNTRIES.COUNTRY_NAME from PEOPLE inner join COUNTRIES on PEOPLE.COUNTRY_CODE = COUNTRIES.CODE order by PEOPLE.ID\", sort=\"PERSONID asc\"),"
|
||||
+ " PERSONID as personId,"
|
||||
+ " NAME as personName,"
|
||||
+ " COUNTRY_NAME as country"
|
||||
|
@ -405,7 +408,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
public void testJDBCSolrInnerJoinRollupExpression() throws Exception{
|
||||
|
||||
StreamFactory factory = new StreamFactory()
|
||||
.withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress())
|
||||
.withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
|
||||
.withFunctionName("search", CloudSolrStream.class)
|
||||
.withFunctionName("select", SelectStream.class)
|
||||
.withFunctionName("hashJoin", HashJoinStream.class)
|
||||
|
@ -448,7 +451,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
.add(id, "6", "rating_f", "3", "personId_i", "16")
|
||||
.add(id, "7", "rating_f", "3", "personId_i", "17")
|
||||
.add(id, "10", "rating_f", "4.8", "personId_i", "20")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
String expression;
|
||||
TupleStream stream;
|
||||
|
@ -459,7 +462,7 @@ public class JDBCStreamTest extends SolrCloudTestCase {
|
|||
"rollup("
|
||||
+ " hashJoin("
|
||||
+ " hashed=select("
|
||||
+ " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " search(" + COLLECTIONORALIAS + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\"),"
|
||||
+ " personId_i as personId,"
|
||||
+ " rating_f as rating"
|
||||
+ " ),"
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -51,25 +51,24 @@ import org.apache.solr.cloud.AbstractDistribZkTestBase;
|
|||
import org.apache.solr.cloud.SolrCloudTestCase;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* All base tests will be done with CloudSolrStream. Under the covers CloudSolrStream uses SolrStream so
|
||||
* SolrStream will get fully exercised through these tests.
|
||||
*
|
||||
**/
|
||||
* All base tests will be done with CloudSolrStream. Under the covers CloudSolrStream uses SolrStream so
|
||||
* SolrStream will get fully exercised through these tests.
|
||||
*
|
||||
**/
|
||||
|
||||
@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
|
||||
public class StreamingTest extends SolrCloudTestCase {
|
||||
|
||||
public static final int TIMEOUT = 30;
|
||||
public static final String COLLECTIONORALIAS = "streams";
|
||||
|
||||
public static final String COLLECTION = "streams";
|
||||
|
||||
private static final StreamFactory streamFactory = new StreamFactory()
|
||||
private static final StreamFactory streamFactory = new StreamFactory()
|
||||
.withFunctionName("search", CloudSolrStream.class)
|
||||
.withFunctionName("merge", MergeStream.class)
|
||||
.withFunctionName("unique", UniqueStream.class)
|
||||
|
@ -79,37 +78,47 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.withFunctionName("rollup", RollupStream.class)
|
||||
.withFunctionName("parallel", ParallelStream.class);
|
||||
|
||||
private static String zkHost;
|
||||
private static String zkHost;
|
||||
|
||||
private static int numShards;
|
||||
private static int numWorkers;
|
||||
private static int numShards;
|
||||
private static int numWorkers;
|
||||
private static boolean useAlias;
|
||||
|
||||
@BeforeClass
|
||||
public static void configureCluster() throws Exception {
|
||||
@BeforeClass
|
||||
public static void configureCluster() throws Exception {
|
||||
numShards = random().nextInt(2) + 1; //1 - 3
|
||||
numWorkers = numShards > 2 ? random().nextInt(numShards - 1) + 1 : numShards;
|
||||
configureCluster(numShards)
|
||||
.addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
|
||||
.configure();
|
||||
|
||||
CollectionAdminRequest.createCollection(COLLECTION, "conf", numShards, 1).process(cluster.getSolrClient());
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), false, true, TIMEOUT);
|
||||
String collection;
|
||||
useAlias = random().nextBoolean();
|
||||
if(useAlias) {
|
||||
collection = COLLECTIONORALIAS + "_collection";
|
||||
CollectionAdminRequest.createAlias(COLLECTIONORALIAS, collection).process(cluster.getSolrClient());
|
||||
} else {
|
||||
collection = COLLECTIONORALIAS;
|
||||
}
|
||||
|
||||
CollectionAdminRequest.createCollection(collection, "conf", numShards, 1).process(cluster.getSolrClient());
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collection, cluster.getSolrClient().getZkStateReader(), false, true, DEFAULT_TIMEOUT);
|
||||
|
||||
zkHost = cluster.getZkServer().getZkAddress();
|
||||
streamFactory.withCollectionZkHost(COLLECTION, zkHost);
|
||||
}
|
||||
streamFactory.withCollectionZkHost(COLLECTIONORALIAS, zkHost);
|
||||
}
|
||||
|
||||
private static final String id = "id";
|
||||
private static final String id = "id";
|
||||
|
||||
@Before
|
||||
public void clearCollection() throws Exception {
|
||||
@Before
|
||||
public void clearCollection() throws Exception {
|
||||
new UpdateRequest()
|
||||
.deleteByQuery("*:*")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
}
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUniqueStream() throws Exception {
|
||||
@Test
|
||||
public void testUniqueStream() throws Exception {
|
||||
|
||||
//Test CloudSolrStream and UniqueStream
|
||||
new UpdateRequest()
|
||||
|
@ -118,19 +127,19 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
|
||||
.add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
|
||||
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParams = StreamingTest.mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f"));
|
||||
List<Tuple> tuples = getTuples(ustream);
|
||||
assertEquals(4, tuples.size());
|
||||
assertOrder(tuples, 0,1,3,4);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSpacesInParams() throws Exception {
|
||||
@Test
|
||||
public void testSpacesInParams() throws Exception {
|
||||
|
||||
SolrParams sParams = StreamingTest.mapParams("q", "*:*", "fl", "id , a_s , a_i , a_f", "sort", "a_f asc , a_i asc");
|
||||
|
||||
|
@ -139,10 +148,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
//a value in the field list.
|
||||
|
||||
CloudSolrStream stream = new CloudSolrStream("", "collection1", sParams);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNonePartitionKeys() throws Exception {
|
||||
@Test
|
||||
public void testNonePartitionKeys() throws Exception {
|
||||
|
||||
new UpdateRequest()
|
||||
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1")
|
||||
|
@ -155,20 +164,20 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = StreamingTest.mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "none");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
ParallelStream pstream = parallelStream(stream, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
|
||||
attachStreamFactory(pstream);
|
||||
List<Tuple> tuples = getTuples(pstream);
|
||||
|
||||
assert(tuples.size() == (10 * numWorkers)); // Each tuple will be double counted.
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelUniqueStream() throws Exception {
|
||||
@Test
|
||||
public void testParallelUniqueStream() throws Exception {
|
||||
|
||||
new UpdateRequest()
|
||||
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
|
||||
|
@ -180,10 +189,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "6", "a_s", "hello1", "a_i", "11", "a_f", "5")
|
||||
.add(id, "7", "a_s", "hello1", "a_i", "12", "a_f", "5")
|
||||
.add(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc", "partitionKeys", "a_f");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f"));
|
||||
ParallelStream pstream = parallelStream(ustream, new FieldComparator("a_f", ComparatorOrder.ASCENDING));
|
||||
attachStreamFactory(pstream);
|
||||
|
@ -196,10 +205,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
Map<String,Tuple> eofTuples = pstream.getEofTuples();
|
||||
assertEquals(numWorkers, eofTuples.size()); //There should be an EOF tuple for each worker.
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleFqClauses() throws Exception {
|
||||
@Test
|
||||
public void testMultipleFqClauses() throws Exception {
|
||||
|
||||
new UpdateRequest()
|
||||
.add(id, "0", "a_ss", "hello0", "a_ss", "hello1", "a_i", "0", "a_f", "0")
|
||||
|
@ -211,20 +220,20 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "6", "a_ss", "hello1", "a_i", "11", "a_f", "5")
|
||||
.add(id, "7", "a_ss", "hello1", "a_i", "12", "a_f", "5")
|
||||
.add(id, "8", "a_ss", "hello1", "a_i", "13", "a_f", "4")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
streamFactory.withCollectionZkHost(COLLECTION, zkHost);
|
||||
streamFactory.withCollectionZkHost(COLLECTIONORALIAS, zkHost);
|
||||
|
||||
ModifiableSolrParams params = new ModifiableSolrParams(mapParams("q", "*:*", "fl", "id,a_i",
|
||||
"sort", "a_i asc", "fq", "a_ss:hello0", "fq", "a_ss:hello1"));
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, params);
|
||||
List<Tuple> tuples = getTuples(stream);
|
||||
assertEquals("Multiple fq clauses should have been honored", 1, tuples.size());
|
||||
assertEquals("should only have gotten back document 0", "0", tuples.get(0).getString("id"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRankStream() throws Exception {
|
||||
@Test
|
||||
public void testRankStream() throws Exception {
|
||||
|
||||
new UpdateRequest()
|
||||
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
|
||||
|
@ -232,21 +241,21 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
|
||||
.add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
|
||||
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
|
||||
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
RankStream rstream = new RankStream(stream, 3, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
|
||||
List<Tuple> tuples = getTuples(rstream);
|
||||
|
||||
assertEquals(3, tuples.size());
|
||||
assertOrder(tuples, 4,3,2);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParallelRankStream() throws Exception {
|
||||
@Test
|
||||
public void testParallelRankStream() throws Exception {
|
||||
|
||||
new UpdateRequest()
|
||||
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
|
||||
|
@ -259,10 +268,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "8", "a_s", "hello1", "a_i", "8", "a_f", "1")
|
||||
.add(id, "9", "a_s", "hello1", "a_i", "9", "a_f", "1")
|
||||
.add(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
RankStream rstream = new RankStream(stream, 11, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
|
||||
ParallelStream pstream = parallelStream(rstream, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
|
||||
attachStreamFactory(pstream);
|
||||
|
@ -271,10 +280,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
assertEquals(10, tuples.size());
|
||||
assertOrder(tuples, 10,9,8,7,6,5,4,3,2,0);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTrace() throws Exception {
|
||||
@Test
|
||||
public void testTrace() throws Exception {
|
||||
|
||||
new UpdateRequest()
|
||||
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1")
|
||||
|
@ -287,17 +296,17 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
//Test with spaces in the parameter lists.
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i,a_f", "sort", "a_s asc,a_f asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
stream.setTrace(true);
|
||||
List<Tuple> tuples = getTuples(stream);
|
||||
assertEquals(COLLECTION, tuples.get(0).get("_COLLECTION_"));
|
||||
assertEquals(COLLECTION, tuples.get(1).get("_COLLECTION_"));
|
||||
assertEquals(COLLECTION, tuples.get(2).get("_COLLECTION_"));
|
||||
assertEquals(COLLECTION, tuples.get(3).get("_COLLECTION_"));
|
||||
assertEquals(COLLECTIONORALIAS, tuples.get(0).get("_COLLECTION_"));
|
||||
assertEquals(COLLECTIONORALIAS, tuples.get(1).get("_COLLECTION_"));
|
||||
assertEquals(COLLECTIONORALIAS, tuples.get(2).get("_COLLECTION_"));
|
||||
assertEquals(COLLECTIONORALIAS, tuples.get(3).get("_COLLECTION_"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -314,11 +323,11 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
//Test with spaces in the parameter lists.
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
ReducerStream rstream = new ReducerStream(stream,
|
||||
new FieldEqualitor("a_s"),
|
||||
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5));
|
||||
|
@ -341,7 +350,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Test with spaces in the parameter lists using a comparator
|
||||
sParamsA = mapParams("q", "*:*", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
rstream = new ReducerStream(stream,
|
||||
new FieldComparator("a_s", ComparatorOrder.ASCENDING),
|
||||
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.DESCENDING), 5));
|
||||
|
@ -379,11 +388,11 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
//Test with spaces in the parameter lists.
|
||||
SolrParams sParamsA = mapParams("q", "blah", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
ReducerStream rstream = new ReducerStream(stream,
|
||||
new FieldEqualitor("a_s"),
|
||||
new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5));
|
||||
|
@ -408,10 +417,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
ReducerStream rstream = new ReducerStream(stream,
|
||||
new FieldEqualitor("a_s"),
|
||||
|
@ -437,7 +446,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
//Test Descending with Ascending subsort
|
||||
|
||||
sParamsA = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s desc,a_f asc", "partitionKeys", "a_s");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
rstream = new ReducerStream(stream,
|
||||
new FieldEqualitor("a_s"),
|
||||
|
@ -477,11 +486,11 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
//Test an error that comes originates from the /select handler
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
ExceptionStream estream = new ExceptionStream(stream);
|
||||
Tuple t = getTuple(estream);
|
||||
assertTrue(t.EOF);
|
||||
|
@ -490,7 +499,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Test an error that comes originates from the /export handler
|
||||
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt", "/export");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
estream = new ExceptionStream(stream);
|
||||
t = getTuple(estream);
|
||||
assertTrue(t.EOF);
|
||||
|
@ -514,11 +523,11 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING));
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
ParallelStream pstream = new ParallelStream(zkHost, COLLECTIONORALIAS, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING));
|
||||
ExceptionStream estream = new ExceptionStream(pstream);
|
||||
Tuple t = getTuple(estream);
|
||||
assertTrue(t.EOF);
|
||||
|
@ -529,8 +538,8 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Test an error that originates from the /select handler
|
||||
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc", "partitionKeys", "a_s");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING));
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
pstream = new ParallelStream(zkHost, COLLECTIONORALIAS, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING));
|
||||
estream = new ExceptionStream(pstream);
|
||||
t = getTuple(estream);
|
||||
assertTrue(t.EOF);
|
||||
|
@ -540,8 +549,8 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Test an error that originates from the /export handler
|
||||
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt", "/export", "partitionKeys", "a_s");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
pstream = new ParallelStream(zkHost, COLLECTIONORALIAS, stream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING));
|
||||
estream = new ExceptionStream(pstream);
|
||||
t = getTuple(estream);
|
||||
assertTrue(t.EOF);
|
||||
|
@ -564,7 +573,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "*:*");
|
||||
|
||||
|
@ -578,7 +587,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
new MeanMetric("a_f"),
|
||||
new CountMetric()};
|
||||
|
||||
StatsStream statsStream = new StatsStream(zkHost, COLLECTION, sParamsA, metrics);
|
||||
StatsStream statsStream = new StatsStream(zkHost, COLLECTIONORALIAS, sParamsA, metrics);
|
||||
|
||||
List<Tuple> tuples = getTuples(statsStream);
|
||||
|
||||
|
@ -624,7 +633,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc");
|
||||
|
||||
|
@ -643,7 +652,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
FieldComparator[] sorts = {new FieldComparator("sum(a_i)",
|
||||
ComparatorOrder.ASCENDING)};
|
||||
|
||||
FacetStream facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
|
||||
FacetStream facetStream = new FacetStream(zkHost, COLLECTIONORALIAS, sParamsA, buckets, metrics, sorts, 100);
|
||||
|
||||
List<Tuple> tuples = getTuples(facetStream);
|
||||
|
||||
|
@ -725,7 +734,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
sorts[0] = new FieldComparator("sum(a_i)", ComparatorOrder.DESCENDING);
|
||||
|
||||
facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
|
||||
facetStream = new FacetStream(zkHost, COLLECTIONORALIAS, sParamsA, buckets, metrics, sorts, 100);
|
||||
|
||||
tuples = getTuples(facetStream);
|
||||
|
||||
|
@ -808,7 +817,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
sorts[0] = new FieldComparator("a_s", ComparatorOrder.DESCENDING);
|
||||
|
||||
|
||||
facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
|
||||
facetStream = new FacetStream(zkHost, COLLECTIONORALIAS, sParamsA, buckets, metrics, sorts, 100);
|
||||
|
||||
tuples = getTuples(facetStream);
|
||||
|
||||
|
@ -889,7 +898,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
sorts[0] = new FieldComparator("a_s", ComparatorOrder.ASCENDING);
|
||||
|
||||
facetStream = new FacetStream(zkHost, COLLECTION, sParamsA, buckets, metrics, sorts, 100);
|
||||
facetStream = new FacetStream(zkHost, COLLECTIONORALIAS, sParamsA, buckets, metrics, sorts, 100);
|
||||
|
||||
tuples = getTuples(facetStream);
|
||||
|
||||
|
@ -1015,7 +1024,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
// }
|
||||
// }
|
||||
// SolrParams exportParams = mapParams("q", "*:*", "qt", "/export", "fl", "id," + field, "sort", field + " " + sortDir + ",id asc");
|
||||
// try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTION, exportParams)) {
|
||||
// try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, exportParams)) {
|
||||
// List<Tuple> tuples = getTuples(solrStream);
|
||||
// assertEquals("There should be exactly 32 responses returned", 32, tuples.size());
|
||||
// // Since the getTuples method doesn't return the EOF tuple, these two entries should be the same size.
|
||||
|
@ -1031,7 +1040,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
List<String> selectOrder = ("asc".equals(sortDir)) ? Arrays.asList(ascOrder) : Arrays.asList(descOrder);
|
||||
List<String> selectOrderBool = ("asc".equals(sortDir)) ? Arrays.asList(ascOrderBool) : Arrays.asList(descOrderBool);
|
||||
SolrParams exportParams = mapParams("q", "*:*", "qt", "/export", "fl", "id," + field, "sort", field + " " + sortDir + ",id asc");
|
||||
try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTION, exportParams)) {
|
||||
try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, exportParams)) {
|
||||
List<Tuple> tuples = getTuples(solrStream);
|
||||
assertEquals("There should be exactly 32 responses returned", 32, tuples.size());
|
||||
// Since the getTuples method doesn't return the EOF tuple, these two entries should be the same size.
|
||||
|
@ -1070,7 +1079,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
}
|
||||
SolrParams sParams = mapParams("q", "*:*", "qt", "/export", "fl", fl.toString(), "sort", "id asc");
|
||||
|
||||
try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTION, sParams)) {
|
||||
try (CloudSolrStream solrStream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams)) {
|
||||
List<Tuple> tuples = getTuples(solrStream);
|
||||
assertEquals("There should be exactly 32 responses returned", 32, tuples.size());
|
||||
|
||||
|
@ -1185,7 +1194,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(docPairs(8, "aaa"))
|
||||
.add(docPairs(8, "ooo"))
|
||||
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
JettySolrRunner jetty = cluster.getJettySolrRunners().get(0);
|
||||
|
||||
|
@ -1216,7 +1225,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "level1_s", "hello3", "level2_s", "b", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "level1_s", "hello3", "level2_s", "b", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "level1_s", "hello0", "level2_s", "b", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_i,a_f");
|
||||
|
||||
|
@ -1229,7 +1238,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
FacetStream facetStream = new FacetStream(
|
||||
zkHost,
|
||||
COLLECTION,
|
||||
COLLECTIONORALIAS,
|
||||
sParamsA,
|
||||
buckets,
|
||||
metrics,
|
||||
|
@ -1309,7 +1318,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
sorts[1] = new FieldComparator("level2_s", ComparatorOrder.DESCENDING );
|
||||
facetStream = new FacetStream(
|
||||
zkHost,
|
||||
COLLECTION,
|
||||
COLLECTIONORALIAS,
|
||||
sParamsA,
|
||||
buckets,
|
||||
metrics,
|
||||
|
@ -1401,10 +1410,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
Bucket[] buckets = {new Bucket("a_s")};
|
||||
|
||||
|
@ -1518,10 +1527,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
//Test will null value in the grouping field
|
||||
new UpdateRequest()
|
||||
.add(id, "12", "a_s", null, "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "qt", "/export");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
Bucket[] buckets1 = {new Bucket("a_s")};
|
||||
|
||||
|
@ -1566,6 +1575,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
@Test
|
||||
public void testDaemonTopicStream() throws Exception {
|
||||
Assume.assumeTrue(!useAlias);
|
||||
|
||||
StreamContext context = new StreamContext();
|
||||
SolrClientCache cache = new SolrClientCache();
|
||||
|
@ -1574,8 +1584,8 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
SolrParams sParams = mapParams("q", "a_s:hello0", "rows", "500", "fl", "id");
|
||||
|
||||
TopicStream topicStream = new TopicStream(zkHost,
|
||||
COLLECTION,
|
||||
COLLECTION,
|
||||
COLLECTIONORALIAS,
|
||||
COLLECTIONORALIAS,
|
||||
"50000000",
|
||||
-1,
|
||||
1000000, sParams);
|
||||
|
@ -1592,7 +1602,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
SolrParams sParams1 = mapParams("qt", "/get", "ids", "50000000", "fl", "id");
|
||||
int count = 0;
|
||||
while(count == 0) {
|
||||
SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/" + COLLECTION, sParams1);
|
||||
SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/" + COLLECTIONORALIAS, sParams1);
|
||||
List<Tuple> tuples = getTuples(solrStream);
|
||||
count = tuples.size();
|
||||
if(count > 0) {
|
||||
|
@ -1609,7 +1619,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "3", "a_s", "hello0", "a_i", "3", "a_f", "3")
|
||||
.add(id, "4", "a_s", "hello0", "a_i", "4", "a_f", "4")
|
||||
.add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
for(int i=0; i<5; i++) {
|
||||
daemonStream.read();
|
||||
|
@ -1618,7 +1628,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
new UpdateRequest()
|
||||
.add(id, "5", "a_s", "hello0", "a_i", "4", "a_f", "4")
|
||||
.add(id, "6", "a_s", "hello0", "a_i", "4", "a_f", "4")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
for(int i=0; i<2; i++) {
|
||||
daemonStream.read();
|
||||
|
@ -1648,10 +1658,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "partitionKeys", "a_s");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
Bucket[] buckets = {new Bucket("a_s")};
|
||||
|
||||
|
@ -1759,10 +1769,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8")
|
||||
.add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
|
||||
.add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParamsA = mapParams("q", "blah", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "a_s");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
ReducerStream rstream = new ReducerStream(stream,
|
||||
new FieldEqualitor("a_s"),
|
||||
new GroupOperation(new FieldComparator("a_s", ComparatorOrder.ASCENDING), 2));
|
||||
|
@ -1779,10 +1789,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
new UpdateRequest()
|
||||
.add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "5.1", "s_multi", "a", "s_multi", "b", "i_multi",
|
||||
"1", "i_multi", "2", "f_multi", "1.2", "f_multi", "1.3")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f,s_multi,i_multi,f_multi", "sort", "a_s asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
List<Tuple> tuples = getTuples(stream);
|
||||
Tuple tuple = tuples.get(0);
|
||||
|
||||
|
@ -1820,14 +1830,14 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
|
||||
.add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
|
||||
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
//Test ascending
|
||||
SolrParams sParamsA = mapParams("q", "id:(4 1)", "fl", "id,a_s,a_i", "sort", "a_i asc");
|
||||
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
SolrParams sParamsB = mapParams("q", "id:(0 2 3)", "fl", "id,a_s,a_i", "sort", "a_i asc");
|
||||
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
|
||||
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
|
||||
|
||||
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
|
||||
List<Tuple> tuples = getTuples(mstream);
|
||||
|
@ -1837,10 +1847,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Test descending
|
||||
sParamsA = mapParams("q", "id:(4 1)", "fl", "id,a_s,a_i", "sort", "a_i desc");
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
sParamsB = mapParams("q", "id:(0 2 3)", "fl", "id,a_s,a_i", "sort", "a_i desc");
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
|
||||
|
||||
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
|
||||
tuples = getTuples(mstream);
|
||||
|
@ -1851,10 +1861,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
//Test compound sort
|
||||
|
||||
sParamsA = mapParams("q", "id:(2 4 1)", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
sParamsB = mapParams("q", "id:(0 3)", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
|
||||
|
||||
mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.ASCENDING)));
|
||||
tuples = getTuples(mstream);
|
||||
|
@ -1863,10 +1873,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
assertOrder(tuples, 0,2,1,3,4);
|
||||
|
||||
sParamsA = mapParams("q", "id:(2 4 1)", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i desc");
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
sParamsB = mapParams("q", "id:(0 3)", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i desc");
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
|
||||
|
||||
mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.DESCENDING)));
|
||||
tuples = getTuples(mstream);
|
||||
|
@ -1890,14 +1900,14 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3")
|
||||
.add(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4")
|
||||
.add(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
//Test ascending
|
||||
SolrParams sParamsA = mapParams("q", "id:(4 1 8 7 9)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
|
||||
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
SolrParams sParamsB = mapParams("q", "id:(0 2 3 6)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
|
||||
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
|
||||
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
|
||||
|
||||
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
|
||||
ParallelStream pstream = parallelStream(mstream, new FieldComparator("a_i", ComparatorOrder.ASCENDING));
|
||||
|
@ -1909,10 +1919,10 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Test descending
|
||||
sParamsA = mapParams("q", "id:(4 1 8 9)", "fl", "id,a_s,a_i", "sort", "a_i desc", "partitionKeys", "a_i");
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
sParamsB = mapParams("q", "id:(0 2 3 6)", "fl", "id,a_s,a_i", "sort", "a_i desc", "partitionKeys", "a_i");
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
|
||||
streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
|
||||
|
||||
mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING));
|
||||
pstream = parallelStream(mstream, new FieldComparator("a_i", ComparatorOrder.DESCENDING));
|
||||
|
@ -1938,14 +1948,14 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3")
|
||||
.add(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4")
|
||||
.add(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
//Test ascending
|
||||
SolrParams sParamsA = mapParams("q", "id:(4 1 8 7 9)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
|
||||
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, sParamsA);
|
||||
CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsA);
|
||||
|
||||
SolrParams sParamsB = mapParams("q", "id:(0 2 3 6)", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i");
|
||||
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, sParamsB);
|
||||
CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParamsB);
|
||||
|
||||
MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING));
|
||||
ParallelStream pstream = parallelStream(mstream, new FieldComparator("a_i", ComparatorOrder.ASCENDING));
|
||||
|
@ -1967,13 +1977,13 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
|
||||
.add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
|
||||
.add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
|
||||
//Basic CloudSolrStream Test with Descending Sort
|
||||
|
||||
SolrParams sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i desc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
List<Tuple> tuples = getTuples(stream);
|
||||
|
||||
assertEquals(5,tuples.size());
|
||||
|
@ -1981,7 +1991,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//With Ascending Sort
|
||||
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
tuples = getTuples(stream);
|
||||
|
||||
assertEquals(5, tuples.size());
|
||||
|
@ -1990,7 +2000,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Test compound sort
|
||||
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i desc");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
tuples = getTuples(stream);
|
||||
|
||||
assertEquals(5, tuples.size());
|
||||
|
@ -1998,7 +2008,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
|
||||
sParams = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_f asc,a_i asc");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
tuples = getTuples(stream);
|
||||
|
||||
assertEquals(5, tuples.size());
|
||||
|
@ -2015,7 +2025,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
.add(id, "2", "b_sing", "false", "dt_sing", "1981-04-04T01:02:03.78Z")
|
||||
.add(id, "1", "b_sing", "true", "dt_sing", "1980-04-04T01:02:03.78Z")
|
||||
.add(id, "4", "b_sing", "true", "dt_sing", "1980-04-04T01:02:03.78Z")
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
|
||||
trySortWithQt("/export");
|
||||
|
@ -2025,7 +2035,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
//Basic CloudSolrStream Test bools desc
|
||||
|
||||
SolrParams sParams = mapParams("q", "*:*", "qt", which, "fl", "id,b_sing", "sort", "b_sing asc,id asc");
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
try {
|
||||
List<Tuple> tuples = getTuples(stream);
|
||||
|
||||
|
@ -2034,7 +2044,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Basic CloudSolrStream Test bools desc
|
||||
sParams = mapParams("q", "*:*", "qt", which, "fl", "id,b_sing", "sort", "b_sing desc,id desc");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
tuples = getTuples(stream);
|
||||
|
||||
assertEquals (5,tuples.size());
|
||||
|
@ -2042,7 +2052,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Basic CloudSolrStream Test dates desc
|
||||
sParams = mapParams("q", "*:*", "qt", which, "fl", "id,dt_sing", "sort", "dt_sing desc,id asc");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
tuples = getTuples(stream);
|
||||
|
||||
assertEquals (5,tuples.size());
|
||||
|
@ -2050,7 +2060,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
|
||||
//Basic CloudSolrStream Test ates desc
|
||||
sParams = mapParams("q", "*:*", "qt", which, "fl", "id,dt_sing", "sort", "dt_sing asc,id desc");
|
||||
stream = new CloudSolrStream(zkHost, COLLECTION, sParams);
|
||||
stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams);
|
||||
tuples = getTuples(stream);
|
||||
|
||||
assertEquals (5,tuples.size());
|
||||
|
@ -2079,7 +2089,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
"dt_sing", "1980-01-02T11:11:33.89Z", "dt_multi", "1981-03-04T01:02:03.78Z", "dt_multi", "1981-05-24T04:05:06.99Z",
|
||||
"b_sing", "true", "b_multi", "false", "b_multi", "true"
|
||||
)
|
||||
.commit(cluster.getSolrClient(), COLLECTION);
|
||||
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
|
||||
|
||||
tryWithQt("/export");
|
||||
tryWithQt("/select");
|
||||
|
@ -2090,7 +2100,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
SolrParams sParams = StreamingTest.mapParams("q", "*:*", "qt", which, "fl",
|
||||
"id,i_sing,i_multi,l_sing,l_multi,f_sing,f_multi,d_sing,d_multi,dt_sing,dt_multi,s_sing,s_multi,b_sing,b_multi",
|
||||
"sort", "i_sing asc");
|
||||
try (CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, sParams)) {
|
||||
try (CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTIONORALIAS, sParams)) {
|
||||
|
||||
Tuple tuple = getTuple(stream); // All I really care about is that all the fields are returned. There's
|
||||
|
||||
|
@ -2225,7 +2235,7 @@ public class StreamingTest extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
private ParallelStream parallelStream(TupleStream stream, FieldComparator comparator) throws IOException {
|
||||
ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, stream, numWorkers, comparator);
|
||||
ParallelStream pstream = new ParallelStream(zkHost, COLLECTIONORALIAS, stream, numWorkers, comparator);
|
||||
return pstream;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue