LUCENE-2858: Reverse merged revision(s) 1-0 from lucene/dev/trunk

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene2858@1235028 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2012-01-23 22:15:15 +00:00
commit b3a23daca8
49 changed files with 31742 additions and 30309 deletions

View File

@ -202,6 +202,12 @@
<testResource>
<directory>src/test-files</directory>
</testResource>
<testResource>
<directory>${project.build.testSourceDirectory}</directory>
<excludes>
<exclude>**/*.java</exclude>
</excludes>
</testResource>
<testResource>
<directory>../solrj/src/test-files</directory>
</testResource>

View File

@ -790,7 +790,7 @@ New Features
input mapping to it) for FSTs that have strictly monotonic long
outputs (such as an ord). (Mike McCandless)
* LUCENE-3121: Add TypeTokenFilter that filters tokens based on
* LUCENE-3671: Add TypeTokenFilter that filters tokens based on
their TypeAttribute. (Tommaso Teofili via Uwe Schindler)
* LUCENE-3690: Added HTMLStripCharFilter, a CharFilter that strips HTML
@ -814,9 +814,11 @@ Bug fixes
* LUCENE-3641: Fixed MultiReader to correctly propagate readerFinishedListeners
to clones/reopened readers. (Uwe Schindler)
* LUCENE-3642: Fixed bugs in CharTokenizer, n-gram filters, and smart chinese
where they would create invalid offsets in some situations, leading to problems
in highlighting. (Max Beutel via Robert Muir)
* LUCENE-3642, SOLR-2891, LUCENE-3717: Fixed bugs in CharTokenizer, n-gram filters,
compound token filters, thai word filter, icutokenizer, pattern analyzer,
wikipediatokenizer, and smart chinese where they would create invalid offsets in
some situations, leading to problems in highlighting.
(Max Beutel, Edwin Steiner via Robert Muir)
* LUCENE-3639: TopDocs.merge was incorrectly setting TopDocs.maxScore to
Float.MIN_VALUE when it should be Float.NaN, when there were 0

View File

@ -206,7 +206,7 @@ public class SegmentTermDocs {
skipListReader = new Lucene40SkipListReader((IndexInput) freqStream.clone(), maxSkipLevels, skipInterval); // lazily clone
if (!haveSkipped) { // lazily initialize skip stream
skipListReader.init(skipPointer, freqBasePointer, proxBasePointer, df, currentFieldStoresPayloads);
skipListReader.init(skipPointer, freqBasePointer, proxBasePointer, df, currentFieldStoresPayloads, false);
haveSkipped = true;
}

View File

@ -85,11 +85,11 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
// LUCENE-3027: past indices were able to write
// storePayloads=true when omitTFAP is also true,
// which is invalid. We correct that, here:
if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
storePayloads = false;
}
hasVectors |= storeTermVector;
hasProx |= isIndexed && indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
hasProx |= isIndexed && indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
// DV Types are packed in one byte
byte val = input.readByte();

View File

@ -58,7 +58,7 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
output.writeVInt(FORMAT_CURRENT);
output.writeVInt(infos.size());
for (FieldInfo fi : infos) {
assert fi.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !fi.storePayloads;
assert fi.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.storePayloads;
byte bits = 0x0;
if (fi.isIndexed) bits |= IS_INDEXED;
if (fi.storeTermVector) bits |= STORE_TERMVECTOR;

View File

@ -197,7 +197,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
// undefined
}
if (fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
if (fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
if (isFirstTerm) {
termState.proxOffset = termState.bytesReader.readVLong();
} else {
@ -245,23 +245,23 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
DocsAndPositionsEnum reuse, boolean needsOffsets)
throws IOException {
if (needsOffsets) {
// TODO: once we index offsets into postings fix this!
return null;
boolean hasOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (needsOffsets && !hasOffsets) {
return null; // not available
}
// TODO: refactor
if (fieldInfo.storePayloads) {
SegmentDocsAndPositionsAndPayloadsEnum docsEnum;
if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsAndPayloadsEnum)) {
docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn);
if (fieldInfo.storePayloads || hasOffsets) {
SegmentFullPositionsEnum docsEnum;
if (reuse == null || !(reuse instanceof SegmentFullPositionsEnum)) {
docsEnum = new SegmentFullPositionsEnum(freqIn, proxIn);
} else {
docsEnum = (SegmentDocsAndPositionsAndPayloadsEnum) reuse;
docsEnum = (SegmentFullPositionsEnum) reuse;
if (docsEnum.startFreqIn != freqIn) {
// If you are using ParellelReader, and pass in a
// reused DocsEnum, it could have come from another
// reader also using standard codec
docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn);
docsEnum = new SegmentFullPositionsEnum(freqIn, proxIn);
}
}
return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs);
@ -295,6 +295,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
protected boolean indexOmitsTF; // does current field omit term freq?
protected boolean storePayloads; // does current field store payloads?
protected boolean storeOffsets; // does current field store offsets?
protected int limit; // number of docs in this posting
protected int ord; // how many docs we've read
@ -324,6 +325,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
DocsEnum reset(FieldInfo fieldInfo, StandardTermState termState) throws IOException {
indexOmitsTF = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY;
storePayloads = fieldInfo.storePayloads;
storeOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
freqOffset = termState.freqOffset;
skipOffset = termState.skipOffset;
@ -471,7 +473,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
skipper.init(freqOffset + skipOffset,
freqOffset, 0,
limit, storePayloads);
limit, storePayloads, storeOffsets);
skipped = true;
}
@ -665,7 +667,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
// TODO specialize DocsAndPosEnum too
// Decodes docs & positions. payloads are not present.
// Decodes docs & positions. payloads nor offsets are present.
private final class SegmentDocsAndPositionsEnum extends DocsAndPositionsEnum {
final IndexInput startFreqIn;
private final IndexInput freqIn;
@ -792,7 +794,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
skipper.init(freqOffset+skipOffset,
freqOffset, proxOffset,
limit, false);
limit, false, false);
skipped = true;
}
@ -868,8 +870,8 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
}
}
// Decodes docs & positions & payloads
private class SegmentDocsAndPositionsAndPayloadsEnum extends DocsAndPositionsEnum {
// Decodes docs & positions & (payloads and/or offsets)
private class SegmentFullPositionsEnum extends DocsAndPositionsEnum {
final IndexInput startFreqIn;
private final IndexInput freqIn;
private final IndexInput proxIn;
@ -895,16 +897,24 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
Lucene40SkipListReader skipper;
private BytesRef payload;
private long lazyProxPointer;
boolean storePayloads;
boolean storeOffsets;
int offsetLength;
int startOffset;
public SegmentDocsAndPositionsAndPayloadsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
public SegmentFullPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
startFreqIn = freqIn;
this.freqIn = (IndexInput) freqIn.clone();
this.proxIn = (IndexInput) proxIn.clone();
}
public SegmentDocsAndPositionsAndPayloadsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
assert fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
assert fieldInfo.storePayloads;
public SegmentFullPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
storeOffsets = fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
storePayloads = fieldInfo.storePayloads;
assert fieldInfo.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
assert storePayloads || storeOffsets;
if (payload == null) {
payload = new BytesRef();
payload.bytes = new byte[1];
@ -923,6 +933,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
doc = -1;
accum = 0;
position = 0;
startOffset = 0;
skipped = false;
posPendingCount = 0;
@ -963,6 +974,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
}
position = 0;
startOffset = 0;
//System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
return (doc = accum);
@ -1001,7 +1013,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
//System.out.println(" init skipper freqOffset=" + freqOffset + " skipOffset=" + skipOffset + " vs len=" + freqIn.length());
skipper.init(freqOffset+skipOffset,
freqOffset, proxOffset,
limit, true);
limit, storePayloads, storeOffsets);
skipped = true;
}
@ -1016,8 +1028,10 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
lazyProxPointer = skipper.getProxPointer();
posPendingCount = 0;
position = 0;
startOffset = 0;
payloadPending = false;
payloadLength = skipper.getPayloadLength();
offsetLength = skipper.getOffsetLength();
}
}
@ -1038,27 +1052,38 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
}
if (payloadPending && payloadLength > 0) {
// payload of last position as never retrieved -- skip it
// payload of last position was never retrieved -- skip it
proxIn.seek(proxIn.getFilePointer() + payloadLength);
payloadPending = false;
}
// scan over any docs that were iterated without their positions
while(posPendingCount > freq) {
final int code = proxIn.readVInt();
if ((code & 1) != 0) {
// new payload length
payloadLength = proxIn.readVInt();
assert payloadLength >= 0;
if (storePayloads) {
if ((code & 1) != 0) {
// new payload length
payloadLength = proxIn.readVInt();
assert payloadLength >= 0;
}
assert payloadLength != -1;
}
assert payloadLength != -1;
proxIn.seek(proxIn.getFilePointer() + payloadLength);
if (storeOffsets) {
if ((proxIn.readVInt() & 1) != 0) {
// new offset length
offsetLength = proxIn.readVInt();
}
}
if (storePayloads) {
proxIn.seek(proxIn.getFilePointer() + payloadLength);
}
posPendingCount--;
position = 0;
startOffset = 0;
payloadPending = false;
//System.out.println("StandardR.D&PE skipPos");
}
@ -1069,16 +1094,28 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
proxIn.seek(proxIn.getFilePointer()+payloadLength);
}
final int code = proxIn.readVInt();
if ((code & 1) != 0) {
// new payload length
payloadLength = proxIn.readVInt();
assert payloadLength >= 0;
}
assert payloadLength != -1;
int code = proxIn.readVInt();
if (storePayloads) {
if ((code & 1) != 0) {
// new payload length
payloadLength = proxIn.readVInt();
assert payloadLength >= 0;
}
assert payloadLength != -1;
payloadPending = true;
position += code >>> 1;
payloadPending = true;
code >>>= 1;
}
position += code;
if (storeOffsets) {
int offsetCode = proxIn.readVInt();
if ((offsetCode & 1) != 0) {
// new offset length
offsetLength = proxIn.readVInt();
}
startOffset += offsetCode >>> 1;
}
posPendingCount--;
@ -1090,32 +1127,36 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
@Override
public int startOffset() throws IOException {
return -1;
return storeOffsets ? startOffset : -1;
}
@Override
public int endOffset() throws IOException {
return -1;
return storeOffsets ? startOffset + offsetLength : -1;
}
/** Returns the payload at this position, or null if no
* payload was indexed. */
@Override
public BytesRef getPayload() throws IOException {
assert lazyProxPointer == -1;
assert posPendingCount < freq;
if (!payloadPending) {
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
}
if (payloadLength > payload.bytes.length) {
payload.grow(payloadLength);
}
if (storePayloads) {
assert lazyProxPointer == -1;
assert posPendingCount < freq;
if (!payloadPending) {
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
}
if (payloadLength > payload.bytes.length) {
payload.grow(payloadLength);
}
proxIn.readBytes(payload.bytes, 0, payloadLength);
payload.length = payloadLength;
payloadPending = false;
proxIn.readBytes(payload.bytes, 0, payloadLength);
payload.length = payloadLength;
payloadPending = false;
return payload;
return payload;
} else {
throw new IOException("No payloads exist for this field!");
}
}
@Override

View File

@ -73,12 +73,15 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
IndexOptions indexOptions;
boolean storePayloads;
boolean storeOffsets;
// Starts a new term
long freqStart;
long proxStart;
FieldInfo fieldInfo;
int lastPayloadLength;
int lastOffsetLength;
int lastPosition;
int lastOffset;
// private String segment;
@ -137,6 +140,8 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
proxStart = proxOut.getFilePointer();
// force first payload to write its length
lastPayloadLength = -1;
// force first offset to write its length
lastOffsetLength = -1;
}
skipListWriter.resetSkip();
}
@ -155,10 +160,8 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
*/
this.fieldInfo = fieldInfo;
indexOptions = fieldInfo.indexOptions;
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
throw new UnsupportedOperationException("this codec cannot index offsets");
}
storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
storePayloads = fieldInfo.storePayloads;
//System.out.println(" set init blockFreqStart=" + freqStart);
//System.out.println(" set init blockProxStart=" + proxStart);
@ -180,7 +183,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
}
if ((++df % skipInterval) == 0) {
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength, storeOffsets, lastOffsetLength);
skipListWriter.bufferSkip(df);
}
@ -197,31 +200,26 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
}
lastPosition = 0;
lastOffset = 0;
}
/** Add a new position & payload */
@Override
public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
//if (DEBUG) System.out.println("SPW: addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS: "invalid indexOptions: " + indexOptions;
assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 : "invalid indexOptions: " + indexOptions;
assert proxOut != null;
// TODO: when we add offsets... often
// endOffset-startOffset will be constant or near
// constant for all docs (eg if the term wasn't stemmed
// then this will usually be the utf16 length of the
// term); would be nice to write that length once up
// front and then not encode endOffset for each
// position..
final int delta = position - lastPosition;
assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it)
lastPosition = position;
int payloadLength = 0;
if (storePayloads) {
final int payloadLength = payload == null ? 0 : payload.length;
payloadLength = payload == null ? 0 : payload.length;
if (payloadLength != lastPayloadLength) {
lastPayloadLength = payloadLength;
@ -230,13 +228,28 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
} else {
proxOut.writeVInt(delta << 1);
}
if (payloadLength > 0) {
proxOut.writeBytes(payload.bytes, payload.offset, payloadLength);
}
} else {
proxOut.writeVInt(delta);
}
if (storeOffsets) {
// don't use startOffset - lastEndOffset, because this creates lots of negative vints for synonyms,
// and the numbers aren't that much smaller anyways.
int offsetDelta = startOffset - lastOffset;
int offsetLength = endOffset - startOffset;
if (offsetLength != lastOffsetLength) {
proxOut.writeVInt(offsetDelta << 1 | 1);
proxOut.writeVInt(offsetLength);
} else {
proxOut.writeVInt(offsetDelta << 1);
}
lastOffset = startOffset;
lastOffsetLength = offsetLength;
}
if (payloadLength > 0) {
proxOut.writeBytes(payload.bytes, payload.offset, payloadLength);
}
}
@Override
@ -304,7 +317,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
assert firstTerm.skipOffset > 0;
bytesWriter.writeVInt(firstTerm.skipOffset);
}
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
bytesWriter.writeVLong(firstTerm.proxStart);
}
long lastFreqStart = firstTerm.freqStart;
@ -319,7 +332,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
assert term.skipOffset > 0;
bytesWriter.writeVInt(term.skipOffset);
}
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
bytesWriter.writeVLong(term.proxStart - lastProxStart);
lastProxStart = term.proxStart;
}

View File

@ -30,13 +30,16 @@ import org.apache.lucene.store.IndexInput;
*/
public class Lucene40SkipListReader extends MultiLevelSkipListReader {
private boolean currentFieldStoresPayloads;
private boolean currentFieldStoresOffsets;
private long freqPointer[];
private long proxPointer[];
private int payloadLength[];
private int offsetLength[];
private long lastFreqPointer;
private long lastProxPointer;
private int lastPayloadLength;
private int lastOffsetLength;
public Lucene40SkipListReader(IndexInput skipStream, int maxSkipLevels, int skipInterval) {
@ -44,17 +47,20 @@ public class Lucene40SkipListReader extends MultiLevelSkipListReader {
freqPointer = new long[maxSkipLevels];
proxPointer = new long[maxSkipLevels];
payloadLength = new int[maxSkipLevels];
offsetLength = new int[maxSkipLevels];
}
public void init(long skipPointer, long freqBasePointer, long proxBasePointer, int df, boolean storesPayloads) {
public void init(long skipPointer, long freqBasePointer, long proxBasePointer, int df, boolean storesPayloads, boolean storesOffsets) {
super.init(skipPointer, df);
this.currentFieldStoresPayloads = storesPayloads;
this.currentFieldStoresOffsets = storesOffsets;
lastFreqPointer = freqBasePointer;
lastProxPointer = proxBasePointer;
Arrays.fill(freqPointer, freqBasePointer);
Arrays.fill(proxPointer, proxBasePointer);
Arrays.fill(payloadLength, 0);
Arrays.fill(offsetLength, 0);
}
/** Returns the freq pointer of the doc to which the last call of
@ -76,12 +82,20 @@ public class Lucene40SkipListReader extends MultiLevelSkipListReader {
return lastPayloadLength;
}
/** Returns the offset length (endOffset-startOffset) of the position stored just before
* the doc to which the last call of {@link MultiLevelSkipListReader#skipTo(int)}
* has skipped. */
public int getOffsetLength() {
return lastOffsetLength;
}
@Override
protected void seekChild(int level) throws IOException {
super.seekChild(level);
freqPointer[level] = lastFreqPointer;
proxPointer[level] = lastProxPointer;
payloadLength[level] = lastPayloadLength;
offsetLength[level] = lastOffsetLength;
}
@Override
@ -90,6 +104,7 @@ public class Lucene40SkipListReader extends MultiLevelSkipListReader {
lastFreqPointer = freqPointer[level];
lastProxPointer = proxPointer[level];
lastPayloadLength = payloadLength[level];
lastOffsetLength = offsetLength[level];
}
@ -110,6 +125,11 @@ public class Lucene40SkipListReader extends MultiLevelSkipListReader {
} else {
delta = skipStream.readVInt();
}
if (currentFieldStoresOffsets) {
offsetLength[level] = skipStream.readVInt();
}
freqPointer[level] += skipStream.readVInt();
proxPointer[level] += skipStream.readVInt();

View File

@ -40,7 +40,9 @@ public class Lucene40SkipListWriter extends MultiLevelSkipListWriter {
private int curDoc;
private boolean curStorePayloads;
private boolean curStoreOffsets;
private int curPayloadLength;
private int curOffsetLength;
private long curFreqPointer;
private long curProxPointer;
@ -58,10 +60,12 @@ public class Lucene40SkipListWriter extends MultiLevelSkipListWriter {
/**
* Sets the values for the current skip data.
*/
public void setSkipData(int doc, boolean storePayloads, int payloadLength) {
public void setSkipData(int doc, boolean storePayloads, int payloadLength, boolean storeOffsets, int offsetLength) {
this.curDoc = doc;
this.curStorePayloads = storePayloads;
this.curPayloadLength = payloadLength;
this.curStoreOffsets = storeOffsets;
this.curOffsetLength = offsetLength;
this.curFreqPointer = freqOutput.getFilePointer();
if (proxOutput != null)
this.curProxPointer = proxOutput.getFilePointer();
@ -116,6 +120,12 @@ public class Lucene40SkipListWriter extends MultiLevelSkipListWriter {
// current field does not store payloads
skipBuffer.writeVInt(curDoc - lastSkipDoc[level]);
}
// TODO: not sure it really helps to shove this somewhere else if its the same as the last skip
if (curStoreOffsets) {
skipBuffer.writeVInt(curOffsetLength);
}
skipBuffer.writeVInt((int) (curFreqPointer - lastSkipFreqPointer[level]));
skipBuffer.writeVInt((int) (curProxPointer - lastSkipProxPointer[level]));

View File

@ -17,6 +17,7 @@ package org.apache.lucene.analysis;
* limitations under the License.
*/
import java.io.Reader;
import java.io.StringReader;
import java.io.IOException;
import java.util.ArrayList;
@ -289,8 +290,12 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
}
}
};
public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength) throws IOException {
checkRandomData(random, a, iterations, maxWordLength, random.nextBoolean());
}
public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean useCharFilter) throws IOException {
for (int i = 0; i < iterations; i++) {
String text;
switch(_TestUtil.nextInt(random, 0, 4)) {
@ -311,7 +316,9 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
System.out.println("NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
}
TokenStream ts = a.tokenStream("dummy", new StringReader(text));
int remainder = random.nextInt(10);
Reader reader = new StringReader(text);
TokenStream ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
assertTrue("has no CharTermAttribute", ts.hasAttribute(CharTermAttribute.class));
CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
OffsetAttribute offsetAtt = ts.hasAttribute(OffsetAttribute.class) ? ts.getAttribute(OffsetAttribute.class) : null;
@ -339,30 +346,38 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
if (VERBOSE) {
System.out.println("NOTE: BaseTokenStreamTestCase: re-run analysis");
}
reader = new StringReader(text);
ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
if (typeAtt != null && posIncAtt != null && offsetAtt != null) {
// offset + pos + type
assertAnalyzesToReuse(a, text,
assertTokenStreamContents(ts,
tokens.toArray(new String[tokens.size()]),
toIntArray(startOffsets),
toIntArray(endOffsets),
types.toArray(new String[types.size()]),
toIntArray(positions));
toIntArray(positions),
text.length());
} else if (posIncAtt != null && offsetAtt != null) {
// offset + pos
assertAnalyzesToReuse(a, text,
assertTokenStreamContents(ts,
tokens.toArray(new String[tokens.size()]),
toIntArray(startOffsets),
toIntArray(endOffsets),
toIntArray(positions));
null,
toIntArray(positions),
text.length());
} else if (offsetAtt != null) {
// offset
assertAnalyzesToReuse(a, text,
assertTokenStreamContents(ts,
tokens.toArray(new String[tokens.size()]),
toIntArray(startOffsets),
toIntArray(endOffsets));
toIntArray(endOffsets),
null,
null,
text.length());
} else {
// terms only
assertAnalyzesToReuse(a, text,
assertTokenStreamContents(ts,
tokens.toArray(new String[tokens.size()]));
}
}

View File

@ -0,0 +1,100 @@
package org.apache.lucene.analysis;
/**
* 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.
*/
import java.io.IOException;
import java.io.Reader;
import java.util.SortedMap;
import java.util.TreeMap;
// the purpose of this charfilter is to send offsets out of bounds
// if the analyzer doesn't use correctOffset or does incorrect offset math.
class MockCharFilter extends CharStream {
final Reader in;
final int remainder;
// for testing only
public MockCharFilter(Reader in, int remainder) {
this.in = in;
this.remainder = remainder;
assert remainder >= 0 && remainder < 10 : "invalid parameter";
}
@Override
public void close() throws IOException {
in.close();
}
int currentOffset = -1;
int delta = 0;
int bufferedCh = -1;
@Override
public int read() throws IOException {
// we have a buffered character, add an offset correction and return it
if (bufferedCh >= 0) {
int ch = bufferedCh;
bufferedCh = -1;
currentOffset++;
addOffCorrectMap(currentOffset+delta, delta-1);
delta--;
return ch;
}
// otherwise actually read one
int ch = in.read();
if (ch < 0)
return ch;
currentOffset++;
if ((ch % 10) != remainder || Character.isHighSurrogate((char)ch) || Character.isLowSurrogate((char)ch)) {
return ch;
}
// we will double this character, so buffer it.
bufferedCh = ch;
return ch;
}
@Override
public int read(char[] cbuf, int off, int len) throws IOException {
int numRead = 0;
for (int i = off; i < off + len; i++) {
int c = read();
if (c == -1) break;
cbuf[i] = (char) c;
numRead++;
}
return numRead == 0 ? -1 : numRead;
}
@Override
public int correctOffset(int currentOff) {
SortedMap<Integer,Integer> subMap = corrections.subMap(0, currentOff+1);
int ret = subMap.isEmpty() ? currentOff : currentOff + subMap.get(subMap.lastKey());
assert ret >= 0 : "currentOff=" + currentOff + ",diff=" + (ret-currentOff);
return ret;
}
protected void addOffCorrectMap(int off, int cumulativeDiff) {
corrections.put(off, cumulativeDiff);
}
TreeMap<Integer,Integer> corrections = new TreeMap<Integer,Integer>();
}

View File

@ -137,7 +137,7 @@ class PreFlexRWFieldsWriter extends FieldsConsumer {
}
if ((++df % termsOut.skipInterval) == 0) {
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength, false, 0);
skipListWriter.bufferSkip(df);
}

View File

@ -0,0 +1,58 @@
package org.apache.lucene.analysis;
import java.io.IOException;
import java.io.Reader;
/**
* 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.
*/
public class TestMockCharFilter extends BaseTokenStreamTestCase {
public void test() throws IOException {
Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, tokenizer);
}
@Override
protected Reader initReader(Reader reader) {
return new MockCharFilter(CharReader.get(reader), 7);
}
};
assertAnalyzesTo(analyzer, "ab",
new String[] { "aab" },
new int[] { 0 },
new int[] { 2 }
);
assertAnalyzesTo(analyzer, "aba",
new String[] { "aabaa" },
new int[] { 0 },
new int[] { 3 }
);
assertAnalyzesTo(analyzer, "abcdefga",
new String[] { "aabcdefgaa" },
new int[] { 0 },
new int[] { 8 }
);
}
}

View File

@ -22,29 +22,46 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.CannedAnalyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockPayloadAnalyzer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.NumericField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.English;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.Assume;
import org.apache.lucene.util._TestUtil;
public class TestPostingsOffsets extends LuceneTestCase {
IndexWriterConfig iwc;
public void setUp() throws Exception {
super.setUp();
// Currently only SimpleText and Lucene40 can index offsets into postings:
assumeTrue("codec does not support offsets", Codec.getDefault().getName().equals("SimpleText") || Codec.getDefault().getName().equals("Lucene40"));
iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
if (Codec.getDefault().getName().equals("Lucene40")) {
// pulsing etc are not implemented
iwc.setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat()));
}
}
public void testBasic() throws Exception {
// Currently only SimpleText can index offsets into postings:
Assume.assumeTrue(Codec.getDefault().getName().equals("SimpleText"));
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random, dir);
RandomIndexWriter w = new RandomIndexWriter(random, dir, iwc);
Document doc = new Document();
FieldType ft = new FieldType(TextField.TYPE_UNSTORED);
@ -94,16 +111,117 @@ public class TestPostingsOffsets extends LuceneTestCase {
r.close();
dir.close();
}
public void testSkipping() throws Exception {
doTestNumbers(false);
}
public void testPayloads() throws Exception {
doTestNumbers(true);
}
public void doTestNumbers(boolean withPayloads) throws Exception {
Directory dir = newDirectory();
Analyzer analyzer = withPayloads ? new MockPayloadAnalyzer() : new MockAnalyzer(random);
iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
if (Codec.getDefault().getName().equals("Lucene40")) {
// pulsing etc are not implemented
iwc.setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat()));
}
iwc.setMergePolicy(newLogMergePolicy()); // will rely on docids a bit for skipping
RandomIndexWriter w = new RandomIndexWriter(random, dir, iwc);
FieldType ft = new FieldType(TextField.TYPE_STORED);
ft.setIndexOptions(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
if (random.nextBoolean()) {
ft.setStoreTermVectors(true);
ft.setStoreTermVectorOffsets(random.nextBoolean());
ft.setStoreTermVectorPositions(random.nextBoolean());
}
int numDocs = atLeast(500);
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
doc.add(new Field("numbers", English.intToEnglish(i), ft));
doc.add(new Field("oddeven", (i % 2) == 0 ? "even" : "odd", ft));
doc.add(new StringField("id", "" + i));
w.addDocument(doc);
}
IndexReader reader = w.getReader();
w.close();
String terms[] = { "one", "two", "three", "four", "five", "six", "seven", "eight", "nine", "ten", "hundred" };
for (String term : terms) {
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef(term), true);
int doc;
while((doc = dp.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
String storedNumbers = reader.document(doc).get("numbers");
int freq = dp.freq();
for (int i = 0; i < freq; i++) {
dp.nextPosition();
int start = dp.startOffset();
assert start >= 0;
int end = dp.endOffset();
assert end >= 0 && end >= start;
// check that the offsets correspond to the term in the src text
assertTrue(storedNumbers.substring(start, end).equals(term));
if (withPayloads) {
// check that we have a payload and it starts with "pos"
assertTrue(dp.hasPayload());
BytesRef payload = dp.getPayload();
assertTrue(payload.utf8ToString().startsWith("pos:"));
} // note: withPayloads=false doesnt necessarily mean we dont have them from MockAnalyzer!
}
}
}
// check we can skip correctly
int numSkippingTests = atLeast(50);
for (int j = 0; j < numSkippingTests; j++) {
int num = _TestUtil.nextInt(random, 100, Math.min(numDocs-1, 999));
DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef("hundred"), true);
int doc = dp.advance(num);
assertEquals(num, doc);
int freq = dp.freq();
for (int i = 0; i < freq; i++) {
String storedNumbers = reader.document(doc).get("numbers");
dp.nextPosition();
int start = dp.startOffset();
assert start >= 0;
int end = dp.endOffset();
assert end >= 0 && end >= start;
// check that the offsets correspond to the term in the src text
assertTrue(storedNumbers.substring(start, end).equals("hundred"));
if (withPayloads) {
// check that we have a payload and it starts with "pos"
assertTrue(dp.hasPayload());
BytesRef payload = dp.getPayload();
assertTrue(payload.utf8ToString().startsWith("pos:"));
} // note: withPayloads=false doesnt necessarily mean we dont have them from MockAnalyzer!
}
}
// check that other fields (without offsets) work correctly
for (int i = 0; i < numDocs; i++) {
DocsEnum dp = MultiFields.getTermDocsEnum(reader, null, "id", new BytesRef("" + i), false);
assertEquals(i, dp.nextDoc());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, dp.nextDoc());
}
reader.close();
dir.close();
}
public void testRandom() throws Exception {
// Currently only SimpleText can index offsets into postings:
Assume.assumeTrue(Codec.getDefault().getName().equals("SimpleText"));
// token -> docID -> tokens
final Map<String,Map<Integer,List<Token>>> actualTokens = new HashMap<String,Map<Integer,List<Token>>>();
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random, dir);
RandomIndexWriter w = new RandomIndexWriter(random, dir, iwc);
final int numDocs = atLeast(20);
//final int numDocs = atLeast(5);

View File

@ -32,6 +32,7 @@ import org.apache.lucene.analysis.util.OpenStringBuilder;
/**
* A CharFilter that wraps another Reader and attempts to strip out HTML constructs.
*/
@SuppressWarnings("fallthrough")
%%
%unicode 6.0
@ -151,6 +152,7 @@ InlineElment = ( [aAbBiIqQsSuU] |
private static final char BR_END_TAG_REPLACEMENT = '\n';
private static final char SCRIPT_REPLACEMENT = '\n';
private static final char STYLE_REPLACEMENT = '\n';
private static final char REPLACEMENT_CHARACTER = '\uFFFD';
private CharArraySet escapedTags = null;
private int inputStart;
@ -350,6 +352,137 @@ InlineElment = ( [aAbBiIqQsSuU] |
yybegin(CHARACTER_REFERENCE_TAIL);
}
"#" { inputSegment.append('#'); yybegin(NUMERIC_CHARACTER); }
// 1 1 11 11
// 0 1 2 3 45 678 9 0 1 23 45
"#" [xX][dD][89aAbB][0-9a-fA-F]{2} ";&#" [xX][dD][c-fC-F][0-9a-fA-F]{2} ";" {
// Handle paired UTF-16 surrogates.
outputSegment = entitySegment;
outputSegment.clear();
String surrogatePair = yytext();
char highSurrogate = '\u0000';
try {
highSurrogate = (char)Integer.parseInt(surrogatePair.substring(2, 6), 16);
} catch(Exception e) { // should never happen
assert false: "Exception parsing high surrogate '"
+ surrogatePair.substring(2, 6) + "'";
}
try {
outputSegment.unsafeWrite
((char)Integer.parseInt(surrogatePair.substring(10, 14), 16));
} catch(Exception e) { // should never happen
assert false: "Exception parsing low surrogate '"
+ surrogatePair.substring(10, 14) + "'";
}
cumulativeDiff += inputSegment.length() + yylength() - 2;
addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
inputSegment.clear();
yybegin(YYINITIAL);
return highSurrogate;
}
// 1 1 11 11
// 01 2 345 678 9 0 1 23 45
"#5" [56] \d{3} ";&#" [xX][dD][c-fC-F][0-9a-fA-F]{2} ";" {
// Handle paired UTF-16 surrogates.
String surrogatePair = yytext();
char highSurrogate = '\u0000';
try { // High surrogates are in decimal range [55296, 56319]
highSurrogate = (char)Integer.parseInt(surrogatePair.substring(1, 6));
} catch(Exception e) { // should never happen
assert false: "Exception parsing high surrogate '"
+ surrogatePair.substring(1, 6) + "'";
}
if (Character.isHighSurrogate(highSurrogate)) {
outputSegment = entitySegment;
outputSegment.clear();
try {
outputSegment.unsafeWrite
((char)Integer.parseInt(surrogatePair.substring(10, 14), 16));
} catch(Exception e) { // should never happen
assert false: "Exception parsing low surrogate '"
+ surrogatePair.substring(10, 14) + "'";
}
cumulativeDiff += inputSegment.length() + yylength() - 2;
addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
inputSegment.clear();
yybegin(YYINITIAL);
return highSurrogate;
}
yypushback(surrogatePair.length() - 1); // Consume only '#'
inputSegment.append('#');
yybegin(NUMERIC_CHARACTER);
}
// 1 111 11
// 0 1 2 3 45 6789 0 123 45
"#" [xX][dD][89aAbB][0-9a-fA-F]{2} ";&#5" [67] \d{3} ";" {
// Handle paired UTF-16 surrogates.
String surrogatePair = yytext();
char highSurrogate = '\u0000';
char lowSurrogate = '\u0000';
try {
highSurrogate = (char)Integer.parseInt(surrogatePair.substring(2, 6), 16);
} catch(Exception e) { // should never happen
assert false: "Exception parsing high surrogate '"
+ surrogatePair.substring(2, 6) + "'";
}
try { // Low surrogates are in decimal range [56320, 57343]
lowSurrogate = (char)Integer.parseInt(surrogatePair.substring(9, 14));
} catch(Exception e) { // should never happen
assert false: "Exception parsing low surrogate '"
+ surrogatePair.substring(9, 14) + "'";
}
if (Character.isLowSurrogate(lowSurrogate)) {
outputSegment = entitySegment;
outputSegment.clear();
outputSegment.unsafeWrite(lowSurrogate);
cumulativeDiff += inputSegment.length() + yylength() - 2;
addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
inputSegment.clear();
yybegin(YYINITIAL);
return highSurrogate;
}
yypushback(surrogatePair.length() - 1); // Consume only '#'
inputSegment.append('#');
yybegin(NUMERIC_CHARACTER);
}
// 1 111 11
// 01 2 345 6789 0 123 45
"#5" [56] \d{3} ";&#5" [67] \d{3} ";" {
// Handle paired UTF-16 surrogates.
String surrogatePair = yytext();
char highSurrogate = '\u0000';
try { // High surrogates are in decimal range [55296, 56319]
highSurrogate = (char)Integer.parseInt(surrogatePair.substring(1, 6));
} catch(Exception e) { // should never happen
assert false: "Exception parsing high surrogate '"
+ surrogatePair.substring(1, 6) + "'";
}
if (Character.isHighSurrogate(highSurrogate)) {
char lowSurrogate = '\u0000';
try { // Low surrogates are in decimal range [56320, 57343]
lowSurrogate = (char)Integer.parseInt(surrogatePair.substring(9, 14));
} catch(Exception e) { // should never happen
assert false: "Exception parsing low surrogate '"
+ surrogatePair.substring(9, 14) + "'";
}
if (Character.isLowSurrogate(lowSurrogate)) {
outputSegment = entitySegment;
outputSegment.clear();
outputSegment.unsafeWrite(lowSurrogate);
cumulativeDiff += inputSegment.length() + yylength() - 2;
addOffCorrectMap(outputCharCount + 2, cumulativeDiff);
inputSegment.clear();
yybegin(YYINITIAL);
return highSurrogate;
}
}
yypushback(surrogatePair.length() - 1); // Consume only '#'
inputSegment.append('#');
yybegin(NUMERIC_CHARACTER);
}
}
<NUMERIC_CHARACTER> {
@ -359,25 +492,27 @@ InlineElment = ( [aAbBiIqQsSuU] |
if (matchLength <= 6) { // 10FFFF: max 6 hex chars
String hexCharRef
= new String(zzBuffer, zzStartRead + 1, matchLength - 1);
int codePoint = 0;
try {
int codePoint = Integer.parseInt(hexCharRef, 16);
if (codePoint <= 0x10FFFF) {
outputSegment = entitySegment;
outputSegment.clear();
codePoint = Integer.parseInt(hexCharRef, 16);
} catch(Exception e) {
assert false: "Exception parsing hex code point '" + hexCharRef + "'";
}
if (codePoint <= 0x10FFFF) {
outputSegment = entitySegment;
outputSegment.clear();
if (codePoint >= Character.MIN_SURROGATE
&& codePoint <= Character.MAX_SURROGATE) {
outputSegment.unsafeWrite(REPLACEMENT_CHARACTER);
} else {
outputSegment.setLength
(Character.toChars(codePoint, outputSegment.getArray(), 0));
yybegin(CHARACTER_REFERENCE_TAIL);
} else {
outputSegment = inputSegment;
yybegin(YYINITIAL);
return outputSegment.nextChar();
}
} catch(NumberFormatException e) {
assert false: "NumberFormatException parsing hex code point '"
+ hexCharRef + "'";
} catch(IllegalArgumentException e) {
assert false: "IllegalArgumentException getting chars "
+ "for hex code point '" + hexCharRef + "'";
yybegin(CHARACTER_REFERENCE_TAIL);
} else {
outputSegment = inputSegment;
yybegin(YYINITIAL);
return outputSegment.nextChar();
}
} else {
outputSegment = inputSegment;
@ -390,25 +525,27 @@ InlineElment = ( [aAbBiIqQsSuU] |
inputSegment.write(zzBuffer, zzStartRead, matchLength);
if (matchLength <= 7) { // 0x10FFFF = 1114111: max 7 decimal chars
String decimalCharRef = yytext();
int codePoint = 0;
try {
int codePoint = Integer.parseInt(decimalCharRef);
if (codePoint <= 0x10FFFF) {
outputSegment = entitySegment;
outputSegment.clear();
codePoint = Integer.parseInt(decimalCharRef);
} catch(Exception e) {
assert false: "Exception parsing code point '" + decimalCharRef + "'";
}
if (codePoint <= 0x10FFFF) {
outputSegment = entitySegment;
outputSegment.clear();
if (codePoint >= Character.MIN_SURROGATE
&& codePoint <= Character.MAX_SURROGATE) {
outputSegment.unsafeWrite(REPLACEMENT_CHARACTER);
} else {
outputSegment.setLength
(Character.toChars(codePoint, outputSegment.getArray(), 0));
yybegin(CHARACTER_REFERENCE_TAIL);
} else {
outputSegment = inputSegment;
yybegin(YYINITIAL);
return outputSegment.nextChar();
}
} catch(NumberFormatException e) {
assert false: "NumberFormatException parsing code point '"
+ decimalCharRef + "'";
} catch(IllegalArgumentException e) {
assert false: "IllegalArgumentException getting chars for code point '"
+ decimalCharRef + "'";
yybegin(CHARACTER_REFERENCE_TAIL);
} else {
outputSegment = inputSegment;
yybegin(YYINITIAL);
return outputSegment.nextChar();
}
} else {
outputSegment = inputSegment;

View File

@ -154,13 +154,22 @@ public abstract class CompoundWordTokenFilterBase extends TokenFilter {
/** Construct the compound token based on a slice of the current {@link CompoundWordTokenFilterBase#termAtt}. */
public CompoundToken(int offset, int length) {
final int newStart = CompoundWordTokenFilterBase.this.offsetAtt.startOffset() + offset;
this.txt = CompoundWordTokenFilterBase.this.termAtt.subSequence(offset, offset + length);
// TODO: This ignores the original endOffset, if a CharFilter/Tokenizer/Filter removed
// chars from the term, offsets may not match correctly (other filters producing tokens
// may also have this problem):
this.startOffset = newStart;
this.endOffset = newStart + length;
// offsets of the original word
int startOff = CompoundWordTokenFilterBase.this.offsetAtt.startOffset();
int endOff = CompoundWordTokenFilterBase.this.offsetAtt.endOffset();
if (endOff - startOff != CompoundWordTokenFilterBase.this.termAtt.length()) {
// if length by start + end offsets doesn't match the term text then assume
// this is a synonym and don't adjust the offsets.
this.startOffset = startOff;
this.endOffset = endOff;
} else {
final int newStart = startOff + offset;
this.startOffset = newStart;
this.endOffset = newStart + length;
}
}
}

View File

@ -60,6 +60,7 @@ public final class HyphenatedWordsFilter extends TokenFilter {
private final StringBuilder hyphenated = new StringBuilder();
private State savedState;
private boolean exhausted = false;
private int lastEndOffset = 0;
/**
* Creates a new HyphenatedWordsFilter
@ -78,6 +79,7 @@ public final class HyphenatedWordsFilter extends TokenFilter {
while (!exhausted && input.incrementToken()) {
char[] term = termAttribute.buffer();
int termLength = termAttribute.length();
lastEndOffset = offsetAttribute.endOffset();
if (termLength > 0 && term[termLength - 1] == '-') {
// a hyphenated word
@ -119,6 +121,7 @@ public final class HyphenatedWordsFilter extends TokenFilter {
hyphenated.setLength(0);
savedState = null;
exhausted = false;
lastEndOffset = 0;
}
// ================================================= Helper Methods ================================================
@ -127,8 +130,6 @@ public final class HyphenatedWordsFilter extends TokenFilter {
* Writes the joined unhyphenated term
*/
private void unhyphenate() {
int endOffset = offsetAttribute.endOffset();
restoreState(savedState);
savedState = null;
@ -140,7 +141,7 @@ public final class HyphenatedWordsFilter extends TokenFilter {
hyphenated.getChars(0, length, term, 0);
termAttribute.setLength(length);
offsetAttribute.setOffset(offsetAttribute.startOffset(), endOffset);
offsetAttribute.setOffset(offsetAttribute.startOffset(), lastEndOffset);
hyphenated.setLength(0);
}
}

View File

@ -183,31 +183,33 @@ public final class PatternAnalyzer extends Analyzer {
*
* @param fieldName
* the name of the field to tokenize (currently ignored).
* @param reader
* reader (e.g. charfilter) of the original text. can be null.
* @param text
* the string to tokenize
* @return a new token stream
*/
public TokenStreamComponents createComponents(String fieldName, String text) {
public TokenStreamComponents createComponents(String fieldName, Reader reader, String text) {
// Ideally the Analyzer superclass should have a method with the same signature,
// with a default impl that simply delegates to the StringReader flavour.
if (text == null)
throw new IllegalArgumentException("text must not be null");
if (pattern == NON_WORD_PATTERN) { // fast path
return new TokenStreamComponents(new FastStringTokenizer(text, true, toLowerCase, stopWords));
return new TokenStreamComponents(new FastStringTokenizer(reader, text, true, toLowerCase, stopWords));
} else if (pattern == WHITESPACE_PATTERN) { // fast path
return new TokenStreamComponents(new FastStringTokenizer(text, false, toLowerCase, stopWords));
return new TokenStreamComponents(new FastStringTokenizer(reader, text, false, toLowerCase, stopWords));
}
Tokenizer tokenizer = new PatternTokenizer(text, pattern, toLowerCase);
Tokenizer tokenizer = new PatternTokenizer(reader, text, pattern, toLowerCase);
TokenStream result = (stopWords != null) ? new StopFilter(matchVersion, tokenizer, stopWords) : tokenizer;
return new TokenStreamComponents(tokenizer, result);
}
/**
* Creates a token stream that tokenizes all the text in the given Reader;
* This implementation forwards to <code>tokenStream(String, String)</code> and is
* less efficient than <code>tokenStream(String, String)</code>.
* This implementation forwards to <code>tokenStream(String, Reader, String)</code> and is
* less efficient than <code>tokenStream(String, Reader, String)</code>.
*
* @param fieldName
* the name of the field to tokenize (currently ignored).
@ -219,7 +221,7 @@ public final class PatternAnalyzer extends Analyzer {
public TokenStreamComponents createComponents(String fieldName, Reader reader) {
try {
String text = toString(reader);
return createComponents(fieldName, text);
return createComponents(fieldName, reader, text);
} catch (IOException e) {
throw new RuntimeException(e);
}
@ -332,7 +334,8 @@ public final class PatternAnalyzer extends Analyzer {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
public PatternTokenizer(String str, Pattern pattern, boolean toLowerCase) {
public PatternTokenizer(Reader input, String str, Pattern pattern, boolean toLowerCase) {
super(input);
this.pattern = pattern;
this.str = str;
this.matcher = pattern.matcher(str);
@ -359,7 +362,7 @@ public final class PatternAnalyzer extends Analyzer {
String text = str.substring(start, end);
if (toLowerCase) text = text.toLowerCase(locale);
termAtt.setEmpty().append(text);
offsetAtt.setOffset(start, end);
offsetAtt.setOffset(correctOffset(start), correctOffset(end));
return true;
}
if (!isMatch) return false;
@ -369,7 +372,7 @@ public final class PatternAnalyzer extends Analyzer {
@Override
public final void end() {
// set final offset
final int finalOffset = str.length();
final int finalOffset = correctOffset(str.length());
this.offsetAtt.setOffset(finalOffset, finalOffset);
}
@ -406,7 +409,8 @@ public final class PatternAnalyzer extends Analyzer {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
public FastStringTokenizer(String str, boolean isLetter, boolean toLowerCase, Set<?> stopWords) {
public FastStringTokenizer(Reader input, String str, boolean isLetter, boolean toLowerCase, Set<?> stopWords) {
super(input);
this.str = str;
this.isLetter = isLetter;
this.toLowerCase = toLowerCase;
@ -458,7 +462,7 @@ public final class PatternAnalyzer extends Analyzer {
return false;
}
termAtt.setEmpty().append(text);
offsetAtt.setOffset(start, i);
offsetAtt.setOffset(correctOffset(start), correctOffset(i));
return true;
}
@ -466,7 +470,7 @@ public final class PatternAnalyzer extends Analyzer {
public final void end() {
// set final offset
final int finalOffset = str.length();
this.offsetAtt.setOffset(finalOffset, finalOffset);
this.offsetAtt.setOffset(correctOffset(finalOffset), correctOffset(finalOffset));
}
private boolean isTokenChar(char c, boolean isLetter) {
@ -479,6 +483,7 @@ public final class PatternAnalyzer extends Analyzer {
@Override
public void reset(Reader input) throws IOException {
super.reset(input);
this.str = PatternAnalyzer.toString(input);
}

View File

@ -68,7 +68,7 @@ public final class TrimFilter extends TokenFilter {
} else {
termAtt.setEmpty();
}
if (updateOffsets) {
if (updateOffsets && len == offsetAtt.endOffset() - offsetAtt.startOffset()) {
int newStart = offsetAtt.startOffset()+start;
int newEnd = offsetAtt.endOffset() - (start<end ? endOff:0);
offsetAtt.setOffset(newStart, newEnd);

View File

@ -405,10 +405,20 @@ public final class WordDelimiterFilter extends TokenFilter {
clearAttributes();
termAttribute.copyBuffer(savedBuffer, iterator.current, iterator.end - iterator.current);
int startOffSet = (isSingleWord || !hasIllegalOffsets) ? savedStartOffset + iterator.current : savedStartOffset;
int endOffSet = (hasIllegalOffsets) ? savedEndOffset : savedStartOffset + iterator.end;
offsetAttribute.setOffset(startOffSet, endOffSet);
int startOffset = savedStartOffset + iterator.current;
int endOffset = savedStartOffset + iterator.end;
if (hasIllegalOffsets) {
// historically this filter did this regardless for 'isSingleWord',
// but we must do a sanity check:
if (isSingleWord && startOffset <= savedEndOffset) {
offsetAttribute.setOffset(startOffset, savedEndOffset);
} else {
offsetAttribute.setOffset(savedStartOffset, savedEndOffset);
}
} else {
offsetAttribute.setOffset(startOffset, endOffset);
}
posIncAttribute.setPositionIncrement(position(false));
typeAttribute.setType(savedType);
}

View File

@ -68,6 +68,7 @@ public final class ThaiWordFilter extends TokenFilter {
private CharTermAttribute clonedTermAtt = null;
private OffsetAttribute clonedOffsetAtt = null;
private boolean hasMoreTokensInClone = false;
private boolean hasIllegalOffsets = false; // only if the length changed before this filter
/** Creates a new ThaiWordFilter with the specified match version. */
public ThaiWordFilter(Version matchVersion, TokenStream input) {
@ -86,7 +87,11 @@ public final class ThaiWordFilter extends TokenFilter {
if (end != BreakIterator.DONE) {
clonedToken.copyTo(this);
termAtt.copyBuffer(clonedTermAtt.buffer(), start, end - start);
offsetAtt.setOffset(clonedOffsetAtt.startOffset() + start, clonedOffsetAtt.startOffset() + end);
if (hasIllegalOffsets) {
offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.endOffset());
} else {
offsetAtt.setOffset(clonedOffsetAtt.startOffset() + start, clonedOffsetAtt.startOffset() + end);
}
if (handlePosIncr) posAtt.setPositionIncrement(1);
return true;
}
@ -102,6 +107,10 @@ public final class ThaiWordFilter extends TokenFilter {
}
hasMoreTokensInClone = true;
// if length by start + end offsets doesn't match the term text then assume
// this is a synonym and don't adjust the offsets.
hasIllegalOffsets = offsetAtt.endOffset() - offsetAtt.startOffset() != termAtt.length();
// we lazy init the cloned token, as in ctor not all attributes may be added
if (clonedToken == null) {
@ -118,7 +127,11 @@ public final class ThaiWordFilter extends TokenFilter {
int end = breaker.next();
if (end != BreakIterator.DONE) {
termAtt.setLength(end);
offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.startOffset() + end);
if (hasIllegalOffsets) {
offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.endOffset());
} else {
offsetAtt.setOffset(clonedOffsetAtt.startOffset(), clonedOffsetAtt.startOffset() + end);
}
// position increment keeps as it is for first token
return true;
}

View File

@ -306,13 +306,14 @@ public final class WikipediaTokenizer extends Tokenizer {
@Override
public void reset() throws IOException {
super.reset();
scanner.yyreset(input);
tokens = null;
scanner.reset();
}
@Override
public void reset(Reader reader) throws IOException {
super.reset(reader);
reset();
scanner.yyreset(input);
}
@Override

View File

@ -1,4 +1,4 @@
/* The following code was generated by JFlex 1.5.0-SNAPSHOT on 9/30/11 12:11 PM */
/* The following code was generated by JFlex 1.5.0-SNAPSHOT on 1/22/12 10:26 PM */
package org.apache.lucene.analysis.wikipedia;
@ -25,8 +25,8 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
/**
* This class is a scanner generated by
* <a href="http://www.jflex.de/">JFlex</a> 1.5.0-SNAPSHOT
* on 9/30/11 12:11 PM from the specification file
* <tt>/lucene/jflex/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex</tt>
* on 1/22/12 10:26 PM from the specification file
* <tt>/home/rmuir/workspace/lucene-clean-trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/wikipedia/WikipediaTokenizerImpl.jflex</tt>
*/
class WikipediaTokenizerImpl {
@ -498,6 +498,14 @@ final int setText(StringBuilder buffer){
return length;
}
final void reset() {
currentTokType = 0;
numBalanced = 0;
positionInc = 1;
numLinkToks = 0;
numWikiTokensSeen = 0;
}

View File

@ -91,6 +91,14 @@ final int setText(StringBuilder buffer){
return length;
}
final void reset() {
currentTokType = 0;
numBalanced = 0;
positionInc = 1;
numLinkToks = 0;
numWikiTokensSeen = 0;
}
%}

View File

@ -503,7 +503,7 @@ public class HTMLStripCharFilterTest extends BaseTokenStreamTestCase {
@Override
protected Reader initReader(Reader reader) {
return new HTMLStripCharFilter(CharReader.get(new BufferedReader(reader)));
return new HTMLStripCharFilter(CharReader.get(reader));
}
};
@ -817,4 +817,47 @@ public class HTMLStripCharFilterTest extends BaseTokenStreamTestCase {
(CharReader.get(new StringReader(text.toString())));
while (reader.read() != -1);
}
public void testUTF16Surrogates() throws Exception {
Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, tokenizer);
}
@Override
protected Reader initReader(Reader reader) {
return new HTMLStripCharFilter(CharReader.get(new BufferedReader(reader)));
}
};
// Paired surrogates
assertAnalyzesTo(analyzer, " one two &#xD86C;&#XdC01;three",
new String[] { "one", "two", "\uD86C\uDC01three" } );
assertAnalyzesTo(analyzer, " &#55404;&#XdC01;", new String[] { "\uD86C\uDC01" } );
assertAnalyzesTo(analyzer, " &#xD86C;&#56321;", new String[] { "\uD86C\uDC01" } );
assertAnalyzesTo(analyzer, " &#55404;&#56321;", new String[] { "\uD86C\uDC01" } );
// Improperly paired surrogates
assertAnalyzesTo(analyzer, " &#55404;&#57999;", new String[] { "\uFFFD\uE28F" } );
assertAnalyzesTo(analyzer, " &#xD86C;&#57999;", new String[] { "\uFFFD\uE28F" } );
assertAnalyzesTo(analyzer, " &#55002;&#XdC01;", new String[] { "\uD6DA\uFFFD" } );
assertAnalyzesTo(analyzer, " &#55002;&#56321;", new String[] { "\uD6DA\uFFFD" } );
// Unpaired high surrogates
assertAnalyzesTo(analyzer, " &#Xd921;", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#Xd921", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#Xd921<br>", new String[] { "&#Xd921" } );
assertAnalyzesTo(analyzer, " &#55528;", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#55528", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#55528<br>", new String[] { "&#55528" } );
// Unpaired low surrogates
assertAnalyzesTo(analyzer, " &#xdfdb;", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#xdfdb", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#xdfdb<br>", new String[] { "&#xdfdb" } );
assertAnalyzesTo(analyzer, " &#57209;", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#57209", new String[] { "\uFFFD" } );
assertAnalyzesTo(analyzer, " &#57209<br>", new String[] { "&#57209" } );
}
}

View File

@ -117,5 +117,10 @@ public class TestChineseTokenizer extends BaseTokenStreamTestCase
assertAnalyzesTo(justFilter, "This is a Test. b c d",
new String[] { "This", "Test." });
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
checkRandomData(random, new ChineseAnalyzer(), 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -306,4 +306,31 @@ public class CommonGramsFilterTest extends BaseTokenStreamTestCase {
TokenFilter nsf = new CommonGramsQueryFilter(cgf);
assertTokenStreamContents(nsf, new String[] { "the_of" });
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
CommonGramsFilter cgf = new CommonGramsFilter(TEST_VERSION_CURRENT, t, commonWords);
return new TokenStreamComponents(t, cgf);
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
Analyzer b = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer t = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
CommonGramsFilter cgf = new CommonGramsFilter(TEST_VERSION_CURRENT, t, commonWords);
return new TokenStreamComponents(t, new CommonGramsQueryFilter(cgf));
}
};
checkRandomData(random, b, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -18,14 +18,19 @@ package org.apache.lucene.analysis.compound;
*/
import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
import java.util.Arrays;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.CharReader;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.charfilter.MappingCharFilter;
import org.apache.lucene.analysis.charfilter.NormalizeCharMap;
import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
import org.apache.lucene.analysis.util.CharArraySet;
@ -299,5 +304,61 @@ public class TestCompoundWordTokenFilter extends BaseTokenStreamTestCase {
}
}
}
// SOLR-2891
// *CompoundWordTokenFilter blindly adds term length to offset, but this can take things out of bounds
// wrt original text if a previous filter increases the length of the word (in this case ü -> ue)
// so in this case we behave like WDF, and preserve any modified offsets
public void testInvalidOffsets() throws Exception {
final CharArraySet dict = makeDictionary("fall");
final NormalizeCharMap normMap = new NormalizeCharMap();
normMap.add("ü", "ue");
Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
TokenFilter filter = new DictionaryCompoundWordTokenFilter(TEST_VERSION_CURRENT, tokenizer, dict);
return new TokenStreamComponents(tokenizer, filter);
}
@Override
protected Reader initReader(Reader reader) {
return new MappingCharFilter(normMap, CharReader.get(reader));
}
};
assertAnalyzesTo(analyzer, "banküberfall",
new String[] { "bankueberfall", "fall" },
new int[] { 0, 0 },
new int[] { 12, 12 });
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
final CharArraySet dict = makeDictionary("a", "e", "i", "o", "u", "y", "bc", "def");
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new DictionaryCompoundWordTokenFilter(TEST_VERSION_CURRENT, tokenizer, dict));
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
InputSource is = new InputSource(getClass().getResource("da_UTF8.xml").toExternalForm());
final HyphenationTree hyphenator = HyphenationCompoundWordTokenFilter.getHyphenationTree(is);
Analyzer b = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
TokenFilter filter = new HyphenationCompoundWordTokenFilter(TEST_VERSION_CURRENT, tokenizer, hyphenator);
return new TokenStreamComponents(tokenizer, filter);
}
};
checkRandomData(random, b, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -18,12 +18,15 @@ package org.apache.lucene.analysis.hunspell;
import java.io.IOException;
import java.io.InputStream;
import java.io.Reader;
import java.io.StringReader;
import java.text.ParseException;
import java.util.Arrays;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.junit.BeforeClass;
@ -57,4 +60,17 @@ public class HunspellStemFilterTest extends BaseTokenStreamTestCase {
filter = new HunspellStemFilter(new KeywordMarkerFilter(tokenizer, set), DICTIONARY);
assertTokenStreamContents(filter, new String[]{"lucene", "is", "awesome"}, new int[] {1, 1, 1});
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new HunspellStemFilter(tokenizer, DICTIONARY));
}
};
checkRandomData(random, analyzer, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -22,6 +22,7 @@ import java.io.StringReader;
import java.util.Arrays;
import java.util.regex.Pattern;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.core.StopAnalyzer;
@ -132,4 +133,10 @@ public class PatternAnalyzerTest extends BaseTokenStreamTestCase {
TokenStream ts2 = analyzer.tokenStream("dummy", new StringReader(document));
assertTokenStreamContents(ts2, expected);
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
Analyzer a = new PatternAnalyzer(TEST_VERSION_CURRENT, Pattern.compile(","), true, StopAnalyzer.ENGLISH_STOP_WORDS_SET);
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -17,11 +17,14 @@ package org.apache.lucene.analysis.miscellaneous;
* limitations under the License.
*/
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import java.io.Reader;
import java.io.StringReader;
import java.util.List;
import java.util.ArrayList;
@ -1907,4 +1910,17 @@ public class TestASCIIFoldingFilter extends BaseTokenStreamTestCase {
assertTrue(stream.incrementToken());
assertEquals(expected, termAtt.toString());
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new ASCIIFoldingFilter(tokenizer));
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -18,12 +18,14 @@
package org.apache.lucene.analysis.miscellaneous;
import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Tokenizer;
@ -117,4 +119,18 @@ public class TestCapitalizationFilter extends BaseTokenStreamTestCase {
new String[] { expected }, onlyFirstWord, keep, forceFirstLetter, okPrefix,
minWordLength, maxWordCount, maxTokenLength);
}
/** blast some random strings through the analyzer */
public void testRandomString() throws Exception {
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new CapitalizationFilter(tokenizer));
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -17,11 +17,14 @@
package org.apache.lucene.analysis.miscellaneous;
import java.io.Reader;
import java.io.StringReader;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
/**
* HyphenatedWordsFilter test
@ -46,5 +49,29 @@ public class TestHyphenatedWordsFilter extends BaseTokenStreamTestCase {
ts = new HyphenatedWordsFilter(ts);
assertTokenStreamContents(ts,
new String[] { "ecological", "develop", "comprehensive-hands-on", "and", "ecology-" });
}
}
public void testOffsets() throws Exception {
String input = "abc- def geh 1234- 5678-";
TokenStream ts = new MockTokenizer(new StringReader(input), MockTokenizer.WHITESPACE, false);
ts = new HyphenatedWordsFilter(ts);
assertTokenStreamContents(ts,
new String[] { "abcdef", "geh", "12345678-" },
new int[] { 0, 9, 13 },
new int[] { 8, 12, 24 });
}
/** blast some random strings through the analyzer */
public void testRandomString() throws Exception {
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new HyphenatedWordsFilter(tokenizer));
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -17,13 +17,16 @@
package org.apache.lucene.analysis.miscellaneous;
import java.io.Reader;
import java.io.StringReader;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.util.CharArraySet;
/** Test {@link KeepWordFilter} */
@ -57,4 +60,23 @@ public class TestKeepWordFilter extends BaseTokenStreamTestCase {
stream = new KeepWordFilter(false, stream, new CharArraySet(TEST_VERSION_CURRENT,words, false));
assertTokenStreamContents(stream, new String[] { "aaa" }, new int[] { 1 });
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
final Set<String> words = new HashSet<String>();
words.add( "a" );
words.add( "b" );
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
TokenStream stream = new KeepWordFilter(true, tokenizer, new CharArraySet(TEST_VERSION_CURRENT, words, true));
return new TokenStreamComponents(tokenizer, stream);
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -17,13 +17,21 @@
package org.apache.lucene.analysis.miscellaneous;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.synonym.SynonymFilter;
import org.apache.lucene.analysis.synonym.SynonymMap;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util._TestUtil;
import java.io.Reader;
import java.util.Iterator;
import java.util.Arrays;
@ -116,6 +124,45 @@ public class TestRemoveDuplicatesTokenFilter extends BaseTokenStreamTestCase {
}
// some helper methods for the below test with synonyms
private String randomNonEmptyString() {
while(true) {
final String s = _TestUtil.randomUnicodeString(random).trim();
if (s.length() != 0 && s.indexOf('\u0000') == -1) {
return s;
}
}
}
private void add(SynonymMap.Builder b, String input, String output, boolean keepOrig) {
b.add(new CharsRef(input.replaceAll(" +", "\u0000")),
new CharsRef(output.replaceAll(" +", "\u0000")),
keepOrig);
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
final int numIters = atLeast(10);
for (int i = 0; i < numIters; i++) {
SynonymMap.Builder b = new SynonymMap.Builder(random.nextBoolean());
final int numEntries = atLeast(10);
for (int j = 0; j < numEntries; j++) {
add(b, randomNonEmptyString(), randomNonEmptyString(), random.nextBoolean());
}
final SynonymMap map = b.build();
final boolean ignoreCase = random.nextBoolean();
final Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
TokenStream stream = new SynonymFilter(tokenizer, map, ignoreCase);
return new TokenStreamComponents(tokenizer, new RemoveDuplicatesTokenFilter(stream));
}
};
checkRandomData(random, analyzer, 1000*RANDOM_MULTIPLIER);
}
}
}

View File

@ -18,11 +18,15 @@
package org.apache.lucene.analysis.miscellaneous;
import java.io.IOException;
import java.io.Reader;
import java.util.Collection;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.tokenattributes.*;
/**
@ -103,4 +107,27 @@ public class TestTrimFilter extends BaseTokenStreamTestCase {
}
}
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.KEYWORD, false);
return new TokenStreamComponents(tokenizer, new TrimFilter(tokenizer, false));
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
Analyzer b = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.KEYWORD, false);
return new TokenStreamComponents(tokenizer, new TrimFilter(tokenizer, true));
}
};
checkRandomData(random, b, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -298,4 +298,28 @@ public class TestWordDelimiterFilter extends BaseTokenStreamTestCase {
new int[] { 10, 15, 15 },
new int[] { 2, 1, 0 });
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
int numIterations = atLeast(5);
for (int i = 0; i < numIterations; i++) {
final int flags = random.nextInt(512);
final CharArraySet protectedWords;
if (random.nextBoolean()) {
protectedWords = new CharArraySet(TEST_VERSION_CURRENT, new HashSet<String>(Arrays.asList("a", "b", "cd")), false);
} else {
protectedWords = null;
}
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new WordDelimiterFilter(tokenizer, flags, protectedWords));
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
}
}
}

View File

@ -160,7 +160,7 @@ public class TestSegmentingTokenizerBase extends BaseTokenStreamTestCase {
hasSentence = false;
clearAttributes();
termAtt.copyBuffer(buffer, sentenceStart, sentenceEnd-sentenceStart);
offsetAtt.setOffset(offset+sentenceStart, offset+sentenceEnd);
offsetAtt.setOffset(correctOffset(offset+sentenceStart), correctOffset(offset+sentenceEnd));
return true;
} else {
return false;
@ -215,7 +215,7 @@ public class TestSegmentingTokenizerBase extends BaseTokenStreamTestCase {
clearAttributes();
termAtt.copyBuffer(buffer, wordStart, wordEnd-wordStart);
offsetAtt.setOffset(offset+wordStart, offset+wordEnd);
offsetAtt.setOffset(correctOffset(offset+wordStart), correctOffset(offset+wordEnd));
posIncAtt.setPositionIncrement(posIncAtt.getPositionIncrement() + posBoost);
posBoost = 0;
return true;

View File

@ -18,12 +18,15 @@
package org.apache.lucene.analysis.wikipedia;
import java.io.Reader;
import java.io.StringReader;
import java.io.IOException;
import java.util.Set;
import java.util.HashSet;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
import static org.apache.lucene.analysis.wikipedia.WikipediaTokenizer.*;
@ -169,4 +172,17 @@ public class WikipediaTokenizerTest extends BaseTokenStreamTestCase {
assertFalse(tf.incrementToken());
tf.close();
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new WikipediaTokenizer(reader);
return new TokenStreamComponents(tokenizer, tokenizer);
}
};
checkRandomData(random, a, 10000*RANDOM_MULTIPLIER);
}
}

View File

@ -111,7 +111,7 @@ public final class ICUTokenizer extends Tokenizer {
@Override
public void end() throws IOException {
final int finalOffset = (length < 0) ? offset : offset + length;
offsetAtt.setOffset(finalOffset, finalOffset);
offsetAtt.setOffset(correctOffset(finalOffset), correctOffset(finalOffset));
}
/*

View File

@ -102,4 +102,9 @@ public class TestMorfologikAnalyzer extends BaseTokenStreamTestCase {
assertPOSToken(ts, "list", "subst:sg:loc.voc:m3");
assertPOSToken(ts, "lista", "subst:sg:dat.loc:f");
}
/** blast some random strings through the analyzer */
public void testRandom() throws Exception {
checkRandomData(random, getTestAnalyzer(), 10000 * RANDOM_MULTIPLIER);
}
}

View File

@ -16,11 +16,17 @@
*/
package org.apache.lucene.analysis.phonetic;
import java.io.Reader;
import java.io.StringReader;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.Analyzer.TokenStreamComponents;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
import org.apache.lucene.util._TestUtil;
public class DoubleMetaphoneFilterTest extends BaseTokenStreamTestCase {
@ -65,4 +71,28 @@ public class DoubleMetaphoneFilterTest extends BaseTokenStreamTestCase {
assertTokenStreamContents(filter, new String[] { "12345", "#$%@#^%&", "HL" });
}
public void testRandom() throws Exception {
final int codeLen = _TestUtil.nextInt(random, 1, 8);
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new DoubleMetaphoneFilter(tokenizer, codeLen, false));
}
};
checkRandomData(random, a, 1000 * RANDOM_MULTIPLIER);
Analyzer b = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new DoubleMetaphoneFilter(tokenizer, codeLen, true));
}
};
checkRandomData(random, b, 1000 * RANDOM_MULTIPLIER);
}
}

View File

@ -17,6 +17,8 @@
package org.apache.lucene.analysis.phonetic;
import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
import org.apache.commons.codec.Encoder;
@ -25,7 +27,9 @@ import org.apache.commons.codec.language.DoubleMetaphone;
import org.apache.commons.codec.language.Metaphone;
import org.apache.commons.codec.language.RefinedSoundex;
import org.apache.commons.codec.language.Soundex;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
@ -70,4 +74,33 @@ public class TestPhoneticFilter extends BaseTokenStreamTestCase {
PhoneticFilter filter = new PhoneticFilter(tokenizer, encoder, inject);
assertTokenStreamContents(filter, expected);
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws IOException {
Encoder encoders[] = new Encoder[] {
new Metaphone(), new DoubleMetaphone(), new Soundex(), new RefinedSoundex(), new Caverphone()
};
for (final Encoder e : encoders) {
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new PhoneticFilter(tokenizer, e, false));
}
};
checkRandomData(random, a, 1000*RANDOM_MULTIPLIER);
Analyzer b = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.WHITESPACE, false);
return new TokenStreamComponents(tokenizer, new PhoneticFilter(tokenizer, e, false));
}
};
checkRandomData(random, b, 1000*RANDOM_MULTIPLIER);
}
}
}

View File

@ -450,6 +450,10 @@ New Features
* SOLR-1709: Distributed support for Date and Numeric Range Faceting
(Peter Sturge, David Smiley, hossman, Simon Willnauer)
* SOLR-3054, LUCENE-3671: Add TypeTokenFilterFactory that creates TypeTokenFilter
that filters tokens based on their TypeAttribute. (Tommaso Teofili via
Uwe Schindler)
Optimizations
----------------------
* SOLR-1931: Speedup for LukeRequestHandler and admin/schema browser. New parameter
@ -509,6 +513,11 @@ Bug Fixes
from Unicode character classes [:ID_Start:] and [:ID_Continue:].
- Uppercase character entities "&QUOT;", "&COPY;", "&GT;", "&LT;", "&REG;",
and "&AMP;" are now recognized and handled as if they were in lowercase.
- The REPLACEMENT CHARACTER U+FFFD is now used to replace numeric character
entities for unpaired UTF-16 low and high surrogates (in the range
[U+D800-U+DFFF]).
- Properly paired numeric character entities for UTF-16 surrogates are now
converted to the corresponding code units.
- Opening tags with unbalanced quotation marks are now properly stripped.
- Literal "<" and ">" characters in opening tags, regardless of whether they
appear inside quotation marks, now inhibit recognition (and stripping) of
@ -526,6 +535,10 @@ Bug Fixes
attributes and their values, are left intact in the output.
(Steve Rowe)
* LUCENE-3717: Fixed offset bugs in TrimFilter, WordDelimiterFilter, and
HyphenatedWordsFilter where they would create invalid offsets in
some situations, leading to problems in highlighting. (Robert Muir)
Other Changes
----------------------
* SOLR-2922: Upgrade commons-io and commons-lang to 2.1 and 2.6, respectively. (koji)

View File

@ -0,0 +1,88 @@
package org.apache.solr.analysis;
/*
* 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.
*/
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.core.TypeTokenFilter;
import org.apache.solr.common.ResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.util.plugin.ResourceLoaderAware;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Factory class for {@link TypeTokenFilter}
* <pre class="prettyprint" >
* &lt;fieldType name="chars" class="solr.TextField" positionIncrementGap="100"&gt;
* &lt;analyzer&gt;
* &lt;tokenizer class="solr.StandardTokenizerFactory"/&gt;
* &lt;filter class="solr.TypeTokenFilterFactory" types="stoptypes.txt" enablePositionIncrements="true"/&gt;
* &lt;/analyzer&gt;
* &lt;/fieldType&gt;</pre>
*/
public class TypeTokenFilterFactory extends BaseTokenFilterFactory implements ResourceLoaderAware {
@Override
public void init(Map<String, String> args) {
super.init(args);
}
@Override
public void inform(ResourceLoader loader) {
String stopTypesFiles = args.get("types");
enablePositionIncrements = getBoolean("enablePositionIncrements", false);
if (stopTypesFiles != null) {
try {
List<String> files = StrUtils.splitFileNames(stopTypesFiles);
if (files.size() > 0) {
stopTypes = new HashSet<String>();
for (String file : files) {
List<String> typesLines = loader.getLines(file.trim());
stopTypes.addAll(typesLines);
}
}
} catch (IOException e) {
throw new RuntimeException(e);
}
} else {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Missing required parameter: types.");
}
}
private Set<String> stopTypes;
private boolean enablePositionIncrements;
public boolean isEnablePositionIncrements() {
return enablePositionIncrements;
}
public Set<String> getStopTypes() {
return stopTypes;
}
@Override
public TokenStream create(TokenStream input) {
return new TypeTokenFilter(enablePositionIncrements, input, stopTypes);
}
}

View File

@ -0,0 +1,17 @@
# 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.
<NUM>
<EMAIL>

View File

@ -0,0 +1,17 @@
# 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.
<HOST>
<APOSTROPHE>

View File

@ -0,0 +1,86 @@
package org.apache.solr.analysis;
/**
* 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.
*/
import org.apache.lucene.analysis.NumericTokenStream;
import org.apache.solr.common.ResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.core.SolrResourceLoader;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
/**
* Testcase for {@link TypeTokenFilterFactory}
*/
public class TestTypeTokenFilterFactory extends BaseTokenTestCase {
@Test
public void testInform() throws Exception {
ResourceLoader loader = new SolrResourceLoader(null, null);
TypeTokenFilterFactory factory = new TypeTokenFilterFactory();
Map<String, String> args = new HashMap<String, String>(DEFAULT_VERSION_PARAM);
args.put("types", "stoptypes-1.txt");
args.put("enablePositionIncrements", "true");
factory.init(args);
factory.inform(loader);
Set<String> types = factory.getStopTypes();
assertTrue("types is null and it shouldn't be", types != null);
assertTrue("types Size: " + types.size() + " is not: " + 2, types.size() == 2);
assertTrue("enablePositionIncrements was set to true but not correctly parsed", factory.isEnablePositionIncrements());
factory = new TypeTokenFilterFactory();
args.put("types", "stoptypes-1.txt, stoptypes-2.txt");
args.put("enablePositionIncrements", "false");
factory.init(args);
factory.inform(loader);
types = factory.getStopTypes();
assertTrue("types is null and it shouldn't be", types != null);
assertTrue("types Size: " + types.size() + " is not: " + 4, types.size() == 4);
assertTrue("enablePositionIncrements was set to false but not correctly parsed", !factory.isEnablePositionIncrements());
}
@Test
public void testCreation() throws Exception {
TypeTokenFilterFactory typeTokenFilterFactory = new TypeTokenFilterFactory();
Map<String, String> args = new HashMap<String, String>(DEFAULT_VERSION_PARAM);
args.put("types", "stoptypes-1.txt, stoptypes-2.txt");
args.put("enablePositionIncrements", "false");
typeTokenFilterFactory.init(args);
NumericTokenStream input = new NumericTokenStream();
input.setIntValue(123);
typeTokenFilterFactory.create(input);
}
@Test
public void testMissingTypesParameter() throws Exception {
try {
TypeTokenFilterFactory typeTokenFilterFactory = new TypeTokenFilterFactory();
Map<String, String> args = new HashMap<String, String>(DEFAULT_VERSION_PARAM);
args.put("enablePositionIncrements", "false");
typeTokenFilterFactory.init(args);
typeTokenFilterFactory.inform(new SolrResourceLoader(null, null));
fail("not supplying 'types' parameter should cause a SolrException");
} catch (SolrException e) {
// everything ok
}
}
}