mirror of https://github.com/apache/lucene.git
LUCENE-2548: don't intern field names
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1138058 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9d469e5111
commit
c039e210b0
|
@ -149,6 +149,9 @@ Changes in backwards compatibility policy
|
|||
files holding stored fields and term vectors) while flushing a
|
||||
segment. (Mike McCandless)
|
||||
|
||||
* LUCENE-2548: Field names (eg in Term, FieldInfo) are no longer
|
||||
interned. (Mike McCandless)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.lucene.index.IndexReader;
|
|||
import org.apache.lucene.index.memory.MemoryIndex;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* {@link Scorer} implementation which scores text fragments by the number of
|
||||
|
@ -88,7 +87,7 @@ public class QueryScorer implements Scorer {
|
|||
* @param defaultField
|
||||
*/
|
||||
public QueryScorer(Query query, IndexReader reader, String field, String defaultField) {
|
||||
this.defaultField = StringHelper.intern(defaultField);
|
||||
this.defaultField = defaultField;
|
||||
init(query, field, reader, true);
|
||||
}
|
||||
|
||||
|
@ -96,7 +95,7 @@ public class QueryScorer implements Scorer {
|
|||
* @param defaultField - The default field for queries with the field name unspecified
|
||||
*/
|
||||
public QueryScorer(Query query, String field, String defaultField) {
|
||||
this.defaultField = StringHelper.intern(defaultField);
|
||||
this.defaultField = defaultField;
|
||||
init(query, field, null, true);
|
||||
}
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.search.BooleanClause;
|
|||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.FilteredQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Utility class used to extract the terms used in a query, plus any weights.
|
||||
|
@ -94,10 +93,6 @@ public final class QueryTermExtractor
|
|||
public static final WeightedTerm[] getTerms(Query query, boolean prohibited, String fieldName)
|
||||
{
|
||||
HashSet<WeightedTerm> terms=new HashSet<WeightedTerm>();
|
||||
if(fieldName!=null)
|
||||
{
|
||||
fieldName= StringHelper.intern(fieldName);
|
||||
}
|
||||
getTerms(query,terms,prohibited,fieldName);
|
||||
return terms.toArray(new WeightedTerm[0]);
|
||||
}
|
||||
|
@ -114,7 +109,6 @@ public final class QueryTermExtractor
|
|||
return getTerms(query,prohibited,null);
|
||||
}
|
||||
|
||||
//fieldname MUST be interned prior to this call
|
||||
private static final void getTerms(Query query, HashSet<WeightedTerm> terms,boolean prohibited, String fieldName)
|
||||
{
|
||||
try
|
||||
|
@ -131,7 +125,7 @@ public final class QueryTermExtractor
|
|||
for (Iterator<Term> iter = nonWeightedTerms.iterator(); iter.hasNext();)
|
||||
{
|
||||
Term term = iter.next();
|
||||
if((fieldName==null)||(term.field()==fieldName))
|
||||
if((fieldName==null)||(term.field().equals(fieldName)))
|
||||
{
|
||||
terms.add(new WeightedTerm(query.getBoost(),term.text()));
|
||||
}
|
||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.lucene.search.spans.SpanOrQuery;
|
|||
import org.apache.lucene.search.spans.SpanQuery;
|
||||
import org.apache.lucene.search.spans.SpanTermQuery;
|
||||
import org.apache.lucene.search.spans.Spans;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Class used to extract {@link WeightedSpanTerm}s from a {@link Query} based on whether
|
||||
|
@ -63,7 +62,7 @@ public class WeightedSpanTermExtractor {
|
|||
|
||||
public WeightedSpanTermExtractor(String defaultField) {
|
||||
if (defaultField != null) {
|
||||
this.defaultField = StringHelper.intern(defaultField);
|
||||
this.defaultField = defaultField;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -314,8 +313,8 @@ public class WeightedSpanTermExtractor {
|
|||
* Necessary to implement matches for queries against <code>defaultField</code>
|
||||
*/
|
||||
private boolean fieldNameComparator(String fieldNameToCheck) {
|
||||
boolean rv = fieldName == null || fieldNameToCheck == fieldName
|
||||
|| fieldNameToCheck == defaultField;
|
||||
boolean rv = fieldName == null || fieldName.equals(fieldNameToCheck)
|
||||
|| (defaultField != null && defaultField.equals(fieldNameToCheck));
|
||||
return rv;
|
||||
}
|
||||
|
||||
|
@ -372,7 +371,7 @@ public class WeightedSpanTermExtractor {
|
|||
public Map<String,WeightedSpanTerm> getWeightedSpanTerms(Query query, TokenStream tokenStream,
|
||||
String fieldName) throws IOException {
|
||||
if (fieldName != null) {
|
||||
this.fieldName = StringHelper.intern(fieldName);
|
||||
this.fieldName = fieldName;
|
||||
} else {
|
||||
this.fieldName = null;
|
||||
}
|
||||
|
@ -408,7 +407,7 @@ public class WeightedSpanTermExtractor {
|
|||
public Map<String,WeightedSpanTerm> getWeightedSpanTermsWithScores(Query query, TokenStream tokenStream, String fieldName,
|
||||
IndexReader reader) throws IOException {
|
||||
if (fieldName != null) {
|
||||
this.fieldName = StringHelper.intern(fieldName);
|
||||
this.fieldName = fieldName;
|
||||
} else {
|
||||
this.fieldName = null;
|
||||
}
|
||||
|
|
|
@ -400,7 +400,7 @@ public class InstantiatedIndexReader extends IndexReader {
|
|||
if (i < 0) {
|
||||
i = -i - 1;
|
||||
}
|
||||
if (i >= orderedTerms.length || orderedTerms[i].field() != field) {
|
||||
if (i >= orderedTerms.length || !orderedTerms[i].field().equals(field)) {
|
||||
// field does not exist
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.lucene.index.Term;
|
|||
import org.apache.lucene.index.TermVectorOffsetInfo;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.SimilarityProvider;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
|
@ -475,7 +474,7 @@ public class InstantiatedIndexWriter implements Closeable {
|
|||
FieldSetting fieldSetting = fieldSettingsByFieldName.get(field.name());
|
||||
if (fieldSetting == null) {
|
||||
fieldSetting = new FieldSetting();
|
||||
fieldSetting.fieldName = StringHelper.intern(field.name());
|
||||
fieldSetting.fieldName = field.name();
|
||||
fieldSettingsByFieldName.put(fieldSetting.fieldName, fieldSetting);
|
||||
fieldNameBuffer.add(fieldSetting.fieldName);
|
||||
}
|
||||
|
|
|
@ -69,11 +69,9 @@ public class InstantiatedTermsEnum extends TermsEnum {
|
|||
return SeekStatus.END;
|
||||
}
|
||||
|
||||
if (terms[upto].field() == field) {
|
||||
if (terms[upto].field().equals(field)) {
|
||||
return SeekStatus.FOUND;
|
||||
} else {
|
||||
// make sure field was interned
|
||||
assert !terms[upto].field().equals(field);
|
||||
return SeekStatus.END;
|
||||
}
|
||||
}
|
||||
|
@ -84,12 +82,10 @@ public class InstantiatedTermsEnum extends TermsEnum {
|
|||
if (upto >= terms.length) {
|
||||
return null;
|
||||
}
|
||||
if (terms[upto].field() == field) {
|
||||
if (terms[upto].field().equals(field)) {
|
||||
br.copy(terms[upto].getTerm().text());
|
||||
return br;
|
||||
} else {
|
||||
// make sure field was interned
|
||||
assert !terms[upto].field().equals(field);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.search.Similarity;
|
|||
import org.apache.lucene.search.SimilarityProvider;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
|
@ -111,7 +110,6 @@ public class FieldNormModifier {
|
|||
* @param field the field whose norms should be reset
|
||||
*/
|
||||
public void reSetNorms(String field) throws IOException {
|
||||
String fieldName = StringHelper.intern(field);
|
||||
Similarity fieldSim = sim.get(field);
|
||||
IndexReader reader = null;
|
||||
try {
|
||||
|
@ -149,7 +147,7 @@ public class FieldNormModifier {
|
|||
for (int d = 0; d < termCounts.length; d++) {
|
||||
if (delDocs == null || !delDocs.get(d)) {
|
||||
invertState.setLength(termCounts[d]);
|
||||
subReader.setNorm(d, fieldName, fieldSim.encodeNormValue(fieldSim.computeNorm(invertState)));
|
||||
subReader.setNorm(d, field, fieldSim.encodeNormValue(fieldSim.computeNorm(invertState)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -15,7 +15,6 @@ package org.apache.lucene.index;
|
|||
*
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -54,9 +53,6 @@ public class TermVectorAccessor {
|
|||
* @throws IOException
|
||||
*/
|
||||
public void accept(IndexReader indexReader, int documentNumber, String fieldName, TermVectorMapper mapper) throws IOException {
|
||||
|
||||
fieldName = StringHelper.intern(fieldName);
|
||||
|
||||
decoratedMapper.decorated = mapper;
|
||||
decoratedMapper.termVectorStored = false;
|
||||
|
||||
|
|
|
@ -190,7 +190,6 @@ public class FuzzyLikeThisQuery extends Query
|
|||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||
|
||||
int corpusNumDocs=reader.numDocs();
|
||||
Term internSavingTemplateTerm =new Term(f.fieldName); //optimization to avoid constructing new Term() objects
|
||||
HashSet<String> processedTerms=new HashSet<String>();
|
||||
ts.reset();
|
||||
while (ts.incrementToken())
|
||||
|
@ -201,7 +200,7 @@ public class FuzzyLikeThisQuery extends Query
|
|||
processedTerms.add(term);
|
||||
ScoreTermQueue variantsQ=new ScoreTermQueue(MAX_VARIANTS_PER_TERM); //maxNum variants considered for any one term
|
||||
float minScore=0;
|
||||
Term startTerm=internSavingTemplateTerm.createTerm(term);
|
||||
Term startTerm=new Term(f.fieldName, term);
|
||||
AttributeSource atts = new AttributeSource();
|
||||
MaxNonCompetitiveBoostAttribute maxBoostAtt =
|
||||
atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
|
||||
|
|
|
@ -71,7 +71,7 @@ public class TermsFilter extends Filter
|
|||
DocsEnum docs = null;
|
||||
for (Iterator<Term> iter = terms.iterator(); iter.hasNext();) {
|
||||
Term term = iter.next();
|
||||
if (term.field() != lastField) {
|
||||
if (!term.field().equals(lastField)) {
|
||||
termsC = fields.terms(term.field());
|
||||
termsEnum = termsC.iterator();
|
||||
lastField = term.field();
|
||||
|
|
|
@ -67,14 +67,7 @@ public class TermsFilterBuilder implements FilterBuilder
|
|||
ts.reset();
|
||||
while (ts.incrementToken()) {
|
||||
termAtt.fillBytesRef();
|
||||
if (term == null)
|
||||
{
|
||||
term = new Term(fieldName, new BytesRef(bytes));
|
||||
} else
|
||||
{
|
||||
// create from previous to save fieldName.intern overhead
|
||||
term = term.createTerm(new BytesRef(bytes));
|
||||
}
|
||||
term = new Term(fieldName, new BytesRef(bytes));
|
||||
tf.addTerm(term);
|
||||
}
|
||||
ts.end();
|
||||
|
|
|
@ -64,14 +64,7 @@ public class TermsQueryBuilder implements QueryBuilder {
|
|||
ts.reset();
|
||||
while (ts.incrementToken()) {
|
||||
termAtt.fillBytesRef();
|
||||
if (term == null)
|
||||
{
|
||||
term = new Term(fieldName, new BytesRef(bytes));
|
||||
} else
|
||||
{
|
||||
// create from previous to save fieldName.intern overhead
|
||||
term = term.createTerm(new BytesRef(bytes));
|
||||
}
|
||||
term = new Term(fieldName, new BytesRef(bytes));
|
||||
bq.add(new BooleanClause(new TermQuery(term),BooleanClause.Occur.SHOULD));
|
||||
}
|
||||
ts.end();
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.index.FieldInvertState; // for javadocs
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.util.StringHelper; // for javadocs
|
||||
|
||||
|
||||
/**
|
||||
|
@ -59,7 +58,7 @@ public abstract class AbstractField implements Fieldable {
|
|||
protected AbstractField(String name, Field.Store store, Field.Index index, Field.TermVector termVector) {
|
||||
if (name == null)
|
||||
throw new NullPointerException("name cannot be null");
|
||||
this.name = StringHelper.intern(name); // field names are interned
|
||||
this.name = name;
|
||||
|
||||
this.isStored = store.isStored();
|
||||
this.isIndexed = index.isIndexed();
|
||||
|
@ -109,7 +108,7 @@ public abstract class AbstractField implements Fieldable {
|
|||
return boost;
|
||||
}
|
||||
|
||||
/** Returns the name of the field as an interned string.
|
||||
/** Returns the name of the field.
|
||||
* For example "date", "title", "body", ...
|
||||
*/
|
||||
public String name() { return name; }
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.Reader;
|
|||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
A field is a section of a Document. Each field has two parts, a name and a
|
||||
|
@ -367,28 +366,6 @@ public final class Field extends AbstractField implements Fieldable {
|
|||
* </ul>
|
||||
*/
|
||||
public Field(String name, String value, Store store, Index index, TermVector termVector) {
|
||||
this(name, true, value, store, index, termVector);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a field by specifying its name, value and how it will
|
||||
* be saved in the index.
|
||||
*
|
||||
* @param name The name of the field
|
||||
* @param internName Whether to .intern() name or not
|
||||
* @param value The string to process
|
||||
* @param store Whether <code>value</code> should be stored in the index
|
||||
* @param index Whether the field should be indexed, and if so, if it should
|
||||
* be tokenized before indexing
|
||||
* @param termVector Whether term vector should be stored
|
||||
* @throws NullPointerException if name or value is <code>null</code>
|
||||
* @throws IllegalArgumentException in any of the following situations:
|
||||
* <ul>
|
||||
* <li>the field is neither stored nor indexed</li>
|
||||
* <li>the field is not indexed but termVector is <code>TermVector.YES</code></li>
|
||||
* </ul>
|
||||
*/
|
||||
public Field(String name, boolean internName, String value, Store store, Index index, TermVector termVector) {
|
||||
if (name == null)
|
||||
throw new NullPointerException("name cannot be null");
|
||||
if (value == null)
|
||||
|
@ -402,9 +379,6 @@ public final class Field extends AbstractField implements Fieldable {
|
|||
throw new IllegalArgumentException("cannot store term vector information "
|
||||
+ "for a field that is not indexed");
|
||||
|
||||
if (internName) // field names are optionally interned
|
||||
name = StringHelper.intern(name);
|
||||
|
||||
this.name = name;
|
||||
|
||||
this.fieldsData = value;
|
||||
|
@ -422,7 +396,7 @@ public final class Field extends AbstractField implements Fieldable {
|
|||
|
||||
setStoreTermVector(termVector);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a tokenized and indexed field that is not stored. Term vectors will
|
||||
* not be stored. The Reader is read only when the Document is added to the index,
|
||||
|
@ -454,7 +428,7 @@ public final class Field extends AbstractField implements Fieldable {
|
|||
if (reader == null)
|
||||
throw new NullPointerException("reader cannot be null");
|
||||
|
||||
this.name = StringHelper.intern(name); // field names are interned
|
||||
this.name = name;
|
||||
this.fieldsData = reader;
|
||||
|
||||
this.isStored = false;
|
||||
|
@ -500,7 +474,7 @@ public final class Field extends AbstractField implements Fieldable {
|
|||
if (tokenStream == null)
|
||||
throw new NullPointerException("tokenStream cannot be null");
|
||||
|
||||
this.name = StringHelper.intern(name); // field names are interned
|
||||
this.name = name;
|
||||
this.fieldsData = null;
|
||||
this.tokenStream = tokenStream;
|
||||
|
||||
|
@ -540,7 +514,7 @@ public final class Field extends AbstractField implements Fieldable {
|
|||
if (value == null)
|
||||
throw new IllegalArgumentException("value cannot be null");
|
||||
|
||||
this.name = StringHelper.intern(name); // field names are interned
|
||||
this.name = name;
|
||||
fieldsData = value;
|
||||
|
||||
isStored = true;
|
||||
|
|
|
@ -71,7 +71,7 @@ public interface Fieldable {
|
|||
*/
|
||||
float getBoost();
|
||||
|
||||
/** Returns the name of the field as an interned string.
|
||||
/** Returns the name of the field.
|
||||
* For example "date", "title", "body", ...
|
||||
*/
|
||||
String name();
|
||||
|
|
|
@ -47,11 +47,12 @@ class BufferedDeletes {
|
|||
key, Integer val, int hash, Entry next
|
||||
(OBJ_HEADER + 3*POINTER + INT). Term is object w/
|
||||
String field and String text (OBJ_HEADER + 2*POINTER).
|
||||
We don't count Term's field since it's interned.
|
||||
Term's field is String (OBJ_HEADER + 4*INT + POINTER +
|
||||
OBJ_HEADER + string.length*CHAR).
|
||||
Term's text is String (OBJ_HEADER + 4*INT + POINTER +
|
||||
OBJ_HEADER + string.length*CHAR). Integer is
|
||||
OBJ_HEADER + INT. */
|
||||
final static int BYTES_PER_DEL_TERM = 8*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 5*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 6*RamUsageEstimator.NUM_BYTES_INT;
|
||||
final static int BYTES_PER_DEL_TERM = 9*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 7*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 10*RamUsageEstimator.NUM_BYTES_INT;
|
||||
|
||||
/* Rough logic: del docIDs are List<Integer>. Say list
|
||||
allocates ~2X size (2*POINTER). Integer is OBJ_HEADER
|
||||
|
@ -189,7 +190,7 @@ class BufferedDeletes {
|
|||
terms.put(term, Integer.valueOf(docIDUpto));
|
||||
numTermDeletes.incrementAndGet();
|
||||
if (current == null) {
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length);
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length + (RamUsageEstimator.NUM_BYTES_CHAR * term.field().length()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -380,7 +380,7 @@ class BufferedDeletesStream {
|
|||
// Since we visit terms sorted, we gain performance
|
||||
// by re-using the same TermsEnum and seeking only
|
||||
// forwards
|
||||
if (term.field() != currentField) {
|
||||
if (!term.field().equals(currentField)) {
|
||||
assert currentField == null || currentField.compareTo(term.field()) < 0;
|
||||
currentField = term.field();
|
||||
Terms terms = fields.terms(currentField);
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/** Access to the Fieldable Info file that describes document fields and whether or
|
||||
* not they are indexed. Each segment has a separate Fieldable Info file. Objects
|
||||
|
@ -475,12 +474,11 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
boolean storeTermVector, boolean storePositionWithTermVector,
|
||||
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions, ValueType docValuesType) {
|
||||
// don't check modifiable here since we use that to initially build up FIs
|
||||
name = StringHelper.intern(name);
|
||||
if (globalFieldNumbers != null) {
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name);
|
||||
}
|
||||
final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions, docValuesType);
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions, docValuesType);
|
||||
putInternal(fi);
|
||||
return fi;
|
||||
}
|
||||
|
@ -659,7 +657,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
final int size = input.readVInt(); //read in the size
|
||||
|
||||
for (int i = 0; i < size; i++) {
|
||||
String name = StringHelper.intern(input.readString());
|
||||
String name = input.readString();
|
||||
// if this is a previous format codec 0 will be preflex!
|
||||
final int fieldNumber = format <= FORMAT_PER_FIELD_CODEC? input.readInt():i;
|
||||
final int codecId = format <= FORMAT_PER_FIELD_CODEC? input.readInt():0;
|
||||
|
|
|
@ -43,10 +43,8 @@ public abstract class FieldsEnum {
|
|||
return atts;
|
||||
}
|
||||
|
||||
/** Increments the enumeration to the next field. The
|
||||
* returned field is always interned, so simple ==
|
||||
* comparison is allowed. Returns null when there are no
|
||||
* more fields.*/
|
||||
/** Increments the enumeration to the next field. Returns
|
||||
* null when there are no more fields.*/
|
||||
public abstract String next() throws IOException;
|
||||
|
||||
/** Get {@link TermsEnum} for the current field. You
|
||||
|
|
|
@ -364,7 +364,6 @@ public final class FieldsReader implements Cloneable {
|
|||
Field.Index index = Field.Index.toIndex(fi.isIndexed, tokenize);
|
||||
Field.TermVector termVector = Field.TermVector.toTermVector(fi.storeTermVector, fi.storeOffsetWithTermVector, fi.storePositionWithTermVector);
|
||||
f = new Field(fi.name, // name
|
||||
false,
|
||||
fieldsStream.readString(), // read value
|
||||
Field.Store.YES,
|
||||
index,
|
||||
|
|
|
@ -234,8 +234,6 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
final TermsConsumer termsConsumer = consumer.addField(fieldInfo);
|
||||
final Comparator<BytesRef> termComp = termsConsumer.getComparator();
|
||||
|
||||
final Term protoTerm = new Term(fieldName);
|
||||
|
||||
final boolean currentFieldOmitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
|
||||
|
||||
final Map<Term,Integer> segDeletes;
|
||||
|
@ -274,7 +272,7 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
|
||||
final int delDocLimit;
|
||||
if (segDeletes != null) {
|
||||
final Integer docIDUpto = segDeletes.get(protoTerm.createTerm(text));
|
||||
final Integer docIDUpto = segDeletes.get(new Term(fieldName, text));
|
||||
if (docIDUpto != null) {
|
||||
delDocLimit = docIDUpto;
|
||||
} else {
|
||||
|
|
|
@ -33,10 +33,11 @@ class FrozenBufferedDeletes {
|
|||
|
||||
/* Rough logic: Term is object w/
|
||||
String field and String text (OBJ_HEADER + 2*POINTER).
|
||||
We don't count Term's field since it's interned.
|
||||
Term's text is String (OBJ_HEADER + 4*INT + POINTER +
|
||||
OBJ_HEADER + string.length*CHAR). */
|
||||
final static int BYTES_PER_DEL_TERM = 3*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 3*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 4*RamUsageEstimator.NUM_BYTES_INT;
|
||||
OBJ_HEADER + text.length*CHAR).
|
||||
Term's field is String (OBJ_HEADER + 4*INT + POINTER +
|
||||
OBJ_HEADER + field.length*CHAR). */
|
||||
final static int BYTES_PER_DEL_TERM = 4*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 4*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 8*RamUsageEstimator.NUM_BYTES_INT;
|
||||
|
||||
/* Query we often undercount (say 24 bytes), plus int. */
|
||||
final static int BYTES_PER_DEL_QUERY = RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_INT + 24;
|
||||
|
|
|
@ -94,7 +94,7 @@ public final class MultiFieldsEnum extends FieldsEnum {
|
|||
if (queue.size() > 0) {
|
||||
while(true) {
|
||||
top[numTop++] = queue.pop();
|
||||
if (queue.size() == 0 || (queue.top()).current != top[0].current) {
|
||||
if (queue.size() == 0 || !(queue.top()).current.equals(top[0].current)) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,15 +17,12 @@ 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;
|
||||
|
||||
/**
|
||||
A Term represents a word from text. This is the unit of search. It is
|
||||
composed of two elements, the text of the word, as a string, and the name of
|
||||
the field that the text occurred in, an interned string.
|
||||
the field that the text occurred in.
|
||||
|
||||
Note that terms may represent more than words from text fields, but also
|
||||
things like dates, email addresses, urls, etc. */
|
||||
|
@ -37,13 +34,14 @@ public final class Term implements Comparable<Term> {
|
|||
/** 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);
|
||||
field = fld;
|
||||
this.bytes = bytes;
|
||||
}
|
||||
|
||||
|
@ -61,28 +59,10 @@ public final class Term implements Comparable<Term> {
|
|||
* @param fld
|
||||
*/
|
||||
public Term(String fld) {
|
||||
this(fld, new BytesRef(), true);
|
||||
this(fld, new BytesRef());
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 text, boolean intern) {
|
||||
this(fld, new BytesRef(text), intern);
|
||||
}
|
||||
|
||||
/** Returns the field of this term, an interned string. The field indicates
|
||||
/** Returns the field of this term. The field indicates
|
||||
the part of a document which this term came from. */
|
||||
public final String field() { return field; }
|
||||
|
||||
|
@ -94,32 +74,6 @@ public final class Term implements Comparable<Term> {
|
|||
/** 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 bytes 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
|
||||
* @param text The text of the new term (field is implicitly same as this Term instance)
|
||||
* @return A new Term
|
||||
*/
|
||||
public Term createTerm(String text)
|
||||
{
|
||||
return new Term(field,text,false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj)
|
||||
|
@ -157,26 +111,11 @@ public final class Term implements Comparable<Term> {
|
|||
|
||||
The ordering of terms is first by field, then by text.*/
|
||||
public final int compareTo(Term other) {
|
||||
if (field == other.field) // fields are interned
|
||||
if (field.equals(other.field)) {
|
||||
return bytes.compareTo(other.bytes);
|
||||
else
|
||||
return field.compareTo(other.field);
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
private static final Comparator<BytesRef> legacyComparator =
|
||||
BytesRef.getUTF8SortedAsUTF16Comparator();
|
||||
|
||||
/**
|
||||
* @deprecated (4.0) 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
|
||||
} else {
|
||||
return field.compareTo(other.field);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -91,7 +91,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
@Override
|
||||
public boolean equals(Object _other) {
|
||||
FieldAndTerm other = (FieldAndTerm) _other;
|
||||
return other.field == field && term.bytesEquals(other.term);
|
||||
return other.field.equals(field) && term.bytesEquals(other.term);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -270,11 +270,11 @@ public class PreFlexFields extends FieldsProducer {
|
|||
private class PreTermsEnum extends TermsEnum {
|
||||
private SegmentTermEnum termEnum;
|
||||
private FieldInfo fieldInfo;
|
||||
private String internedFieldName;
|
||||
private boolean skipNext;
|
||||
private BytesRef current;
|
||||
|
||||
private SegmentTermEnum seekTermEnum;
|
||||
private Term protoTerm;
|
||||
|
||||
private static final byte UTF8_NON_BMP_LEAD = (byte) 0xf0;
|
||||
private static final byte UTF8_HIGH_BMP_LEAD = (byte) 0xee;
|
||||
|
@ -334,7 +334,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
}
|
||||
|
||||
// Seek "back":
|
||||
getTermsDict().seekEnum(te, protoTerm.createTerm(term), true);
|
||||
getTermsDict().seekEnum(te, new Term(fieldInfo.name, term), true);
|
||||
|
||||
// Test if the term we seek'd to in fact found a
|
||||
// surrogate pair at the same position as the E:
|
||||
|
@ -343,7 +343,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
// Cannot be null (or move to next field) because at
|
||||
// "worst" it'd seek to the same term we are on now,
|
||||
// unless we are being called from seek
|
||||
if (t2 == null || t2.field() != fieldInfo.name) {
|
||||
if (t2 == null || t2.field() != internedFieldName) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -461,13 +461,13 @@ public class PreFlexFields extends FieldsProducer {
|
|||
|
||||
// TODO: more efficient seek? can we simply swap
|
||||
// the enums?
|
||||
getTermsDict().seekEnum(termEnum, protoTerm.createTerm(scratchTerm), true);
|
||||
getTermsDict().seekEnum(termEnum, new Term(fieldInfo.name, scratchTerm), true);
|
||||
|
||||
final Term t2 = termEnum.term();
|
||||
|
||||
// We could hit EOF or different field since this
|
||||
// was a seek "forward":
|
||||
if (t2 != null && t2.field() == fieldInfo.name) {
|
||||
if (t2 != null && t2.field() == internedFieldName) {
|
||||
|
||||
if (DEBUG_SURROGATES) {
|
||||
System.out.println(" got term=" + UnicodeUtil.toHexString(t2.text()) + " " + t2.bytes());
|
||||
|
@ -552,7 +552,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
// current term.
|
||||
|
||||
// TODO: can we avoid this copy?
|
||||
if (termEnum.term() == null || termEnum.term().field() != fieldInfo.name) {
|
||||
if (termEnum.term() == null || termEnum.term().field() != internedFieldName) {
|
||||
scratchTerm.length = 0;
|
||||
} else {
|
||||
scratchTerm.copy(termEnum.term().bytes());
|
||||
|
@ -637,7 +637,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
|
||||
// Seek "forward":
|
||||
// TODO: more efficient seek?
|
||||
getTermsDict().seekEnum(seekTermEnum, protoTerm.createTerm(scratchTerm), true);
|
||||
getTermsDict().seekEnum(seekTermEnum, new Term(fieldInfo.name, scratchTerm), true);
|
||||
|
||||
scratchTerm.bytes[upTo] = scratch[0];
|
||||
scratchTerm.bytes[upTo+1] = scratch[1];
|
||||
|
@ -659,7 +659,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
// EOF or a different field:
|
||||
boolean matches;
|
||||
|
||||
if (t2 != null && t2.field() == fieldInfo.name) {
|
||||
if (t2 != null && t2.field() == internedFieldName) {
|
||||
final BytesRef b2 = t2.bytes();
|
||||
assert b2.offset == 0;
|
||||
if (b2.length >= upTo+3 && isHighBMPChar(b2.bytes, upTo)) {
|
||||
|
@ -713,20 +713,21 @@ public class PreFlexFields extends FieldsProducer {
|
|||
void reset(FieldInfo fieldInfo) throws IOException {
|
||||
//System.out.println("pff.reset te=" + termEnum);
|
||||
this.fieldInfo = fieldInfo;
|
||||
protoTerm = new Term(fieldInfo.name);
|
||||
internedFieldName = fieldInfo.name.intern();
|
||||
final Term term = new Term(internedFieldName);
|
||||
if (termEnum == null) {
|
||||
termEnum = getTermsDict().terms(protoTerm);
|
||||
seekTermEnum = getTermsDict().terms(protoTerm);
|
||||
termEnum = getTermsDict().terms(term);
|
||||
seekTermEnum = getTermsDict().terms(term);
|
||||
//System.out.println(" term=" + termEnum.term());
|
||||
} else {
|
||||
getTermsDict().seekEnum(termEnum, protoTerm, true);
|
||||
getTermsDict().seekEnum(termEnum, term, true);
|
||||
}
|
||||
skipNext = true;
|
||||
|
||||
unicodeSortOrder = sortTermsByUnicode();
|
||||
|
||||
final Term t = termEnum.term();
|
||||
if (t != null && t.field() == fieldInfo.name) {
|
||||
if (t != null && t.field() == internedFieldName) {
|
||||
newSuffixStart = 0;
|
||||
prevTerm.length = 0;
|
||||
surrogateDance();
|
||||
|
@ -761,7 +762,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
}
|
||||
skipNext = false;
|
||||
final TermInfosReader tis = getTermsDict();
|
||||
final Term t0 = protoTerm.createTerm(term);
|
||||
final Term t0 = new Term(fieldInfo.name, term);
|
||||
|
||||
assert termEnum != null;
|
||||
|
||||
|
@ -769,7 +770,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
|
||||
final Term t = termEnum.term();
|
||||
|
||||
if (t != null && t.field() == fieldInfo.name && term.bytesEquals(t.bytes())) {
|
||||
if (t != null && t.field() == internedFieldName && term.bytesEquals(t.bytes())) {
|
||||
// If we found an exact match, no need to do the
|
||||
// surrogate dance
|
||||
if (DEBUG_SURROGATES) {
|
||||
|
@ -777,7 +778,7 @@ public class PreFlexFields extends FieldsProducer {
|
|||
}
|
||||
current = t.bytes();
|
||||
return SeekStatus.FOUND;
|
||||
} else if (t == null || t.field() != fieldInfo.name) {
|
||||
} else if (t == null || t.field() != internedFieldName) {
|
||||
|
||||
// TODO: maybe we can handle this like the next()
|
||||
// into null? set term as prevTerm then dance?
|
||||
|
@ -840,8 +841,9 @@ public class PreFlexFields extends FieldsProducer {
|
|||
surrogateDance();
|
||||
|
||||
final Term t2 = termEnum.term();
|
||||
if (t2 == null || t2.field() != fieldInfo.name) {
|
||||
assert t2 == null || !t2.field().equals(fieldInfo.name); // make sure fields are in fact interned
|
||||
if (t2 == null || t2.field() != internedFieldName) {
|
||||
// PreFlex codec interns field names; verify:
|
||||
assert t2 == null || !t2.field().equals(internedFieldName);
|
||||
current = null;
|
||||
return SeekStatus.END;
|
||||
} else {
|
||||
|
@ -885,7 +887,8 @@ public class PreFlexFields extends FieldsProducer {
|
|||
skipNext = false;
|
||||
if (termEnum.term() == null) {
|
||||
return null;
|
||||
} else if (termEnum.term().field() != fieldInfo.name) {
|
||||
// PreFlex codec interns field names:
|
||||
} else if (termEnum.term().field() != internedFieldName) {
|
||||
return null;
|
||||
} else {
|
||||
return current = termEnum.term().bytes();
|
||||
|
@ -895,15 +898,16 @@ public class PreFlexFields extends FieldsProducer {
|
|||
// TODO: can we use STE's prevBuffer here?
|
||||
prevTerm.copy(termEnum.term().bytes());
|
||||
|
||||
if (termEnum.next() && termEnum.term().field() == fieldInfo.name) {
|
||||
if (termEnum.next() && termEnum.term().field() == internedFieldName) {
|
||||
newSuffixStart = termEnum.newSuffixStart;
|
||||
if (DEBUG_SURROGATES) {
|
||||
System.out.println(" newSuffixStart=" + newSuffixStart);
|
||||
}
|
||||
surrogateDance();
|
||||
final Term t = termEnum.term();
|
||||
if (t == null || t.field() != fieldInfo.name) {
|
||||
assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
|
||||
if (t == null || t.field() != internedFieldName) {
|
||||
// PreFlex codec interns field names; verify:
|
||||
assert t == null || !t.field().equals(internedFieldName);
|
||||
current = null;
|
||||
} else {
|
||||
current = t.bytes();
|
||||
|
@ -920,8 +924,9 @@ public class PreFlexFields extends FieldsProducer {
|
|||
surrogateDance();
|
||||
|
||||
final Term t = termEnum.term();
|
||||
if (t == null || t.field() != fieldInfo.name) {
|
||||
assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
|
||||
if (t == null || t.field() != internedFieldName) {
|
||||
// PreFlex codec interns field names; verify:
|
||||
assert t == null || !t.field().equals(internedFieldName);
|
||||
return null;
|
||||
} else {
|
||||
current = t.bytes();
|
||||
|
|
|
@ -37,12 +37,17 @@ final class TermBuffer implements Cloneable {
|
|||
|
||||
private BytesRef bytes = new BytesRef(10);
|
||||
|
||||
// Cannot be -1 since (strangely) we write that
|
||||
// fieldNumber into index for first indexed term:
|
||||
private int currentFieldNumber = -2;
|
||||
|
||||
private static final Comparator<BytesRef> utf8AsUTF16Comparator = BytesRef.getUTF8SortedAsUTF16Comparator();
|
||||
|
||||
int newSuffixStart; // only valid right after .read is called
|
||||
|
||||
public int compareTo(TermBuffer other) {
|
||||
if (field == other.field) // fields are interned
|
||||
// (only by PreFlex codec)
|
||||
return utf8AsUTF16Comparator.compare(bytes, other.bytes);
|
||||
else
|
||||
return field.compareTo(other.field);
|
||||
|
@ -59,7 +64,13 @@ final class TermBuffer implements Cloneable {
|
|||
}
|
||||
bytes.length = totalLength;
|
||||
input.readBytes(bytes.bytes, newSuffixStart, length);
|
||||
this.field = fieldInfos.fieldName(input.readVInt());
|
||||
final int fieldNumber = input.readVInt();
|
||||
if (fieldNumber != currentFieldNumber) {
|
||||
currentFieldNumber = fieldNumber;
|
||||
field = fieldInfos.fieldName(currentFieldNumber).intern();
|
||||
} else {
|
||||
assert field.equals(fieldInfos.fieldName(fieldNumber)): "currentFieldNumber=" + currentFieldNumber + " field=" + field + " vs " + fieldInfos.fieldName(fieldNumber);
|
||||
}
|
||||
}
|
||||
|
||||
public void set(Term term) {
|
||||
|
@ -68,12 +79,14 @@ final class TermBuffer implements Cloneable {
|
|||
return;
|
||||
}
|
||||
bytes.copy(term.bytes());
|
||||
field = term.field();
|
||||
field = term.field().intern();
|
||||
currentFieldNumber = -1;
|
||||
this.term = term;
|
||||
}
|
||||
|
||||
public void set(TermBuffer other) {
|
||||
field = other.field;
|
||||
currentFieldNumber = other.currentFieldNumber;
|
||||
// dangerous to copy Term over, since the underlying
|
||||
// BytesRef could subsequently be modified:
|
||||
term = null;
|
||||
|
@ -83,6 +96,7 @@ final class TermBuffer implements Cloneable {
|
|||
public void reset() {
|
||||
field = null;
|
||||
term = null;
|
||||
currentFieldNumber= -1;
|
||||
}
|
||||
|
||||
public Term toTerm() {
|
||||
|
@ -90,8 +104,7 @@ final class TermBuffer implements Cloneable {
|
|||
return null;
|
||||
|
||||
if (term == null) {
|
||||
term = new Term(field, new BytesRef(bytes), false);
|
||||
//term = new Term(field, bytes, false);
|
||||
term = new Term(field, new BytesRef(bytes));
|
||||
}
|
||||
|
||||
return term;
|
||||
|
|
|
@ -18,12 +18,14 @@ package org.apache.lucene.index.codecs.preflex;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CloseableThreadLocal;
|
||||
import org.apache.lucene.util.DoubleBarrelLRUCache;
|
||||
|
||||
|
@ -188,7 +190,17 @@ public final class TermInfosReader {
|
|||
}
|
||||
return resources;
|
||||
}
|
||||
|
||||
private static final Comparator<BytesRef> legacyComparator =
|
||||
BytesRef.getUTF8SortedAsUTF16Comparator();
|
||||
|
||||
private final int compareAsUTF16(Term term1, Term term2) {
|
||||
if (term1.field().equals(term2.field())) {
|
||||
return legacyComparator.compare(term1.bytes(), term2.bytes());
|
||||
} else {
|
||||
return term1.field().compareTo(term2.field());
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns the offset of the greatest index entry which is less than or equal to term.*/
|
||||
private int getIndexOffset(Term term) {
|
||||
|
@ -198,7 +210,7 @@ public final class TermInfosReader {
|
|||
while (hi >= lo) {
|
||||
int mid = (lo + hi) >>> 1;
|
||||
assert indexTerms[mid] != null : "indexTerms = " + indexTerms.length + " mid=" + mid;
|
||||
int delta = term.compareToUTF16(indexTerms[mid]);
|
||||
int delta = compareAsUTF16(term, indexTerms[mid]);
|
||||
if (delta < 0)
|
||||
hi = mid - 1;
|
||||
else if (delta > 0)
|
||||
|
@ -256,16 +268,16 @@ 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.compareToUTF16(enumerator.prev())> 0)
|
||||
|| term.compareToUTF16(enumerator.term()) >= 0)) {
|
||||
&& ((enumerator.prev() != null && compareAsUTF16(term, enumerator.prev())> 0)
|
||||
|| compareAsUTF16(term, enumerator.term()) >= 0)) {
|
||||
int enumOffset = (int)(enumerator.position/totalIndexInterval)+1;
|
||||
if (indexTerms.length == enumOffset // but before end of block
|
||||
|| term.compareToUTF16(indexTerms[enumOffset]) < 0) {
|
||||
|| compareAsUTF16(term, indexTerms[enumOffset]) < 0) {
|
||||
// no need to seek
|
||||
|
||||
final TermInfo ti;
|
||||
int numScans = enumerator.scanTo(term);
|
||||
if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
|
||||
if (enumerator.term() != null && compareAsUTF16(term, enumerator.term()) == 0) {
|
||||
ti = enumerator.termInfo;
|
||||
if (numScans > 1) {
|
||||
// we only want to put this TermInfo into the cache if
|
||||
|
@ -303,7 +315,7 @@ public final class TermInfosReader {
|
|||
enumerator.scanTo(term);
|
||||
final TermInfo ti;
|
||||
|
||||
if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
|
||||
if (enumerator.term() != null && compareAsUTF16(term, enumerator.term()) == 0) {
|
||||
ti = enumerator.termInfo;
|
||||
if (tiOrd == null) {
|
||||
if (useCache) {
|
||||
|
@ -360,9 +372,9 @@ public final class TermInfosReader {
|
|||
SegmentTermEnum enumerator = getThreadResources().termEnum;
|
||||
seekEnum(enumerator, indexOffset);
|
||||
|
||||
while(term.compareToUTF16(enumerator.term()) > 0 && enumerator.next()) {}
|
||||
while(compareAsUTF16(term, enumerator.term()) > 0 && enumerator.next()) {}
|
||||
|
||||
if (term.compareToUTF16(enumerator.term()) == 0)
|
||||
if (compareAsUTF16(term, enumerator.term()) == 0)
|
||||
return enumerator.position;
|
||||
else
|
||||
return -1;
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.lucene.store.IndexInput;
|
|||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
|
@ -102,9 +101,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
return null;
|
||||
}
|
||||
if (scratch.startsWith(FIELD)) {
|
||||
String field = StringHelper.intern(new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8"));
|
||||
current = field;
|
||||
return field;
|
||||
return current = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,7 +117,7 @@ public class AutomatonQuery extends MultiTermQuery {
|
|||
factory = new TermsEnumFactory() {
|
||||
@Override
|
||||
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
|
||||
return new SingleTermsEnum(terms.iterator(), term.createTerm(singleton));
|
||||
return new SingleTermsEnum(terms.iterator(), new Term(field, singleton));
|
||||
}
|
||||
};
|
||||
} else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
|
||||
|
@ -126,7 +126,7 @@ public class AutomatonQuery extends MultiTermQuery {
|
|||
factory = new TermsEnumFactory() {
|
||||
@Override
|
||||
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
|
||||
return new PrefixTermsEnum(terms.iterator(), term.createTerm(commonPrefix));
|
||||
return new PrefixTermsEnum(terms.iterator(), new Term(field, commonPrefix));
|
||||
}
|
||||
};
|
||||
} else {
|
||||
|
|
|
@ -100,14 +100,13 @@ class ConstantScoreAutoRewrite extends TermCollectingRewrite<BooleanQuery> {
|
|||
return getTopLevelQuery();
|
||||
} else {
|
||||
final BooleanQuery bq = getTopLevelQuery();
|
||||
final Term placeholderTerm = new Term(query.field);
|
||||
final BytesRefHash pendingTerms = col.pendingTerms;
|
||||
final int sort[] = pendingTerms.sort(col.termsEnum.getComparator());
|
||||
for(int i = 0; i < size; i++) {
|
||||
final int pos = sort[i];
|
||||
// docFreq is not used for constant score here, we pass 1
|
||||
// to explicitely set a fake value, so it's not calculated
|
||||
addClause(bq, placeholderTerm.createTerm(pendingTerms.get(pos, new BytesRef())), 1, 1.0f, col.array.termState[pos]);
|
||||
addClause(bq, new Term(query.field, pendingTerms.get(pos, new BytesRef())), 1, 1.0f, col.array.termState[pos]);
|
||||
}
|
||||
// Strip scores
|
||||
final Query result = new ConstantScoreQuery(bq);
|
||||
|
|
|
@ -42,7 +42,6 @@ import org.apache.lucene.search.cache.CachedArray.IntValues;
|
|||
import org.apache.lucene.search.cache.CachedArray.LongValues;
|
||||
import org.apache.lucene.search.cache.CachedArray.ShortValues;
|
||||
import org.apache.lucene.util.FieldCacheSanityChecker;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Expert: The default cache implementation, storing all values in memory.
|
||||
|
@ -246,7 +245,7 @@ public class FieldCacheImpl implements FieldCache { // Made Public so that
|
|||
|
||||
/** Creates one of these objects for a custom comparator/parser. */
|
||||
Entry (String field, EntryCreator<T> custom) {
|
||||
this.field = StringHelper.intern(field);
|
||||
this.field = field;
|
||||
this.creator = custom;
|
||||
}
|
||||
|
||||
|
@ -255,7 +254,7 @@ public class FieldCacheImpl implements FieldCache { // Made Public so that
|
|||
public boolean equals (Object o) {
|
||||
if (o instanceof Entry) {
|
||||
Entry other = (Entry) o;
|
||||
if (other.field == field) {
|
||||
if (other.field.equals(field)) {
|
||||
if (other.creator == null) {
|
||||
if (creator == null) return true;
|
||||
} else if (other.creator.equals (creator)) {
|
||||
|
|
|
@ -88,7 +88,7 @@ public class MultiPhraseQuery extends Query {
|
|||
field = terms[0].field();
|
||||
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
if (terms[i].field() != field) {
|
||||
if (!terms[i].field().equals(field)) {
|
||||
throw new IllegalArgumentException(
|
||||
"All phrase terms must be in the same field (" + field + "): "
|
||||
+ terms[i]);
|
||||
|
|
|
@ -85,14 +85,15 @@ public class PhraseQuery extends Query {
|
|||
* @param position
|
||||
*/
|
||||
public void add(Term term, int position) {
|
||||
if (terms.size() == 0)
|
||||
field = term.field();
|
||||
else if (term.field() != field)
|
||||
throw new IllegalArgumentException("All phrase terms must be in the same field: " + term);
|
||||
if (terms.size() == 0) {
|
||||
field = term.field();
|
||||
} else if (!term.field().equals(field)) {
|
||||
throw new IllegalArgumentException("All phrase terms must be in the same field: " + term);
|
||||
}
|
||||
|
||||
terms.add(term);
|
||||
positions.add(Integer.valueOf(position));
|
||||
if (position > maxPosition) maxPosition = position;
|
||||
terms.add(term);
|
||||
positions.add(Integer.valueOf(position));
|
||||
if (position > maxPosition) maxPosition = position;
|
||||
}
|
||||
|
||||
/** Returns the set of terms in this phrase. */
|
||||
|
|
|
@ -113,7 +113,6 @@ public abstract class ScoringRewrite<Q extends Query> extends TermCollectingRewr
|
|||
final ParallelArraysTermCollector col = new ParallelArraysTermCollector();
|
||||
collectTerms(reader, query, col);
|
||||
|
||||
final Term placeholderTerm = new Term(query.field);
|
||||
final int size = col.terms.size();
|
||||
if (size > 0) {
|
||||
final int sort[] = col.terms.sort(col.termsEnum.getComparator());
|
||||
|
@ -121,7 +120,7 @@ public abstract class ScoringRewrite<Q extends Query> extends TermCollectingRewr
|
|||
final PerReaderTermState[] termStates = col.array.termState;
|
||||
for (int i = 0; i < size; i++) {
|
||||
final int pos = sort[i];
|
||||
final Term term = placeholderTerm.createTerm(col.terms.get(pos, new BytesRef()));
|
||||
final Term term = new Term(query.getField(), col.terms.get(pos, new BytesRef()));
|
||||
assert reader.docFreq(term) == termStates[pos].docFreq();
|
||||
addClause(result, term, termStates[pos].docFreq(), query.getBoost() * boost[pos], termStates[pos]);
|
||||
}
|
||||
|
|
|
@ -21,8 +21,8 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.search.cache.*;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
// TODO(simonw) -- for cleaner transition, maybe we should make
|
||||
// a new SortField that subclasses this one and always uses
|
||||
|
@ -166,7 +166,7 @@ public class SortField {
|
|||
if (field == null) {
|
||||
throw new IllegalArgumentException("field can only be null when type is SCORE or DOC");
|
||||
}
|
||||
this.field = StringHelper.intern(field);
|
||||
this.field = field;
|
||||
this.reverse = reverse;
|
||||
|
||||
if (parser instanceof FieldCache.IntParser) {
|
||||
|
@ -205,7 +205,7 @@ public class SortField {
|
|||
*/
|
||||
public SortField( CachedArrayCreator<?> creator, boolean reverse )
|
||||
{
|
||||
this.field = StringHelper.intern(creator.field);
|
||||
this.field = creator.field;
|
||||
this.reverse = reverse;
|
||||
this.creator = creator;
|
||||
this.type = creator.getSortTypeID();
|
||||
|
@ -253,7 +253,7 @@ public class SortField {
|
|||
if (type != SCORE && type != DOC)
|
||||
throw new IllegalArgumentException("field can only be null when type is SCORE or DOC");
|
||||
} else {
|
||||
this.field = StringHelper.intern(field);
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
if( creator != null ) {
|
||||
|
@ -381,7 +381,7 @@ public class SortField {
|
|||
if (!(o instanceof SortField)) return false;
|
||||
final SortField other = (SortField)o;
|
||||
return (
|
||||
other.field == this.field // field is always interned
|
||||
StringHelper.equals(other.field, this.field)
|
||||
&& other.type == this.type
|
||||
&& other.reverse == this.reverse
|
||||
&& (other.comparatorSource == null ? this.comparatorSource == null : other.comparatorSource.equals(this.comparatorSource))
|
||||
|
|
|
@ -131,12 +131,11 @@ public abstract class TopTermsRewrite<Q extends Query> extends TermCollectingRew
|
|||
}
|
||||
});
|
||||
|
||||
final Term placeholderTerm = new Term(query.field);
|
||||
final Q q = getTopLevelQuery();
|
||||
final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
|
||||
ArrayUtil.mergeSort(scoreTerms, scoreTermSortByTermComp);
|
||||
for (final ScoreTerm st : scoreTerms) {
|
||||
final Term term = placeholderTerm.createTerm(st.bytes);
|
||||
final Term term = new Term(query.field, st.bytes);
|
||||
assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq();
|
||||
addClause(q, term, st.termState.docFreq(), query.getBoost() * st.boost, st.termState); // add to query
|
||||
}
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.lucene.search.FieldCache.DocTerms;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.GrowableWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -66,7 +65,6 @@ public class DocTermsCreator extends EntryCreatorWithOptions<DocTerms>
|
|||
@Override
|
||||
public DocTerms create(IndexReader reader) throws IOException {
|
||||
|
||||
String field = StringHelper.intern(this.field); // TODO?? necessary?
|
||||
Terms terms = MultiFields.getTerms(reader, field);
|
||||
|
||||
final boolean fasterButMoreRAM = hasOption( FASTER_BUT_MORE_RAM );
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.GrowableWriter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
|
@ -70,7 +69,6 @@ public class DocTermsIndexCreator extends EntryCreatorWithOptions<DocTermsIndex>
|
|||
@Override
|
||||
public DocTermsIndex create(IndexReader reader) throws IOException
|
||||
{
|
||||
String field = StringHelper.intern(this.field); // TODO?? necessary?
|
||||
Terms terms = MultiFields.getTerms(reader, field);
|
||||
|
||||
final boolean fasterButMoreRAM = hasOption(FASTER_BUT_MORE_RAM);
|
||||
|
|
|
@ -208,7 +208,7 @@ public final class FieldCacheSanityChecker {
|
|||
|
||||
if (seen.contains(rf)) continue;
|
||||
|
||||
List<Object> kids = getAllDecendentReaderKeys(rf.readerKey);
|
||||
List<Object> kids = getAllDescendentReaderKeys(rf.readerKey);
|
||||
for (Object kidKey : kids) {
|
||||
ReaderField kid = new ReaderField(kidKey, rf.fieldName);
|
||||
|
||||
|
@ -252,7 +252,7 @@ public final class FieldCacheSanityChecker {
|
|||
badness = badEntries.toArray(badness);
|
||||
|
||||
insanity.add(new Insanity(InsanityType.SUBREADER,
|
||||
"Found caches for decendents of " +
|
||||
"Found caches for descendants of " +
|
||||
parent.toString(),
|
||||
badness));
|
||||
}
|
||||
|
@ -266,7 +266,7 @@ public final class FieldCacheSanityChecker {
|
|||
* the hierarchy of subReaders building up a list of the objects
|
||||
* returned by obj.getFieldCacheKey()
|
||||
*/
|
||||
private List<Object> getAllDecendentReaderKeys(Object seed) {
|
||||
private List<Object> getAllDescendentReaderKeys(Object seed) {
|
||||
List<Object> all = new ArrayList<Object>(17); // will grow as we iter
|
||||
all.add(seed);
|
||||
for (int i = 0; i < all.size(); i++) {
|
||||
|
|
|
@ -1,85 +0,0 @@
|
|||
package org.apache.lucene.util;
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
|
||||
/**
|
||||
* Simple lockless and memory barrier free String intern cache that is guaranteed
|
||||
* to return the same String instance as String.intern()
|
||||
* does.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class SimpleStringInterner extends StringInterner {
|
||||
|
||||
private static class Entry {
|
||||
final private String str;
|
||||
final private int hash;
|
||||
private Entry next;
|
||||
private Entry(String str, int hash, Entry next) {
|
||||
this.str = str;
|
||||
this.hash = hash;
|
||||
this.next = next;
|
||||
}
|
||||
}
|
||||
|
||||
private final Entry[] cache;
|
||||
private final int maxChainLength;
|
||||
|
||||
/**
|
||||
* @param tableSize Size of the hash table, should be a power of two.
|
||||
* @param maxChainLength Maximum length of each bucket, after which the oldest item inserted is dropped.
|
||||
*/
|
||||
public SimpleStringInterner(int tableSize, int maxChainLength) {
|
||||
cache = new Entry[Math.max(1,BitUtil.nextHighestPowerOfTwo(tableSize))];
|
||||
this.maxChainLength = Math.max(2,maxChainLength);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String intern(String s) {
|
||||
int h = s.hashCode();
|
||||
// In the future, it may be worth augmenting the string hash
|
||||
// if the lower bits need better distribution.
|
||||
int slot = h & (cache.length-1);
|
||||
|
||||
Entry first = this.cache[slot];
|
||||
Entry nextToLast = null;
|
||||
|
||||
int chainLength = 0;
|
||||
|
||||
for(Entry e=first; e!=null; e=e.next) {
|
||||
if (e.hash == h && (e.str == s || e.str.compareTo(s)==0)) {
|
||||
// if (e.str == s || (e.hash == h && e.str.compareTo(s)==0)) {
|
||||
return e.str;
|
||||
}
|
||||
|
||||
chainLength++;
|
||||
if (e.next != null) {
|
||||
nextToLast = e;
|
||||
}
|
||||
}
|
||||
|
||||
// insertion-order cache: add new entry at head
|
||||
s = s.intern();
|
||||
this.cache[slot] = new Entry(s, h, first);
|
||||
if (chainLength >= maxChainLength) {
|
||||
// prune last entry
|
||||
nextToLast.next = null;
|
||||
}
|
||||
return s;
|
||||
}
|
||||
}
|
|
@ -27,17 +27,6 @@ import java.util.StringTokenizer;
|
|||
* @lucene.internal
|
||||
*/
|
||||
public abstract class StringHelper {
|
||||
/**
|
||||
* Expert:
|
||||
* The StringInterner implementation used by Lucene.
|
||||
* This shouldn't be changed to an incompatible implementation after other Lucene APIs have been used.
|
||||
*/
|
||||
public static StringInterner interner = new SimpleStringInterner(1024,8);
|
||||
|
||||
/** Return the same string object for all equal strings */
|
||||
public static String intern(String s) {
|
||||
return interner.intern(s);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares two byte[] arrays, element by element, and returns the
|
||||
|
@ -95,4 +84,12 @@ public abstract class StringHelper {
|
|||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
public static boolean equals(String s1, String s2) {
|
||||
if (s1 == null) {
|
||||
return s2 == null;
|
||||
} else {
|
||||
return s1.equals(s2);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,37 +0,0 @@
|
|||
package org.apache.lucene.util;
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Subclasses of StringInterner are required to
|
||||
* return the same single String object for all equal strings.
|
||||
* Depending on the implementation, this may not be
|
||||
* the same object returned as String.intern().
|
||||
*
|
||||
* This StringInterner base class simply delegates to String.intern().
|
||||
*/
|
||||
public class StringInterner {
|
||||
/** Returns a single object instance for each equal string. */
|
||||
public String intern(String s) {
|
||||
return s.intern();
|
||||
}
|
||||
|
||||
/** Returns a single object instance for each equal string. */
|
||||
public String intern(char[] arr, int offset, int len) {
|
||||
return intern(new String(arr, offset, len));
|
||||
}
|
||||
}
|
|
@ -713,8 +713,12 @@ public abstract class LuceneTestCase extends Assert {
|
|||
throw e;
|
||||
}
|
||||
|
||||
if (insanity.length != 0) {
|
||||
reportAdditionalFailureInfo();
|
||||
}
|
||||
|
||||
assertEquals(msg + ": Insane FieldCache usage(s) found",
|
||||
0, insanity.length);
|
||||
0, insanity.length);
|
||||
insanity = null;
|
||||
} finally {
|
||||
|
||||
|
@ -1104,11 +1108,17 @@ public abstract class LuceneTestCase extends Assert {
|
|||
/** Returns a new field instance, using the specified random.
|
||||
* See {@link #newField(String, String, Field.Store, Field.Index, Field.TermVector)} for more information */
|
||||
public static Field newField(Random random, String name, String value, Store store, Index index, TermVector tv) {
|
||||
|
||||
if (usually(random)) {
|
||||
// most of the time, don't modify the params
|
||||
return new Field(name, value, store, index, tv);
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
// tickle any code still relying on field names being interned:
|
||||
name = new String(name);
|
||||
}
|
||||
|
||||
if (!index.isIndexed())
|
||||
return new Field(name, value, store, index, tv);
|
||||
|
||||
|
|
|
@ -39,7 +39,6 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
|||
for (int i = 0; i < ids.length; i++) {
|
||||
ids[i] = random.nextInt();
|
||||
}
|
||||
Term template = new Term("id");
|
||||
DeleteSlice slice1 = queue.newSlice();
|
||||
DeleteSlice slice2 = queue.newSlice();
|
||||
BufferedDeletes bd1 = new BufferedDeletes(false);
|
||||
|
@ -50,7 +49,7 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
|||
for (int j = 0; j < ids.length; j++) {
|
||||
Integer i = ids[j];
|
||||
// create an array here since we compare identity below against tailItem
|
||||
Term[] term = new Term[] {template.createTerm(i.toString())};
|
||||
Term[] term = new Term[] {new Term("id", i.toString())};
|
||||
uniqueValues.add(term[0]);
|
||||
queue.addDelete(term);
|
||||
if (random.nextInt(20) == 0 || j == ids.length - 1) {
|
||||
|
@ -79,16 +78,14 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
|||
|
||||
private void assertAllBetween(int start, int end, BufferedDeletes deletes,
|
||||
Integer[] ids) {
|
||||
Term template = new Term("id");
|
||||
for (int i = start; i <= end; i++) {
|
||||
assertEquals(Integer.valueOf(end), deletes.terms.get(template
|
||||
.createTerm(ids[i].toString())));
|
||||
assertEquals(Integer.valueOf(end), deletes.terms.get(new Term("id",
|
||||
ids[i].toString())));
|
||||
}
|
||||
}
|
||||
|
||||
public void testClear() {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
|
||||
Term template = new Term("id");
|
||||
assertFalse(queue.anyChanges());
|
||||
queue.clear();
|
||||
assertFalse(queue.anyChanges());
|
||||
|
@ -96,7 +93,7 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
|||
int termsSinceFreeze = 0;
|
||||
int queriesSinceFreeze = 0;
|
||||
for (int i = 0; i < size; i++) {
|
||||
Term term = template.createTerm("" + i);
|
||||
Term term = new Term("id", "" + i);
|
||||
if (random.nextInt(10) == 0) {
|
||||
queue.addDelete(new TermQuery(term));
|
||||
queriesSinceFreeze++;
|
||||
|
@ -116,12 +113,11 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
|||
|
||||
public void testAnyChanges() {
|
||||
DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue();
|
||||
Term template = new Term("id");
|
||||
final int size = 200 + random.nextInt(500) * RANDOM_MULTIPLIER;
|
||||
int termsSinceFreeze = 0;
|
||||
int queriesSinceFreeze = 0;
|
||||
for (int i = 0; i < size; i++) {
|
||||
Term term = template.createTerm("" + i);
|
||||
Term term = new Term("id", "" + i);
|
||||
if (random.nextInt(10) == 0) {
|
||||
queue.addDelete(new TermQuery(term));
|
||||
queriesSinceFreeze++;
|
||||
|
@ -147,10 +143,9 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
|||
Set<Term> uniqueValues = new HashSet<Term>();
|
||||
final int size = 10000 + random.nextInt(500) * RANDOM_MULTIPLIER;
|
||||
Integer[] ids = new Integer[size];
|
||||
Term template = new Term("id");
|
||||
for (int i = 0; i < ids.length; i++) {
|
||||
ids[i] = random.nextInt();
|
||||
uniqueValues.add(template.createTerm(ids[i].toString()));
|
||||
uniqueValues.add(new Term("id", ids[i].toString()));
|
||||
}
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicInteger index = new AtomicInteger(0);
|
||||
|
@ -204,10 +199,9 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
|||
} catch (InterruptedException e) {
|
||||
throw new ThreadInterruptedException(e);
|
||||
}
|
||||
Term template = new Term("id");
|
||||
int i = 0;
|
||||
while ((i = index.getAndIncrement()) < ids.length) {
|
||||
Term term = template.createTerm(ids[i].toString());
|
||||
Term term = new Term("id", ids[i].toString());
|
||||
queue.add(term, slice);
|
||||
assertTrue(slice.isTailItem(term));
|
||||
slice.apply(deletes, BufferedDeletes.MAX_INT);
|
||||
|
|
|
@ -30,9 +30,6 @@ import org.apache.lucene.analysis.MockTokenizer;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.Field.Index;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field.TermVector;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
|
@ -940,7 +937,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
|
|||
count++;
|
||||
}
|
||||
}
|
||||
assertTrue("flush happened too quickly during " + (doIndexing ? "indexing" : "deleting") + " count=" + count, count > 3000);
|
||||
assertTrue("flush happened too quickly during " + (doIndexing ? "indexing" : "deleting") + " count=" + count, count > 2500);
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
|
|
|
@ -40,13 +40,12 @@ public class TestRollback extends LuceneTestCase {
|
|||
// If buffer size is small enough to cause a flush, errors ensue...
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMaxBufferedDocs(2).setOpenMode(IndexWriterConfig.OpenMode.APPEND));
|
||||
|
||||
Term pkTerm = new Term("pk", "");
|
||||
for (int i = 0; i < 3; i++) {
|
||||
Document doc = new Document();
|
||||
String value = Integer.toString(i);
|
||||
doc.add(newField("pk", value, Store.YES, Index.ANALYZED_NO_NORMS));
|
||||
doc.add(newField("text", "foo", Store.YES, Index.ANALYZED_NO_NORMS));
|
||||
w.updateDocument(pkTerm.createTerm(value), doc);
|
||||
w.updateDocument(new Term("pk", value), doc);
|
||||
}
|
||||
w.rollback();
|
||||
|
||||
|
|
|
@ -312,7 +312,6 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
int[] r2r1 = new int[r2.maxDoc()]; // r2 id to r1 id mapping
|
||||
|
||||
// create mapping from id2 space to id2 based on idField
|
||||
idField = StringHelper.intern(idField);
|
||||
final Fields f1 = MultiFields.getFields(r1);
|
||||
if (f1 == null) {
|
||||
// make sure r2 is empty
|
||||
|
@ -669,7 +668,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
|
||||
ArrayList<Field> fields = new ArrayList<Field>();
|
||||
String idString = getIdString();
|
||||
Field idField = newField(idTerm.field(), idString, Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
|
||||
Field idField = newField("id", idString, Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
|
||||
fields.add(idField);
|
||||
|
||||
int nFields = nextInt(maxFields);
|
||||
|
@ -720,7 +719,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": indexing id:" + idString);
|
||||
}
|
||||
w.updateDocument(idTerm.createTerm(idString), d);
|
||||
w.updateDocument(new Term("id", idString), d);
|
||||
//System.out.println(Thread.currentThread().getName() + ": indexing "+d);
|
||||
docs.put(idString, d);
|
||||
}
|
||||
|
@ -730,7 +729,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": del id:" + idString);
|
||||
}
|
||||
w.deleteDocuments(idTerm.createTerm(idString));
|
||||
w.deleteDocuments(new Term("id", idString));
|
||||
docs.remove(idString);
|
||||
}
|
||||
|
||||
|
@ -739,7 +738,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
if (VERBOSE) {
|
||||
System.out.println(Thread.currentThread().getName() + ": del query id:" + idString);
|
||||
}
|
||||
w.deleteDocuments(new TermQuery(idTerm.createTerm(idString)));
|
||||
w.deleteDocuments(new TermQuery(new Term("id", idString)));
|
||||
docs.remove(idString);
|
||||
}
|
||||
|
||||
|
|
|
@ -75,8 +75,15 @@ public class TestSurrogates extends LuceneTestCase {
|
|||
}
|
||||
|
||||
private static class SortTermAsUTF16Comparator implements Comparator<Term> {
|
||||
public int compare(Term o1, Term o2) {
|
||||
return o1.compareToUTF16(o2);
|
||||
private static final Comparator<BytesRef> legacyComparator =
|
||||
BytesRef.getUTF8SortedAsUTF16Comparator();
|
||||
|
||||
public int compare(Term term1, Term term2) {
|
||||
if (term1.field().equals(term2.field())) {
|
||||
return legacyComparator.compare(term1.bytes(), term2.bytes());
|
||||
} else {
|
||||
return term1.field().compareTo(term2.field());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -171,7 +178,7 @@ public class TestSurrogates extends LuceneTestCase {
|
|||
break;
|
||||
}
|
||||
term = fieldTerms.get(1+spot+i);
|
||||
if (term.field() != field) {
|
||||
if (!term.field().equals(field)) {
|
||||
assertNull(te.next());
|
||||
break;
|
||||
} else {
|
||||
|
@ -224,7 +231,7 @@ public class TestSurrogates extends LuceneTestCase {
|
|||
|
||||
spot = -spot - 1;
|
||||
|
||||
if (spot == fieldTerms.size() || fieldTerms.get(spot).field() != field) {
|
||||
if (spot == fieldTerms.size() || !fieldTerms.get(spot).field().equals(field)) {
|
||||
assertEquals(TermsEnum.SeekStatus.END, te.seek(tx.bytes()));
|
||||
} else {
|
||||
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seek(tx.bytes()));
|
||||
|
@ -247,7 +254,7 @@ public class TestSurrogates extends LuceneTestCase {
|
|||
break;
|
||||
}
|
||||
Term term = fieldTerms.get(1+spot+i);
|
||||
if (term.field() != field) {
|
||||
if (!term.field().equals(field)) {
|
||||
assertNull(te.next());
|
||||
break;
|
||||
} else {
|
||||
|
|
|
@ -1,89 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.util;
|
||||
import java.util.Random;
|
||||
|
||||
public class TestStringIntern extends LuceneTestCase {
|
||||
String[] testStrings;
|
||||
String[] internedStrings;
|
||||
|
||||
private String randStr(int len) {
|
||||
char[] arr = new char[len];
|
||||
for (int i=0; i<len; i++) {
|
||||
arr[i] = (char)('a' + random.nextInt(26));
|
||||
}
|
||||
return new String(arr);
|
||||
}
|
||||
|
||||
private void makeStrings(int sz) {
|
||||
testStrings = new String[sz];
|
||||
internedStrings = new String[sz];
|
||||
for (int i=0; i<sz; i++) {
|
||||
testStrings[i] = randStr(random.nextInt(8)+3);
|
||||
}
|
||||
}
|
||||
|
||||
public void testStringIntern() throws InterruptedException {
|
||||
makeStrings(1024*10); // something greater than the capacity of the default cache size
|
||||
// makeStrings(100); // realistic for perf testing
|
||||
int nThreads = 20;
|
||||
// final int iter=100000;
|
||||
final int iter = atLeast(100000);
|
||||
|
||||
// try native intern
|
||||
// StringHelper.interner = new StringInterner();
|
||||
|
||||
Thread[] threads = new Thread[nThreads];
|
||||
for (int i=0; i<nThreads; i++) {
|
||||
final int seed = i;
|
||||
threads[i] = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
Random rand = new Random(seed);
|
||||
String[] myInterned = new String[testStrings.length];
|
||||
for (int j=0; j<iter; j++) {
|
||||
int idx = rand.nextInt(testStrings.length);
|
||||
String s = testStrings[idx];
|
||||
if (rand.nextBoolean()) s = new String(s); // make a copy half of the time
|
||||
String interned = StringHelper.intern(s);
|
||||
String prevInterned = myInterned[idx];
|
||||
String otherInterned = internedStrings[idx];
|
||||
|
||||
// test against other threads
|
||||
if (otherInterned != null && otherInterned != interned) {
|
||||
fail();
|
||||
}
|
||||
internedStrings[idx] = interned;
|
||||
|
||||
// test against local copy
|
||||
if (prevInterned != null && prevInterned != interned) {
|
||||
fail();
|
||||
}
|
||||
myInterned[idx] = interned;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
threads[i].start();
|
||||
}
|
||||
|
||||
for (int i=0; i<nThreads; i++) {
|
||||
threads[i].join();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -481,7 +481,7 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
|
|||
FieldsEnum fields = MultiFields.getFields(reader).iterator();
|
||||
String fieldName = null;
|
||||
while((fieldName = fields.next()) != null) {
|
||||
if (fieldName == DocMaker.ID_FIELD || fieldName == DocMaker.DATE_MSEC_FIELD || fieldName == DocMaker.TIME_SEC_FIELD) {
|
||||
if (fieldName.equals(DocMaker.ID_FIELD) || fieldName.equals(DocMaker.DATE_MSEC_FIELD) || fieldName.equals(DocMaker.TIME_SEC_FIELD)) {
|
||||
continue;
|
||||
}
|
||||
TermsEnum terms = fields.terms();
|
||||
|
|
|
@ -328,8 +328,9 @@ public class DirectSpellChecker {
|
|||
if (minQueryLength > 0 && text.codePointCount(0, text.length()) < minQueryLength)
|
||||
return new SuggestWord[0];
|
||||
|
||||
if (lowerCaseTerms)
|
||||
term = term.createTerm(text.toLowerCase(Locale.ENGLISH));
|
||||
if (lowerCaseTerms) {
|
||||
term = new Term(term.field(), text.toLowerCase(Locale.ENGLISH));
|
||||
}
|
||||
|
||||
int docfreq = ir.docFreq(term);
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.search.spell.Dictionary;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
|
@ -47,7 +46,7 @@ public class HighFrequencyDictionary implements Dictionary {
|
|||
|
||||
public HighFrequencyDictionary(IndexReader reader, String field, float thresh) {
|
||||
this.reader = reader;
|
||||
this.field = StringHelper.intern(field);
|
||||
this.field = field;
|
||||
this.thresh = thresh;
|
||||
}
|
||||
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
import java.io.*;
|
||||
|
||||
|
@ -46,7 +45,7 @@ public class LuceneDictionary implements Dictionary {
|
|||
|
||||
public LuceneDictionary(IndexReader reader, String field) {
|
||||
this.reader = reader;
|
||||
this.field = StringHelper.intern(field);
|
||||
this.field = field;
|
||||
}
|
||||
|
||||
public final Iterator<String> getWordsIterator() {
|
||||
|
|
|
@ -77,8 +77,6 @@ public class SpellChecker implements java.io.Closeable {
|
|||
*/
|
||||
public static final String F_WORD = "word";
|
||||
|
||||
private static final Term F_WORD_TERM = new Term(F_WORD);
|
||||
|
||||
/**
|
||||
* the spell index
|
||||
*/
|
||||
|
@ -486,7 +484,7 @@ public class SpellChecker implements java.io.Closeable {
|
|||
// obtainSearcher calls ensureOpen
|
||||
final IndexSearcher indexSearcher = obtainSearcher();
|
||||
try{
|
||||
return indexSearcher.docFreq(F_WORD_TERM.createTerm(word)) > 0;
|
||||
return indexSearcher.docFreq(new Term(F_WORD, word)) > 0;
|
||||
} finally {
|
||||
releaseSearcher(indexSearcher);
|
||||
}
|
||||
|
|
|
@ -46,7 +46,6 @@ import org.apache.lucene.index.IndexReader;
|
|||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.solr.cloud.ZkController;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.solr.common.SolrException;
|
||||
|
@ -161,7 +160,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
|
|||
throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
|
||||
"QueryElevationComponent requires the schema to have a uniqueKeyField implemented using StrField" );
|
||||
}
|
||||
idField = StringHelper.intern(sf.getName());
|
||||
idField = sf.getName();
|
||||
|
||||
forceElevation = initArgs.getBool( QueryElevationParams.FORCE_ELEVATION, forceElevation );
|
||||
try {
|
||||
|
|
|
@ -296,7 +296,7 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
|
|||
private boolean useOffsets, usePositions;
|
||||
//private Map<String, Integer> idfCache;
|
||||
private NamedList<Object> fieldNL;
|
||||
private Term currentTerm;
|
||||
private String field;
|
||||
|
||||
|
||||
public TVMapper(IndexReader reader) {
|
||||
|
@ -337,9 +337,8 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
|
|||
|
||||
private int getDocFreq(BytesRef term) {
|
||||
int result = 1;
|
||||
currentTerm = currentTerm.createTerm(term);
|
||||
try {
|
||||
Terms terms = MultiFields.getTerms(reader, currentTerm.field());
|
||||
Terms terms = MultiFields.getTerms(reader, field);
|
||||
if (terms != null) {
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
if (termsEnum.seek(term) == TermsEnum.SeekStatus.FOUND) {
|
||||
|
@ -354,10 +353,7 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
|
|||
|
||||
@Override
|
||||
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
|
||||
|
||||
if (fieldOptions.docFreq == true && reader != null) {
|
||||
this.currentTerm = new Term(field);
|
||||
}
|
||||
this.field = field;
|
||||
useOffsets = storeOffsets && fieldOptions.offsets;
|
||||
usePositions = storePositions && fieldOptions.positions;
|
||||
fieldNL = new NamedList<Object>();
|
||||
|
|
|
@ -22,7 +22,6 @@ import org.apache.lucene.queryParser.ParseException;
|
|||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.packed.Direct16;
|
||||
import org.apache.lucene.util.packed.Direct32;
|
||||
|
@ -371,10 +370,9 @@ public class SimpleFacets {
|
|||
FieldType ft = searcher.getSchema().getFieldType(field);
|
||||
List<String> terms = StrUtils.splitSmart(termList, ",", true);
|
||||
NamedList<Integer> res = new NamedList<Integer>();
|
||||
Term t = new Term(field);
|
||||
for (String term : terms) {
|
||||
String internal = ft.toInternal(term);
|
||||
int count = searcher.numDocs(new TermQuery(t.createTerm(internal)), base);
|
||||
int count = searcher.numDocs(new TermQuery(new Term(field, internal)), base);
|
||||
res.add(term, count);
|
||||
}
|
||||
return res;
|
||||
|
@ -674,7 +672,7 @@ public class SimpleFacets {
|
|||
|
||||
if (deState==null) {
|
||||
deState = new SolrIndexSearcher.DocsEnumState();
|
||||
deState.fieldName = StringHelper.intern(field);
|
||||
deState.fieldName = field;
|
||||
deState.deletedDocs = MultiFields.getDeletedDocs(r);
|
||||
deState.termsEnum = termsEnum;
|
||||
deState.docsEnum = docsEnum;
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.lucene.index.Term;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TermRangeQuery;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.solr.common.params.FacetParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.SolrException;
|
||||
|
@ -123,7 +122,7 @@ public class UnInvertedField extends DocTermOrds {
|
|||
|
||||
if (deState == null) {
|
||||
deState = new SolrIndexSearcher.DocsEnumState();
|
||||
deState.fieldName = StringHelper.intern(field);
|
||||
deState.fieldName = field;
|
||||
// deState.termsEnum = te.tenum;
|
||||
deState.termsEnum = te; // TODO: check for MultiTermsEnum in SolrIndexSearcher could now fail?
|
||||
deState.docsEnum = docsEnum;
|
||||
|
|
|
@ -27,7 +27,6 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.document.Fieldable;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.solr.common.SolrDocument;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
|
@ -75,7 +74,7 @@ class JSONWriter extends TextResponseWriter {
|
|||
|
||||
public JSONWriter(Writer writer, SolrQueryRequest req, SolrQueryResponse rsp) {
|
||||
super(writer, req, rsp);
|
||||
namedListStyle = StringHelper.intern(req.getParams().get(JSON_NL_STYLE, JSON_NL_FLAT));
|
||||
namedListStyle = req.getParams().get(JSON_NL_STYLE, JSON_NL_FLAT).intern();
|
||||
wrapperFunction = req.getParams().get(JSON_WRAPPER_FUNCTION);
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ import org.apache.lucene.search.*;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
|
@ -290,14 +289,14 @@ class JoinQuery extends Query {
|
|||
Bits toDeletedDocs = fromSearcher == toSearcher ? fromDeletedDocs : MultiFields.getDeletedDocs(toSearcher.getIndexReader());
|
||||
|
||||
fromDeState = new SolrIndexSearcher.DocsEnumState();
|
||||
fromDeState.fieldName = StringHelper.intern(fromField);
|
||||
fromDeState.fieldName = fromField;
|
||||
fromDeState.deletedDocs = fromDeletedDocs;
|
||||
fromDeState.termsEnum = termsEnum;
|
||||
fromDeState.docsEnum = null;
|
||||
fromDeState.minSetSizeCached = minDocFreqFrom;
|
||||
|
||||
toDeState = new SolrIndexSearcher.DocsEnumState();
|
||||
toDeState.fieldName = StringHelper.intern(toField);
|
||||
toDeState.fieldName = toField;
|
||||
toDeState.deletedDocs = toDeletedDocs;
|
||||
toDeState.termsEnum = toTermsEnum;
|
||||
toDeState.docsEnum = null;
|
||||
|
|
|
@ -715,7 +715,7 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
|
|||
TermQuery key = null;
|
||||
|
||||
if (useCache) {
|
||||
key = new TermQuery(new Term(deState.fieldName, new BytesRef(deState.termsEnum.term()), false));
|
||||
key = new TermQuery(new Term(deState.fieldName, new BytesRef(deState.termsEnum.term())));
|
||||
DocSet result = filterCache.get(key);
|
||||
if (result != null) return result;
|
||||
}
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
|||
import org.apache.lucene.index.IndexReader.ReaderContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.handler.RequestHandlerBase;
|
||||
import org.apache.solr.handler.RequestHandlerUtils;
|
||||
|
@ -224,7 +223,7 @@ public class FileFloatSource extends ValueSource {
|
|||
|
||||
BufferedReader r = new BufferedReader(new InputStreamReader(is));
|
||||
|
||||
String idName = StringHelper.intern(ffs.keyField.getName());
|
||||
String idName = ffs.keyField.getName();
|
||||
FieldType idType = ffs.keyField.getType();
|
||||
|
||||
// warning: lucene's termEnum.skipTo() is not optimized... it simply does a next()
|
||||
|
|
|
@ -181,13 +181,13 @@ public abstract class AbstractLuceneSpellChecker extends SolrSpellChecker {
|
|||
}
|
||||
|
||||
if (options.extendedResults == true && reader != null && field != null) {
|
||||
term = term.createTerm(tokenText);
|
||||
term = new Term(field, tokenText);
|
||||
result.addFrequency(token, reader.docFreq(term));
|
||||
int countLimit = Math.min(options.count, suggestions.length);
|
||||
if(countLimit>0)
|
||||
{
|
||||
for (int i = 0; i < countLimit; i++) {
|
||||
term = term.createTerm(suggestions[i]);
|
||||
term = new Term(field, suggestions[i]);
|
||||
result.add(token, suggestions[i], reader.docFreq(term));
|
||||
}
|
||||
} else if(shardRequest) {
|
||||
|
|
|
@ -169,13 +169,13 @@ public class DirectUpdateHandler2 extends UpdateHandler {
|
|||
if (cmd.indexedId == null) {
|
||||
cmd.indexedId = getIndexedId(cmd.doc);
|
||||
}
|
||||
Term idTerm = this.idTerm.createTerm(cmd.indexedId);
|
||||
Term idTerm = new Term(idField.getName(), cmd.indexedId);
|
||||
boolean del = false;
|
||||
if (cmd.updateTerm == null) {
|
||||
updateTerm = idTerm;
|
||||
} else {
|
||||
del = true;
|
||||
updateTerm = cmd.updateTerm;
|
||||
updateTerm = cmd.updateTerm;
|
||||
}
|
||||
|
||||
writer.updateDocument(updateTerm, cmd.getLuceneDocument(schema));
|
||||
|
@ -214,7 +214,7 @@ public class DirectUpdateHandler2 extends UpdateHandler {
|
|||
iwCommit.lock();
|
||||
try {
|
||||
openWriter();
|
||||
writer.deleteDocuments(idTerm.createTerm(idFieldType.toInternal(cmd.id)));
|
||||
writer.deleteDocuments(new Term(idField.getName(), idFieldType.toInternal(cmd.id)));
|
||||
} finally {
|
||||
iwCommit.unlock();
|
||||
}
|
||||
|
|
|
@ -54,7 +54,6 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
|
||||
protected final SchemaField idField;
|
||||
protected final FieldType idFieldType;
|
||||
protected final Term idTerm; // prototype term to avoid interning fieldname
|
||||
|
||||
protected Vector<SolrEventListener> commitCallbacks = new Vector<SolrEventListener>();
|
||||
protected Vector<SolrEventListener> optimizeCallbacks = new Vector<SolrEventListener>();
|
||||
|
@ -93,7 +92,6 @@ public abstract class UpdateHandler implements SolrInfoMBean {
|
|||
schema = core.getSchema();
|
||||
idField = schema.getUniqueKeyField();
|
||||
idFieldType = idField!=null ? idField.getType() : null;
|
||||
idTerm = idField!=null ? new Term(idField.getName(),"") : null;
|
||||
parseEventListeners();
|
||||
}
|
||||
|
||||
|
|
|
@ -61,8 +61,6 @@ public class SignatureUpdateProcessorFactory
|
|||
|
||||
signatureField = params.get("signatureField", "signatureField");
|
||||
|
||||
signatureTerm = new Term(signatureField, "");
|
||||
|
||||
signatureClass = params.get("signatureClass",
|
||||
"org.apache.solr.update.processor.Lookup3Signature");
|
||||
this.params = params;
|
||||
|
@ -173,7 +171,7 @@ public class SignatureUpdateProcessorFactory
|
|||
doc.addField(signatureField, sigString);
|
||||
|
||||
if (overwriteDupes) {
|
||||
cmd.updateTerm = signatureTerm.createTerm(sigString);
|
||||
cmd.updateTerm = new Term(signatureField, sigString);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue