LUCENE-2514: Term is no longer character based

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@960484 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2010-07-05 08:33:25 +00:00
parent 36b388859a
commit 0c333c60dd
60 changed files with 515 additions and 349 deletions

View File

@ -102,6 +102,15 @@ Changes in backwards compatibility policy
API Changes
* LUCENE-2302, LUCENE-1458, LUCENE-2111, LUCENE-2514: Terms are no longer
required to be character based. Lucene views a term as an arbitrary byte[]:
during analysis, character-based terms are converted to UTF8 byte[],
but analyzers are free to directly create terms as byte[]
(NumericField does this, for example). The term data is buffered as
byte[] during indexing, written as byte[] into the terms dictionary,
and iterated as byte[] (wrapped in a BytesRef) by IndexReader for
searching.
* LUCENE-1458, LUCENE-2111: IndexReader now directly exposes its
deleted docs (getDeletedDocs), providing a new Bits interface to
directly query by doc ID.
@ -147,15 +156,6 @@ New features
standard codec), and int block (really a "base" for using
block-based compressors like PForDelta for storing postings data).
* LUCENE-2302, LUCENE-1458, LUCENE-2111: Terms are no longer required
to be character based. Lucene views a term as an arbitrary byte[]:
during analysis, character-based terms are converted to UTF8 byte[],
but analyzers are free to directly create terms as byte[]
(NumericField does this, for example). The term data is buffered as
byte[] during indexing, written as byte[] into the terms dictionary,
and iterated as byte[] (wrapped in a BytesRef) by IndexReader for
searching.
* LUCENE-2385: Moved NoDeletionPolicy from benchmark to core. NoDeletionPolicy
can be used to prevent commits from ever getting deleted from the index.
(Shai Erera)

View File

@ -36,6 +36,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.TermFreqVector;
import org.apache.lucene.index.TermPositionVector;
import org.apache.lucene.index.TermVectorOffsetInfo;
import org.apache.lucene.util.BytesRef;
/**
* Hides implementation issues associated with obtaining a TokenStream for use
@ -176,7 +177,7 @@ public class TokenSources {
}
}
// code to reconstruct the original sequence of Tokens
String[] terms = tpv.getTerms();
BytesRef[] terms = tpv.getTerms();
int[] freq = tpv.getTermFrequencies();
int totalTokens = 0;
@ -204,7 +205,7 @@ public class TokenSources {
unsortedTokens = new ArrayList<Token>();
}
for (int tp = 0; tp < offsets.length; tp++) {
Token token = new Token(terms[t], offsets[tp].getStartOffset(), offsets[tp]
Token token = new Token(terms[t].utf8ToString(), offsets[tp].getStartOffset(), offsets[tp]
.getEndOffset());
unsortedTokens.add(token);
}
@ -220,7 +221,7 @@ public class TokenSources {
// tokens stored with positions - can use this to index straight into
// sorted array
for (int tp = 0; tp < pos.length; tp++) {
Token token = new Token(terms[t], offsets[tp].getStartOffset(),
Token token = new Token(terms[t].utf8ToString(), offsets[tp].getStartOffset(),
offsets[tp].getEndOffset());
tokensInOriginalOrder[pos[tp]] = token;
}

View File

@ -30,6 +30,7 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.index.TermPositionVector;
import org.apache.lucene.index.TermVectorOffsetInfo;
import org.apache.lucene.util.BytesRef;
public final class TokenStreamFromTermPositionVector extends TokenStream {
@ -54,18 +55,18 @@ public final class TokenStreamFromTermPositionVector extends TokenStream {
termAttribute = addAttribute(CharTermAttribute.class);
positionIncrementAttribute = addAttribute(PositionIncrementAttribute.class);
offsetAttribute = addAttribute(OffsetAttribute.class);
final String[] terms = termPositionVector.getTerms();
final BytesRef[] terms = termPositionVector.getTerms();
for (int i = 0; i < terms.length; i++) {
final TermVectorOffsetInfo[] offsets = termPositionVector.getOffsets(i);
final int[] termPositions = termPositionVector.getTermPositions(i);
for (int j = 0; j < termPositions.length; j++) {
Token token;
if (offsets != null) {
token = new Token(terms[i].toCharArray(), 0, terms[i].length(),
token = new Token(terms[i].utf8ToString(),
offsets[j].getStartOffset(), offsets[j].getEndOffset());
} else {
token = new Token();
token.setEmpty().append(terms[i]);
token.setEmpty().append(terms[i].utf8ToString());
}
// Yes - this is the position, not the increment! This is for
// sorting. This value

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.TermFreqVector;
import org.apache.lucene.index.TermPositionVector;
import org.apache.lucene.index.TermVectorOffsetInfo;
import org.apache.lucene.util.BytesRef;
/**
* <code>FieldTermStack</code> is a stack that keeps query terms in the specified field
@ -80,15 +81,15 @@ public class FieldTermStack {
// just return to make null snippet if un-matched fieldName specified when fieldMatch == true
if( termSet == null ) return;
for( String term : tpv.getTerms() ){
if( !termSet.contains( term ) ) continue;
for( BytesRef term : tpv.getTerms() ){
if( !termSet.contains( term.utf8ToString() ) ) continue;
int index = tpv.indexOf( term );
TermVectorOffsetInfo[] tvois = tpv.getOffsets( index );
if( tvois == null ) return; // just return to make null snippets
int[] poss = tpv.getTermPositions( index );
if( poss == null ) return; // just return to make null snippets
for( int i = 0; i < tvois.length; i++ )
termList.add( new TermInfo( term, tvois[i].getStartOffset(), tvois[i].getEndOffset(), poss[i] ) );
termList.add( new TermInfo( term.utf8ToString(), tvois[i].getStartOffset(), tvois[i].getEndOffset(), poss[i] ) );
}
// sort by position

View File

@ -290,7 +290,7 @@ public class InstantiatedIndex
TermPositionVector termPositionVector = (TermPositionVector) sourceIndexReader.getTermFreqVector(document.getDocumentNumber(), field.name());
if (termPositionVector != null) {
for (int i = 0; i < termPositionVector.getTerms().length; i++) {
String token = termPositionVector.getTerms()[i];
String token = termPositionVector.getTerms()[i].utf8ToString();
InstantiatedTerm term = findTerm(field.name(), token);
InstantiatedTermDocumentInformation termDocumentInformation = term.getAssociatedDocument(document.getDocumentNumber());
termDocumentInformation.setTermOffsets(termPositionVector.getOffsets(i));

View File

@ -464,7 +464,7 @@ public class InstantiatedIndexReader extends IndexReader {
List<InstantiatedTermDocumentInformation> tv = doc.getVectorSpace().get(field);
mapper.setExpectations(field, tv.size(), true, true);
for (InstantiatedTermDocumentInformation tdi : tv) {
mapper.map(tdi.getTerm().text(), tdi.getTermPositions().length, tdi.getTermOffsets(), tdi.getTermPositions());
mapper.map(tdi.getTerm().getTerm().bytes(), tdi.getTermPositions().length, tdi.getTermOffsets(), tdi.getTermPositions());
}
}
}
@ -475,7 +475,7 @@ public class InstantiatedIndexReader extends IndexReader {
for (Map.Entry<String, List<InstantiatedTermDocumentInformation>> e : doc.getVectorSpace().entrySet()) {
mapper.setExpectations(e.getKey(), e.getValue().size(), true, true);
for (InstantiatedTermDocumentInformation tdi : e.getValue()) {
mapper.map(tdi.getTerm().text(), tdi.getTermPositions().length, tdi.getTermOffsets(), tdi.getTermPositions());
mapper.map(tdi.getTerm().getTerm().bytes(), tdi.getTermPositions().length, tdi.getTermOffsets(), tdi.getTermPositions());
}
}
}

View File

@ -1,6 +1,7 @@
package org.apache.lucene.store.instantiated;
import org.apache.lucene.index.TermFreqVector;
import org.apache.lucene.util.BytesRef;
import java.io.Serializable;
import java.util.Arrays;
@ -34,18 +35,18 @@ public class InstantiatedTermFreqVector
private final List<InstantiatedTermDocumentInformation> termDocumentInformations;
private final String field;
private final String terms[];
private final BytesRef terms[];
private final int termFrequencies[];
public InstantiatedTermFreqVector(InstantiatedDocument document, String field) {
this.field = field;
termDocumentInformations = document.getVectorSpace().get(field);
terms = new String[termDocumentInformations.size()];
terms = new BytesRef[termDocumentInformations.size()];
termFrequencies = new int[termDocumentInformations.size()];
for (int i = 0; i < termDocumentInformations.size(); i++) {
InstantiatedTermDocumentInformation termDocumentInformation = termDocumentInformations.get(i);
terms[i] = termDocumentInformation.getTerm().text();
terms[i] = termDocumentInformation.getTerm().getTerm().bytes();
termFrequencies[i] = termDocumentInformation.getTermPositions().length;
}
}
@ -77,7 +78,7 @@ public class InstantiatedTermFreqVector
return terms == null ? 0 : terms.length;
}
public String[] getTerms() {
public BytesRef[] getTerms() {
return terms;
}
@ -85,14 +86,14 @@ public class InstantiatedTermFreqVector
return termFrequencies;
}
public int indexOf(String termText) {
public int indexOf(BytesRef termText) {
if (terms == null)
return -1;
int res = Arrays.binarySearch(terms, termText);
return res >= 0 ? res : -1;
}
public int[] indexesOf(String[] termNumbers, int start, int len) {
public int[] indexesOf(BytesRef[] termNumbers, int start, int len) {
// TODO: there must be a more efficient way of doing this.
// At least, we could advance the lower bound of the terms array
// as we find valid indices. Also, it might be possible to leverage

View File

@ -41,14 +41,14 @@ public class InstantiatedTermsEnum extends TermsEnum {
@Override
public SeekStatus seek(BytesRef text, boolean useCache) {
final Term t = new Term(field, text.utf8ToString());
final Term t = new Term(field, text);
int loc = Arrays.binarySearch(terms, t, InstantiatedTerm.termComparator);
if (loc < 0) {
upto = -loc - 1;
if (upto >= terms.length) {
return SeekStatus.END;
} else {
br.copy(terms[upto].getTerm().text());
br.copy(terms[upto].getTerm().bytes());
return SeekStatus.NOT_FOUND;
}
} else {

View File

@ -207,7 +207,7 @@ public class MemoryIndex implements Serializable {
if (o1 instanceof Map.Entry<?,?>) o1 = ((Map.Entry<?,?>) o1).getKey();
if (o2 instanceof Map.Entry<?,?>) o2 = ((Map.Entry<?,?>) o2).getKey();
if (o1 == o2) return 0;
return ((String) o1).compareTo((String) o2);
return ((Comparable) o1).compareTo((Comparable) o2);
}
};
@ -341,21 +341,19 @@ public class MemoryIndex implements Serializable {
if (fields.get(fieldName) != null)
throw new IllegalArgumentException("field must not be added more than once");
HashMap<String,ArrayIntList> terms = new HashMap<String,ArrayIntList>();
HashMap<BytesRef,ArrayIntList> terms = new HashMap<BytesRef,ArrayIntList>();
int numTokens = 0;
int numOverlapTokens = 0;
int pos = -1;
TermToBytesRefAttribute termAtt = stream.addAttribute(TermToBytesRefAttribute.class);
TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
BytesRef ref = new BytesRef(10);
stream.reset();
while (stream.incrementToken()) {
termAtt.toBytesRef(ref);
// TODO: support non-UTF8 strings (like numerics) here
String term = ref.utf8ToString();
if (term.length() == 0) continue; // nothing to do
if (ref.length == 0) continue; // nothing to do
// if (DEBUG) System.err.println("token='" + term + "'");
numTokens++;
final int posIncr = posIncrAttribute.getPositionIncrement();
@ -363,10 +361,10 @@ public class MemoryIndex implements Serializable {
numOverlapTokens++;
pos += posIncr;
ArrayIntList positions = terms.get(term);
ArrayIntList positions = terms.get(ref);
if (positions == null) { // term not seen before
positions = new ArrayIntList(stride);
terms.put(term, positions);
terms.put(new BytesRef(ref), positions);
}
if (stride == 1) {
positions.add(pos);
@ -490,9 +488,10 @@ public class MemoryIndex implements Serializable {
int len = info.terms.size();
size += VM.sizeOfHashMap(len);
Iterator<Map.Entry<String,ArrayIntList>> iter2 = info.terms.entrySet().iterator();
Iterator<Map.Entry<BytesRef,ArrayIntList>> iter2 = info.terms.entrySet().iterator();
while (--len >= 0) { // for each term
Map.Entry<String,ArrayIntList> e = iter2.next();
Map.Entry<BytesRef,ArrayIntList> e = iter2.next();
// FIXME: this calculation is probably not correct since we use bytes now.
size += VM.sizeOfObject(PTR + 3*INT); // assumes substring() memory overlay
// size += STR + 2 * ((String) e.getKey()).length();
ArrayIntList positions = e.getValue();
@ -534,7 +533,7 @@ public class MemoryIndex implements Serializable {
public String toString() {
StringBuilder result = new StringBuilder(256);
sortFields();
int sumChars = 0;
int sumBytes = 0;
int sumPositions = 0;
int sumTerms = 0;
@ -545,32 +544,32 @@ public class MemoryIndex implements Serializable {
info.sortTerms();
result.append(fieldName + ":\n");
int numChars = 0;
int numBytes = 0;
int numPositions = 0;
for (int j=0; j < info.sortedTerms.length; j++) {
Map.Entry<String,ArrayIntList> e = info.sortedTerms[j];
String term = e.getKey();
Map.Entry<BytesRef,ArrayIntList> e = info.sortedTerms[j];
BytesRef term = e.getKey();
ArrayIntList positions = e.getValue();
result.append("\t'" + term + "':" + numPositions(positions) + ":");
result.append(positions.toString(stride)); // ignore offsets
result.append("\n");
numPositions += numPositions(positions);
numChars += term.length();
numBytes += term.length;
}
result.append("\tterms=" + info.sortedTerms.length);
result.append(", positions=" + numPositions);
result.append(", Kchars=" + (numChars/1000.0f));
result.append(", Kbytes=" + (numBytes/1000.0f));
result.append("\n");
sumPositions += numPositions;
sumChars += numChars;
sumBytes += numBytes;
sumTerms += info.sortedTerms.length;
}
result.append("\nfields=" + sortedFields.length);
result.append(", terms=" + sumTerms);
result.append(", positions=" + sumPositions);
result.append(", Kchars=" + (sumChars/1000.0f));
result.append(", Kbytes=" + (sumBytes/1000.0f));
return result.toString();
}
@ -588,10 +587,10 @@ public class MemoryIndex implements Serializable {
* Term strings and their positions for this field: Map <String
* termText, ArrayIntList positions>
*/
private final HashMap<String,ArrayIntList> terms;
private final HashMap<BytesRef,ArrayIntList> terms;
/** Terms sorted ascending by term text; computed on demand */
private transient Map.Entry<String,ArrayIntList>[] sortedTerms;
private transient Map.Entry<BytesRef,ArrayIntList>[] sortedTerms;
/** Number of added tokens for this field */
private final int numTokens;
@ -607,7 +606,7 @@ public class MemoryIndex implements Serializable {
private static final long serialVersionUID = 2882195016849084649L;
public Info(HashMap<String,ArrayIntList> terms, int numTokens, int numOverlapTokens, float boost) {
public Info(HashMap<BytesRef,ArrayIntList> terms, int numTokens, int numOverlapTokens, float boost) {
this.terms = terms;
this.numTokens = numTokens;
this.numOverlapTokens = numOverlapTokens;
@ -627,7 +626,7 @@ public class MemoryIndex implements Serializable {
}
/** note that the frequency can be calculated as numPosition(getPositions(x)) */
public ArrayIntList getPositions(String term) {
public ArrayIntList getPositions(BytesRef term) {
return terms.get(term);
}
@ -759,7 +758,7 @@ public class MemoryIndex implements Serializable {
public int docFreq(Term term) {
Info info = getInfo(term.field());
int freq = 0;
if (info != null) freq = info.getPositions(term.text()) != null ? 1 : 0;
if (info != null) freq = info.getPositions(term.bytes()) != null ? 1 : 0;
if (DEBUG) System.err.println("MemoryIndexReader.docFreq: " + term + ", freq:" + freq);
return freq;
}
@ -833,8 +832,7 @@ public class MemoryIndex implements Serializable {
@Override
public SeekStatus seek(BytesRef text, boolean useCache) {
final String s = text.utf8ToString();
termUpto = Arrays.binarySearch(info.sortedTerms, s, termComparator);
termUpto = Arrays.binarySearch(info.sortedTerms, text, termComparator);
if (termUpto < 0) { // not found; choose successor
termUpto = -termUpto -1;
if (termUpto >= info.sortedTerms.length) {
@ -1061,7 +1059,7 @@ public class MemoryIndex implements Serializable {
return new TermPositionVector() {
private final Map.Entry<String,ArrayIntList>[] sortedTerms = info.sortedTerms;
private final Map.Entry<BytesRef,ArrayIntList>[] sortedTerms = info.sortedTerms;
public String getField() {
return fieldName;
@ -1071,8 +1069,8 @@ public class MemoryIndex implements Serializable {
return sortedTerms.length;
}
public String[] getTerms() {
String[] terms = new String[sortedTerms.length];
public BytesRef[] getTerms() {
BytesRef[] terms = new BytesRef[sortedTerms.length];
for (int i=sortedTerms.length; --i >= 0; ) {
terms[i] = sortedTerms[i].getKey();
}
@ -1087,12 +1085,12 @@ public class MemoryIndex implements Serializable {
return freqs;
}
public int indexOf(String term) {
public int indexOf(BytesRef term) {
int i = Arrays.binarySearch(sortedTerms, term, termComparator);
return i >= 0 ? i : -1;
}
public int[] indexesOf(String[] terms, int start, int len) {
public int[] indexesOf(BytesRef[] terms, int start, int len) {
int[] indexes = new int[len];
for (int i=0; i < len; i++) {
indexes[i] = indexOf(terms[start++]);

View File

@ -69,7 +69,7 @@ public class TermVectorAccessor {
}
/** Instance reused to save garbage collector some time */
private List<String> tokens;
private List<BytesRef> tokens;
/** Instance reused to save garbage collector some time */
private List<int[]> positions;
@ -91,7 +91,7 @@ public class TermVectorAccessor {
private void build(IndexReader indexReader, String field, TermVectorMapper mapper, int documentNumber) throws IOException {
if (tokens == null) {
tokens = new ArrayList<String>(500);
tokens = new ArrayList<BytesRef>(500);
positions = new ArrayList<int[]>(500);
frequencies = new ArrayList<Integer>(500);
} else {
@ -122,7 +122,7 @@ public class TermVectorAccessor {
if (docID == documentNumber) {
frequencies.add(Integer.valueOf(docs.freq()));
tokens.add(text.utf8ToString());
tokens.add(new BytesRef(text));
if (!mapper.isIgnoringPositions()) {
int[] positions = new int[docs.freq()];
@ -173,7 +173,7 @@ public class TermVectorAccessor {
}
@Override
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
decorated.map(term, frequency, offsets, positions);
}

View File

@ -76,21 +76,21 @@ public class TestTermVectorAccessor extends LuceneTestCase {
mapper = new ParallelArrayTermVectorMapper();
accessor.accept(ir, i, "a", mapper);
tfv = mapper.materializeVector();
assertEquals("doc " + i, "a", tfv.getTerms()[0]);
assertEquals("doc " + i, "a", tfv.getTerms()[0].utf8ToString());
assertEquals("doc " + i, 8, tfv.getTermFrequencies()[0]);
mapper = new ParallelArrayTermVectorMapper();
accessor.accept(ir, i, "b", mapper);
tfv = mapper.materializeVector();
assertEquals("doc " + i, 8, tfv.getTermFrequencies().length);
assertEquals("doc " + i, "b", tfv.getTerms()[1]);
assertEquals("doc " + i, "b", tfv.getTerms()[1].utf8ToString());
assertEquals("doc " + i, 7, tfv.getTermFrequencies()[1]);
mapper = new ParallelArrayTermVectorMapper();
accessor.accept(ir, i, "c", mapper);
tfv = mapper.materializeVector();
assertEquals("doc " + i, 8, tfv.getTermFrequencies().length);
assertEquals("doc " + i, "c", tfv.getTerms()[2]);
assertEquals("doc " + i, "c", tfv.getTerms()[2].utf8ToString());
assertEquals("doc " + i, 7, tfv.getTermFrequencies()[2]);
mapper = new ParallelArrayTermVectorMapper();

View File

@ -213,7 +213,7 @@ public class FuzzyLikeThisQuery extends Query
totalVariantDocFreqs+=fe.docFreq();
float score=boostAtt.getBoost();
if (variantsQ.size() < MAX_VARIANTS_PER_TERM || score > minScore){
ScoreTerm st=new ScoreTerm(new Term(startTerm.field(), possibleMatch.utf8ToString()),score,startTerm);
ScoreTerm st=new ScoreTerm(new Term(startTerm.field(), new BytesRef(possibleMatch)),score,startTerm);
variantsQ.insertWithOverflow(st);
minScore = variantsQ.top().score; // maintain minScore
}

View File

@ -77,7 +77,7 @@ public class TermsFilter extends Filter
}
if (terms != null) {
br.copy(term.text());
br.copy(term.bytes());
if (termsEnum.seek(br) == TermsEnum.SeekStatus.FOUND) {
docs = termsEnum.docs(delDocs, docs);
while(docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {

View File

@ -47,6 +47,7 @@ import org.apache.lucene.search.Similarity;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
@ -848,10 +849,10 @@ public final class MoreLikeThis {
*/
private void addTermFrequencies(Map<String,Int> termFreqMap, TermFreqVector vector)
{
String[] terms = vector.getTerms();
BytesRef[] terms = vector.getTerms();
int freqs[]=vector.getTermFrequencies();
for (int j = 0; j < terms.length; j++) {
String term = terms[j];
String term = terms[j].utf8ToString();
if(isNoiseWord(term)){
continue;

View File

@ -1073,7 +1073,6 @@ final class DocumentsWriter {
TermsEnum termsEnum = null;
String currentField = null;
BytesRef termRef = new BytesRef();
DocsEnum docs = null;
for (Entry<Term, BufferedDeletes.Num> entry: deletesFlushed.terms.entrySet()) {
@ -1097,9 +1096,7 @@ final class DocumentsWriter {
}
assert checkDeleteTerm(term);
termRef.copy(term.text());
if (termsEnum.seek(termRef, false) == TermsEnum.SeekStatus.FOUND) {
if (termsEnum.seek(term.bytes(), false) == TermsEnum.SeekStatus.FOUND) {
DocsEnum docsEnum = termsEnum.docs(reader.getDeletedDocs(), docs);
if (docsEnum != null) {
@ -1166,7 +1163,7 @@ final class DocumentsWriter {
num.setNum(docIDUpto);
deletesInRAM.numTerms++;
deletesInRAM.addBytesUsed(BYTES_PER_DEL_TERM + term.text.length()*CHAR_NUM_BYTE);
deletesInRAM.addBytesUsed(BYTES_PER_DEL_TERM + term.bytes.length);
}
// Buffer a specific docID for deletion. Currently only

View File

@ -2,6 +2,8 @@ package org.apache.lucene.index;
import java.util.*;
import org.apache.lucene.util.BytesRef;
/**
* Copyright 2007 The Apache Software Foundation
* <p/>
@ -44,7 +46,7 @@ public class FieldSortedTermVectorMapper extends TermVectorMapper{
}
@Override
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
TermVectorEntry entry = new TermVectorEntry(currentField, term, frequency, offsets, positions);
currentSet.add(entry);
}

View File

@ -883,7 +883,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
public abstract Fields fields() throws IOException;
public int docFreq(Term term) throws IOException {
return docFreq(term.field(), new BytesRef(term.text()));
return docFreq(term.field(), term.bytes());
}
/** Returns the number of documents containing the term
@ -1000,7 +1000,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
DocsEnum docs = MultiFields.getTermDocsEnum(this,
MultiFields.getDeletedDocs(this),
term.field(),
new BytesRef(term.text()));
term.bytes());
if (docs == null) return 0;
int n = 0;
int doc;

View File

@ -21,6 +21,8 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.util.BytesRef;
/**
* For each Field, store position by position information. It ignores frequency information
* <p/>
@ -69,7 +71,7 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
* @param positions
*/
@Override
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
for (int i = 0; i < positions.length; i++) {
Integer posVal = Integer.valueOf(positions[i]);
TVPositionInfo pos = currentPositions.get(posVal);
@ -120,20 +122,20 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
public static class TVPositionInfo{
private int position;
private List<String> terms;
private List<BytesRef> terms;
private List<TermVectorOffsetInfo> offsets;
public TVPositionInfo(int position, boolean storeOffsets) {
this.position = position;
terms = new ArrayList<String>();
terms = new ArrayList<BytesRef>();
if (storeOffsets) {
offsets = new ArrayList<TermVectorOffsetInfo>();
}
}
void addTerm(String term, TermVectorOffsetInfo info)
void addTerm(BytesRef term, TermVectorOffsetInfo info)
{
terms.add(term);
if (offsets != null) {
@ -151,9 +153,9 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
/**
* Note, there may be multiple terms at the same position
* @return A List of Strings
* @return A List of BytesRefs
*/
public List<String> getTerms() {
public List<BytesRef> getTerms() {
return terms;
}

View File

@ -1,5 +1,7 @@
package org.apache.lucene.index;
import org.apache.lucene.util.BytesRef;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -22,7 +24,7 @@ class SegmentTermPositionVector extends SegmentTermVector implements TermPositio
protected TermVectorOffsetInfo[][] offsets;
public static final int[] EMPTY_TERM_POS = new int[0];
public SegmentTermPositionVector(String field, String terms[], int termFreqs[], int[][] positions, TermVectorOffsetInfo[][] offsets) {
public SegmentTermPositionVector(String field, BytesRef terms[], int termFreqs[], int[][] positions, TermVectorOffsetInfo[][] offsets) {
super(field, terms, termFreqs);
this.offsets = offsets;
this.positions = positions;

View File

@ -19,13 +19,15 @@ package org.apache.lucene.index;
import java.util.*;
import org.apache.lucene.util.BytesRef;
class SegmentTermVector implements TermFreqVector {
private String field;
private String terms[];
private BytesRef terms[];
private int termFreqs[];
SegmentTermVector(String field, String terms[], int termFreqs[]) {
SegmentTermVector(String field, BytesRef terms[], int termFreqs[]) {
this.field = field;
this.terms = terms;
this.termFreqs = termFreqs;
@ -59,7 +61,7 @@ class SegmentTermVector implements TermFreqVector {
return terms == null ? 0 : terms.length;
}
public String [] getTerms() {
public BytesRef [] getTerms() {
return terms;
}
@ -67,14 +69,14 @@ class SegmentTermVector implements TermFreqVector {
return termFreqs;
}
public int indexOf(String termText) {
public int indexOf(BytesRef termBytes) {
if(terms == null)
return -1;
int res = Arrays.binarySearch(terms, termText);
int res = Arrays.binarySearch(terms, termBytes);
return res >= 0 ? res : -1;
}
public int[] indexesOf(String [] termNumbers, int start, int len) {
public int[] indexesOf(BytesRef [] termNumbers, int start, int len) {
// TODO: there must be a more efficient way of doing this.
// At least, we could advance the lower bound of the terms array
// as we find valid indexes. Also, it might be possible to leverage

View File

@ -17,6 +17,8 @@ package org.apache.lucene.index;
import java.util.*;
import org.apache.lucene.util.BytesRef;
/**
* Store a sorted collection of {@link org.apache.lucene.index.TermVectorEntry}s. Collects all term information
* into a single, SortedSet.
@ -30,7 +32,7 @@ public class SortedTermVectorMapper extends TermVectorMapper{
private SortedSet<TermVectorEntry> currentSet;
private Map<String,TermVectorEntry> termToTVE = new HashMap<String,TermVectorEntry>();
private Map<BytesRef,TermVectorEntry> termToTVE = new HashMap<BytesRef,TermVectorEntry>();
private boolean storeOffsets;
private boolean storePositions;
/**
@ -61,7 +63,7 @@ public class SortedTermVectorMapper extends TermVectorMapper{
*/
//We need to combine any previous mentions of the term
@Override
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
TermVectorEntry entry = termToTVE.get(term);
if (entry == null) {
entry = new TermVectorEntry(ALL, term, frequency,

View File

@ -17,6 +17,9 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.util.Comparator;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
/**
@ -29,14 +32,26 @@ import org.apache.lucene.util.StringHelper;
public final class Term implements Comparable<Term>, java.io.Serializable {
String field;
String text;
BytesRef bytes;
/** Constructs a Term with the given field and bytes.
* <p>Note that a null field or null bytes value results in undefined
* behavior for most Lucene APIs that accept a Term parameter.
* <p>WARNING: the provided BytesRef is not copied, but used directly.
* Therefore the bytes should not be modified after construction, for
* example, you should clone a copy rather than pass reused bytes from
* a TermsEnum.
*/
public Term(String fld, BytesRef bytes) {
field = fld == null ? null : StringHelper.intern(fld);
this.bytes = bytes;
}
/** Constructs a Term with the given field and text.
* <p>Note that a null field or null text value results in undefined
* behavior for most Lucene APIs that accept a Term parameter. */
public Term(String fld, String txt) {
field = fld == null ? null : StringHelper.intern(fld);
text = txt;
public Term(String fld, String text) {
this(fld, new BytesRef(text));
}
/** Constructs a Term with the given field and empty text.
@ -46,15 +61,27 @@ public final class Term implements Comparable<Term>, java.io.Serializable {
* @param fld
*/
public Term(String fld) {
this(fld, "", true);
this(fld, new BytesRef(), true);
}
/**
* WARNING: the provided BytesRef is not copied, but used directly.
* Therefore the bytes should not be modified after construction, for
* example, you should clone a copy rather than pass reused bytes from
* a TermsEnum.
*
* @lucene.experimental
*/
public Term(String fld, BytesRef bytes, boolean intern) {
field = intern ? StringHelper.intern(fld) : fld; // field names are interned
this.bytes = bytes; // unless already known to be
}
/** @lucene.experimental */
public Term(String fld, String txt, boolean intern) {
field = intern ? StringHelper.intern(fld) : fld; // field names are interned
text = txt; // unless already known to be
public Term(String fld, String text, boolean intern) {
this(fld, new BytesRef(text), intern);
}
/** Returns the field of this term, an interned string. The field indicates
the part of a document which this term came from. */
public final String field() { return field; }
@ -62,8 +89,26 @@ public final class Term implements Comparable<Term>, java.io.Serializable {
/** Returns the text of this term. In the case of words, this is simply the
text of the word. In the case of dates and other types, this is an
encoding of the object as a string. */
public final String text() { return text; }
public final String text() { return bytes.utf8ToString(); }
/** Returns the bytes of this term. */
public final BytesRef bytes() { return bytes; }
/**
* Optimized construction of new Terms by reusing same field as this Term
* - avoids field.intern() overhead
* <p>WARNING: the provided BytesRef is not copied, but used directly.
* Therefore the bytes should not be modified after construction, for
* example, you should clone a copy rather than pass reused bytes from
* a TermsEnum.
* @param text The bytes of the new term (field is implicitly same as this Term instance)
* @return A new Term
*/
public Term createTerm(BytesRef bytes)
{
return new Term(field,bytes,false);
}
/**
* Optimized construction of new Terms by reusing same field as this Term
* - avoids field.intern() overhead
@ -89,10 +134,10 @@ public final class Term implements Comparable<Term>, java.io.Serializable {
return false;
} else if (!field.equals(other.field))
return false;
if (text == null) {
if (other.text != null)
if (bytes == null) {
if (other.bytes != null)
return false;
} else if (!text.equals(other.text))
} else if (!bytes.equals(other.bytes))
return false;
return true;
}
@ -102,7 +147,7 @@ public final class Term implements Comparable<Term>, java.io.Serializable {
final int prime = 31;
int result = 1;
result = prime * result + ((field == null) ? 0 : field.hashCode());
result = prime * result + ((text == null) ? 0 : text.hashCode());
result = prime * result + ((bytes == null) ? 0 : bytes.hashCode());
return result;
}
@ -113,19 +158,47 @@ public final class Term implements Comparable<Term>, java.io.Serializable {
The ordering of terms is first by field, then by text.*/
public final int compareTo(Term other) {
if (field == other.field) // fields are interned
return text.compareTo(other.text);
return bytes.compareTo(other.bytes);
else
return field.compareTo(other.field);
}
@Deprecated
private static final Comparator<BytesRef> legacyComparator =
BytesRef.getUTF8SortedAsUTF16Comparator();
/**
* @deprecated For internal backwards compatibility use only
* @lucene.internal
*/
@Deprecated
public final int compareToUTF16(Term other) {
if (field == other.field) // fields are interned
return legacyComparator.compare(this.bytes, other.bytes);
else
return field.compareTo(other.field);
}
/**
* Resets the field and text of a Term.
* <p>WARNING: the provided BytesRef is not copied, but used directly.
* Therefore the bytes should not be modified after construction, for
* example, you should clone a copy rather than pass reused bytes from
* a TermsEnum.
*/
final void set(String fld, BytesRef bytes) {
field = fld;
this.bytes = bytes;
}
/** Resets the field and text of a Term. */
final void set(String fld, String txt) {
field = fld;
text = txt;
this.bytes = new BytesRef(txt);
}
@Override
public final String toString() { return field + ":" + text; }
public final String toString() { return field + ":" + bytes.utf8ToString(); }
private void readObject(java.io.ObjectInputStream in)
throws java.io.IOException, ClassNotFoundException

View File

@ -1,5 +1,7 @@
package org.apache.lucene.index;
import org.apache.lucene.util.BytesRef;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -38,7 +40,7 @@ public interface TermFreqVector {
/**
* @return An Array of term texts in ascending order.
*/
public String[] getTerms();
public BytesRef[] getTerms();
/** Array of term frequencies. Locations of the array correspond one to one
@ -54,7 +56,7 @@ public interface TermFreqVector {
* <code>term</code> appears. If this term does not appear in the array,
* return -1.
*/
public int indexOf(String term);
public int indexOf(BytesRef term);
/** Just like <code>indexOf(int)</code> but searches for a number of terms
@ -66,6 +68,6 @@ public interface TermFreqVector {
* @param start index in the array where the list of terms starts
* @param len the number of terms in the list
*/
public int[] indexesOf(String[] terms, int start, int len);
public int[] indexesOf(BytesRef[] terms, int start, int len);
}

View File

@ -1,5 +1,7 @@
package org.apache.lucene.index;
import org.apache.lucene.util.BytesRef;
/**
* Copyright 2007 The Apache Software Foundation
* <p/>
@ -21,7 +23,7 @@ package org.apache.lucene.index;
*/
public class TermVectorEntry {
private String field;
private String term;
private BytesRef term;
private int frequency;
private TermVectorOffsetInfo [] offsets;
int [] positions;
@ -30,7 +32,7 @@ public class TermVectorEntry {
public TermVectorEntry() {
}
public TermVectorEntry(String field, String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public TermVectorEntry(String field, BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
this.field = field;
this.term = term;
this.frequency = frequency;
@ -55,7 +57,7 @@ public class TermVectorEntry {
return positions;
}
public String getTerm() {
public BytesRef getTerm() {
return term;
}

View File

@ -1,4 +1,7 @@
package org.apache.lucene.index;
import org.apache.lucene.util.BytesRef;
/**
* Copyright 2007 The Apache Software Foundation
*
@ -62,7 +65,7 @@ public abstract class TermVectorMapper {
* @param offsets null if the offset is not specified, otherwise the offset into the field of the term
* @param positions null if the position is not specified, otherwise the position in the field of the term
*/
public abstract void map(String term, int frequency, TermVectorOffsetInfo [] offsets, int [] positions);
public abstract void map(BytesRef term, int frequency, TermVectorOffsetInfo [] offsets, int [] positions);
/**
* Indicate to Lucene that even if there are positions stored, this mapper is not interested in them and they

View File

@ -21,6 +21,7 @@ import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.util.Arrays;
@ -415,14 +416,15 @@ class TermVectorsReader implements Cloneable {
deltaLength = tvf.readVInt();
totalLength = start + deltaLength;
final String term;
final BytesRef term = new BytesRef(totalLength);
// Term stored as utf8 bytes
if (byteBuffer.length < totalLength) {
byteBuffer = ArrayUtil.grow(byteBuffer, totalLength);
}
tvf.readBytes(byteBuffer, start, deltaLength);
term = new String(byteBuffer, 0, totalLength, "UTF-8");
System.arraycopy(byteBuffer, 0, term.bytes, 0, totalLength);
term.length = totalLength;
int freq = tvf.readVInt();
int [] positions = null;
if (storePositions) { //read in the positions
@ -491,7 +493,7 @@ class TermVectorsReader implements Cloneable {
class ParallelArrayTermVectorMapper extends TermVectorMapper
{
private String[] terms;
private BytesRef[] terms;
private int[] termFreqs;
private int positions[][];
private TermVectorOffsetInfo offsets[][];
@ -503,7 +505,7 @@ class ParallelArrayTermVectorMapper extends TermVectorMapper
@Override
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
this.field = field;
terms = new String[numTerms];
terms = new BytesRef[numTerms];
termFreqs = new int[numTerms];
this.storingOffsets = storeOffsets;
this.storingPositions = storePositions;
@ -514,7 +516,7 @@ class ParallelArrayTermVectorMapper extends TermVectorMapper
}
@Override
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
terms[currentPosition] = term;
termFreqs[currentPosition] = frequency;
if (storingOffsets)

View File

@ -21,7 +21,6 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
import java.io.IOException;
@ -29,7 +28,6 @@ final class TermVectorsWriter {
private IndexOutput tvx = null, tvd = null, tvf = null;
private FieldInfos fieldInfos;
final BytesRef[] utf8Results = new BytesRef[] {new BytesRef(10), new BytesRef(10)};
public TermVectorsWriter(Directory directory, String segment,
FieldInfos fieldInfos)
@ -97,25 +95,19 @@ final class TermVectorsWriter {
tvf.writeVInt(bits);
final String[] terms = vectors[i].getTerms();
final BytesRef[] terms = vectors[i].getTerms();
final int[] freqs = vectors[i].getTermFrequencies();
int utf8Upto = 0;
utf8Results[1].length = 0;
for (int j=0; j<numTerms; j++) {
UnicodeUtil.UTF16toUTF8(terms[j], 0, terms[j].length(), utf8Results[utf8Upto]);
int start = StringHelper.bytesDifference(utf8Results[1-utf8Upto].bytes,
utf8Results[1-utf8Upto].length,
utf8Results[utf8Upto].bytes,
utf8Results[utf8Upto].length);
int length = utf8Results[utf8Upto].length - start;
int start = j == 0 ? 0 : StringHelper.bytesDifference(terms[j-1].bytes,
terms[j-1].length,
terms[j].bytes,
terms[j].length);
int length = terms[j].length - start;
tvf.writeVInt(start); // write shared prefix length
tvf.writeVInt(length); // write delta length
tvf.writeBytes(utf8Results[utf8Upto].bytes, start, length); // write delta bytes
utf8Upto = 1-utf8Upto;
tvf.writeBytes(terms[j].bytes, start, length); // write delta bytes
final int termFreq = freqs[j];

View File

@ -237,7 +237,6 @@ public class PreFlexFields extends FieldsProducer {
private FieldInfo fieldInfo;
private boolean skipNext;
private BytesRef current;
private final BytesRef scratchBytesRef = new BytesRef();
private int[] surrogateSeekPending = new int[1];
private boolean[] surrogateDidSeekBack = new boolean[1];
@ -319,7 +318,8 @@ public class PreFlexFields extends FieldsProducer {
assert pendingPrefix != null;
assert pendingPrefix.length > seekPrefix;
pendingPrefix[seekPrefix] = UnicodeUtil.UNI_SUR_HIGH_START;
Term t2 = protoTerm.createTerm(new String(pendingPrefix, 0, 1+seekPrefix));
pendingPrefix[1+seekPrefix] = UnicodeUtil.UNI_SUR_LOW_START;
Term t2 = protoTerm.createTerm(new BytesRef(pendingPrefix, 0, 2+seekPrefix));
if (DEBUG_SURROGATES) {
System.out.println(" do pop; seek back to " + UnicodeUtil.toHexString(t2.text()));
}
@ -334,7 +334,7 @@ public class PreFlexFields extends FieldsProducer {
assert pendingPrefix != null;
assert pendingPrefix.length > seekPrefix;
pendingPrefix[seekPrefix] = 0xffff;
Term t2 = protoTerm.createTerm(new String(pendingPrefix, 0, 1+seekPrefix));
Term t2 = protoTerm.createTerm(new BytesRef(pendingPrefix, 0, 1+seekPrefix));
if (DEBUG_SURROGATES) {
System.out.println(" finish pop; seek fwd to " + UnicodeUtil.toHexString(t2.text()));
}
@ -358,6 +358,9 @@ public class PreFlexFields extends FieldsProducer {
return false;
}
private UnicodeUtil.UTF16Result termBuffer = new UnicodeUtil.UTF16Result();
private UnicodeUtil.UTF16Result seekBuffer = new UnicodeUtil.UTF16Result();
private boolean pushNewSurrogate() throws IOException {
if (DEBUG_SURROGATES) {
System.out.println(" check push newSuffix=" + newSuffixStart + " stack=" + getStack());
@ -366,11 +369,12 @@ public class PreFlexFields extends FieldsProducer {
if (t == null || t.field() != fieldInfo.name) {
return false;
}
final String text = t.text();
final int textLen = text.length();
for(int i=Math.max(0,newSuffixStart);i<textLen;i++) {
final char ch = text.charAt(i);
final BytesRef bytes = t.bytes();
UnicodeUtil.UTF8toUTF16(bytes.bytes, bytes.offset, bytes.length, termBuffer);
for(int i=Math.max(0,newSuffixStart);i<termBuffer.length;i++) {
final char ch = termBuffer.result[i];
if (ch >= UnicodeUtil.UNI_SUR_HIGH_START && ch <= UnicodeUtil.UNI_SUR_HIGH_END && (surrogateSeekUpto == 0 || i > surrogateSeekPending[surrogateSeekUpto-1])) {
if (DEBUG_SURROGATES) {
@ -385,24 +389,27 @@ public class PreFlexFields extends FieldsProducer {
// surrogate range; if so, we must first iterate
// them, then seek back to the surrogates
char[] testPrefix = new char[i+1];
char[] testPrefix = new char[i+2];
for(int j=0;j<i;j++) {
testPrefix[j] = text.charAt(j);
testPrefix[j] = termBuffer.result[j];
}
testPrefix[i] = 1+UnicodeUtil.UNI_SUR_LOW_END;
getTermsDict().seekEnum(seekTermEnum, protoTerm.createTerm(new String(testPrefix)));
getTermsDict().seekEnum(seekTermEnum, protoTerm.createTerm(new BytesRef(testPrefix, 0, i+1)));
Term t2 = seekTermEnum.term();
boolean isPrefix;
if (t2 != null && t2.field() == fieldInfo.name) {
String seekText = t2.text();
final BytesRef seekBytes = t2.bytes();
UnicodeUtil.UTF8toUTF16(seekBytes.bytes, seekBytes.offset, seekBytes.length, seekBuffer);
isPrefix = true;
if (DEBUG_SURROGATES) {
System.out.println(" seek found " + UnicodeUtil.toHexString(seekText));
System.out.println(" seek found " + UnicodeUtil.toHexString(t2.text()));
}
for(int j=0;j<i;j++) {
if (testPrefix[j] != seekText.charAt(j)) {
if (testPrefix[j] != seekBuffer.result[j]) {
isPrefix = false;
break;
}
@ -481,7 +488,7 @@ public class PreFlexFields extends FieldsProducer {
}
skipNext = false;
final TermInfosReader tis = getTermsDict();
final Term t0 = protoTerm.createTerm(term.utf8ToString());
final Term t0 = protoTerm.createTerm(term);
assert termEnum != null;
@ -496,13 +503,7 @@ public class PreFlexFields extends FieldsProducer {
final Term t = termEnum.term();
final BytesRef tr;
if (t != null) {
tr = scratchBytesRef;
scratchBytesRef.copy(t.text());
} else {
tr = null;
}
final BytesRef tr = t == null ? null : t.bytes();
if (t != null && t.field() == fieldInfo.name && term.bytesEquals(tr)) {
current = tr;
@ -526,8 +527,7 @@ public class PreFlexFields extends FieldsProducer {
if (termEnum.term() == null) {
return null;
} else {
scratchBytesRef.copy(termEnum.term().text());
return current = scratchBytesRef;
return current = termEnum.term().bytes();
}
}
if (termEnum.next() && termEnum.term().field() == fieldInfo.name) {
@ -541,8 +541,7 @@ public class PreFlexFields extends FieldsProducer {
assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
current = null;
} else {
scratchBytesRef.copy(t.text());
current = scratchBytesRef;
current = t.bytes();
}
return current;
} else {
@ -557,8 +556,7 @@ public class PreFlexFields extends FieldsProducer {
assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
return null;
} else {
scratchBytesRef.copy(t.text());
current = scratchBytesRef;
current = t.bytes();
return current;
}
}

View File

@ -96,10 +96,9 @@ final class TermBuffer implements Cloneable {
reset();
return;
}
final String termText = term.text();
final int termLen = termText.length();
text.setLength(termLen);
termText.getChars(0, termLen, text.result, 0);
final BytesRef termBytes = term.bytes();
UnicodeUtil.UTF8toUTF16(termBytes.bytes, termBytes.offset, termBytes.length, text);
dirty = true;
field = term.field();
this.term = term;
@ -124,7 +123,7 @@ final class TermBuffer implements Cloneable {
return null;
if (term == null)
term = new Term(field, new String(text.result, 0, text.length), false);
term = new Term(field, new BytesRef(text.result, 0, text.length), false);
return term;
}

View File

@ -189,7 +189,7 @@ public final class TermInfosReader {
while (hi >= lo) {
int mid = (lo + hi) >>> 1;
int delta = term.compareTo(indexTerms[mid]);
int delta = term.compareToUTF16(indexTerms[mid]);
if (delta < 0)
hi = mid - 1;
else if (delta > 0)
@ -234,17 +234,17 @@ public final class TermInfosReader {
// optimize sequential access: first try scanning cached enum w/o seeking
if (enumerator.term() != null // term is at or past current
&& ((enumerator.prev() != null && term.compareTo(enumerator.prev())> 0)
|| term.compareTo(enumerator.term()) >= 0)) {
&& ((enumerator.prev() != null && term.compareToUTF16(enumerator.prev())> 0)
|| term.compareToUTF16(enumerator.term()) >= 0)) {
int enumOffset = (int)(enumerator.position/totalIndexInterval)+1;
if (indexTerms.length == enumOffset // but before end of block
|| term.compareTo(indexTerms[enumOffset]) < 0) {
|| term.compareToUTF16(indexTerms[enumOffset]) < 0) {
// no need to seek
final TermInfo ti;
int numScans = enumerator.scanTo(term);
if (enumerator.term() != null && term.compareTo(enumerator.term()) == 0) {
if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
ti = enumerator.termInfo();
if (numScans > 1) {
// we only want to put this TermInfo into the cache if
@ -279,7 +279,7 @@ public final class TermInfosReader {
seekEnum(enumerator, indexPos);
enumerator.scanTo(term);
final TermInfo ti;
if (enumerator.term() != null && term.compareTo(enumerator.term()) == 0) {
if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
ti = enumerator.termInfo();
if (tiOrd == null) {
termsCache.put(new CloneableTerm(term), new TermInfoAndOrd(ti, (int) enumerator.position));
@ -328,9 +328,9 @@ public final class TermInfosReader {
SegmentTermEnum enumerator = getThreadResources().termEnum;
seekEnum(enumerator, indexOffset);
while(term.compareTo(enumerator.term()) > 0 && enumerator.next()) {}
while(term.compareToUTF16(enumerator.term()) > 0 && enumerator.next()) {}
if (term.compareTo(enumerator.term()) == 0)
if (term.compareToUTF16(enumerator.term()) == 0)
return enumerator.position;
else
return -1;

View File

@ -95,11 +95,18 @@ import org.apache.lucene.util.BytesRef;
public class FieldCacheTermsFilter extends Filter {
private String field;
private String[] terms;
private BytesRef[] terms;
public FieldCacheTermsFilter(String field, BytesRef... terms) {
this.field = field;
this.terms = terms;
}
public FieldCacheTermsFilter(String field, String... terms) {
this.field = field;
this.terms = terms;
this.terms = new BytesRef[terms.length];
for (int i = 0; i < terms.length; i++)
this.terms[i] = new BytesRef(terms[i]);
}
public FieldCache getFieldCache() {
@ -121,7 +128,7 @@ public class FieldCacheTermsFilter extends Filter {
openBitSet = new OpenBitSet(this.fcsi.size());
final BytesRef spare = new BytesRef();
for (int i=0;i<terms.length;i++) {
int termNumber = this.fcsi.binarySearchLookup(new BytesRef(terms[i]), spare);
int termNumber = this.fcsi.binarySearchLookup(terms[i], spare);
if (termNumber > 0) {
openBitSet.fastSet(termNumber);
}

View File

@ -499,14 +499,13 @@ class UnionDocsAndPositionsEnum extends DocsAndPositionsEnum {
List<DocsAndPositionsEnum> docsEnums = new LinkedList<DocsAndPositionsEnum>();
final Bits delDocs = MultiFields.getDeletedDocs(indexReader);
for (int i = 0; i < terms.length; i++) {
final BytesRef text = new BytesRef(terms[i].text());
DocsAndPositionsEnum postings = indexReader.termPositionsEnum(delDocs,
terms[i].field(),
text);
terms[i].bytes());
if (postings != null) {
docsEnums.add(postings);
} else {
if (MultiFields.getTermDocsEnum(indexReader, delDocs, terms[i].field(), text) != null) {
if (MultiFields.getTermDocsEnum(indexReader, delDocs, terms[i].field(), terms[i].bytes()) != null) {
// term does exist, but has no positions
throw new IllegalStateException("field \"" + terms[i].field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run PhraseQuery (term=" + terms[i].text() + ")");
}

View File

@ -32,6 +32,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.queryParser.QueryParser; // for javadoc
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.PagedBytes;
/**
* An abstract {@link Query} that matches documents
@ -177,11 +178,6 @@ public abstract class MultiTermQuery extends Query {
private abstract static class BooleanQueryRewrite extends RewriteMethod {
protected final int collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
if (query.field == null) {
throw new NullPointerException("If you implement getTermsEnum(), you must specify a non-null field in the constructor of MultiTermQuery.");
}
final Fields fields = MultiFields.getFields(reader);
if (fields == null) {
// reader has no fields
@ -203,10 +199,9 @@ public abstract class MultiTermQuery extends Query {
termsEnum.attributes().addAttribute(BoostAttribute.class);
collector.boostAtt = boostAtt;
int count = 0;
BytesRef term;
final Term placeholderTerm = new Term(query.field);
while ((term = termsEnum.next()) != null) {
if (collector.collect(placeholderTerm.createTerm(term.utf8ToString()), boostAtt.getBoost())) {
BytesRef bytes;
while ((bytes = termsEnum.next()) != null) {
if (collector.collect(bytes, boostAtt.getBoost())) {
count++;
} else {
break;
@ -217,15 +212,15 @@ public abstract class MultiTermQuery extends Query {
}
protected static abstract class TermCollector {
/** this field is only set if a boostAttribute is used (e.g. {@link FuzzyTermsEnum}) */
private BoostAttribute boostAtt = null;
/** return false to stop collecting */
public abstract boolean collect(Term t, float boost) throws IOException;
public abstract boolean collect(BytesRef bytes, float boost) throws IOException;
/** set the minimum boost as a hint for the term producer */
protected final void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost) {
if (boostAtt != null) boostAtt.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost);
assert boostAtt != null;
boostAtt.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost);
}
}
}
@ -234,9 +229,11 @@ public abstract class MultiTermQuery extends Query {
@Override
public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException {
final BooleanQuery result = new BooleanQuery(true);
final Term placeholderTerm = new Term(query.field);
query.incTotalNumberOfTerms(collectTerms(reader, query, new TermCollector() {
public boolean collect(Term t, float boost) {
TermQuery tq = new TermQuery(t); // found a match
public boolean collect(BytesRef bytes, float boost) {
// add new TQ, we must clone the term, else it may get overwritten!
TermQuery tq = new TermQuery(placeholderTerm.createTerm(new BytesRef(bytes)));
tq.setBoost(query.getBoost() * boost); // set the boost
result.add(tq, BooleanClause.Occur.SHOULD); // add to query
return true;
@ -297,16 +294,16 @@ public abstract class MultiTermQuery extends Query {
protected abstract Query getQuery(Term term);
@Override
public Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException {
public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException {
final int maxSize = Math.min(size, BooleanQuery.getMaxClauseCount());
final PriorityQueue<ScoreTerm> stQueue = new PriorityQueue<ScoreTerm>();
collectTerms(reader, query, new TermCollector() {
public boolean collect(Term t, float boost) {
public boolean collect(BytesRef bytes, float boost) {
// ignore uncompetetive hits
if (stQueue.size() >= maxSize && boost <= stQueue.peek().boost)
return true;
// add new entry in PQ
st.term = t;
// add new entry in PQ, we must clone the term, else it may get overwritten!
st.bytes.copy(bytes);
st.boost = boost;
stQueue.offer(st);
// possibly drop entries from queue
@ -319,9 +316,11 @@ public abstract class MultiTermQuery extends Query {
private ScoreTerm st = new ScoreTerm();
});
final Term placeholderTerm = new Term(query.field);
final BooleanQuery bq = new BooleanQuery(true);
for (final ScoreTerm st : stQueue) {
Query tq = getQuery(st.term); // found a match
// add new query, we must clone the term, else it may get overwritten!
Query tq = getQuery(placeholderTerm.createTerm(st.bytes));
tq.setBoost(query.getBoost() * st.boost); // set the boost
bq.add(tq, BooleanClause.Occur.SHOULD); // add to query
}
@ -348,12 +347,13 @@ public abstract class MultiTermQuery extends Query {
}
private static class ScoreTerm implements Comparable<ScoreTerm> {
public Term term;
public final BytesRef bytes = new BytesRef();
public float boost;
public int compareTo(ScoreTerm other) {
if (this.boost == other.boost)
return other.term.compareTo(this.term);
// TODO: is it OK to use default compare here?
return other.bytes.compareTo(this.bytes);
else
return Float.compare(this.boost, other.boost);
}
@ -530,58 +530,67 @@ public abstract class MultiTermQuery extends Query {
final int docCountCutoff = (int) ((docCountPercent / 100.) * reader.maxDoc());
final int termCountLimit = Math.min(BooleanQuery.getMaxClauseCount(), termCountCutoff);
final CutOffTermCollector col = new CutOffTermCollector(reader, docCountCutoff, termCountLimit);
final CutOffTermCollector col = new CutOffTermCollector(reader, query.field, docCountCutoff, termCountLimit);
collectTerms(reader, query, col);
if (col.hasCutOff) {
return CONSTANT_SCORE_FILTER_REWRITE.rewrite(reader, query);
} else if (col.termCount == 0) {
return new BooleanQuery(true);
} else {
final Query result;
if (col.pendingTerms.isEmpty()) {
result = new BooleanQuery(true);
} else {
BooleanQuery bq = new BooleanQuery(true);
for(Term term : col.pendingTerms) {
TermQuery tq = new TermQuery(term);
bq.add(tq, BooleanClause.Occur.SHOULD);
final PagedBytes.Reader bytesReader = col.pendingTerms.freeze(false);
try {
final BooleanQuery bq = new BooleanQuery(true);
final Term placeholderTerm = new Term(query.field);
long start = col.startOffset;
for(int i = 0; i < col.termCount; i++) {
final BytesRef bytes = new BytesRef();
start = bytesReader.fillUsingLengthPrefix3(bytes, start);
bq.add(new TermQuery(placeholderTerm.createTerm(bytes)), BooleanClause.Occur.SHOULD);
}
// Strip scores
result = new ConstantScoreQuery(new QueryWrapperFilter(bq));
final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq));
result.setBoost(query.getBoost());
query.incTotalNumberOfTerms(col.termCount);
return result;
} finally {
bytesReader.close();
}
query.incTotalNumberOfTerms(col.pendingTerms.size());
return result;
}
}
private static final class CutOffTermCollector extends TermCollector {
CutOffTermCollector(IndexReader reader, int docCountCutoff, int termCountLimit) {
CutOffTermCollector(IndexReader reader, String field, int docCountCutoff, int termCountLimit) {
this.reader = reader;
this.field = field;
this.docCountCutoff = docCountCutoff;
this.termCountLimit = termCountLimit;
}
public boolean collect(Term t, float boost) throws IOException {
pendingTerms.add(t);
if (pendingTerms.size() >= termCountLimit || docVisitCount >= docCountCutoff) {
public boolean collect(BytesRef bytes, float boost) throws IOException {
termCount++;
if (termCount >= termCountLimit || docVisitCount >= docCountCutoff) {
hasCutOff = true;
return false;
}
pendingTerms.copyUsingLengthPrefix(bytes);
// Loading the TermInfo from the terms dict here
// should not be costly, because 1) the
// query/filter will load the TermInfo when it
// runs, and 2) the terms dict has a cache:
// @deprecated: in 4.0 use BytesRef for collectTerms()
docVisitCount += reader.docFreq(t);
docVisitCount += reader.docFreq(field, bytes);
return true;
}
int docVisitCount = 0;
boolean hasCutOff = false;
int termCount = 0;
final IndexReader reader;
final String field;
final int docCountCutoff, termCountLimit;
final ArrayList<Term> pendingTerms = new ArrayList<Term>();
final PagedBytes pendingTerms = new PagedBytes(15); // max term size is 32 KiB
final long startOffset = pendingTerms.getPointer();
}
@Override
@ -647,18 +656,7 @@ public abstract class MultiTermQuery extends Query {
*/
public MultiTermQuery(final String field) {
this.field = field;
}
/**
* Constructs a query matching terms that cannot be represented with a single
* Term.
* @deprecated Use {@link #MultiTermQuery(String)}, as the flex branch can
* only work on one field per terms enum. If you override
* {@link #getTermsEnum(IndexReader)}, you cannot use this ctor.
*/
@Deprecated
public MultiTermQuery() {
this(null);
assert field != null;
}
/** Returns the field name for this query */

View File

@ -106,10 +106,6 @@ public class MultiTermQueryWrapperFilter<Q extends MultiTermQuery> extends Filte
*/
@Override
public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
if (query.field == null) {
throw new NullPointerException("If you implement getTermsEnum(), you must specify a non-null field in the constructor of MultiTermQuery.");
}
final Fields fields = MultiFields.getFields(reader);
if (fields == null) {
// reader has no fields

View File

@ -184,15 +184,14 @@ public class PhraseQuery extends Query {
final Bits delDocs = MultiFields.getDeletedDocs(reader);
for (int i = 0; i < terms.size(); i++) {
final Term t = terms.get(i);
final BytesRef text = new BytesRef(t.text());
DocsAndPositionsEnum postingsEnum = MultiFields.getTermPositionsEnum(reader,
delDocs,
t.field(),
text);
t.bytes());
// PhraseQuery on a field that did not index
// positions.
if (postingsEnum == null) {
if (MultiFields.getTermDocsEnum(reader, delDocs, t.field(), text) != null) {
if (MultiFields.getTermDocsEnum(reader, delDocs, t.field(), t.bytes()) != null) {
// term does exist, but has no positions
throw new IllegalStateException("field \"" + t.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run PhraseQuery (term=" + t.text() + ")");
} else {
@ -200,7 +199,7 @@ public class PhraseQuery extends Query {
return null;
}
}
postingsFreqs[i] = new PostingsAndFreq(postingsEnum, reader.docFreq(t.field(), text), positions.get(i).intValue());
postingsFreqs[i] = new PostingsAndFreq(postingsEnum, reader.docFreq(t.field(), t.bytes()), positions.get(i).intValue());
}
// sort by increasing docFreq order

View File

@ -46,7 +46,7 @@ public class PrefixQuery extends MultiTermQuery {
@Override
protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
if (prefix.text().length() == 0) {
if (prefix.bytes().length == 0) {
// no prefix -- match all terms for this field:
final Terms terms = MultiFields.getTerms(reader, getField());
return (terms != null) ? terms.iterator() : TermsEnum.EMPTY;

View File

@ -36,7 +36,7 @@ public class PrefixTermsEnum extends FilteredTermsEnum {
public PrefixTermsEnum(IndexReader reader, Term prefix) throws IOException {
super(reader, prefix.field());
setInitialSeekTerm(prefixRef = new BytesRef(prefix.text()));
setInitialSeekTerm(prefixRef = prefix.bytes());
}
@Override

View File

@ -29,14 +29,16 @@ import java.util.Map;
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.TermToBytesRefAttribute;
import org.apache.lucene.index.TermFreqVector;
import org.apache.lucene.util.BytesRef;
/**
*
*
**/
public class QueryTermVector implements TermFreqVector {
private String [] terms = new String[0];
private BytesRef [] terms = new BytesRef[0];
private int [] termFreqs = new int[0];
public String getField() { return null; }
@ -45,7 +47,7 @@ public class QueryTermVector implements TermFreqVector {
*
* @param queryTerms The original list of terms from the query, can contain duplicates
*/
public QueryTermVector(String [] queryTerms) {
public QueryTermVector(BytesRef [] queryTerms) {
processTerms(queryTerms);
}
@ -56,35 +58,37 @@ public class QueryTermVector implements TermFreqVector {
TokenStream stream = analyzer.tokenStream("", new StringReader(queryString));
if (stream != null)
{
List<String> terms = new ArrayList<String>();
List<BytesRef> terms = new ArrayList<BytesRef>();
try {
boolean hasMoreTokens = false;
stream.reset();
final CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
final TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
hasMoreTokens = stream.incrementToken();
while (hasMoreTokens) {
terms.add(termAtt.toString());
BytesRef bytes = new BytesRef();
termAtt.toBytesRef(bytes);
terms.add(bytes);
hasMoreTokens = stream.incrementToken();
}
processTerms(terms.toArray(new String[terms.size()]));
processTerms(terms.toArray(new BytesRef[terms.size()]));
} catch (IOException e) {
}
}
}
}
private void processTerms(String[] queryTerms) {
private void processTerms(BytesRef[] queryTerms) {
if (queryTerms != null) {
Arrays.sort(queryTerms);
Map<String,Integer> tmpSet = new HashMap<String,Integer>(queryTerms.length);
Map<BytesRef,Integer> tmpSet = new HashMap<BytesRef,Integer>(queryTerms.length);
//filter out duplicates
List<String> tmpList = new ArrayList<String>(queryTerms.length);
List<BytesRef> tmpList = new ArrayList<BytesRef>(queryTerms.length);
List<Integer> tmpFreqs = new ArrayList<Integer>(queryTerms.length);
int j = 0;
for (int i = 0; i < queryTerms.length; i++) {
String term = queryTerms[i];
BytesRef term = queryTerms[i];
Integer position = tmpSet.get(term);
if (position == null) {
tmpSet.put(term, Integer.valueOf(j++));
@ -112,7 +116,7 @@ public class QueryTermVector implements TermFreqVector {
sb.append('{');
for (int i=0; i<terms.length; i++) {
if (i>0) sb.append(", ");
sb.append(terms[i]).append('/').append(termFreqs[i]);
sb.append(terms[i].utf8ToString()).append('/').append(termFreqs[i]);
}
sb.append('}');
return sb.toString();
@ -123,7 +127,7 @@ public class QueryTermVector implements TermFreqVector {
return terms.length;
}
public String[] getTerms() {
public BytesRef[] getTerms() {
return terms;
}
@ -131,12 +135,12 @@ public class QueryTermVector implements TermFreqVector {
return termFreqs;
}
public int indexOf(String term) {
public int indexOf(BytesRef term) {
int res = Arrays.binarySearch(terms, term);
return res >= 0 ? res : -1;
}
public int[] indexesOf(String[] terms, int start, int len) {
public int[] indexesOf(BytesRef[] terms, int start, int len) {
int res[] = new int[len];
for (int i=0; i < len; i++) {

View File

@ -41,7 +41,7 @@ public final class SingleTermsEnum extends FilteredTermsEnum {
*/
public SingleTermsEnum(IndexReader reader, Term singleTerm) throws IOException {
super(reader, singleTerm.field());
singleRef = new BytesRef(singleTerm.text());
singleRef = singleTerm.bytes();
setInitialSeekTerm(singleRef);
}

View File

@ -75,7 +75,7 @@ public class TermQuery extends Query {
public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
// NOTE: debateably, the caller should never pass in a
// multi reader...
DocsEnum docs = MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), new BytesRef(term.text()));
DocsEnum docs = MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), term.bytes());
if (docs == null) {
return null;
}
@ -118,7 +118,7 @@ public class TermQuery extends Query {
Explanation tfExplanation = new Explanation();
int tf = 0;
DocsEnum docs = reader.termDocsEnum(MultiFields.getDeletedDocs(reader), term.field(), new BytesRef(term.text()));
DocsEnum docs = reader.termDocsEnum(MultiFields.getDeletedDocs(reader), term.field(), term.bytes());
if (docs != null) {
int newDoc = docs.advance(doc);
if (newDoc == doc) {

View File

@ -85,16 +85,15 @@ public class SpanTermQuery extends SpanQuery {
public Spans getSpans(final IndexReader reader) throws IOException {
// NOTE: debateably, the caller should never pass in a
// multi reader...
final BytesRef textBytes = new BytesRef(term.text());
final DocsAndPositionsEnum postings = MultiFields.getTermPositionsEnum(reader,
MultiFields.getDeletedDocs(reader),
term.field(),
textBytes);
term.bytes());
if (postings != null) {
return new TermSpans(postings, term);
} else {
if (MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), textBytes) != null) {
if (MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), term.bytes()) != null) {
// term does exist, but has no positions
throw new IllegalStateException("field \"" + term.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run SpanTermQuery (term=" + term.text() + ")");
} else {

View File

@ -77,6 +77,16 @@ public final class BytesRef implements Comparable<BytesRef>, Externalizable {
this();
copy(text);
}
/**
* @param text Initialize the byte[] from the UTF8 bytes
* for the provided array. This must be well-formed
* unicode text, with no unpaired surrogates or U+FFFF.
*/
public BytesRef(char text[], int offset, int length) {
this(length * 4);
copy(text, offset, length);
}
public BytesRef(BytesRef other) {
this();
@ -106,6 +116,15 @@ public final class BytesRef implements Comparable<BytesRef>, Externalizable {
UnicodeUtil.UTF16toUTF8(text, 0, text.length(), this);
}
/**
* Copies the UTF8 bytes for this string.
*
* @param text Must be well-formed unicode text, with no
* unpaired surrogates or invalid UTF16 code units.
*/
public void copy(char text[], int offset, int length) {
UnicodeUtil.UTF16toUTF8(text, offset, length, this);
}
public boolean bytesEquals(BytesRef other) {
if (length == other.length) {
int otherUpto = other.offset;
@ -277,6 +296,62 @@ public final class BytesRef implements Comparable<BytesRef>, Externalizable {
}
}
private final static Comparator<BytesRef> utf8SortedAsUTF16SortOrder = new UTF8SortedAsUTF16Comparator();
public static Comparator<BytesRef> getUTF8SortedAsUTF16Comparator() {
return utf8SortedAsUTF16SortOrder;
}
private static class UTF8SortedAsUTF16Comparator implements Comparator<BytesRef> {
// Only singleton
private UTF8SortedAsUTF16Comparator() {};
public int compare(BytesRef a, BytesRef b) {
final byte[] aBytes = a.bytes;
int aUpto = a.offset;
final byte[] bBytes = b.bytes;
int bUpto = b.offset;
final int aStop;
if (a.length < b.length) {
aStop = aUpto + a.length;
} else {
aStop = aUpto + b.length;
}
while(aUpto < aStop) {
int aByte = aBytes[aUpto++] & 0xff;
int bByte = bBytes[bUpto++] & 0xff;
if (aByte != bByte) {
// See http://icu-project.org/docs/papers/utf16_code_point_order.html#utf-8-in-utf-16-order
// We know the terms are not equal, but, we may
// have to carefully fixup the bytes at the
// difference to match UTF16's sort order:
if (aByte >= 0xee && bByte >= 0xee) {
if ((aByte & 0xfe) == 0xee) {
aByte += 0x10;
}
if ((bByte&0xfe) == 0xee) {
bByte += 0x10;
}
}
return aByte - bByte;
}
}
// One is a prefix of the other, or, they are equal:
return a.length - b.length;
}
public boolean equals(Object other) {
return this == other;
}
}
public void writeExternal(ObjectOutput out)
throws IOException
{

View File

@ -125,6 +125,26 @@ public final class PagedBytes {
return index;
}
/** @lucene.internal Reads length as 1 or 2 byte vInt prefix, starting @ start.
* Returns the start offset of the next part, suitable as start parameter on next call
* to sequentially read all BytesRefs. */
public long fillUsingLengthPrefix3(BytesRef b, long start) {
final int index = (int) (start >> blockBits);
final int offset = (int) (start & blockMask);
final byte[] block = b.bytes = blocks[index];
if ((block[offset] & 128) == 0) {
b.length = block[offset];
b.offset = offset+1;
start += 1L + b.length;
} else {
b.length = (((int) (block[offset] & 0x7f)) << 8) | (block[1+offset] & 0xff);
b.offset = offset+2;
start += 2L + b.length;
assert b.length > 0;
}
return start;
}
/** @lucene.internal */
public byte[][] getBlocks() {
@ -230,7 +250,7 @@ public final class PagedBytes {
/** Commits final byte[], trimming it if necessary and if trim=true */
public Reader freeze(boolean trim) {
if (upto < blockSize) {
if (trim && upto < blockSize) {
final byte[] newBlock = new byte[upto];
System.arraycopy(currentBlock, 0, newBlock, 0, upto);
currentBlock = newBlock;

View File

@ -464,7 +464,7 @@ public class TestAddIndexes extends LuceneTestCase {
private void verifyTermDocs(Directory dir, Term term, int numDocs)
throws IOException {
IndexReader reader = IndexReader.open(dir, true);
DocsEnum docsEnum = MultiFields.getTermDocsEnum(reader, null, term.field, new BytesRef(term.text));
DocsEnum docsEnum = MultiFields.getTermDocsEnum(reader, null, term.field, term.bytes);
int count = 0;
while (docsEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS)
count++;

View File

@ -188,7 +188,7 @@ public class TestPayloads extends LuceneTestCase {
Term[] terms = generateTerms(fieldName, numTerms);
StringBuilder sb = new StringBuilder();
for (int i = 0; i < terms.length; i++) {
sb.append(terms[i].text);
sb.append(terms[i].text());
sb.append(" ");
}
String content = sb.toString();

View File

@ -15,6 +15,7 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import java.io.IOException;
@ -65,7 +66,7 @@ public class TestPositionBasedTermVectorMapper extends LuceneTestCase {
//Test single position
for (int i = 0; i < tokens.length; i++) {
String token = tokens[i];
mapper.map(token, 1, null, thePositions[i]);
mapper.map(new BytesRef(token), 1, null, thePositions[i]);
}
Map<String,Map<Integer,PositionBasedTermVectorMapper.TVPositionInfo>> map = mapper.getFieldToTerms();

View File

@ -100,7 +100,7 @@ public class TestSegmentMerger extends LuceneTestCase {
TermFreqVector vector = mergedReader.getTermFreqVector(0, DocHelper.TEXT_FIELD_2_KEY);
assertTrue(vector != null);
String [] terms = vector.getTerms();
BytesRef [] terms = vector.getTerms();
assertTrue(terms != null);
//System.out.println("Terms size: " + terms.length);
assertTrue(terms.length == 3);
@ -110,7 +110,7 @@ public class TestSegmentMerger extends LuceneTestCase {
assertTrue(vector instanceof TermPositionVector == true);
for (int i = 0; i < terms.length; i++) {
String term = terms[i];
String term = terms[i].utf8ToString();
int freq = freqs[i];
//System.out.println("Term: " + term + " Freq: " + freq);
assertTrue(DocHelper.FIELD_2_TEXT.indexOf(term) != -1);

View File

@ -192,11 +192,11 @@ public class TestSegmentReader extends LuceneTestCase {
public void testTermVectors() throws IOException {
TermFreqVector result = reader.getTermFreqVector(0, DocHelper.TEXT_FIELD_2_KEY);
assertTrue(result != null);
String [] terms = result.getTerms();
BytesRef [] terms = result.getTerms();
int [] freqs = result.getTermFrequencies();
assertTrue(terms != null && terms.length == 3 && freqs != null && freqs.length == 3);
for (int i = 0; i < terms.length; i++) {
String term = terms[i];
String term = terms[i].utf8ToString();
int freq = freqs[i];
assertTrue(DocHelper.FIELD_2_TEXT.indexOf(term) != -1);
assertTrue(freq > 0);

View File

@ -516,8 +516,8 @@ public class TestStressIndexing2 extends MultiCodecTestCase {
System.out.println("v1=" + v1 + " v2=" + v2 + " i=" + i + " of " + d1.length);
assertEquals(v1.size(), v2.size());
int numTerms = v1.size();
String[] terms1 = v1.getTerms();
String[] terms2 = v2.getTerms();
BytesRef[] terms1 = v1.getTerms();
BytesRef[] terms2 = v2.getTerms();
int[] freq1 = v1.getTermFrequencies();
int[] freq2 = v2.getTermFrequencies();
for(int j=0;j<numTerms;j++) {

View File

@ -32,6 +32,7 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.store.MockRAMDirectory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class TestTermVectorsReader extends LuceneTestCase {
@ -170,11 +171,11 @@ public class TestTermVectorsReader extends LuceneTestCase {
for (int j = 0; j < 5; j++) {
TermFreqVector vector = reader.get(j, testFields[0]);
assertTrue(vector != null);
String[] terms = vector.getTerms();
BytesRef[] terms = vector.getTerms();
assertTrue(terms != null);
assertTrue(terms.length == testTerms.length);
for (int i = 0; i < terms.length; i++) {
String term = terms[i];
String term = terms[i].utf8ToString();
//System.out.println("Term: " + term);
assertTrue(term.equals(testTerms[i]));
}
@ -184,14 +185,14 @@ public class TestTermVectorsReader extends LuceneTestCase {
public void testPositionReader() throws IOException {
TermVectorsReader reader = new TermVectorsReader(dir, seg, fieldInfos);
TermPositionVector vector;
String[] terms;
BytesRef[] terms;
vector = (TermPositionVector) reader.get(0, testFields[0]);
assertTrue(vector != null);
terms = vector.getTerms();
assertTrue(terms != null);
assertTrue(terms.length == testTerms.length);
for (int i = 0; i < terms.length; i++) {
String term = terms[i];
String term = terms[i].utf8ToString();
//System.out.println("Term: " + term);
assertTrue(term.equals(testTerms[i]));
int[] positions = vector.getTermPositions(i);
@ -217,7 +218,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
assertTrue(terms != null);
assertTrue(terms.length == testTerms.length);
for (int i = 0; i < terms.length; i++) {
String term = terms[i];
String term = terms[i].utf8ToString();
//System.out.println("Term: " + term);
assertTrue(term.equals(testTerms[i]));
}
@ -227,11 +228,11 @@ public class TestTermVectorsReader extends LuceneTestCase {
TermVectorsReader reader = new TermVectorsReader(dir, seg, fieldInfos);
TermPositionVector vector = (TermPositionVector) reader.get(0, testFields[0]);
assertTrue(vector != null);
String[] terms = vector.getTerms();
BytesRef[] terms = vector.getTerms();
assertTrue(terms != null);
assertTrue(terms.length == testTerms.length);
for (int i = 0; i < terms.length; i++) {
String term = terms[i];
String term = terms[i].utf8ToString();
//System.out.println("Term: " + term);
assertTrue(term.equals(testTerms[i]));
int[] positions = vector.getTermPositions(i);
@ -413,7 +414,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
}
@Override
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
if (documentNumber == -1) {
throw new RuntimeException("Documentnumber should be set at this point!");
}

View File

@ -76,7 +76,6 @@ final class TermInfosWriter {
private int lastFieldNumber = -1;
private TermInfosWriter other;
private BytesRef utf8Result = new BytesRef(10);
TermInfosWriter(Directory directory, String segment, FieldInfos fis,
int interval)
@ -106,8 +105,7 @@ final class TermInfosWriter {
}
void add(Term term, TermInfo ti) throws IOException {
UnicodeUtil.UTF16toUTF8(term.text(), 0, term.text().length(), utf8Result);
add(fieldInfos.fieldNumber(term.field()), utf8Result.bytes, utf8Result.length, ti);
add(fieldInfos.fieldNumber(term.field()), term.bytes().bytes, term.bytes().length, ti);
}
// Currently used only by assert statements

View File

@ -30,25 +30,6 @@ import org.junit.Test;
public class TestSurrogates extends LuceneTestCaseJ4 {
// like Term, but uses BytesRef for text
private static class FieldAndText implements Comparable<FieldAndText> {
String field;
BytesRef text;
public FieldAndText(Term t) {
field = t.field();
text = new BytesRef(t.text());
}
public int compareTo(FieldAndText other) {
if (other.field == field) {
return text.compareTo(other.text);
} else {
return field.compareTo(other.field);
}
}
}
// chooses from a very limited alphabet to exacerbate the
// surrogate seeking required
private static String makeDifficultRandomUnicodeString(Random r) {
@ -76,7 +57,7 @@ public class TestSurrogates extends LuceneTestCaseJ4 {
return new String(buffer, 0, end);
}
private SegmentInfo makePreFlexSegment(Random r, String segName, Directory dir, FieldInfos fieldInfos, Codec codec, List<FieldAndText> fieldTerms) throws IOException {
private SegmentInfo makePreFlexSegment(Random r, String segName, Directory dir, FieldInfos fieldInfos, Codec codec, List<Term> fieldTerms) throws IOException {
final int numField = _TestUtil.nextInt(r, 2, 5);
@ -110,11 +91,14 @@ public class TestSurrogates extends LuceneTestCaseJ4 {
fieldInfos.write(dir, segName);
// sorts in UTF16 order, just like preflex:
Collections.sort(terms);
Collections.sort(terms, new Comparator<Term>() {
public int compare(Term o1, Term o2) {
return o1.compareToUTF16(o2);
}
});
TermInfosWriter w = new TermInfosWriter(dir, segName, fieldInfos, 128);
TermInfo ti = new TermInfo();
BytesRef utf8 = new BytesRef(10);
String lastText = null;
int uniqueTermCount = 0;
if (VERBOSE) {
@ -127,23 +111,22 @@ public class TestSurrogates extends LuceneTestCaseJ4 {
if (lastText != null && lastText.equals(text)) {
continue;
}
fieldTerms.add(new FieldAndText(t));
fieldTerms.add(t);
uniqueTermCount++;
lastText = text;
UnicodeUtil.UTF16toUTF8(text, 0, text.length(), utf8);
if (VERBOSE) {
System.out.println(" " + toHexString(t));
}
w.add(fi.number, utf8.bytes, utf8.length, ti);
w.add(fi.number, t.bytes().bytes, t.bytes().length, ti);
}
w.close();
Collections.sort(fieldTerms);
if (VERBOSE) {
System.out.println("\nTEST: codepoint order");
for(FieldAndText t: fieldTerms) {
System.out.println(" " + t.field + ":" + UnicodeUtil.toHexString(t.text.utf8ToString()));
for(Term t: fieldTerms) {
System.out.println(" " + t.field() + ":" + toHexString(t));
}
}
@ -166,7 +149,7 @@ public class TestSurrogates extends LuceneTestCaseJ4 {
Random r = newRandom();
FieldInfos fieldInfos = new FieldInfos();
List<FieldAndText> fieldTerms = new ArrayList<FieldAndText>();
List<Term> fieldTerms = new ArrayList<Term>();
SegmentInfo si = makePreFlexSegment(r, "_0", dir, fieldInfos, codec, fieldTerms);
// hack alert!!
@ -188,8 +171,8 @@ public class TestSurrogates extends LuceneTestCaseJ4 {
BytesRef text;
BytesRef lastText = null;
while((text = termsEnum.next()) != null) {
UnicodeUtil.UTF8toUTF16(text.bytes, text.offset, text.length, utf16);
if (VERBOSE) {
UnicodeUtil.UTF8toUTF16(text.bytes, text.offset, text.length, utf16);
System.out.println("got term=" + field + ":" + UnicodeUtil.toHexString(new String(utf16.result, 0, utf16.length)));
System.out.println();
}
@ -199,8 +182,8 @@ public class TestSurrogates extends LuceneTestCaseJ4 {
assertTrue(lastText.compareTo(text) < 0);
lastText.copy(text);
}
assertEquals(fieldTerms.get(termCount).field, field);
assertEquals(fieldTerms.get(termCount).text, text);
assertEquals(fieldTerms.get(termCount).field(), field);
assertEquals(fieldTerms.get(termCount).bytes(), text);
termCount++;
}
if (VERBOSE) {

View File

@ -17,6 +17,7 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.*;
@ -175,11 +176,11 @@ class MultiThreadTermVectorsReader implements Runnable {
private void verifyVectors(TermFreqVector[] vectors, int num) {
StringBuilder temp = new StringBuilder();
String[] terms = null;
BytesRef[] terms = null;
for (int i = 0; i < vectors.length; i++) {
terms = vectors[i].getTerms();
for (int z = 0; z < terms.length; z++) {
temp.append(terms[z]);
temp.append(terms[z].utf8ToString());
}
}

View File

@ -17,6 +17,7 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.analysis.MockAnalyzer;
@ -28,12 +29,14 @@ public class TestQueryTermVector extends LuceneTestCase {
}
public void testConstructor() {
String [] queryTerm = {"foo", "bar", "foo", "again", "foo", "bar", "go", "go", "go"};
BytesRef [] queryTerm = {new BytesRef("foo"), new BytesRef("bar"), new BytesRef("foo"),
new BytesRef("again"), new BytesRef("foo"), new BytesRef("bar"), new BytesRef("go"),
new BytesRef("go"), new BytesRef("go")};
//Items are sorted lexicographically
String [] gold = {"again", "bar", "foo", "go"};
BytesRef [] gold = {new BytesRef("again"), new BytesRef("bar"), new BytesRef("foo"), new BytesRef("go")};
int [] goldFreqs = {1, 2, 3, 3};
QueryTermVector result = new QueryTermVector(queryTerm);
String [] terms = result.getTerms();
BytesRef [] terms = result.getTerms();
assertTrue(terms.length == 4);
int [] freq = result.getTermFrequencies();
assertTrue(freq.length == 4);
@ -49,7 +52,7 @@ public class TestQueryTermVector extends LuceneTestCase {
checkGold(terms, gold, freq, goldFreqs);
}
private void checkGold(String[] terms, String[] gold, int[] freq, int[] goldFreqs) {
private void checkGold(BytesRef[] terms, BytesRef[] gold, int[] freq, int[] goldFreqs) {
for (int i = 0; i < terms.length; i++) {
assertTrue(terms[i].equals(gold[i]));
assertTrue(freq[i] == goldFreqs[i]);

View File

@ -17,6 +17,7 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
@ -123,11 +124,11 @@ public class TestTermVectors extends LuceneTestCase {
for(int i=0;i<v.length;i++) {
TermPositionVector posVec = (TermPositionVector) v[i];
assertEquals(expectedFields[i], posVec.getField());
String[] terms = posVec.getTerms();
BytesRef[] terms = posVec.getTerms();
assertEquals(3, terms.length);
assertEquals("content", terms[0]);
assertEquals("here", terms[1]);
assertEquals("some", terms[2]);
assertEquals("content", terms[0].utf8ToString());
assertEquals("here", terms[1].utf8ToString());
assertEquals("some", terms[2].utf8ToString());
for(int j=0;j<3;j++) {
int[] positions = posVec.getTermPositions(j);
assertEquals(1, positions.length);
@ -156,7 +157,7 @@ public class TestTermVectors extends LuceneTestCase {
if(shouldBePosVector || shouldBeOffVector){
TermPositionVector posVec = (TermPositionVector)vector[0];
String [] terms = posVec.getTerms();
BytesRef [] terms = posVec.getTerms();
assertTrue(terms != null && terms.length > 0);
for (int j = 0; j < terms.length; j++) {
@ -184,7 +185,7 @@ public class TestTermVectors extends LuceneTestCase {
}
catch(ClassCastException ignore){
TermFreqVector freqVec = vector[0];
String [] terms = freqVec.getTerms();
BytesRef [] terms = freqVec.getTerms();
assertTrue(terms != null && terms.length > 0);
}
@ -277,11 +278,11 @@ public class TestTermVectors extends LuceneTestCase {
//float coord = sim.coord()
//System.out.println("TF: " + tf + " IDF: " + idf + " LenNorm: " + lNorm);
assertTrue(vector != null);
String[] vTerms = vector.getTerms();
BytesRef[] vTerms = vector.getTerms();
int [] freqs = vector.getTermFrequencies();
for (int i = 0; i < vTerms.length; i++)
{
if (text.equals(vTerms[i]))
if (text.equals(vTerms[i].utf8ToString()))
{
assertTrue(freqs[i] == freq);
}
@ -306,11 +307,11 @@ public class TestTermVectors extends LuceneTestCase {
TermFreqVector vector = knownSearcher.reader.getTermFreqVector(hits[1].doc, "field");
assertTrue(vector != null);
//System.out.println("Vector: " + vector);
String[] terms = vector.getTerms();
BytesRef[] terms = vector.getTerms();
int [] freqs = vector.getTermFrequencies();
assertTrue(terms != null && terms.length == 10);
for (int i = 0; i < terms.length; i++) {
String term = terms[i];
String term = terms[i].utf8ToString();
//System.out.println("Term: " + term);
int freq = freqs[i];
assertTrue(test4.indexOf(term) != -1);
@ -327,7 +328,7 @@ public class TestTermVectors extends LuceneTestCase {
if (tve != null && last != null)
{
assertTrue("terms are not properly sorted", last.getFrequency() >= tve.getFrequency());
Integer expectedFreq = test4Map.get(tve.getTerm());
Integer expectedFreq = test4Map.get(tve.getTerm().utf8ToString());
//we expect double the expectedFreq, since there are two fields with the exact same text and we are collapsing all fields
assertTrue("Frequency is not correct:", tve.getFrequency() == 2*expectedFreq.intValue());
}
@ -421,9 +422,9 @@ public class TestTermVectors extends LuceneTestCase {
assertTrue(vector.length == 1);
TermPositionVector tfv = (TermPositionVector) vector[0];
assertTrue(tfv.getField().equals("field"));
String[] terms = tfv.getTerms();
BytesRef[] terms = tfv.getTerms();
assertEquals(1, terms.length);
assertEquals(terms[0], "one");
assertEquals(terms[0].utf8ToString(), "one");
assertEquals(5, tfv.getTermFrequencies()[0]);
int[] positions = tfv.getTermPositions(0);
@ -447,7 +448,7 @@ public class TestTermVectors extends LuceneTestCase {
}
@Override
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
}
}

View File

@ -265,7 +265,7 @@ public class LukeRequestHandler extends RequestHandlerBase
if( v != null ) {
SimpleOrderedMap<Integer> tfv = new SimpleOrderedMap<Integer>();
for( int i=0; i<v.size(); i++ ) {
tfv.add( v.getTerms()[i], v.getTermFrequencies()[i] );
tfv.add( v.getTerms()[i].utf8ToString(), v.getTermFrequencies()[i] );
}
f.add( "termVector", tfv );
}

View File

@ -292,9 +292,9 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
this.reader = reader;
}
public void map(String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
NamedList termInfo = new NamedList();
fieldNL.add(term, termInfo);
fieldNL.add(term.utf8ToString(), termInfo);
if (fieldOptions.termFreq == true) {
termInfo.add("tf", frequency);
}
@ -323,14 +323,14 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
}
}
private int getDocFreq(String term) {
private int getDocFreq(BytesRef term) {
int result = 1;
currentTerm = currentTerm.createTerm(term);
try {
Terms terms = MultiFields.getTerms(reader, currentTerm.field());
if (terms != null) {
TermsEnum termsEnum = terms.iterator();
if (termsEnum.seek(new BytesRef(term)) == TermsEnum.SeekStatus.FOUND) {
if (termsEnum.seek(term) == TermsEnum.SeekStatus.FOUND) {
result = termsEnum.docFreq();
}
}

View File

@ -256,7 +256,7 @@ public class UnInvertedField {
deState.termsEnum = te.tenum;
deState.reuse = te.docsEnum;
}
DocSet set = searcher.getDocSet(new TermQuery(new Term(ti.field, topTerm.term.utf8ToString())), deState);
DocSet set = searcher.getDocSet(new TermQuery(new Term(ti.field, topTerm.term)), deState);
te.docsEnum = deState.reuse;
maxTermCounts[termNum] = set.size();
@ -514,7 +514,7 @@ public class UnInvertedField {
for (TopTerm tt : bigTerms.values()) {
// TODO: counts could be deferred if sorted==false
if (tt.termNum >= startTerm && tt.termNum < endTerm) {
counts[tt.termNum] = searcher.numDocs(new TermQuery(new Term(ti.field, tt.term.utf8ToString())), docs);
counts[tt.termNum] = searcher.numDocs(new TermQuery(new Term(ti.field, tt.term)), docs);
}
}
@ -712,7 +712,7 @@ public class UnInvertedField {
for (TopTerm tt : bigTerms.values()) {
// TODO: counts could be deferred if sorted==false
if (tt.termNum >= 0 && tt.termNum < numTermsInField) {
final Term t = new Term(ti.field, tt.term.utf8ToString());
final Term t = new Term(ti.field, tt.term);
if (finfo.length == 0) {
counts[tt.termNum] = searcher.numDocs(new TermQuery(t), docs);
} else {

View File

@ -480,7 +480,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
if (fields == null) return -1;
Terms terms = fields.terms(t.field());
if (terms == null) return -1;
BytesRef termBytes = new BytesRef(t.text());
BytesRef termBytes = t.bytes();
DocsEnum docs = terms.docs(MultiFields.getDeletedDocs(reader), termBytes, null);
if (docs == null) return -1;
int id = docs.nextDoc();
@ -754,7 +754,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
Fields fields = sir.fields();
Terms terms = fields.terms(t.field());
BytesRef termBytes = new BytesRef(t.text());
BytesRef termBytes = t.bytes();
Bits skipDocs = sir.getDeletedDocs();
DocsEnum docsEnum = terms==null ? null : terms.docs(skipDocs, termBytes, null);

View File

@ -118,7 +118,7 @@ public class DirectUpdateHandler extends UpdateHandler {
DocsEnum tdocs = MultiFields.getTermDocsEnum(ir,
MultiFields.getDeletedDocs(ir),
idTerm.field(),
new BytesRef(idTerm.text()));
idTerm.bytes());
if (tdocs != null) {
return tdocs.nextDoc() != DocsEnum.NO_MORE_DOCS;
} else {