mirror of https://github.com/apache/lucene.git
LUCENE-3622: separate IndexDocValues interface from implementation (phase 1)
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1213844 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
8d7749ea4f
|
@ -505,23 +505,23 @@ New features
|
|||
bytes in RAM. (Mike McCandless)
|
||||
|
||||
* LUCENE-3108, LUCENE-2935, LUCENE-2168, LUCENE-1231: Changes from
|
||||
IndexDocValues (ColumnStrideFields):
|
||||
DocValues (ColumnStrideFields):
|
||||
|
||||
- IndexWriter now supports typesafe dense per-document values stored in
|
||||
a column like storage. IndexDocValues are stored on a per-document
|
||||
a column like storage. DocValues are stored on a per-document
|
||||
basis where each documents field can hold exactly one value of a given
|
||||
type. IndexDocValues are provided via Fieldable and can be used in
|
||||
type. DocValues are provided via Fieldable and can be used in
|
||||
conjunction with stored and indexed values.
|
||||
|
||||
- IndexDocValues provides an entirely RAM resident document id to value
|
||||
- DocValues provides an entirely RAM resident document id to value
|
||||
mapping per field as well as a DocIdSetIterator based disk-resident
|
||||
sequential access API relying on filesystem-caches.
|
||||
|
||||
- Both APIs are exposed via IndexReader and the Codec / Flex API allowing
|
||||
expert users to integrate customized IndexDocValues reader and writer
|
||||
expert users to integrate customized DocValues reader and writer
|
||||
implementations by extending existing Codecs.
|
||||
|
||||
- IndexDocValues provides implementations for primitive datatypes like int,
|
||||
- DocValues provides implementations for primitive datatypes like int,
|
||||
long, float, double and arrays of byte. Byte based implementations further
|
||||
provide storage variants like straight or dereferenced stored bytes, fixed
|
||||
and variable length bytes as well as index time sorted based on
|
||||
|
|
|
@ -383,6 +383,8 @@ LUCENE-1458, LUCENE-2111: Flexible Indexing
|
|||
- o.a.l.search.function.ValueSource -> o.a.l.queries.function.ValueSource
|
||||
- o.a.l.search.function.ValueSourceQuery -> o.a.l.queries.function.FunctionQuery
|
||||
|
||||
DocValues are now named FunctionValues, to not confuse with Lucene's per-document values.
|
||||
|
||||
* LUCENE-2392: Enable flexible scoring:
|
||||
|
||||
The existing "Similarity" api is now TFIDFSimilarity, if you were extending
|
||||
|
|
|
@ -41,12 +41,12 @@ import org.apache.lucene.index.FieldInvertState;
|
|||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.OrdTermState;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermState;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.Collector;
|
||||
|
@ -1159,7 +1159,7 @@ public class MemoryIndex {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,8 +28,8 @@ import org.apache.lucene.index.FieldInfo;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.IndexableFieldType;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** Defers actually loading a field's value until you ask
|
||||
|
@ -157,20 +157,20 @@ public class LazyDocument {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocFieldValues docValues() {
|
||||
public DocValue docValue() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).docValues();
|
||||
return getDocument().getField(name).docValue();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].docValues();
|
||||
return getDocument().getFields(name)[num].docValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType docValuesType() {
|
||||
public DocValues.Type docValueType() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).docValuesType();
|
||||
return getDocument().getField(name).docValueType();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].docValuesType();
|
||||
return getDocument().getFields(name)[num].docValueType();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,8 +20,9 @@ import java.io.Reader;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.IndexableFieldType;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
|
@ -31,14 +32,14 @@ import org.apache.lucene.util.BytesRef;
|
|||
* example usage, adding an int value:
|
||||
*
|
||||
* <pre>
|
||||
* document.add(new IndexDocValuesField(name).setInt(value));
|
||||
* document.add(new DocValuesField(name).setInt(value));
|
||||
* </pre>
|
||||
*
|
||||
* For optimal performance, re-use the <code>DocValuesField</code> and
|
||||
* {@link Document} instance for more than one document:
|
||||
*
|
||||
* <pre>
|
||||
* IndexDocValuesField field = new IndexDocValuesField(name);
|
||||
* DocValuesField field = new DocValuesField(name);
|
||||
* Document document = new Document();
|
||||
* document.add(field);
|
||||
*
|
||||
|
@ -56,7 +57,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* when creating the field:
|
||||
*
|
||||
* <pre>
|
||||
* IndexDocValuesField field = new IndexDocValuesField(name, StringField.TYPE_STORED);
|
||||
* DocValuesField field = new DocValuesField(name, StringField.TYPE_STORED);
|
||||
* Document document = new Document();
|
||||
* document.add(field);
|
||||
* for(all documents) {
|
||||
|
@ -68,40 +69,39 @@ import org.apache.lucene.util.BytesRef;
|
|||
* </pre>
|
||||
*
|
||||
* */
|
||||
// TODO: maybe rename to DocValuesField?
|
||||
public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
||||
public class DocValuesField extends Field implements DocValue {
|
||||
|
||||
protected BytesRef bytes;
|
||||
protected double doubleValue;
|
||||
protected long longValue;
|
||||
protected ValueType type;
|
||||
protected DocValues.Type type;
|
||||
protected Comparator<BytesRef> bytesComparator;
|
||||
|
||||
/**
|
||||
* Creates a new {@link IndexDocValuesField} with the given name.
|
||||
* Creates a new {@link DocValuesField} with the given name.
|
||||
*/
|
||||
public IndexDocValuesField(String name) {
|
||||
public DocValuesField(String name) {
|
||||
this(name, new FieldType());
|
||||
}
|
||||
|
||||
public IndexDocValuesField(String name, IndexableFieldType type) {
|
||||
public DocValuesField(String name, IndexableFieldType type) {
|
||||
this(name, type, null);
|
||||
}
|
||||
|
||||
public IndexDocValuesField(String name, IndexableFieldType type, String value) {
|
||||
public DocValuesField(String name, IndexableFieldType type, String value) {
|
||||
super(name, type);
|
||||
fieldsData = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocFieldValues docValues() {
|
||||
public DocValue docValue() {
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>long</code> value and sets the field's {@link ValueType} to
|
||||
* {@link ValueType#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(ValueType)}.
|
||||
* Sets the given <code>long</code> value and sets the field's {@link Type} to
|
||||
* {@link Type#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(DocValues.Type)}.
|
||||
*/
|
||||
public void setInt(long value) {
|
||||
setInt(value, false);
|
||||
|
@ -113,20 +113,20 @@ public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
|||
* @param value
|
||||
* the value to set
|
||||
* @param fixed
|
||||
* if <code>true</code> {@link ValueType#FIXED_INTS_64} is used
|
||||
* otherwise {@link ValueType#VAR_INTS}
|
||||
* if <code>true</code> {@link Type#FIXED_INTS_64} is used
|
||||
* otherwise {@link Type#VAR_INTS}
|
||||
*/
|
||||
public void setInt(long value, boolean fixed) {
|
||||
if (type == null) {
|
||||
type = fixed ? ValueType.FIXED_INTS_64 : ValueType.VAR_INTS;
|
||||
type = fixed ? DocValues.Type.FIXED_INTS_64 : DocValues.Type.VAR_INTS;
|
||||
}
|
||||
longValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>int</code> value and sets the field's {@link ValueType} to
|
||||
* {@link ValueType#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(ValueType)}.
|
||||
* Sets the given <code>int</code> value and sets the field's {@link Type} to
|
||||
* {@link Type#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(DocValues.Type)}.
|
||||
*/
|
||||
public void setInt(int value) {
|
||||
setInt(value, false);
|
||||
|
@ -138,20 +138,20 @@ public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
|||
* @param value
|
||||
* the value to set
|
||||
* @param fixed
|
||||
* if <code>true</code> {@link ValueType#FIXED_INTS_32} is used
|
||||
* otherwise {@link ValueType#VAR_INTS}
|
||||
* if <code>true</code> {@link Type#FIXED_INTS_32} is used
|
||||
* otherwise {@link Type#VAR_INTS}
|
||||
*/
|
||||
public void setInt(int value, boolean fixed) {
|
||||
if (type == null) {
|
||||
type = fixed ? ValueType.FIXED_INTS_32 : ValueType.VAR_INTS;
|
||||
type = fixed ? DocValues.Type.FIXED_INTS_32 : DocValues.Type.VAR_INTS;
|
||||
}
|
||||
longValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>short</code> value and sets the field's {@link ValueType} to
|
||||
* {@link ValueType#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(ValueType)}.
|
||||
* Sets the given <code>short</code> value and sets the field's {@link Type} to
|
||||
* {@link Type#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(DocValues.Type)}.
|
||||
*/
|
||||
public void setInt(short value) {
|
||||
setInt(value, false);
|
||||
|
@ -163,20 +163,20 @@ public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
|||
* @param value
|
||||
* the value to set
|
||||
* @param fixed
|
||||
* if <code>true</code> {@link ValueType#FIXED_INTS_16} is used
|
||||
* otherwise {@link ValueType#VAR_INTS}
|
||||
* if <code>true</code> {@link Type#FIXED_INTS_16} is used
|
||||
* otherwise {@link Type#VAR_INTS}
|
||||
*/
|
||||
public void setInt(short value, boolean fixed) {
|
||||
if (type == null) {
|
||||
type = fixed ? ValueType.FIXED_INTS_16 : ValueType.VAR_INTS;
|
||||
type = fixed ? DocValues.Type.FIXED_INTS_16 : DocValues.Type.VAR_INTS;
|
||||
}
|
||||
longValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>byte</code> value and sets the field's {@link ValueType} to
|
||||
* {@link ValueType#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(ValueType)}.
|
||||
* Sets the given <code>byte</code> value and sets the field's {@link Type} to
|
||||
* {@link Type#VAR_INTS} unless already set. If you want to change the
|
||||
* default type use {@link #setDocValuesType(DocValues.Type)}.
|
||||
*/
|
||||
public void setInt(byte value) {
|
||||
setInt(value, false);
|
||||
|
@ -188,59 +188,59 @@ public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
|||
* @param value
|
||||
* the value to set
|
||||
* @param fixed
|
||||
* if <code>true</code> {@link ValueType#FIXED_INTS_8} is used
|
||||
* otherwise {@link ValueType#VAR_INTS}
|
||||
* if <code>true</code> {@link Type#FIXED_INTS_8} is used
|
||||
* otherwise {@link Type#VAR_INTS}
|
||||
*/
|
||||
public void setInt(byte value, boolean fixed) {
|
||||
if (type == null) {
|
||||
type = fixed ? ValueType.FIXED_INTS_8 : ValueType.VAR_INTS;
|
||||
type = fixed ? DocValues.Type.FIXED_INTS_8 : DocValues.Type.VAR_INTS;
|
||||
}
|
||||
longValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>float</code> value and sets the field's {@link ValueType}
|
||||
* to {@link ValueType#FLOAT_32} unless already set. If you want to
|
||||
* change the type use {@link #setDocValuesType(ValueType)}.
|
||||
* Sets the given <code>float</code> value and sets the field's {@link Type}
|
||||
* to {@link Type#FLOAT_32} unless already set. If you want to
|
||||
* change the type use {@link #setDocValuesType(DocValues.Type)}.
|
||||
*/
|
||||
public void setFloat(float value) {
|
||||
if (type == null) {
|
||||
type = ValueType.FLOAT_32;
|
||||
type = DocValues.Type.FLOAT_32;
|
||||
}
|
||||
doubleValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given <code>double</code> value and sets the field's {@link ValueType}
|
||||
* to {@link ValueType#FLOAT_64} unless already set. If you want to
|
||||
* change the default type use {@link #setDocValuesType(ValueType)}.
|
||||
* Sets the given <code>double</code> value and sets the field's {@link Type}
|
||||
* to {@link Type#FLOAT_64} unless already set. If you want to
|
||||
* change the default type use {@link #setDocValuesType(DocValues.Type)}.
|
||||
*/
|
||||
public void setFloat(double value) {
|
||||
if (type == null) {
|
||||
type = ValueType.FLOAT_64;
|
||||
type = DocValues.Type.FLOAT_64;
|
||||
}
|
||||
doubleValue = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value and the field's {@link ValueType}. The
|
||||
* Sets the given {@link BytesRef} value and the field's {@link Type}. The
|
||||
* comparator for this field is set to <code>null</code>. If a
|
||||
* <code>null</code> comparator is set the default comparator for the given
|
||||
* {@link ValueType} is used.
|
||||
* {@link Type} is used.
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type) {
|
||||
public void setBytes(BytesRef value, DocValues.Type type) {
|
||||
setBytes(value, type, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value, the field's {@link ValueType} and the
|
||||
* Sets the given {@link BytesRef} value, the field's {@link Type} and the
|
||||
* field's comparator. If the {@link Comparator} is set to <code>null</code>
|
||||
* the default for the given {@link ValueType} is used instead.
|
||||
* the default for the given {@link Type} is used instead.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* if the value or the type are null
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type, Comparator<BytesRef> comp) {
|
||||
public void setBytes(BytesRef value, DocValues.Type type, Comparator<BytesRef> comp) {
|
||||
if (value == null) {
|
||||
throw new IllegalArgumentException("value must not be null");
|
||||
}
|
||||
|
@ -283,16 +283,16 @@ public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
|||
|
||||
/**
|
||||
* Sets the {@link BytesRef} comparator for this field. If the field has a
|
||||
* numeric {@link ValueType} the comparator will be ignored.
|
||||
* numeric {@link Type} the comparator will be ignored.
|
||||
*/
|
||||
public void setBytesComparator(Comparator<BytesRef> comp) {
|
||||
this.bytesComparator = comp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link ValueType} for this field.
|
||||
* Sets the {@link Type} for this field.
|
||||
*/
|
||||
public void setDocValuesType(ValueType type) {
|
||||
public void setDocValuesType(DocValues.Type type) {
|
||||
if (type == null) {
|
||||
throw new IllegalArgumentException("Type must not be null");
|
||||
}
|
||||
|
@ -307,7 +307,7 @@ public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueType docValuesType() {
|
||||
public DocValues.Type docValueType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
|
@ -348,21 +348,21 @@ public class IndexDocValuesField extends Field implements PerDocFieldValues {
|
|||
default:
|
||||
throw new IllegalArgumentException("unknown type: " + type);
|
||||
}
|
||||
return "<" + name() + ": IndexDocValuesField " + value + ">";
|
||||
return "<" + name() + ": DocValuesField " + value + ">";
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an IndexDocValuesField holding the value from
|
||||
* Returns an DocValuesField holding the value from
|
||||
* the provided string field, as the specified type. The
|
||||
* incoming field must have a string value. The name, {@link
|
||||
* FieldType} and string value are carried over from the
|
||||
* incoming Field.
|
||||
*/
|
||||
public static IndexDocValuesField build(Field field, ValueType type) {
|
||||
if (field instanceof IndexDocValuesField) {
|
||||
return (IndexDocValuesField) field;
|
||||
public static DocValuesField build(Field field, DocValues.Type type) {
|
||||
if (field instanceof DocValuesField) {
|
||||
return (DocValuesField) field;
|
||||
}
|
||||
final IndexDocValuesField valField = new IndexDocValuesField(field.name(), field.fieldType(), field.stringValue());
|
||||
final DocValuesField valField = new DocValuesField(field.name(), field.fieldType(), field.stringValue());
|
||||
switch (type) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_STRAIGHT:
|
|
@ -25,10 +25,10 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexableFieldType;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
|
@ -51,7 +51,7 @@ public class Field implements IndexableField {
|
|||
// pre-analyzed tokenStream for indexed fields
|
||||
protected TokenStream tokenStream;
|
||||
// length/offset for all primitive types
|
||||
protected PerDocFieldValues docValues;
|
||||
protected DocValue docValue;
|
||||
|
||||
protected float boost = 1.0f;
|
||||
|
||||
|
@ -292,17 +292,17 @@ public class Field implements IndexableField {
|
|||
return result.toString();
|
||||
}
|
||||
|
||||
public void setDocValues(PerDocFieldValues docValues) {
|
||||
this.docValues = docValues;
|
||||
public void setDocValue(DocValue docValue) {
|
||||
this.docValue = docValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocFieldValues docValues() {
|
||||
public DocValue docValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType docValuesType() {
|
||||
public DocValues.Type docValueType() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.Collection;
|
|||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
@ -156,9 +155,9 @@ abstract class BaseMultiReader<R extends IndexReader> extends IndexReader implem
|
|||
public ReaderContext getTopReaderContext() {
|
||||
return topLevelContext;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiPerDocValues.getPerDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiDocValues#getDocValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level DocValues");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,18 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.document.FieldType; // for javadocs
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
@ -39,9 +51,6 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.index.codecs.Codec;
|
||||
|
||||
import org.apache.lucene.index.codecs.BlockTreeTermsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -640,11 +649,26 @@ public class CheckIndex {
|
|||
if (infoStream != null) {
|
||||
infoStream.print(" test: field norms.........");
|
||||
}
|
||||
FieldInfos infos = reader.fieldInfos();
|
||||
byte[] b;
|
||||
for (final String fieldName : fieldNames) {
|
||||
FieldInfo info = infos.fieldInfo(fieldName);
|
||||
if (reader.hasNorms(fieldName)) {
|
||||
b = reader.norms(fieldName);
|
||||
if (b.length != reader.maxDoc()) {
|
||||
throw new RuntimeException("norms for field: " + fieldName + " are of the wrong size");
|
||||
}
|
||||
if (!info.isIndexed || info.omitNorms) {
|
||||
throw new RuntimeException("field: " + fieldName + " should omit norms but has them!");
|
||||
}
|
||||
++status.totFields;
|
||||
} else {
|
||||
if (reader.norms(fieldName) != null) {
|
||||
throw new RuntimeException("field: " + fieldName + " should omit norms but has them!");
|
||||
}
|
||||
if (info.isIndexed && !info.omitNorms) {
|
||||
throw new RuntimeException("field: " + fieldName + " should have norms but omits them!");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1134,39 +1158,92 @@ public class CheckIndex {
|
|||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
status.totalValueFields++;
|
||||
final PerDocValues perDocValues = reader.perDocValues();
|
||||
final IndexDocValues docValues = perDocValues.docValues(fieldInfo.name);
|
||||
final DocValues docValues = reader.docValues(fieldInfo.name);
|
||||
if (docValues == null) {
|
||||
continue;
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " omits docvalues but should have them!");
|
||||
}
|
||||
DocValues.Type type = docValues.type();
|
||||
if (type != fieldInfo.getDocValuesType()) {
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " has type: " + type + " but fieldInfos says:" + fieldInfo.getDocValuesType());
|
||||
}
|
||||
final Source values = docValues.getDirectSource();
|
||||
final int maxDoc = reader.maxDoc();
|
||||
int size = docValues.getValueSize();
|
||||
for (int i = 0; i < maxDoc; i++) {
|
||||
switch (fieldInfo.docValues) {
|
||||
switch (fieldInfo.getDocValuesType()) {
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
values.getBytes(i, new BytesRef());
|
||||
BytesRef bytes = new BytesRef();
|
||||
values.getBytes(i, bytes);
|
||||
if (size != -1 && size != bytes.length) {
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " returned wrongly sized bytes, was: " + bytes.length + " should be: " + size);
|
||||
}
|
||||
break;
|
||||
case FLOAT_32:
|
||||
assert size == 4;
|
||||
values.getFloat(i);
|
||||
break;
|
||||
case FLOAT_64:
|
||||
assert size == 8;
|
||||
values.getFloat(i);
|
||||
break;
|
||||
case VAR_INTS:
|
||||
assert size == -1;
|
||||
values.getInt(i);
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
assert size == 2;
|
||||
values.getInt(i);
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
assert size == 4;
|
||||
values.getInt(i);
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
assert size == 8;
|
||||
values.getInt(i);
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
assert size == 1;
|
||||
values.getInt(i);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Field: " + fieldInfo.name
|
||||
+ " - no such DocValues type: " + fieldInfo.docValues);
|
||||
+ " - no such DocValues type: " + fieldInfo.getDocValuesType());
|
||||
}
|
||||
}
|
||||
if (type == DocValues.Type.BYTES_FIXED_SORTED || type == DocValues.Type.BYTES_VAR_SORTED) {
|
||||
// check sorted bytes
|
||||
SortedSource sortedValues = values.asSortedSource();
|
||||
Comparator<BytesRef> comparator = sortedValues.getComparator();
|
||||
int lastOrd = -1;
|
||||
BytesRef lastBytes = new BytesRef();
|
||||
for (int i = 0; i < maxDoc; i++) {
|
||||
int ord = sortedValues.ord(i);
|
||||
if (ord < 0 || ord > maxDoc) {
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " ord is out of bounds: " + ord);
|
||||
}
|
||||
BytesRef bytes = new BytesRef();
|
||||
sortedValues.getByOrd(ord, bytes);
|
||||
if (lastOrd != -1) {
|
||||
int ordComp = Integer.signum(new Integer(ord).compareTo(new Integer(lastOrd)));
|
||||
int bytesComp = Integer.signum(comparator.compare(bytes, lastBytes));
|
||||
if (ordComp != bytesComp) {
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " ord comparison is wrong: " + ordComp + " comparator claims: " + bytesComp);
|
||||
}
|
||||
}
|
||||
lastOrd = ord;
|
||||
lastBytes = bytes;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (reader.docValues(fieldInfo.name) != null) {
|
||||
throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -29,8 +29,6 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
|
|
|
@ -26,11 +26,11 @@ import java.util.Map;
|
|||
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
@ -224,7 +224,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
// needs to be more "pluggable" such that if I want
|
||||
// to have a new "thing" my Fields can do, I can
|
||||
// easily add it
|
||||
FieldInfo fi = fieldInfos.addOrUpdate(fieldName, field.fieldType(), false, field.docValuesType());
|
||||
FieldInfo fi = fieldInfos.addOrUpdate(fieldName, field.fieldType());
|
||||
|
||||
fp = new DocFieldProcessorPerField(this, fi);
|
||||
fp.next = fieldHash[hashPos];
|
||||
|
@ -235,7 +235,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
rehash();
|
||||
}
|
||||
} else {
|
||||
fieldInfos.addOrUpdate(fp.fieldInfo.name, field.fieldType(), false, field.docValuesType());
|
||||
fieldInfos.addOrUpdate(fp.fieldInfo.name, field.fieldType());
|
||||
}
|
||||
|
||||
if (thisFieldGen != fp.lastGen) {
|
||||
|
@ -259,9 +259,9 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
if (field.fieldType().stored()) {
|
||||
fieldsWriter.addField(field, fp.fieldInfo);
|
||||
}
|
||||
final PerDocFieldValues docValues = field.docValues();
|
||||
if (docValues != null) {
|
||||
docValuesConsumer(docState, fp.fieldInfo).add(docState.docID, docValues);
|
||||
final DocValue docValue = field.docValue();
|
||||
if (docValue != null) {
|
||||
docValuesConsumer(field.docValueType(), docState, fp.fieldInfo).add(docState.docID, docValue);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -310,12 +310,12 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
final private Map<String, DocValuesConsumerAndDocID> docValues = new HashMap<String, DocValuesConsumerAndDocID>();
|
||||
final private Map<Integer, PerDocConsumer> perDocConsumers = new HashMap<Integer, PerDocConsumer>();
|
||||
|
||||
DocValuesConsumer docValuesConsumer(DocState docState, FieldInfo fieldInfo)
|
||||
DocValuesConsumer docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo)
|
||||
throws IOException {
|
||||
DocValuesConsumerAndDocID docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
|
||||
if (docValuesConsumerAndDocID != null) {
|
||||
if (docState.docID == docValuesConsumerAndDocID.docID) {
|
||||
throw new IllegalArgumentException("IndexDocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
|
||||
throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
|
||||
}
|
||||
assert docValuesConsumerAndDocID.docID < docState.docID;
|
||||
docValuesConsumerAndDocID.docID = docState.docID;
|
||||
|
@ -329,17 +329,9 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
perDocConsumer = dvFormat.docsConsumer(perDocWriteState);
|
||||
perDocConsumers.put(0, perDocConsumer);
|
||||
}
|
||||
boolean success = false;
|
||||
DocValuesConsumer docValuesConsumer = null;
|
||||
try {
|
||||
docValuesConsumer = perDocConsumer.addValuesField(fieldInfo);
|
||||
fieldInfo.commitDocValues();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
fieldInfo.revertUncommitted();
|
||||
}
|
||||
}
|
||||
|
||||
DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
|
||||
fieldInfo.setDocValuesType(valueType);
|
||||
|
||||
docValuesConsumerAndDocID = new DocValuesConsumerAndDocID(docValuesConsumer);
|
||||
docValuesConsumerAndDocID.docID = docState.docID;
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -18,17 +18,19 @@ package org.apache.lucene.index.values;
|
|||
*/
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer; // javadocs
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Per document and field values consumed by {@link DocValuesConsumer}.
|
||||
* @see IndexDocValuesField
|
||||
* @see DocValuesField
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public interface PerDocFieldValues {
|
||||
public interface DocValue {
|
||||
|
||||
/**
|
||||
* Sets the given <code>long</code> value.
|
||||
|
@ -46,19 +48,19 @@ public interface PerDocFieldValues {
|
|||
public void setFloat(double value);
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value and the field's {@link ValueType}. The
|
||||
* Sets the given {@link BytesRef} value and the field's {@link Type}. The
|
||||
* comparator for this field is set to <code>null</code>. If a
|
||||
* <code>null</code> comparator is set the default comparator for the given
|
||||
* {@link ValueType} is used.
|
||||
* {@link Type} is used.
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type);
|
||||
public void setBytes(BytesRef value, DocValues.Type type);
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value, the field's {@link ValueType} and the
|
||||
* Sets the given {@link BytesRef} value, the field's {@link Type} and the
|
||||
* field's comparator. If the {@link Comparator} is set to <code>null</code>
|
||||
* the default for the given {@link ValueType} is used instead.
|
||||
* the default for the given {@link Type} is used instead.
|
||||
*/
|
||||
public void setBytes(BytesRef value, ValueType type, Comparator<BytesRef> comp);
|
||||
public void setBytes(BytesRef value, DocValues.Type type, Comparator<BytesRef> comp);
|
||||
|
||||
/**
|
||||
* Returns the set {@link BytesRef} or <code>null</code> if not set.
|
||||
|
@ -82,17 +84,17 @@ public interface PerDocFieldValues {
|
|||
|
||||
/**
|
||||
* Sets the {@link BytesRef} comparator for this field. If the field has a
|
||||
* numeric {@link ValueType} the comparator will be ignored.
|
||||
* numeric {@link Type} the comparator will be ignored.
|
||||
*/
|
||||
public void setBytesComparator(Comparator<BytesRef> comp);
|
||||
|
||||
/**
|
||||
* Sets the {@link ValueType}
|
||||
* Sets the {@link Type}
|
||||
*/
|
||||
public void setDocValuesType(ValueType type);
|
||||
public void setDocValuesType(DocValues.Type type);
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType}
|
||||
* Returns the {@link Type}
|
||||
*/
|
||||
public ValueType docValuesType();
|
||||
public DocValues.Type docValueType();
|
||||
}
|
|
@ -0,0 +1,687 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* {@link DocValues} provides a dense per-document typed storage for fast
|
||||
* value access based on the lucene internal document id. {@link DocValues}
|
||||
* exposes two distinct APIs:
|
||||
* <ul>
|
||||
* <li>via {@link #getSource()} providing RAM resident random access</li>
|
||||
* <li>via {@link #getDirectSource()} providing on disk random access</li>
|
||||
* </ul> {@link DocValues} are exposed via
|
||||
* {@link IndexReader#perDocValues()} on a per-segment basis. For best
|
||||
* performance {@link DocValues} should be consumed per-segment just like
|
||||
* IndexReader.
|
||||
* <p>
|
||||
* {@link DocValues} are fully integrated into the {@link DocValuesFormat} API.
|
||||
*
|
||||
* @see Type for limitations and default implementation documentation
|
||||
* @see DocValuesField for adding values to the index
|
||||
* @see DocValuesFormat#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
|
||||
* customization
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class DocValues implements Closeable {
|
||||
|
||||
public static final DocValues[] EMPTY_ARRAY = new DocValues[0];
|
||||
|
||||
private volatile SourceCache cache = new SourceCache.DirectSourceCache();
|
||||
private final Object cacheLock = new Object();
|
||||
|
||||
/**
|
||||
* Loads a new {@link Source} instance for this {@link DocValues} field
|
||||
* instance. Source instances returned from this method are not cached. It is
|
||||
* the callers responsibility to maintain the instance and release its
|
||||
* resources once the source is not needed anymore.
|
||||
* <p>
|
||||
* For managed {@link Source} instances see {@link #getSource()}.
|
||||
*
|
||||
* @see #getSource()
|
||||
* @see #setCache(SourceCache)
|
||||
*/
|
||||
public abstract Source load() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link Source} instance through the current {@link SourceCache}.
|
||||
* Iff no {@link Source} has been loaded into the cache so far the source will
|
||||
* be loaded through {@link #load()} and passed to the {@link SourceCache}.
|
||||
* The caller of this method should not close the obtained {@link Source}
|
||||
* instance unless it is not needed for the rest of its life time.
|
||||
* <p>
|
||||
* {@link Source} instances obtained from this method are closed / released
|
||||
* from the cache once this {@link DocValues} instance is closed by the
|
||||
* {@link IndexReader}, {@link Fields} or {@link FieldsEnum} the
|
||||
* {@link DocValues} was created from.
|
||||
*/
|
||||
public Source getSource() throws IOException {
|
||||
return cache.load(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a disk resident {@link Source} instance. Direct Sources are not
|
||||
* cached in the {@link SourceCache} and should not be shared between threads.
|
||||
*/
|
||||
public abstract Source getDirectSource() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns the {@link Type} of this {@link DocValues} instance
|
||||
*/
|
||||
public abstract Type type();
|
||||
|
||||
/**
|
||||
* Closes this {@link DocValues} instance. This method should only be called
|
||||
* by the creator of this {@link DocValues} instance. API users should not
|
||||
* close {@link DocValues} instances.
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
cache.close(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the size per value in bytes or <code>-1</code> iff size per value
|
||||
* is variable.
|
||||
*
|
||||
* @return the size per value in bytes or <code>-1</code> iff size per value
|
||||
* is variable.
|
||||
*/
|
||||
public int getValueSize() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link SourceCache} used by this {@link DocValues} instance. This
|
||||
* method should be called before {@link #load()} is called. All {@link Source} instances in the currently used cache will be closed
|
||||
* before the new cache is installed.
|
||||
* <p>
|
||||
* Note: All instances previously obtained from {@link #load()} will be lost.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* if the given cache is <code>null</code>
|
||||
*
|
||||
*/
|
||||
public void setCache(SourceCache cache) {
|
||||
if (cache == null)
|
||||
throw new IllegalArgumentException("cache must not be null");
|
||||
synchronized (cacheLock) {
|
||||
SourceCache toClose = this.cache;
|
||||
this.cache = cache;
|
||||
toClose.close(this);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Source of per document values like long, double or {@link BytesRef}
|
||||
* depending on the {@link DocValues} fields {@link Type}. Source
|
||||
* implementations provide random access semantics similar to array lookups
|
||||
* <p>
|
||||
* @see DocValues#getSource()
|
||||
* @see DocValues#getDirectSource()
|
||||
*/
|
||||
public static abstract class Source {
|
||||
|
||||
protected final Type type;
|
||||
|
||||
protected Source(Type type) {
|
||||
this.type = type;
|
||||
}
|
||||
/**
|
||||
* Returns a <tt>long</tt> for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>long</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>long</tt> values.
|
||||
*/
|
||||
public long getInt(int docID) {
|
||||
throw new UnsupportedOperationException("ints are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a <tt>double</tt> for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>double</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>double</tt> values.
|
||||
*/
|
||||
public double getFloat(int docID) {
|
||||
throw new UnsupportedOperationException("floats are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link BytesRef} for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>byte[]</tt> values.
|
||||
* @throws IOException
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>byte[]</tt> values.
|
||||
*/
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
throw new UnsupportedOperationException("bytes are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link Type} of this source.
|
||||
*
|
||||
* @return the {@link Type} of this source.
|
||||
*/
|
||||
public Type type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff this {@link Source} exposes an array via
|
||||
* {@link #getArray()} otherwise <code>false</code>.
|
||||
*
|
||||
* @return <code>true</code> iff this {@link Source} exposes an array via
|
||||
* {@link #getArray()} otherwise <code>false</code>.
|
||||
*/
|
||||
public boolean hasArray() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the internal array representation iff this {@link Source} uses an
|
||||
* array as its inner representation, otherwise <code>UOE</code>.
|
||||
*/
|
||||
public Object getArray() {
|
||||
throw new UnsupportedOperationException("getArray is not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* If this {@link Source} is sorted this method will return an instance of
|
||||
* {@link SortedSource} otherwise <code>UOE</code>
|
||||
*/
|
||||
public SortedSource asSortedSource() {
|
||||
throw new UnsupportedOperationException("asSortedSource is not supported");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A sorted variant of {@link Source} for <tt>byte[]</tt> values per document.
|
||||
* <p>
|
||||
*/
|
||||
public static abstract class SortedSource extends Source {
|
||||
|
||||
private final Comparator<BytesRef> comparator;
|
||||
|
||||
protected SortedSource(Type type, Comparator<BytesRef> comparator) {
|
||||
super(type);
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final int ord = ord(docID);
|
||||
if (ord < 0) {
|
||||
bytesRef.length = 0;
|
||||
} else {
|
||||
getByOrd(ord , bytesRef);
|
||||
}
|
||||
return bytesRef;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns ord for specified docID. Ord is dense, ie, starts at 0, then increments by 1
|
||||
* for the next (as defined by {@link Comparator} value.
|
||||
*/
|
||||
public abstract int ord(int docID);
|
||||
|
||||
/** Returns value for specified ord. */
|
||||
public abstract BytesRef getByOrd(int ord, BytesRef bytesRef);
|
||||
|
||||
/** Return true if it's safe to call {@link
|
||||
* #getDocToOrd}. */
|
||||
public boolean hasPackedDocToOrd() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the PackedInts.Reader impl that maps document to ord.
|
||||
*/
|
||||
public abstract PackedInts.Reader getDocToOrd();
|
||||
|
||||
/**
|
||||
* Returns the comparator used to order the BytesRefs.
|
||||
*/
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs a lookup by value.
|
||||
*
|
||||
* @param value
|
||||
* the value to look up
|
||||
* @param spare
|
||||
* a spare {@link BytesRef} instance used to compare internal
|
||||
* values to the given value. Must not be <code>null</code>
|
||||
* @return the given values ordinal if found or otherwise
|
||||
* <code>(-(ord)-1)</code>, defined as the ordinal of the first
|
||||
* element that is greater than the given value. This guarantees
|
||||
* that the return value will always be >= 0 if the given value
|
||||
* is found.
|
||||
*/
|
||||
public int getByValue(BytesRef value, BytesRef spare) {
|
||||
return binarySearch(value, spare, 0, getValueCount() - 1);
|
||||
}
|
||||
|
||||
private int binarySearch(BytesRef b, BytesRef bytesRef, int low,
|
||||
int high) {
|
||||
int mid = 0;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
getByOrd(mid, bytesRef);
|
||||
final int cmp = comparator.compare(bytesRef, b);
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
return mid;
|
||||
}
|
||||
}
|
||||
assert comparator.compare(bytesRef, b) != 0;
|
||||
return -(low + 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSource asSortedSource() {
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of unique values in this sorted source
|
||||
*/
|
||||
public abstract int getValueCount();
|
||||
}
|
||||
|
||||
/** Returns a Source that always returns default (missing)
|
||||
* values for all documents. */
|
||||
public static Source getDefaultSource(final Type type) {
|
||||
return new Source(type) {
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
return 0.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** Returns a SortedSource that always returns default (missing)
|
||||
* values for all documents. */
|
||||
public static SortedSource getDefaultSortedSource(final Type type, final int size) {
|
||||
|
||||
final PackedInts.Reader docToOrd = new PackedInts.Reader() {
|
||||
@Override
|
||||
public long get(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBitsPerValue() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getArray() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
return new SortedSource(type, BytesRef.getUTF8SortedAsUnicodeComparator()) {
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int ord(int docID) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
|
||||
assert ord == 0;
|
||||
bytesRef.length = 0;
|
||||
return bytesRef;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPackedDocToOrd() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PackedInts.Reader getDocToOrd() {
|
||||
return docToOrd;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getByValue(BytesRef value, BytesRef spare) {
|
||||
if (value.length == 0) {
|
||||
return 0;
|
||||
} else {
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return 1;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* <code>Type</code> specifies the {@link DocValues} type for a
|
||||
* certain field. A <code>Type</code> only defines the data type for a field
|
||||
* while the actual implementation used to encode and decode the values depends
|
||||
* on the the {@link DocValuesFormat#docsConsumer} and {@link DocValuesFormat#docsProducer} methods.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static enum Type {
|
||||
|
||||
/**
|
||||
* A variable bit signed integer value. By default this type uses
|
||||
* {@link PackedInts} to compress the values, as an offset
|
||||
* from the minimum value, as long as the value range
|
||||
* fits into 2<sup>63</sup>-1. Otherwise,
|
||||
* the default implementation falls back to fixed size 64bit
|
||||
* integers ({@link #FIXED_INTS_64}).
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
VAR_INTS,
|
||||
|
||||
/**
|
||||
* A 8 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>byte</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_8,
|
||||
|
||||
/**
|
||||
* A 16 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>short</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_16,
|
||||
|
||||
/**
|
||||
* A 32 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>int</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_32,
|
||||
|
||||
/**
|
||||
* A 64 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>long</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_64,
|
||||
/**
|
||||
* A 32 bit floating point value. By default there is no compression
|
||||
* applied. To fit custom float values into less than 32bit either a custom
|
||||
* implementation is needed or values must be encoded into a
|
||||
* {@link #BYTES_FIXED_STRAIGHT} type. {@link Source} instances of
|
||||
* this type return a <tt>float</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0.0f</tt> as the default value without any
|
||||
* distinction between provided <tt>0.0f</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0.0f</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FLOAT_32,
|
||||
/**
|
||||
*
|
||||
* A 64 bit floating point value. By default there is no compression
|
||||
* applied. To fit custom float values into less than 64bit either a custom
|
||||
* implementation is needed or values must be encoded into a
|
||||
* {@link #BYTES_FIXED_STRAIGHT} type. {@link Source} instances of
|
||||
* this type return a <tt>double</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0.0d</tt> as the default value without any
|
||||
* distinction between provided <tt>0.0d</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0.0d</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FLOAT_64,
|
||||
|
||||
// TODO(simonw): -- shouldn't lucene decide/detect straight vs
|
||||
// deref, as well fixed vs var?
|
||||
/**
|
||||
* A fixed length straight byte[]. All values added to
|
||||
* such a field must be of the same length. All bytes are stored sequentially
|
||||
* for fast offset access.
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0 byte</tt> filled byte[] based on the length of the first seen
|
||||
* value as the default value without any distinction between explicitly
|
||||
* provided values during indexing. All documents without an explicit value
|
||||
* will use the default instead.Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_FIXED_STRAIGHT,
|
||||
|
||||
/**
|
||||
* A fixed length dereferenced byte[] variant. Fields with
|
||||
* this type only store distinct byte values and store an additional offset
|
||||
* pointer per document to dereference the shared byte[].
|
||||
* Use this type if your documents may share the same byte[].
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_FIXED_DEREF,
|
||||
|
||||
/**
|
||||
* Variable length straight stored byte[] variant. All bytes are
|
||||
* stored sequentially for compactness. Usage of this type via the
|
||||
* disk-resident API might yield performance degradation since no additional
|
||||
* index is used to advance by more than one document value at a time.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without an
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* byte[] reference. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_VAR_STRAIGHT,
|
||||
|
||||
/**
|
||||
* A variable length dereferenced byte[]. Just like
|
||||
* {@link #BYTES_FIXED_DEREF}, but allowing each
|
||||
* document's value to be a different length.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_VAR_DEREF,
|
||||
|
||||
|
||||
/**
|
||||
* A variable length pre-sorted byte[] variant. Just like
|
||||
* {@link #BYTES_FIXED_SORTED}, but allowing each
|
||||
* document's value to be a different length.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference.Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*
|
||||
* @see SortedSource
|
||||
*/
|
||||
BYTES_VAR_SORTED,
|
||||
|
||||
/**
|
||||
* A fixed length pre-sorted byte[] variant. Fields with this type only
|
||||
* store distinct byte values and store an additional offset pointer per
|
||||
* document to dereference the shared byte[]. The stored
|
||||
* byte[] is presorted, by default by unsigned byte order,
|
||||
* and allows access via document id, ordinal and by-value.
|
||||
* Use this type if your documents may share the same byte[].
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. Custom default values must be assigned
|
||||
* explicitly.
|
||||
* </p>
|
||||
*
|
||||
* @see SortedSource
|
||||
*/
|
||||
BYTES_FIXED_SORTED
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Abstract base class for {@link DocValues} {@link Source} cache.
|
||||
* <p>
|
||||
* {@link Source} instances loaded via {@link DocValues#load()} are entirely memory resident
|
||||
* and need to be maintained by the caller. Each call to
|
||||
* {@link DocValues#load()} will cause an entire reload of
|
||||
* the underlying data. Source instances obtained from
|
||||
* {@link DocValues#getSource()} and {@link DocValues#getSource()}
|
||||
* respectively are maintained by a {@link SourceCache} that is closed (
|
||||
* {@link #close(DocValues)}) once the {@link IndexReader} that created the
|
||||
* {@link DocValues} instance is closed.
|
||||
* <p>
|
||||
* Unless {@link Source} instances are managed by another entity it is
|
||||
* recommended to use the cached variants to obtain a source instance.
|
||||
* <p>
|
||||
* Implementation of this API must be thread-safe.
|
||||
*
|
||||
* @see DocValues#setCache(SourceCache)
|
||||
* @see DocValues#getSource()
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static abstract class SourceCache {
|
||||
|
||||
/**
|
||||
* Atomically loads a {@link Source} into the cache from the given
|
||||
* {@link DocValues} and returns it iff no other {@link Source} has already
|
||||
* been cached. Otherwise the cached source is returned.
|
||||
* <p>
|
||||
* This method will not return <code>null</code>
|
||||
*/
|
||||
public abstract Source load(DocValues values) throws IOException;
|
||||
|
||||
/**
|
||||
* Atomically invalidates the cached {@link Source}
|
||||
* instances if any and empties the cache.
|
||||
*/
|
||||
public abstract void invalidate(DocValues values);
|
||||
|
||||
/**
|
||||
* Atomically closes the cache and frees all resources.
|
||||
*/
|
||||
public synchronized void close(DocValues values) {
|
||||
invalidate(values);
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple per {@link DocValues} instance cache implementation that holds a
|
||||
* {@link Source} a member variable.
|
||||
* <p>
|
||||
* If a {@link DirectSourceCache} instance is closed or invalidated the cached
|
||||
* reference are simply set to <code>null</code>
|
||||
*/
|
||||
public static final class DirectSourceCache extends SourceCache {
|
||||
private Source ref;
|
||||
|
||||
public synchronized Source load(DocValues values) throws IOException {
|
||||
if (ref == null) {
|
||||
ref = values.load();
|
||||
}
|
||||
return ref;
|
||||
}
|
||||
|
||||
public synchronized void invalidate(DocValues values) {
|
||||
ref = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -248,7 +248,6 @@ public class DocumentsWriterPerThread {
|
|||
// mark document as deleted
|
||||
deleteDocID(docState.docID);
|
||||
numDocsInRAM++;
|
||||
fieldInfos.revertUncommitted();
|
||||
} else {
|
||||
abort();
|
||||
}
|
||||
|
@ -312,7 +311,6 @@ public class DocumentsWriterPerThread {
|
|||
// Incr here because finishDocument will not
|
||||
// be called (because an exc is being thrown):
|
||||
numDocsInRAM++;
|
||||
fieldInfos.revertUncommitted();
|
||||
} else {
|
||||
abort();
|
||||
}
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -25,7 +25,7 @@ public final class FieldInfo {
|
|||
public final int number;
|
||||
|
||||
public boolean isIndexed;
|
||||
ValueType docValues;
|
||||
private DocValues.Type docValues;
|
||||
|
||||
|
||||
// true if term vector for this field should be stored
|
||||
|
@ -57,7 +57,7 @@ public final class FieldInfo {
|
|||
*/
|
||||
public FieldInfo(String name, boolean isIndexed, int number, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
|
||||
this.name = name;
|
||||
this.isIndexed = isIndexed;
|
||||
this.number = number;
|
||||
|
@ -118,13 +118,14 @@ public final class FieldInfo {
|
|||
}
|
||||
assert this.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !this.storePayloads;
|
||||
}
|
||||
void setDocValues(ValueType v) {
|
||||
|
||||
void setDocValuesType(DocValues.Type v) {
|
||||
if (docValues == null) {
|
||||
docValues = v;
|
||||
}
|
||||
}
|
||||
|
||||
public void resetDocValues(ValueType v) {
|
||||
public void resetDocValuesType(DocValues.Type v) {
|
||||
if (docValues != null) {
|
||||
docValues = v;
|
||||
}
|
||||
|
@ -134,42 +135,13 @@ public final class FieldInfo {
|
|||
return docValues != null;
|
||||
}
|
||||
|
||||
public ValueType getDocValues() {
|
||||
public DocValues.Type getDocValuesType() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
private boolean vectorsCommitted;
|
||||
private boolean docValuesCommitted;
|
||||
|
||||
/**
|
||||
* Reverts all uncommitted changes on this {@link FieldInfo}
|
||||
* @see #commitVectors()
|
||||
*/
|
||||
void revertUncommitted() {
|
||||
if (storeTermVector && !vectorsCommitted) {
|
||||
storeOffsetWithTermVector = false;
|
||||
storePositionWithTermVector = false;
|
||||
storeTermVector = false;
|
||||
}
|
||||
|
||||
if (docValues != null && !docValuesCommitted) {
|
||||
docValues = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Commits term vector modifications. Changes to term-vectors must be
|
||||
* explicitly committed once the necessary files are created. If those changes
|
||||
* are not committed subsequent {@link #revertUncommitted()} will reset the
|
||||
* all term-vector flags before the next document.
|
||||
*/
|
||||
void commitVectors() {
|
||||
assert storeTermVector;
|
||||
vectorsCommitted = true;
|
||||
}
|
||||
|
||||
void commitDocValues() {
|
||||
assert hasDocValues();
|
||||
docValuesCommitted = true;
|
||||
public void setStoreTermVectors(boolean withPositions, boolean withOffsets) {
|
||||
storeTermVector = true;
|
||||
storePositionWithTermVector |= withPositions;
|
||||
storeOffsetWithTermVector |= withOffsets;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.SortedMap;
|
|||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
||||
/** Access to the Field Info file that describes document fields and whether or
|
||||
* not they are indexed. Each segment has a separate Field Info file. Objects
|
||||
|
@ -299,20 +299,30 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
*/
|
||||
synchronized public FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
|
||||
return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
}
|
||||
|
||||
synchronized public FieldInfo addOrUpdate(String name, IndexableFieldType fieldType, boolean scorePayloads, ValueType docValues) {
|
||||
return addOrUpdateInternal(name, -1, fieldType.indexed(), fieldType.storeTermVectors(),
|
||||
fieldType.storeTermVectorPositions(), fieldType.storeTermVectorOffsets(), fieldType.omitNorms(), scorePayloads,
|
||||
fieldType.indexOptions(), docValues);
|
||||
// NOTE: this method does not carry over termVector
|
||||
// booleans nor docValuesType; the indexer chain
|
||||
// (TermVectorsConsumerPerField, DocFieldProcessor) must
|
||||
// set these fields when they succeed in consuming
|
||||
// the document:
|
||||
public FieldInfo addOrUpdate(String name, IndexableFieldType fieldType) {
|
||||
// TODO: really, indexer shouldn't even call this
|
||||
// method (it's only called from DocFieldProcessor);
|
||||
// rather, each component in the chain should update
|
||||
// what it "owns". EG fieldType.indexOptions() should
|
||||
// be updated by maybe FreqProxTermsWriterPerField:
|
||||
return addOrUpdateInternal(name, -1, fieldType.indexed(), false, false, false,
|
||||
fieldType.omitNorms(), false,
|
||||
fieldType.indexOptions(), null);
|
||||
}
|
||||
|
||||
synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
|
||||
if (globalFieldNumbers == null) {
|
||||
throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos");
|
||||
}
|
||||
|
@ -322,7 +332,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
} else {
|
||||
fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions);
|
||||
fi.setDocValues(docValues);
|
||||
fi.setDocValuesType(docValues);
|
||||
}
|
||||
version++;
|
||||
return fi;
|
||||
|
@ -333,7 +343,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
return addOrUpdateInternal(fi.name, fi.number, fi.isIndexed, fi.storeTermVector,
|
||||
fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
|
||||
fi.omitNorms, fi.storePayloads,
|
||||
fi.indexOptions, fi.docValues);
|
||||
fi.indexOptions, fi.getDocValuesType());
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -341,7 +351,7 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
*/
|
||||
private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean storePositionWithTermVector,
|
||||
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValuesType) {
|
||||
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValuesType) {
|
||||
// don't check modifiable here since we use that to initially build up FIs
|
||||
if (globalFieldNumbers != null) {
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name);
|
||||
|
@ -429,16 +439,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
return version;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reverts all uncommitted changes
|
||||
* @see FieldInfo#revertUncommitted()
|
||||
*/
|
||||
void revertUncommitted() {
|
||||
for (FieldInfo fieldInfo : this) {
|
||||
fieldInfo.revertUncommitted();
|
||||
}
|
||||
}
|
||||
|
||||
final FieldInfos asReadOnly() {
|
||||
if (isReadOnly()) {
|
||||
return this;
|
||||
|
|
|
@ -17,7 +17,6 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -422,9 +421,9 @@ public class FilterIndexReader extends IndexReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return in.perDocValues();
|
||||
return in.docValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -28,8 +28,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.search.FieldCache; // javadocs
|
||||
import org.apache.lucene.search.SearcherManager; // javadocs
|
||||
import org.apache.lucene.store.*;
|
||||
|
@ -772,21 +770,6 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
* through them yourself. */
|
||||
public abstract Fields fields() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns {@link PerDocValues} for this reader.
|
||||
* This method may return null if the reader has no per-document
|
||||
* values stored.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this is a multi reader ({@link
|
||||
* #getSequentialSubReaders} is not null) then this
|
||||
* method will throw UnsupportedOperationException. If
|
||||
* you really need {@link PerDocValues} for such a reader,
|
||||
* use {@link MultiPerDocValues#getPerDocs(IndexReader)}. However, for
|
||||
* performance reasons, it's best to get all sub-readers
|
||||
* using {@link ReaderUtil#gatherSubReaders} and iterate
|
||||
* through them yourself. */
|
||||
public abstract PerDocValues perDocValues() throws IOException;
|
||||
|
||||
public final int docFreq(Term term) throws IOException {
|
||||
return docFreq(term.field(), term.bytes());
|
||||
}
|
||||
|
@ -1161,14 +1144,20 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
throw new UnsupportedOperationException("This reader does not support this method.");
|
||||
}
|
||||
|
||||
public final IndexDocValues docValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
final PerDocValues perDoc = perDocValues();
|
||||
if (perDoc == null) {
|
||||
return null;
|
||||
}
|
||||
return perDoc.docValues(field);
|
||||
}
|
||||
/**
|
||||
* Returns {@link DocValues} for this field.
|
||||
* This method may return null if the reader has no per-document
|
||||
* values stored.
|
||||
*
|
||||
* <p><b>NOTE</b>: if this is a multi reader ({@link
|
||||
* #getSequentialSubReaders} is not null) then this
|
||||
* method will throw UnsupportedOperationException. If
|
||||
* you really need {@link DocValues} for such a reader,
|
||||
* use {@link MultiDocValues#getDocValues(IndexReader,String)}. However, for
|
||||
* performance reasons, it's best to get all sub-readers
|
||||
* using {@link ReaderUtil#gatherSubReaders} and iterate
|
||||
* through them yourself. */
|
||||
public abstract DocValues docValues(String field) throws IOException;
|
||||
|
||||
private volatile Fields fields;
|
||||
|
||||
|
@ -1184,21 +1173,6 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
return fields;
|
||||
}
|
||||
|
||||
private volatile PerDocValues perDocValues;
|
||||
|
||||
/** @lucene.internal */
|
||||
void storePerDoc(PerDocValues perDocValues) {
|
||||
ensureOpen();
|
||||
this.perDocValues = perDocValues;
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
PerDocValues retrievePerDoc() {
|
||||
ensureOpen();
|
||||
return perDocValues;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A struct like class that represents a hierarchical relationship between
|
||||
* {@link IndexReader} instances.
|
||||
|
|
|
@ -23,8 +23,7 @@ import java.io.Reader;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.document.NumericField;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
// TODO: how to handle versioning here...?
|
||||
|
@ -77,10 +76,10 @@ public interface IndexableField {
|
|||
public IndexableFieldType fieldType();
|
||||
|
||||
/** Non-null if doc values should be indexed */
|
||||
public PerDocFieldValues docValues();
|
||||
public DocValue docValue();
|
||||
|
||||
/** DocValues type; only used if docValues is non-null */
|
||||
public ValueType docValuesType();
|
||||
/** DocValues type; only used if docValue is non-null */
|
||||
public DocValues.Type docValueType();
|
||||
|
||||
/**
|
||||
* Creates the TokenStream used for indexing this field. If appropriate,
|
||||
|
|
|
@ -0,0 +1,247 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.ReaderUtil.Gather;
|
||||
|
||||
/**
|
||||
* A wrapper for compound IndexReader providing access to per segment
|
||||
* {@link DocValues}
|
||||
*
|
||||
* @lucene.experimental
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class MultiDocValues extends DocValues {
|
||||
|
||||
public static class DocValuesSlice {
|
||||
public final static DocValuesSlice[] EMPTY_ARRAY = new DocValuesSlice[0];
|
||||
final int start;
|
||||
final int length;
|
||||
DocValues docValues;
|
||||
|
||||
public DocValuesSlice(DocValues docValues, int start, int length) {
|
||||
this.docValues = docValues;
|
||||
this.start = start;
|
||||
this.length = length;
|
||||
}
|
||||
}
|
||||
|
||||
private DocValuesSlice[] slices;
|
||||
private int[] starts;
|
||||
private Type type;
|
||||
private int valueSize;
|
||||
|
||||
private MultiDocValues(DocValuesSlice[] slices, int[] starts, TypePromoter promotedType) {
|
||||
this.starts = starts;
|
||||
this.slices = slices;
|
||||
this.type = promotedType.type();
|
||||
this.valueSize = promotedType.getValueSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a single {@link DocValues} instance for this field, merging
|
||||
* their values on the fly.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: this is a slow way to access DocValues. It's better to get the
|
||||
* sub-readers (using {@link Gather}) and iterate through them yourself.
|
||||
*/
|
||||
public static DocValues getDocValues(IndexReader r, final String field) throws IOException {
|
||||
final IndexReader[] subs = r.getSequentialSubReaders();
|
||||
if (subs == null) {
|
||||
// already an atomic reader
|
||||
return r.docValues(field);
|
||||
} else if (subs.length == 0) {
|
||||
// no fields
|
||||
return null;
|
||||
} else if (subs.length == 1) {
|
||||
return getDocValues(subs[0], field);
|
||||
} else {
|
||||
final List<DocValuesSlice> slices = new ArrayList<DocValuesSlice>();
|
||||
|
||||
final TypePromoter promotedType[] = new TypePromoter[1];
|
||||
promotedType[0] = TypePromoter.getIdentityPromoter();
|
||||
|
||||
// gather all docvalues fields, accumulating a promoted type across
|
||||
// potentially incompatible types
|
||||
|
||||
new ReaderUtil.Gather(r) {
|
||||
@Override
|
||||
protected void add(int base, IndexReader r) throws IOException {
|
||||
final DocValues d = r.docValues(field);
|
||||
if (d != null) {
|
||||
TypePromoter incoming = TypePromoter.create(d.type(), d.getValueSize());
|
||||
promotedType[0] = promotedType[0].promote(incoming);
|
||||
}
|
||||
slices.add(new DocValuesSlice(d, base, r.maxDoc()));
|
||||
}
|
||||
}.run();
|
||||
|
||||
// return null if no docvalues encountered anywhere
|
||||
if (promotedType[0] == TypePromoter.getIdentityPromoter()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// populate starts and fill gaps with empty docvalues
|
||||
int starts[] = new int[slices.size()];
|
||||
for (int i = 0; i < slices.size(); i++) {
|
||||
DocValuesSlice slice = slices.get(i);
|
||||
starts[i] = slice.start;
|
||||
if (slice.docValues == null) {
|
||||
slice.docValues = new EmptyDocValues(slice.length, promotedType[0].type());
|
||||
}
|
||||
}
|
||||
|
||||
return new MultiDocValues(slices.toArray(new DocValuesSlice[slices.size()]), starts, promotedType[0]);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return new MultiSource(slices, starts, false, type);
|
||||
}
|
||||
|
||||
public static class EmptyDocValues extends DocValues {
|
||||
final int maxDoc;
|
||||
final Source emptySource;
|
||||
|
||||
public EmptyDocValues(int maxDoc, Type type) {
|
||||
this.maxDoc = maxDoc;
|
||||
this.emptySource = new EmptySource(type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return emptySource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return emptySource.type();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return emptySource;
|
||||
}
|
||||
}
|
||||
|
||||
private static class MultiSource extends Source {
|
||||
private int numDocs = 0;
|
||||
private int start = 0;
|
||||
private Source current;
|
||||
private final int[] starts;
|
||||
private final DocValuesSlice[] slices;
|
||||
private boolean direct;
|
||||
|
||||
public MultiSource(DocValuesSlice[] slices, int[] starts, boolean direct, Type type) {
|
||||
super(type);
|
||||
this.slices = slices;
|
||||
this.starts = starts;
|
||||
assert slices.length != 0;
|
||||
this.direct = direct;
|
||||
}
|
||||
|
||||
public long getInt(int docID) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getInt(doc);
|
||||
}
|
||||
|
||||
private final int ensureSource(int docID) {
|
||||
if (docID >= start && docID < start+numDocs) {
|
||||
return docID - start;
|
||||
} else {
|
||||
final int idx = ReaderUtil.subIndex(docID, starts);
|
||||
assert idx >= 0 && idx < slices.length : "idx was " + idx
|
||||
+ " for doc id: " + docID + " slices : " + Arrays.toString(starts);
|
||||
assert slices[idx] != null;
|
||||
try {
|
||||
if (direct) {
|
||||
current = slices[idx].docValues.getDirectSource();
|
||||
} else {
|
||||
current = slices[idx].docValues.getSource();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("load failed", e); // TODO how should we
|
||||
// handle this
|
||||
}
|
||||
|
||||
start = slices[idx].start;
|
||||
numDocs = slices[idx].length;
|
||||
return docID - start;
|
||||
}
|
||||
}
|
||||
|
||||
public double getFloat(int docID) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getFloat(doc);
|
||||
}
|
||||
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getBytes(doc, bytesRef);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: this is dup of DocValues.getDefaultSource()?
|
||||
private static class EmptySource extends Source {
|
||||
|
||||
public EmptySource(Type type) {
|
||||
super(type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
return 0d;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueSize() {
|
||||
return valueSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return new MultiSource(slices, starts, true, type);
|
||||
}
|
||||
}
|
|
@ -1,162 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.MultiIndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.values.MultiIndexDocValues.DocValuesIndex;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.ReaderUtil.Gather;
|
||||
|
||||
/**
|
||||
* Exposes per-document values, merged from per-document values API of
|
||||
* sub-segments. This is useful when you're interacting with an {@link IndexReader}
|
||||
* implementation that consists of sequential sub-readers (eg DirectoryReader
|
||||
* or {@link MultiReader}).
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: for multi readers, you'll get better performance by gathering
|
||||
* the sub readers using {@link ReaderUtil#gatherSubReaders} and then operate
|
||||
* per-reader, instead of using this class.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class MultiPerDocValues extends PerDocValues {
|
||||
private final PerDocValues[] subs;
|
||||
private final ReaderUtil.Slice[] subSlices;
|
||||
private final Map<String, IndexDocValues> docValues = new ConcurrentHashMap<String, IndexDocValues>();
|
||||
private final TreeSet<String> fields;
|
||||
|
||||
public MultiPerDocValues(PerDocValues[] subs, ReaderUtil.Slice[] subSlices) {
|
||||
this.subs = subs;
|
||||
this.subSlices = subSlices;
|
||||
fields = new TreeSet<String>();
|
||||
for (PerDocValues sub : subs) {
|
||||
fields.addAll(sub.fields());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a single {@link PerDocValues} instance for this reader, merging
|
||||
* their values on the fly. This method will not return <code>null</code>.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: this is a slow way to access postings. It's better to get the
|
||||
* sub-readers (using {@link Gather}) and iterate through them yourself.
|
||||
*/
|
||||
public static PerDocValues getPerDocs(IndexReader r) throws IOException {
|
||||
final IndexReader[] subs = r.getSequentialSubReaders();
|
||||
if (subs == null) {
|
||||
// already an atomic reader
|
||||
return r.perDocValues();
|
||||
} else if (subs.length == 0) {
|
||||
// no fields
|
||||
return null;
|
||||
} else if (subs.length == 1) {
|
||||
return getPerDocs(subs[0]);
|
||||
}
|
||||
PerDocValues perDocValues = r.retrievePerDoc();
|
||||
if (perDocValues == null) {
|
||||
|
||||
final List<PerDocValues> producer = new ArrayList<PerDocValues>();
|
||||
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
|
||||
|
||||
new ReaderUtil.Gather(r) {
|
||||
@Override
|
||||
protected void add(int base, IndexReader r) throws IOException {
|
||||
final PerDocValues f = r.perDocValues();
|
||||
if (f != null) {
|
||||
producer.add(f);
|
||||
slices
|
||||
.add(new ReaderUtil.Slice(base, r.maxDoc(), producer.size() - 1));
|
||||
}
|
||||
}
|
||||
}.run();
|
||||
|
||||
if (producer.size() == 0) {
|
||||
return null;
|
||||
} else if (producer.size() == 1) {
|
||||
perDocValues = producer.get(0);
|
||||
} else {
|
||||
perDocValues = new MultiPerDocValues(
|
||||
producer.toArray(PerDocValues.EMPTY_ARRAY),
|
||||
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
|
||||
}
|
||||
r.storePerDoc(perDocValues);
|
||||
}
|
||||
return perDocValues;
|
||||
}
|
||||
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
IndexDocValues result = docValues.get(field);
|
||||
if (result == null) {
|
||||
// Lazy init: first time this field is requested, we
|
||||
// create & add to docValues:
|
||||
final List<MultiIndexDocValues.DocValuesIndex> docValuesIndex = new ArrayList<MultiIndexDocValues.DocValuesIndex>();
|
||||
int docsUpto = 0;
|
||||
ValueType type = null;
|
||||
// Gather all sub-readers that share this field
|
||||
for (int i = 0; i < subs.length; i++) {
|
||||
IndexDocValues values = subs[i].docValues(field);
|
||||
final int start = subSlices[i].start;
|
||||
final int length = subSlices[i].length;
|
||||
if (values != null) {
|
||||
if (docsUpto != start) {
|
||||
type = values.type();
|
||||
docValuesIndex.add(new MultiIndexDocValues.DocValuesIndex(
|
||||
new MultiIndexDocValues.EmptyDocValues(start, type), docsUpto, start
|
||||
- docsUpto));
|
||||
}
|
||||
docValuesIndex.add(new MultiIndexDocValues.DocValuesIndex(values, start,
|
||||
length));
|
||||
docsUpto = start + length;
|
||||
|
||||
} else if (i + 1 == subs.length && !docValuesIndex.isEmpty()) {
|
||||
docValuesIndex.add(new MultiIndexDocValues.DocValuesIndex(
|
||||
new MultiIndexDocValues.EmptyDocValues(start, type), docsUpto, start
|
||||
- docsUpto));
|
||||
}
|
||||
}
|
||||
if (docValuesIndex.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
result = new MultiIndexDocValues(
|
||||
docValuesIndex.toArray(DocValuesIndex.EMPTY_ARRAY));
|
||||
docValues.put(field, result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(this.subs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return fields;
|
||||
}
|
||||
}
|
|
@ -21,8 +21,6 @@ import java.io.IOException;
|
|||
import java.util.Collection;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
||||
/** An IndexReader which reads multiple indexes, appending
|
||||
|
@ -187,5 +185,4 @@ public class MultiReader extends BaseMultiReader<IndexReader> {
|
|||
sub.removeReaderFinishedListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,8 +17,6 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
@ -59,7 +57,6 @@ public class ParallelReader extends IndexReader {
|
|||
private boolean hasDeletions;
|
||||
|
||||
private final ParallelFields fields = new ParallelFields();
|
||||
private final ParallelPerDocs perDocs = new ParallelPerDocs();
|
||||
|
||||
/** Construct a ParallelReader.
|
||||
* <p>Note that all subreaders are closed if this ParallelReader is closed.</p>
|
||||
|
@ -132,7 +129,6 @@ public class ParallelReader extends IndexReader {
|
|||
if (fieldToReader.get(field) == null) {
|
||||
fieldToReader.put(field, reader);
|
||||
this.fields.addField(field, MultiFields.getFields(reader).terms(field));
|
||||
this.perDocs.addField(field, reader);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -464,41 +460,10 @@ public class ParallelReader extends IndexReader {
|
|||
}
|
||||
}
|
||||
|
||||
// TODO: I suspect this is completely untested!!!!!
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
ensureOpen();
|
||||
return perDocs;
|
||||
}
|
||||
|
||||
// Single instance of this, per ParallelReader instance
|
||||
private static final class ParallelPerDocs extends PerDocValues {
|
||||
final TreeMap<String,IndexDocValues> fields = new TreeMap<String,IndexDocValues>();
|
||||
|
||||
void addField(String field, IndexReader r) throws IOException {
|
||||
PerDocValues perDocs = MultiPerDocValues.getPerDocs(r);
|
||||
if (perDocs != null) {
|
||||
fields.put(field, perDocs.docValues(field));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
// nothing to do here
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
return fields.get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return fields.keySet();
|
||||
}
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
IndexReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : MultiDocValues.getDocValues(reader, field);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -22,10 +22,10 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.NormsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -48,7 +48,7 @@ final class SegmentCoreReaders {
|
|||
final FieldInfos fieldInfos;
|
||||
|
||||
final FieldsProducer fields;
|
||||
final PerDocValues perDocProducer;
|
||||
final PerDocProducer perDocProducer;
|
||||
final NormsReader norms;
|
||||
|
||||
final Directory dir;
|
||||
|
|
|
@ -20,10 +20,13 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
|
@ -31,7 +34,6 @@ import org.apache.lucene.index.codecs.NormsWriter;
|
|||
import org.apache.lucene.index.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -131,9 +133,7 @@ final class SegmentMerger {
|
|||
int numMerged = mergeVectors();
|
||||
assert numMerged == mergeState.mergedDocCount;
|
||||
}
|
||||
// write FIS once merge is done. IDV might change types or drops fields
|
||||
FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
|
||||
fieldInfosWriter.write(directory, segment, mergeState.fieldInfos, context);
|
||||
|
||||
return mergeState;
|
||||
}
|
||||
|
||||
|
@ -186,15 +186,40 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
// returns an updated typepromoter (tracking type and size) given a previous one,
|
||||
// and a newly encountered docvalues
|
||||
private TypePromoter mergeDocValuesType(TypePromoter previous, DocValues docValues) {
|
||||
TypePromoter incoming = TypePromoter.create(docValues.type(), docValues.getValueSize());
|
||||
if (previous == null) {
|
||||
previous = TypePromoter.getIdentityPromoter();
|
||||
}
|
||||
TypePromoter promoted = previous.promote(incoming);
|
||||
if (promoted == null) {
|
||||
// type is incompatible: promote to BYTES_VAR_STRAIGHT
|
||||
return TypePromoter.create(DocValues.Type.BYTES_VAR_STRAIGHT, TypePromoter.VAR_TYPE_VALUE_SIZE);
|
||||
} else {
|
||||
return promoted;
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeFieldInfos() throws IOException {
|
||||
// mapping from all docvalues fields found to their promoted types
|
||||
// this is because FieldInfos does not store the valueSize
|
||||
Map<FieldInfo,TypePromoter> docValuesTypes = new HashMap<FieldInfo,TypePromoter>();
|
||||
|
||||
for (MergeState.IndexReaderAndLiveDocs readerAndLiveDocs : mergeState.readers) {
|
||||
final IndexReader reader = readerAndLiveDocs.reader;
|
||||
if (reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader;
|
||||
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
|
||||
for (FieldInfo fi : readerFieldInfos) {
|
||||
mergeState.fieldInfos.add(fi);
|
||||
FieldInfo merged = mergeState.fieldInfos.add(fi);
|
||||
// update the type promotion mapping for this reader
|
||||
if (fi.hasDocValues()) {
|
||||
TypePromoter previous = docValuesTypes.get(merged);
|
||||
docValuesTypes.put(merged, mergeDocValuesType(previous, reader.docValues(fi.name)));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
|
@ -209,10 +234,33 @@ final class SegmentMerger {
|
|||
Collection<String> dvNames = reader.getFieldNames(FieldOption.DOC_VALUES);
|
||||
mergeState.fieldInfos.addOrUpdate(dvNames, false);
|
||||
for (String dvName : dvNames) {
|
||||
mergeState.fieldInfos.fieldInfo(dvName).setDocValues(reader.docValues(dvName).type());
|
||||
FieldInfo merged = mergeState.fieldInfos.fieldInfo(dvName);
|
||||
DocValues docValues = reader.docValues(dvName);
|
||||
merged.setDocValuesType(docValues.type());
|
||||
TypePromoter previous = docValuesTypes.get(merged);
|
||||
docValuesTypes.put(merged, mergeDocValuesType(previous, docValues));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// update any promoted doc values types:
|
||||
for (Map.Entry<FieldInfo,TypePromoter> e : docValuesTypes.entrySet()) {
|
||||
FieldInfo fi = e.getKey();
|
||||
TypePromoter promoter = e.getValue();
|
||||
if (promoter == null) {
|
||||
fi.resetDocValuesType(null);
|
||||
} else {
|
||||
assert promoter != TypePromoter.getIdentityPromoter();
|
||||
if (fi.getDocValuesType() != promoter.type()) {
|
||||
// reset the type if we got promoted
|
||||
fi.resetDocValuesType(promoter.type());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// write the merged infos
|
||||
FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
|
||||
fieldInfosWriter.write(directory, segment, mergeState.fieldInfos, context);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -27,8 +27,8 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
|
@ -425,9 +425,13 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return core.perDocProducer;
|
||||
final PerDocProducer perDoc = core.perDocProducer;
|
||||
if (perDoc == null) {
|
||||
return null;
|
||||
}
|
||||
return perDoc.docValues(field);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.util.ReaderUtil; // javadoc
|
|||
|
||||
import org.apache.lucene.index.DirectoryReader; // javadoc
|
||||
import org.apache.lucene.index.MultiReader; // javadoc
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
|
||||
/**
|
||||
* This class forces a composite reader (eg a {@link
|
||||
|
@ -35,11 +34,12 @@ import org.apache.lucene.index.codecs.PerDocValues;
|
|||
* IndexReader#getSequentialSubReaders}) to emulate an
|
||||
* atomic reader. This requires implementing the postings
|
||||
* APIs on-the-fly, using the static methods in {@link
|
||||
* MultiFields}, by stepping through the sub-readers to
|
||||
* merge fields/terms, appending docs, etc.
|
||||
* MultiFields}, {@link MultiNorms}, {@link MultiDocValues},
|
||||
* by stepping through the sub-readers to merge fields/terms,
|
||||
* appending docs, etc.
|
||||
*
|
||||
* <p>If you ever hit an UnsupportedOperationException saying
|
||||
* "please use MultiFields.XXX instead", the simple
|
||||
* "please use MultiXXX.YYY instead", the simple
|
||||
* but non-performant workaround is to wrap your reader
|
||||
* using this class.</p>
|
||||
*
|
||||
|
@ -72,9 +72,9 @@ public final class SlowMultiReaderWrapper extends FilterIndexReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return MultiPerDocValues.getPerDocs(in);
|
||||
return MultiDocValues.getDocValues(in, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -22,10 +22,10 @@ import java.util.Collections;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.MergeState.IndexReaderAndLiveDocs;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -35,17 +35,22 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
/**
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class SortedBytesMergeUtils {
|
||||
// TODO: generalize this a bit more:
|
||||
// * remove writing (like indexoutput) from here
|
||||
// * just take IndexReaders (not IR&LiveDocs), doesnt care about liveDocs
|
||||
// * hook into MultiDocValues to make a MultiSortedSource
|
||||
// * maybe DV merging should then just use MultiDocValues for simplicity?
|
||||
public final class SortedBytesMergeUtils {
|
||||
|
||||
private SortedBytesMergeUtils() {
|
||||
// no instance
|
||||
}
|
||||
|
||||
static MergeContext init(ValueType type, IndexDocValues[] docValues,
|
||||
public static MergeContext init(Type type, DocValues[] docValues,
|
||||
Comparator<BytesRef> comp, MergeState mergeState) {
|
||||
int size = -1;
|
||||
if (type == ValueType.BYTES_FIXED_SORTED) {
|
||||
for (IndexDocValues indexDocValues : docValues) {
|
||||
if (type == Type.BYTES_FIXED_SORTED) {
|
||||
for (DocValues indexDocValues : docValues) {
|
||||
if (indexDocValues != null) {
|
||||
size = indexDocValues.getValueSize();
|
||||
break;
|
||||
|
@ -59,14 +64,14 @@ final class SortedBytesMergeUtils {
|
|||
public static final class MergeContext {
|
||||
private final Comparator<BytesRef> comp;
|
||||
private final BytesRef missingValue = new BytesRef();
|
||||
final int sizePerValues; // -1 if var length
|
||||
final ValueType type;
|
||||
final int[] docToEntry;
|
||||
long[] offsets; // if non-null #mergeRecords collects byte offsets here
|
||||
public final int sizePerValues; // -1 if var length
|
||||
final Type type;
|
||||
public final int[] docToEntry;
|
||||
public long[] offsets; // if non-null #mergeRecords collects byte offsets here
|
||||
|
||||
public MergeContext(Comparator<BytesRef> comp, MergeState mergeState,
|
||||
int size, ValueType type) {
|
||||
assert type == ValueType.BYTES_FIXED_SORTED || type == ValueType.BYTES_VAR_SORTED;
|
||||
int size, Type type) {
|
||||
assert type == Type.BYTES_FIXED_SORTED || type == Type.BYTES_VAR_SORTED;
|
||||
this.comp = comp;
|
||||
this.sizePerValues = size;
|
||||
this.type = type;
|
||||
|
@ -78,8 +83,8 @@ final class SortedBytesMergeUtils {
|
|||
}
|
||||
}
|
||||
|
||||
static List<SortedSourceSlice> buildSlices(MergeState mergeState,
|
||||
IndexDocValues[] docValues, MergeContext ctx) throws IOException {
|
||||
public static List<SortedSourceSlice> buildSlices(MergeState mergeState,
|
||||
DocValues[] docValues, MergeContext ctx) throws IOException {
|
||||
final List<SortedSourceSlice> slices = new ArrayList<SortedSourceSlice>();
|
||||
for (int i = 0; i < docValues.length; i++) {
|
||||
final SortedSourceSlice nextSlice;
|
||||
|
@ -140,7 +145,7 @@ final class SortedBytesMergeUtils {
|
|||
}
|
||||
}
|
||||
|
||||
static int mergeRecords(MergeContext ctx, IndexOutput datOut,
|
||||
public static int mergeRecords(MergeContext ctx, IndexOutput datOut,
|
||||
List<SortedSourceSlice> slices) throws IOException {
|
||||
final RecordMerger merger = new RecordMerger(new MergeQueue(slices.size(),
|
||||
ctx.comp), slices.toArray(new SortedSourceSlice[0]));
|
||||
|
@ -215,7 +220,7 @@ final class SortedBytesMergeUtils {
|
|||
}
|
||||
}
|
||||
|
||||
static class SortedSourceSlice {
|
||||
public static class SortedSourceSlice {
|
||||
final SortedSource source;
|
||||
final int readerIdx;
|
||||
/* global array indexed by docID containg the relative ord for the doc */
|
||||
|
@ -265,7 +270,7 @@ final class SortedBytesMergeUtils {
|
|||
return null;
|
||||
}
|
||||
|
||||
void writeOrds(PackedInts.Writer writer) throws IOException {
|
||||
public void writeOrds(PackedInts.Writer writer) throws IOException {
|
||||
for (int i = docToOrdStart; i < docToOrdEnd; i++) {
|
||||
final int mappedOrd = docIDToRelativeOrd[i];
|
||||
assert mappedOrd < ordMapping.length;
|
|
@ -113,7 +113,6 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
// of a given field in the doc. At this point we flush
|
||||
// our hash into the DocWriter.
|
||||
|
||||
assert fieldInfo.storeTermVector;
|
||||
assert termsWriter.vectorFieldsInOrder(fieldInfo);
|
||||
|
||||
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
|
||||
|
@ -150,8 +149,9 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
}
|
||||
|
||||
termsHashPerField.reset();
|
||||
|
||||
// commit the termVectors once successful success - FI will otherwise reset them
|
||||
fieldInfo.commitVectors();
|
||||
fieldInfo.setStoreTermVectors(doVectorPositions, doVectorOffsets);
|
||||
}
|
||||
|
||||
void shrinkHash() {
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,15 +19,17 @@ package org.apache.lucene.index.values;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
|
||||
/**
|
||||
* Type promoter that promotes {@link IndexDocValues} during merge based on
|
||||
* their {@link ValueType} and {@link #getValueSize()}
|
||||
* Type promoter that promotes {@link DocValues} during merge based on
|
||||
* their {@link Type} and {@link #getValueSize()}
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class TypePromoter {
|
||||
class TypePromoter {
|
||||
|
||||
private final static Map<Integer, ValueType> FLAGS_MAP = new HashMap<Integer, ValueType>();
|
||||
private final static Map<Integer, Type> FLAGS_MAP = new HashMap<Integer, Type>();
|
||||
private static final TypePromoter IDENTITY_PROMOTER = new IdentityTypePromoter();
|
||||
public static final int VAR_TYPE_VALUE_SIZE = -1;
|
||||
|
||||
|
@ -49,7 +51,7 @@ public class TypePromoter {
|
|||
private static final int IS_32_BIT = 1 << 10 | 1 << 11;
|
||||
private static final int IS_64_BIT = 1 << 11;
|
||||
|
||||
private final ValueType type;
|
||||
private final Type type;
|
||||
private final int flags;
|
||||
private final int valueSize;
|
||||
|
||||
|
@ -65,7 +67,7 @@ public class TypePromoter {
|
|||
}
|
||||
|
||||
static {
|
||||
for (ValueType type : ValueType.values()) {
|
||||
for (Type type : Type.values()) {
|
||||
TypePromoter create = create(type, VAR_TYPE_VALUE_SIZE);
|
||||
FLAGS_MAP.put(create.flags, type);
|
||||
}
|
||||
|
@ -75,13 +77,13 @@ public class TypePromoter {
|
|||
* Creates a new {@link TypePromoter}
|
||||
*
|
||||
* @param type
|
||||
* the {@link ValueType} this promoter represents
|
||||
* the {@link Type} this promoter represents
|
||||
* @param flags
|
||||
* the promoters flags
|
||||
* @param valueSize
|
||||
* the value size if {@link #IS_FIXED} or <code>-1</code> otherwise.
|
||||
*/
|
||||
protected TypePromoter(ValueType type, int flags, int valueSize) {
|
||||
protected TypePromoter(Type type, int flags, int valueSize) {
|
||||
this.type = type;
|
||||
this.flags = flags;
|
||||
this.valueSize = valueSize;
|
||||
|
@ -117,11 +119,11 @@ public class TypePromoter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType} of this {@link TypePromoter}
|
||||
* Returns the {@link Type} of this {@link TypePromoter}
|
||||
*
|
||||
* @return the {@link ValueType} of this {@link TypePromoter}
|
||||
* @return the {@link Type} of this {@link TypePromoter}
|
||||
*/
|
||||
public ValueType type() {
|
||||
public Type type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
|
@ -134,13 +136,13 @@ public class TypePromoter {
|
|||
* Creates a new {@link TypePromoter} for the given type and size per value.
|
||||
*
|
||||
* @param type
|
||||
* the {@link ValueType} to create the promoter for
|
||||
* the {@link Type} to create the promoter for
|
||||
* @param valueSize
|
||||
* the size per value in bytes or <code>-1</code> iff the types have
|
||||
* variable length.
|
||||
* @return a new {@link TypePromoter}
|
||||
*/
|
||||
public static TypePromoter create(ValueType type, int valueSize) {
|
||||
public static TypePromoter create(Type type, int valueSize) {
|
||||
if (type == null) {
|
||||
return null;
|
||||
}
|
|
@ -17,76 +17,51 @@ package org.apache.lucene.index.codecs;
|
|||
* limitations under the License.
|
||||
*/
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.Writer;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.Counter;
|
||||
|
||||
/**
|
||||
* Abstract API that consumes {@link PerDocFieldValues}.
|
||||
* Abstract API that consumes {@link DocValue}s.
|
||||
* {@link DocValuesConsumer} are always associated with a specific field and
|
||||
* segments. Concrete implementations of this API write the given
|
||||
* {@link PerDocFieldValues} into a implementation specific format depending on
|
||||
* {@link DocValue} into a implementation specific format depending on
|
||||
* the fields meta-data.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class DocValuesConsumer {
|
||||
|
||||
protected final Counter bytesUsed;
|
||||
|
||||
/**
|
||||
* Creates a new {@link DocValuesConsumer}.
|
||||
*
|
||||
* @param bytesUsed
|
||||
* bytes-usage tracking reference used by implementation to track
|
||||
* internally allocated memory. All tracked bytes must be released
|
||||
* once {@link #finish(int)} has been called.
|
||||
*/
|
||||
protected DocValuesConsumer(Counter bytesUsed) {
|
||||
this.bytesUsed = bytesUsed == null ? Counter.newCounter() : bytesUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the given {@link PerDocFieldValues} instance to this
|
||||
* Adds the given {@link DocValue} instance to this
|
||||
* {@link DocValuesConsumer}
|
||||
*
|
||||
* @param docID
|
||||
* the document ID to add the value for. The docID must always
|
||||
* increase or be <tt>0</tt> if it is the first call to this method.
|
||||
* @param docValues
|
||||
* the values to add
|
||||
* @param docValue
|
||||
* the value to add
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public abstract void add(int docID, PerDocFieldValues docValues)
|
||||
public abstract void add(int docID, DocValue docValue)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Called when the consumer of this API is doc with adding
|
||||
* {@link PerDocFieldValues} to this {@link DocValuesConsumer}
|
||||
* {@link DocValue} to this {@link DocValuesConsumer}
|
||||
*
|
||||
* @param docCount
|
||||
* the total number of documents in this {@link DocValuesConsumer}.
|
||||
* Must be greater than or equal the last given docID to
|
||||
* {@link #add(int, PerDocFieldValues)}.
|
||||
* {@link #add(int, DocValue)}.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract void finish(int docCount) throws IOException;
|
||||
|
||||
/**
|
||||
* Gathers files associated with this {@link DocValuesConsumer}
|
||||
*
|
||||
* @param files
|
||||
* the of files to add the consumers files to.
|
||||
*/
|
||||
public abstract void files(Collection<String> files) throws IOException;
|
||||
|
||||
/**
|
||||
* Merges the given {@link org.apache.lucene.index.MergeState} into
|
||||
* this {@link DocValuesConsumer}.
|
||||
|
@ -95,18 +70,18 @@ public abstract class DocValuesConsumer {
|
|||
* the state to merge
|
||||
* @param docValues docValues array containing one instance per reader (
|
||||
* {@link org.apache.lucene.index.MergeState#readers}) or <code>null</code> if the reader has
|
||||
* no {@link IndexDocValues} instance.
|
||||
* no {@link DocValues} instance.
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public void merge(MergeState mergeState, IndexDocValues[] docValues) throws IOException {
|
||||
public void merge(MergeState mergeState, DocValues[] docValues) throws IOException {
|
||||
assert mergeState != null;
|
||||
boolean hasMerged = false;
|
||||
for(int readerIDX=0;readerIDX<mergeState.readers.size();readerIDX++) {
|
||||
final org.apache.lucene.index.MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(readerIDX);
|
||||
if (docValues[readerIDX] != null) {
|
||||
hasMerged = true;
|
||||
merge(new Writer.SingleSubMergeState(docValues[readerIDX], mergeState.docBase[readerIDX], reader.reader.maxDoc(),
|
||||
merge(new SingleSubMergeState(docValues[readerIDX], mergeState.docBase[readerIDX], reader.reader.maxDoc(),
|
||||
reader.liveDocs));
|
||||
}
|
||||
}
|
||||
|
@ -124,6 +99,9 @@ public abstract class DocValuesConsumer {
|
|||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
// TODO: can't we have a default implementation here that merges naively with our apis?
|
||||
// this is how stored fields and term vectors work. its a pain to have to impl merging
|
||||
// (should be an optimization to override it)
|
||||
protected abstract void merge(SingleSubMergeState mergeState) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -137,7 +115,7 @@ public abstract class DocValuesConsumer {
|
|||
* the source reader for this MergeState - merged values should be read from
|
||||
* this instance
|
||||
*/
|
||||
public final IndexDocValues reader;
|
||||
public final DocValues reader;
|
||||
/** the absolute docBase for this MergeState within the resulting segment */
|
||||
public final int docBase;
|
||||
/** the number of documents in this MergeState */
|
||||
|
@ -145,7 +123,7 @@ public abstract class DocValuesConsumer {
|
|||
/** the not deleted bits for this MergeState */
|
||||
public final Bits liveDocs;
|
||||
|
||||
public SingleSubMergeState(IndexDocValues reader, int docBase, int docCount, Bits liveDocs) {
|
||||
public SingleSubMergeState(DocValues reader, int docBase, int docCount, Bits liveDocs) {
|
||||
assert reader != null;
|
||||
this.reader = reader;
|
||||
this.docBase = docBase;
|
||||
|
|
|
@ -27,6 +27,6 @@ import org.apache.lucene.store.Directory;
|
|||
|
||||
public abstract class DocValuesFormat {
|
||||
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
|
||||
public abstract PerDocValues docsProducer(SegmentReadState state) throws IOException;
|
||||
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
||||
|
|
|
@ -26,23 +26,24 @@ import java.util.TreeMap;
|
|||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.values.Bytes;
|
||||
import org.apache.lucene.index.values.Floats;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.Ints;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Floats;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Ints;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Abstract base class for PerDocValues implementations
|
||||
* Abstract base class for PerDocProducer implementations
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class DocValuesReaderBase extends PerDocValues {
|
||||
// TODO: this needs to go under lucene40 codec (its specific to its impl)
|
||||
public abstract class DocValuesReaderBase extends PerDocProducer {
|
||||
|
||||
protected abstract void closeInternal(Collection<? extends Closeable> closeables) throws IOException;
|
||||
protected abstract Map<String, IndexDocValues> docValues();
|
||||
protected abstract Map<String, DocValues> docValues();
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
@ -50,24 +51,19 @@ public abstract class DocValuesReaderBase extends PerDocValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValues docValues(String field) throws IOException {
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
return docValues().get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> fields() {
|
||||
return docValues().keySet();
|
||||
}
|
||||
|
||||
public Comparator<BytesRef> getComparator() throws IOException {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
// Only opens files... doesn't actually load any values
|
||||
protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
|
||||
protected TreeMap<String, DocValues> load(FieldInfos fieldInfos,
|
||||
String segment, int docCount, Directory dir, IOContext context)
|
||||
throws IOException {
|
||||
TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
|
||||
TreeMap<String, DocValues> values = new TreeMap<String, DocValues>();
|
||||
boolean success = false;
|
||||
try {
|
||||
|
||||
|
@ -79,7 +75,7 @@ public abstract class DocValuesReaderBase extends PerDocValues {
|
|||
final String id = DocValuesWriterBase.docValuesId(segment,
|
||||
fieldInfo.number);
|
||||
values.put(field,
|
||||
loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), context));
|
||||
loadDocValues(docCount, dir, id, fieldInfo.getDocValuesType(), context));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
@ -93,26 +89,26 @@ public abstract class DocValuesReaderBase extends PerDocValues {
|
|||
}
|
||||
|
||||
/**
|
||||
* Loads a {@link IndexDocValues} instance depending on the given {@link ValueType}.
|
||||
* Codecs that use different implementations for a certain {@link ValueType} can
|
||||
* Loads a {@link DocValues} instance depending on the given {@link Type}.
|
||||
* Codecs that use different implementations for a certain {@link Type} can
|
||||
* simply override this method and return their custom implementations.
|
||||
*
|
||||
* @param docCount
|
||||
* number of documents in the segment
|
||||
* @param dir
|
||||
* the {@link Directory} to load the {@link IndexDocValues} from
|
||||
* the {@link Directory} to load the {@link DocValues} from
|
||||
* @param id
|
||||
* the unique file ID within the segment
|
||||
* @param type
|
||||
* the type to load
|
||||
* @return a {@link IndexDocValues} instance for the given type
|
||||
* @return a {@link DocValues} instance for the given type
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
* @throws IllegalArgumentException
|
||||
* if the given {@link ValueType} is not supported
|
||||
* if the given {@link Type} is not supported
|
||||
*/
|
||||
protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
|
||||
ValueType type, IOContext context) throws IOException {
|
||||
protected DocValues loadDocValues(int docCount, Directory dir, String id,
|
||||
DocValues.Type type, IOContext context) throws IOException {
|
||||
switch (type) {
|
||||
case FIXED_INTS_16:
|
||||
case FIXED_INTS_32:
|
||||
|
|
|
@ -22,7 +22,8 @@ import java.util.Comparator;
|
|||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.values.Writer;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Writer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -32,6 +33,7 @@ import org.apache.lucene.util.Counter;
|
|||
* Abstract base class for PerDocConsumer implementations
|
||||
* @lucene.experimental
|
||||
*/
|
||||
//TODO: this needs to go under lucene40 codec (its specific to its impl)
|
||||
public abstract class DocValuesWriterBase extends PerDocConsumer {
|
||||
protected final String segmentName;
|
||||
protected final String segmentSuffix;
|
||||
|
@ -52,8 +54,8 @@ public abstract class DocValuesWriterBase extends PerDocConsumer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
|
||||
return Writer.create(field.getDocValues(),
|
||||
public DocValuesConsumer addValuesField(DocValues.Type valueType, FieldInfo field) throws IOException {
|
||||
return Writer.create(valueType,
|
||||
docValuesId(segmentName, field.number),
|
||||
getDirectory(), getComparator(), bytesUsed, context);
|
||||
}
|
||||
|
|
|
@ -19,11 +19,8 @@ import java.io.Closeable;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.TypePromoter;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
|
||||
/**
|
||||
* Abstract API that consumes per document values. Concrete implementations of
|
||||
|
@ -37,82 +34,26 @@ import org.apache.lucene.index.values.ValueType;
|
|||
*/
|
||||
public abstract class PerDocConsumer implements Closeable{
|
||||
/** Adds a new DocValuesField */
|
||||
public abstract DocValuesConsumer addValuesField(FieldInfo field)
|
||||
public abstract DocValuesConsumer addValuesField(DocValues.Type type, FieldInfo field)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Consumes and merges the given {@link PerDocValues} producer
|
||||
* Consumes and merges the given {@link PerDocProducer} producer
|
||||
* into this consumers format.
|
||||
*/
|
||||
public void merge(MergeState mergeState)
|
||||
throws IOException {
|
||||
final FieldInfos fieldInfos = mergeState.fieldInfos;
|
||||
final IndexDocValues[] docValues = new IndexDocValues[mergeState.readers.size()];
|
||||
final PerDocValues[] perDocValues = new PerDocValues[mergeState.readers.size()];
|
||||
// pull all PerDocValues
|
||||
for (int i = 0; i < perDocValues.length; i++) {
|
||||
perDocValues[i] = mergeState.readers.get(i).reader.perDocValues();
|
||||
}
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
mergeState.fieldInfo = fieldInfo;
|
||||
TypePromoter currentPromoter = TypePromoter.getIdentityPromoter();
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
final DocValues[] docValues = new DocValues[mergeState.readers.size()];
|
||||
|
||||
for (FieldInfo fieldInfo : mergeState.fieldInfos) {
|
||||
mergeState.fieldInfo = fieldInfo; // set the field we are merging
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
for (int i = 0; i < perDocValues.length; i++) {
|
||||
if (perDocValues[i] != null) { // get all IDV to merge
|
||||
docValues[i] = perDocValues[i].docValues(fieldInfo.name);
|
||||
if (docValues[i] != null) {
|
||||
currentPromoter = promoteValueType(fieldInfo, docValues[i], currentPromoter);
|
||||
if (currentPromoter == null) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
for (int i = 0; i < docValues.length; i++) {
|
||||
docValues[i] = mergeState.readers.get(i).reader.docValues(fieldInfo.name);
|
||||
}
|
||||
|
||||
if (currentPromoter == null) {
|
||||
fieldInfo.resetDocValues(null);
|
||||
continue;
|
||||
}
|
||||
assert currentPromoter != TypePromoter.getIdentityPromoter();
|
||||
if (fieldInfo.getDocValues() != currentPromoter.type()) {
|
||||
// reset the type if we got promoted
|
||||
fieldInfo.resetDocValues(currentPromoter.type());
|
||||
}
|
||||
|
||||
final DocValuesConsumer docValuesConsumer = addValuesField(mergeState.fieldInfo);
|
||||
final DocValuesConsumer docValuesConsumer = addValuesField(fieldInfo.getDocValuesType(), fieldInfo);
|
||||
assert docValuesConsumer != null;
|
||||
docValuesConsumer.merge(mergeState, docValues);
|
||||
}
|
||||
}
|
||||
/* NOTE: don't close the perDocProducers here since they are private segment producers
|
||||
* and will be closed once the SegmentReader goes out of scope */
|
||||
}
|
||||
|
||||
protected TypePromoter promoteValueType(final FieldInfo fieldInfo, final IndexDocValues docValues,
|
||||
TypePromoter currentPromoter) {
|
||||
assert currentPromoter != null;
|
||||
final TypePromoter incomingPromoter = TypePromoter.create(docValues.type(), docValues.getValueSize());
|
||||
assert incomingPromoter != null;
|
||||
final TypePromoter newPromoter = currentPromoter.promote(incomingPromoter);
|
||||
return newPromoter == null ? handleIncompatibleValueType(fieldInfo, incomingPromoter, currentPromoter) : newPromoter;
|
||||
}
|
||||
|
||||
/**
|
||||
* Resolves a conflicts of incompatible {@link TypePromoter}s. The default
|
||||
* implementation promotes incompatible types to
|
||||
* {@link ValueType#BYTES_VAR_STRAIGHT} and preserves all values. If this
|
||||
* method returns <code>null</code> all docvalues for the given
|
||||
* {@link FieldInfo} are dropped and all values are lost.
|
||||
*
|
||||
* @param incomingPromoter
|
||||
* the incompatible incoming promoter
|
||||
* @param currentPromoter
|
||||
* the current promoter
|
||||
* @return a promoted {@link TypePromoter} or <code>null</code> iff this index
|
||||
* docvalues should be dropped for this field.
|
||||
*/
|
||||
protected TypePromoter handleIncompatibleValueType(FieldInfo fieldInfo, TypePromoter incomingPromoter, TypePromoter currentPromoter) {
|
||||
return TypePromoter.create(ValueType.BYTES_VAR_STRAIGHT, TypePromoter.VAR_TYPE_VALUE_SIZE);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,9 +17,8 @@ package org.apache.lucene.index.codecs;
|
|||
*/
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
||||
/**
|
||||
* Abstract API that provides access to one or more per-document storage
|
||||
|
@ -27,28 +26,23 @@ import org.apache.lucene.index.values.IndexDocValues;
|
|||
* storage on a per-document basis corresponding to their actual
|
||||
* {@link PerDocConsumer} counterpart.
|
||||
* <p>
|
||||
* The {@link PerDocValues} API is accessible through the
|
||||
* The {@link PerDocProducer} API is accessible through the
|
||||
* {@link PostingsFormat} - API providing per field consumers and producers for inverted
|
||||
* data (terms, postings) as well as per-document data.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class PerDocValues implements Closeable {
|
||||
public abstract class PerDocProducer implements Closeable {
|
||||
/**
|
||||
* Returns {@link IndexDocValues} for the current field.
|
||||
* Returns {@link DocValues} for the current field.
|
||||
*
|
||||
* @param field
|
||||
* the field name
|
||||
* @return the {@link IndexDocValues} for this field or <code>null</code> if not
|
||||
* @return the {@link DocValues} for this field or <code>null</code> if not
|
||||
* applicable.
|
||||
* @throws IOException
|
||||
*/
|
||||
public abstract IndexDocValues docValues(String field) throws IOException;
|
||||
public abstract DocValues docValues(String field) throws IOException;
|
||||
|
||||
public static final PerDocValues[] EMPTY_ARRAY = new PerDocValues[0];
|
||||
|
||||
/**
|
||||
* Returns all fields this {@link PerDocValues} contains values for.
|
||||
*/
|
||||
public abstract Collection<String> fields();
|
||||
public static final PerDocProducer[] EMPTY_ARRAY = new PerDocProducer[0];
|
||||
}
|
|
@ -27,9 +27,9 @@ import org.apache.lucene.index.codecs.Codec;
|
|||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.index.codecs.NormsFormat;
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.TermVectorsFormat;
|
||||
|
@ -75,7 +75,7 @@ public class Lucene3xCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.lucene.index.SegmentInfo;
|
|||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
public class Lucene40DocValuesFormat extends DocValuesFormat {
|
||||
|
@ -36,7 +36,7 @@ public class Lucene40DocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene40DocValuesProducer(state);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,25 +24,25 @@ import java.util.Collection;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.DocValuesReaderBase;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Default PerDocValues implementation that uses compound file.
|
||||
* Default PerDocProducer implementation that uses compound file.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Lucene40DocValuesProducer extends DocValuesReaderBase {
|
||||
protected final TreeMap<String,IndexDocValues> docValues;
|
||||
protected final TreeMap<String,DocValues> docValues;
|
||||
private final Directory cfs;
|
||||
|
||||
/**
|
||||
* Creates a new {@link Lucene40DocValuesProducer} instance and loads all
|
||||
* {@link IndexDocValues} instances for this segment and codec.
|
||||
* {@link DocValues} instances for this segment and codec.
|
||||
*/
|
||||
public Lucene40DocValuesProducer(SegmentReadState state) throws IOException {
|
||||
if (state.fieldInfos.anyDocValuesFields()) {
|
||||
|
@ -53,12 +53,12 @@ public class Lucene40DocValuesProducer extends DocValuesReaderBase {
|
|||
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, cfs, state.context);
|
||||
} else {
|
||||
cfs = null;
|
||||
docValues = new TreeMap<String,IndexDocValues>();
|
||||
docValues = new TreeMap<String,DocValues>();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String,IndexDocValues> docValues() {
|
||||
protected Map<String,DocValues> docValues() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ import org.apache.lucene.index.IndexFormatTooOldException;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.codecs.FieldInfosReader;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -95,7 +95,7 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
|
|||
hasVectors |= storeTermVector;
|
||||
hasProx |= isIndexed && indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
|
||||
hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
|
||||
ValueType docValuesType = null;
|
||||
DocValues.Type docValuesType = null;
|
||||
if (format <= Lucene40FieldInfosWriter.FORMAT_FLEX) {
|
||||
final byte b = input.readByte();
|
||||
switch(b) {
|
||||
|
@ -103,43 +103,43 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
|
|||
docValuesType = null;
|
||||
break;
|
||||
case 1:
|
||||
docValuesType = ValueType.VAR_INTS;
|
||||
docValuesType = DocValues.Type.VAR_INTS;
|
||||
break;
|
||||
case 2:
|
||||
docValuesType = ValueType.FLOAT_32;
|
||||
docValuesType = DocValues.Type.FLOAT_32;
|
||||
break;
|
||||
case 3:
|
||||
docValuesType = ValueType.FLOAT_64;
|
||||
docValuesType = DocValues.Type.FLOAT_64;
|
||||
break;
|
||||
case 4:
|
||||
docValuesType = ValueType.BYTES_FIXED_STRAIGHT;
|
||||
docValuesType = DocValues.Type.BYTES_FIXED_STRAIGHT;
|
||||
break;
|
||||
case 5:
|
||||
docValuesType = ValueType.BYTES_FIXED_DEREF;
|
||||
docValuesType = DocValues.Type.BYTES_FIXED_DEREF;
|
||||
break;
|
||||
case 6:
|
||||
docValuesType = ValueType.BYTES_VAR_STRAIGHT;
|
||||
docValuesType = DocValues.Type.BYTES_VAR_STRAIGHT;
|
||||
break;
|
||||
case 7:
|
||||
docValuesType = ValueType.BYTES_VAR_DEREF;
|
||||
docValuesType = DocValues.Type.BYTES_VAR_DEREF;
|
||||
break;
|
||||
case 8:
|
||||
docValuesType = ValueType.FIXED_INTS_16;
|
||||
docValuesType = DocValues.Type.FIXED_INTS_16;
|
||||
break;
|
||||
case 9:
|
||||
docValuesType = ValueType.FIXED_INTS_32;
|
||||
docValuesType = DocValues.Type.FIXED_INTS_32;
|
||||
break;
|
||||
case 10:
|
||||
docValuesType = ValueType.FIXED_INTS_64;
|
||||
docValuesType = DocValues.Type.FIXED_INTS_64;
|
||||
break;
|
||||
case 11:
|
||||
docValuesType = ValueType.FIXED_INTS_8;
|
||||
docValuesType = DocValues.Type.FIXED_INTS_8;
|
||||
break;
|
||||
case 12:
|
||||
docValuesType = ValueType.BYTES_FIXED_SORTED;
|
||||
docValuesType = DocValues.Type.BYTES_FIXED_SORTED;
|
||||
break;
|
||||
case 13:
|
||||
docValuesType = ValueType.BYTES_VAR_SORTED;
|
||||
docValuesType = DocValues.Type.BYTES_VAR_SORTED;
|
||||
break;
|
||||
|
||||
default:
|
||||
|
|
|
@ -83,7 +83,7 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
|
|||
if (!fi.hasDocValues()) {
|
||||
b = 0;
|
||||
} else {
|
||||
switch(fi.getDocValues()) {
|
||||
switch(fi.getDocValuesType()) {
|
||||
case VAR_INTS:
|
||||
b = 1;
|
||||
break;
|
||||
|
@ -124,7 +124,7 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
|
|||
b = 13;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("unhandled indexValues type " + fi.getDocValues());
|
||||
throw new IllegalStateException("unhandled indexValues type " + fi.getDocValuesType());
|
||||
}
|
||||
}
|
||||
output.writeByte(b);
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,13 +19,15 @@ package org.apache.lucene.index.values;
|
|||
|
||||
/** Base class for specific Bytes Reader/Writer implementations */
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
@ -148,13 +150,13 @@ public final class Bytes {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link IndexDocValues} instance that provides either memory
|
||||
* Creates a new {@link DocValues} instance that provides either memory
|
||||
* resident or iterative access to a per-document stored <tt>byte[]</tt>
|
||||
* value. The returned {@link IndexDocValues} instance will be initialized without
|
||||
* value. The returned {@link DocValues} instance will be initialized without
|
||||
* consuming a significant amount of memory.
|
||||
*
|
||||
* @param dir
|
||||
* the directory to load the {@link IndexDocValues} from.
|
||||
* the directory to load the {@link DocValues} from.
|
||||
* @param id
|
||||
* the file ID in the {@link Directory} to load the values from.
|
||||
* @param mode
|
||||
|
@ -167,11 +169,11 @@ public final class Bytes {
|
|||
* @param sortComparator {@link BytesRef} comparator used by sorted variants.
|
||||
* If <code>null</code> {@link BytesRef#getUTF8SortedAsUnicodeComparator()}
|
||||
* is used instead
|
||||
* @return an initialized {@link IndexDocValues} instance.
|
||||
* @return an initialized {@link DocValues} instance.
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
public static IndexDocValues getValues(Directory dir, String id, Mode mode,
|
||||
public static DocValues getValues(Directory dir, String id, Mode mode,
|
||||
boolean fixedSize, int maxDoc, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
|
||||
if (sortComparator == null) {
|
||||
sortComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
|
@ -183,7 +185,7 @@ public final class Bytes {
|
|||
} else if (mode == Mode.DEREF) {
|
||||
return new FixedDerefBytesImpl.FixedDerefReader(dir, id, maxDoc, context);
|
||||
} else if (mode == Mode.SORTED) {
|
||||
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc, context, ValueType.BYTES_FIXED_SORTED, sortComparator);
|
||||
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc, context, Type.BYTES_FIXED_SORTED, sortComparator);
|
||||
}
|
||||
} else {
|
||||
if (mode == Mode.STRAIGHT) {
|
||||
|
@ -191,7 +193,7 @@ public final class Bytes {
|
|||
} else if (mode == Mode.DEREF) {
|
||||
return new VarDerefBytesImpl.VarDerefReader(dir, id, maxDoc, context);
|
||||
} else if (mode == Mode.SORTED) {
|
||||
return new VarSortedBytesImpl.Reader(dir, id, maxDoc,context, ValueType.BYTES_VAR_SORTED, sortComparator);
|
||||
return new VarSortedBytesImpl.Reader(dir, id, maxDoc,context, Type.BYTES_VAR_SORTED, sortComparator);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -209,7 +211,7 @@ public final class Bytes {
|
|||
|
||||
|
||||
protected BytesSourceBase(IndexInput datIn, IndexInput idxIn,
|
||||
PagedBytes pagedBytes, long bytesToRead, ValueType type) throws IOException {
|
||||
PagedBytes pagedBytes, long bytesToRead, Type type) throws IOException {
|
||||
super(type);
|
||||
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
|
||||
+ (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
|
||||
|
@ -300,39 +302,27 @@ public final class Bytes {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, PerDocFieldValues docValues) throws IOException {
|
||||
public void add(int docID, DocValue docValue) throws IOException {
|
||||
final BytesRef ref;
|
||||
if ((ref = docValues.getBytes()) != null) {
|
||||
if ((ref = docValue.getBytes()) != null) {
|
||||
add(docID, ref);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Collection<String> files) throws IOException {
|
||||
assert datOut != null;
|
||||
files.add(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX, DATA_EXTENSION));
|
||||
if (idxOut != null) { // called after flush - so this must be initialized
|
||||
// if needed or present
|
||||
final String idxFile = IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
|
||||
INDEX_EXTENSION);
|
||||
files.add(idxFile);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens all necessary files, but does not read any data in until you call
|
||||
* {@link #load}.
|
||||
*/
|
||||
static abstract class BytesReaderBase extends IndexDocValues {
|
||||
static abstract class BytesReaderBase extends DocValues {
|
||||
protected final IndexInput idxIn;
|
||||
protected final IndexInput datIn;
|
||||
protected final int version;
|
||||
protected final String id;
|
||||
protected final ValueType type;
|
||||
protected final Type type;
|
||||
|
||||
protected BytesReaderBase(Directory dir, String id, String codecName,
|
||||
int maxVersion, boolean doIndex, IOContext context, ValueType type) throws IOException {
|
||||
int maxVersion, boolean doIndex, IOContext context, Type type) throws IOException {
|
||||
IndexInput dataIn = null;
|
||||
IndexInput indexIn = null;
|
||||
boolean success = false;
|
||||
|
@ -385,7 +375,7 @@ public final class Bytes {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
public Type type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
|
@ -569,12 +559,12 @@ public final class Bytes {
|
|||
protected final PagedBytes.Reader data;
|
||||
|
||||
protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
|
||||
Comparator<BytesRef> comp, long bytesToRead, ValueType type, boolean hasOffsets) throws IOException {
|
||||
Comparator<BytesRef> comp, long bytesToRead, Type type, boolean hasOffsets) throws IOException {
|
||||
this(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), bytesToRead, type, hasOffsets);
|
||||
}
|
||||
|
||||
protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
|
||||
Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead, ValueType type, boolean hasOffsets)
|
||||
Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead, Type type, boolean hasOffsets)
|
||||
throws IOException {
|
||||
super(type, comp);
|
||||
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
|
||||
|
@ -588,6 +578,11 @@ public final class Bytes {
|
|||
docToOrdIndex = PackedInts.getReader(idxIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPackedDocToOrd() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PackedInts.Reader getDocToOrd() {
|
||||
return docToOrdIndex;
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -25,7 +25,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class BytesRefUtils {
|
||||
public final class BytesRefUtils {
|
||||
|
||||
private BytesRefUtils() {
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,7 +19,9 @@ package org.apache.lucene.index.values;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -27,13 +29,13 @@ import org.apache.lucene.util.BytesRef;
|
|||
* Base class for disk resident source implementations
|
||||
* @lucene.internal
|
||||
*/
|
||||
abstract class DirectSource extends Source {
|
||||
public abstract class DirectSource extends Source {
|
||||
|
||||
protected final IndexInput data;
|
||||
private final ToNumeric toNumeric;
|
||||
protected final long baseOffset;
|
||||
|
||||
DirectSource(IndexInput input, ValueType type) {
|
||||
public DirectSource(IndexInput input, Type type) {
|
||||
super(type);
|
||||
this.data = input;
|
||||
baseOffset = input.getFilePointer();
|
|
@ -1,11 +1,12 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
@ -30,30 +31,30 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
abstract class IndexDocValuesArray extends Source {
|
||||
abstract class DocValuesArray extends Source {
|
||||
|
||||
static final Map<ValueType, IndexDocValuesArray> TEMPLATES;
|
||||
static final Map<Type, DocValuesArray> TEMPLATES;
|
||||
|
||||
static {
|
||||
EnumMap<ValueType, IndexDocValuesArray> templates = new EnumMap<ValueType, IndexDocValuesArray>(
|
||||
ValueType.class);
|
||||
templates.put(ValueType.FIXED_INTS_16, new ShortValues());
|
||||
templates.put(ValueType.FIXED_INTS_32, new IntValues());
|
||||
templates.put(ValueType.FIXED_INTS_64, new LongValues());
|
||||
templates.put(ValueType.FIXED_INTS_8, new ByteValues());
|
||||
templates.put(ValueType.FLOAT_32, new FloatValues());
|
||||
templates.put(ValueType.FLOAT_64, new DoubleValues());
|
||||
EnumMap<Type, DocValuesArray> templates = new EnumMap<Type, DocValuesArray>(
|
||||
Type.class);
|
||||
templates.put(Type.FIXED_INTS_16, new ShortValues());
|
||||
templates.put(Type.FIXED_INTS_32, new IntValues());
|
||||
templates.put(Type.FIXED_INTS_64, new LongValues());
|
||||
templates.put(Type.FIXED_INTS_8, new ByteValues());
|
||||
templates.put(Type.FLOAT_32, new FloatValues());
|
||||
templates.put(Type.FLOAT_64, new DoubleValues());
|
||||
TEMPLATES = Collections.unmodifiableMap(templates);
|
||||
}
|
||||
|
||||
protected final int bytesPerValue;
|
||||
|
||||
IndexDocValuesArray(int bytesPerValue, ValueType type) {
|
||||
DocValuesArray(int bytesPerValue, Type type) {
|
||||
super(type);
|
||||
this.bytesPerValue = bytesPerValue;
|
||||
}
|
||||
|
||||
public abstract IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
public abstract DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException;
|
||||
|
||||
@Override
|
||||
|
@ -69,16 +70,16 @@ abstract class IndexDocValuesArray extends Source {
|
|||
BytesRefUtils.copyLong(bytesRef, Double.doubleToRawLongBits(value));
|
||||
}
|
||||
|
||||
final static class ByteValues extends IndexDocValuesArray {
|
||||
final static class ByteValues extends DocValuesArray {
|
||||
private final byte[] values;
|
||||
|
||||
ByteValues() {
|
||||
super(1, ValueType.FIXED_INTS_8);
|
||||
super(1, Type.FIXED_INTS_8);
|
||||
values = new byte[0];
|
||||
}
|
||||
|
||||
private ByteValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(1, ValueType.FIXED_INTS_8);
|
||||
super(1, Type.FIXED_INTS_8);
|
||||
values = new byte[numDocs];
|
||||
input.readBytes(values, 0, values.length, false);
|
||||
}
|
||||
|
@ -95,7 +96,7 @@ abstract class IndexDocValuesArray extends Source {
|
|||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new ByteValues(input, numDocs);
|
||||
}
|
||||
|
@ -106,16 +107,16 @@ abstract class IndexDocValuesArray extends Source {
|
|||
|
||||
};
|
||||
|
||||
final static class ShortValues extends IndexDocValuesArray {
|
||||
final static class ShortValues extends DocValuesArray {
|
||||
private final short[] values;
|
||||
|
||||
ShortValues() {
|
||||
super(RamUsageEstimator.NUM_BYTES_SHORT, ValueType.FIXED_INTS_16);
|
||||
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
|
||||
values = new short[0];
|
||||
}
|
||||
|
||||
private ShortValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(RamUsageEstimator.NUM_BYTES_SHORT, ValueType.FIXED_INTS_16);
|
||||
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
|
||||
values = new short[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readShort();
|
||||
|
@ -134,7 +135,7 @@ abstract class IndexDocValuesArray extends Source {
|
|||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new ShortValues(input, numDocs);
|
||||
}
|
||||
|
@ -145,16 +146,16 @@ abstract class IndexDocValuesArray extends Source {
|
|||
|
||||
};
|
||||
|
||||
final static class IntValues extends IndexDocValuesArray {
|
||||
final static class IntValues extends DocValuesArray {
|
||||
private final int[] values;
|
||||
|
||||
IntValues() {
|
||||
super(RamUsageEstimator.NUM_BYTES_INT, ValueType.FIXED_INTS_32);
|
||||
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
|
||||
values = new int[0];
|
||||
}
|
||||
|
||||
private IntValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(RamUsageEstimator.NUM_BYTES_INT, ValueType.FIXED_INTS_32);
|
||||
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
|
||||
values = new int[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readInt();
|
||||
|
@ -173,7 +174,7 @@ abstract class IndexDocValuesArray extends Source {
|
|||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new IntValues(input, numDocs);
|
||||
}
|
||||
|
@ -184,16 +185,16 @@ abstract class IndexDocValuesArray extends Source {
|
|||
|
||||
};
|
||||
|
||||
final static class LongValues extends IndexDocValuesArray {
|
||||
final static class LongValues extends DocValuesArray {
|
||||
private final long[] values;
|
||||
|
||||
LongValues() {
|
||||
super(RamUsageEstimator.NUM_BYTES_LONG, ValueType.FIXED_INTS_64);
|
||||
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
|
||||
values = new long[0];
|
||||
}
|
||||
|
||||
private LongValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(RamUsageEstimator.NUM_BYTES_LONG, ValueType.FIXED_INTS_64);
|
||||
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
|
||||
values = new long[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readLong();
|
||||
|
@ -212,23 +213,23 @@ abstract class IndexDocValuesArray extends Source {
|
|||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new LongValues(input, numDocs);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class FloatValues extends IndexDocValuesArray {
|
||||
final static class FloatValues extends DocValuesArray {
|
||||
private final float[] values;
|
||||
|
||||
FloatValues() {
|
||||
super(RamUsageEstimator.NUM_BYTES_FLOAT, ValueType.FLOAT_32);
|
||||
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
|
||||
values = new float[0];
|
||||
}
|
||||
|
||||
private FloatValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(RamUsageEstimator.NUM_BYTES_FLOAT, ValueType.FLOAT_32);
|
||||
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
|
||||
values = new float[numDocs];
|
||||
/*
|
||||
* we always read BIG_ENDIAN here since the writer serialized plain bytes
|
||||
|
@ -257,22 +258,22 @@ abstract class IndexDocValuesArray extends Source {
|
|||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new FloatValues(input, numDocs);
|
||||
}
|
||||
};
|
||||
|
||||
final static class DoubleValues extends IndexDocValuesArray {
|
||||
final static class DoubleValues extends DocValuesArray {
|
||||
private final double[] values;
|
||||
|
||||
DoubleValues() {
|
||||
super(RamUsageEstimator.NUM_BYTES_DOUBLE, ValueType.FLOAT_64);
|
||||
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
|
||||
values = new double[0];
|
||||
}
|
||||
|
||||
private DoubleValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(RamUsageEstimator.NUM_BYTES_DOUBLE, ValueType.FLOAT_64);
|
||||
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
|
||||
values = new double[numDocs];
|
||||
/*
|
||||
* we always read BIG_ENDIAN here since the writer serialized plain bytes
|
||||
|
@ -295,7 +296,7 @@ abstract class IndexDocValuesArray extends Source {
|
|||
}
|
||||
|
||||
@Override
|
||||
public IndexDocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new DoubleValues(input, numDocs);
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,9 +19,10 @@ package org.apache.lucene.index.values;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -70,7 +71,7 @@ class FixedDerefBytesImpl {
|
|||
private final int size;
|
||||
private final int numValuesStored;
|
||||
FixedDerefReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, ValueType.BYTES_FIXED_DEREF);
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, Type.BYTES_FIXED_DEREF);
|
||||
size = datIn.readInt();
|
||||
numValuesStored = idxIn.readInt();
|
||||
}
|
||||
|
@ -99,7 +100,7 @@ class FixedDerefBytesImpl {
|
|||
|
||||
protected FixedDerefSource(IndexInput datIn, IndexInput idxIn, int size, long numValues) throws IOException {
|
||||
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * numValues,
|
||||
ValueType.BYTES_FIXED_DEREF);
|
||||
Type.BYTES_FIXED_DEREF);
|
||||
this.size = size;
|
||||
addresses = PackedInts.getReader(idxIn);
|
||||
}
|
||||
|
@ -116,7 +117,7 @@ class FixedDerefBytesImpl {
|
|||
private final PackedInts.Reader index;
|
||||
private final int size;
|
||||
|
||||
DirectFixedDerefSource(IndexInput data, IndexInput index, int size, ValueType type)
|
||||
DirectFixedDerefSource(IndexInput data, IndexInput index, int size, Type type)
|
||||
throws IOException {
|
||||
super(data, type);
|
||||
this.size = size;
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -21,13 +21,16 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.SortedBytesMergeUtils;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.SortedBytesMergeUtils.MergeContext;
|
||||
import org.apache.lucene.index.SortedBytesMergeUtils.SortedSourceSlice;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.SortedBytesMergeUtils.MergeContext;
|
||||
import org.apache.lucene.index.values.SortedBytesMergeUtils.SortedSourceSlice;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesSortedSourceBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -59,11 +62,11 @@ class FixedSortedBytesImpl {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void merge(MergeState mergeState, IndexDocValues[] docValues)
|
||||
public void merge(MergeState mergeState, DocValues[] docValues)
|
||||
throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
final MergeContext ctx = SortedBytesMergeUtils.init(ValueType.BYTES_FIXED_SORTED, docValues, comp, mergeState);
|
||||
final MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_FIXED_SORTED, docValues, comp, mergeState);
|
||||
List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState, docValues, ctx);
|
||||
final IndexOutput datOut = getOrCreateDataOut();
|
||||
datOut.writeInt(ctx.sizePerValues);
|
||||
|
@ -122,7 +125,7 @@ class FixedSortedBytesImpl {
|
|||
private final Comparator<BytesRef> comparator;
|
||||
|
||||
public Reader(Directory dir, String id, int maxDoc, IOContext context,
|
||||
ValueType type, Comparator<BytesRef> comparator) throws IOException {
|
||||
Type type, Comparator<BytesRef> comparator) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, type);
|
||||
size = datIn.readInt();
|
||||
valueCount = idxIn.readInt();
|
||||
|
@ -153,7 +156,7 @@ class FixedSortedBytesImpl {
|
|||
|
||||
FixedSortedSource(IndexInput datIn, IndexInput idxIn, int size,
|
||||
int numValues, Comparator<BytesRef> comp) throws IOException {
|
||||
super(datIn, idxIn, comp, size * numValues, ValueType.BYTES_FIXED_SORTED,
|
||||
super(datIn, idxIn, comp, size * numValues, Type.BYTES_FIXED_SORTED,
|
||||
false);
|
||||
this.size = size;
|
||||
this.valueCount = numValues;
|
||||
|
@ -179,7 +182,7 @@ class FixedSortedBytesImpl {
|
|||
private final int valueCount;
|
||||
|
||||
DirectFixedSortedSource(IndexInput datIn, IndexInput idxIn, int size,
|
||||
int valueCount, Comparator<BytesRef> comp, ValueType type)
|
||||
int valueCount, Comparator<BytesRef> comp, Type type)
|
||||
throws IOException {
|
||||
super(type, comp);
|
||||
docToOrdIndex = PackedInts.getDirectReader(idxIn);
|
||||
|
@ -194,6 +197,11 @@ class FixedSortedBytesImpl {
|
|||
return (int) docToOrdIndex.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPackedDocToOrd() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PackedInts.Reader getDocToOrd() {
|
||||
return docToOrdIndex;
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -21,11 +21,12 @@ import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.index.values.DirectSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -179,7 +180,7 @@ class FixedStraightBytesImpl {
|
|||
}
|
||||
}
|
||||
|
||||
protected boolean tryBulkMerge(IndexDocValues docValues) {
|
||||
protected boolean tryBulkMerge(DocValues docValues) {
|
||||
return docValues instanceof FixedStraightReader;
|
||||
}
|
||||
|
||||
|
@ -255,10 +256,10 @@ class FixedStraightBytesImpl {
|
|||
protected final int maxDoc;
|
||||
|
||||
FixedStraightReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
|
||||
this(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, ValueType.BYTES_FIXED_STRAIGHT);
|
||||
this(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, Type.BYTES_FIXED_STRAIGHT);
|
||||
}
|
||||
|
||||
protected FixedStraightReader(Directory dir, String id, String codec, int version, int maxDoc, IOContext context, ValueType type) throws IOException {
|
||||
protected FixedStraightReader(Directory dir, String id, String codec, int version, int maxDoc, IOContext context, Type type) throws IOException {
|
||||
super(dir, id, codec, version, false, context, type);
|
||||
size = datIn.readInt();
|
||||
this.maxDoc = maxDoc;
|
||||
|
@ -291,7 +292,7 @@ class FixedStraightBytesImpl {
|
|||
private final byte[] data;
|
||||
|
||||
public SingleByteSource(IndexInput datIn, int maxDoc) throws IOException {
|
||||
super(ValueType.BYTES_FIXED_STRAIGHT);
|
||||
super(Type.BYTES_FIXED_STRAIGHT);
|
||||
try {
|
||||
data = new byte[maxDoc];
|
||||
datIn.readBytes(data, 0, data.length, false);
|
||||
|
@ -323,7 +324,7 @@ class FixedStraightBytesImpl {
|
|||
private final static class FixedStraightSource extends BytesSourceBase {
|
||||
private final int size;
|
||||
|
||||
public FixedStraightSource(IndexInput datIn, int size, int maxDoc, ValueType type)
|
||||
public FixedStraightSource(IndexInput datIn, int size, int maxDoc, Type type)
|
||||
throws IOException {
|
||||
super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc,
|
||||
type);
|
||||
|
@ -339,7 +340,7 @@ class FixedStraightBytesImpl {
|
|||
public final static class DirectFixedStraightSource extends DirectSource {
|
||||
private final int size;
|
||||
|
||||
DirectFixedStraightSource(IndexInput input, int size, ValueType type) {
|
||||
DirectFixedStraightSource(IndexInput input, int size, Type type) {
|
||||
super(input, type);
|
||||
this.size = size;
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -18,7 +18,10 @@ package org.apache.lucene.index.values;
|
|||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -42,16 +45,16 @@ public class Floats {
|
|||
protected static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
public static Writer getWriter(Directory dir, String id, Counter bytesUsed,
|
||||
IOContext context, ValueType type) throws IOException {
|
||||
IOContext context, Type type) throws IOException {
|
||||
return new FloatsWriter(dir, id, bytesUsed, context, type);
|
||||
}
|
||||
|
||||
public static IndexDocValues getValues(Directory dir, String id, int maxDoc, IOContext context, ValueType type)
|
||||
public static DocValues getValues(Directory dir, String id, int maxDoc, IOContext context, Type type)
|
||||
throws IOException {
|
||||
return new FloatsReader(dir, id, maxDoc, context, type);
|
||||
}
|
||||
|
||||
private static int typeToSize(ValueType type) {
|
||||
private static int typeToSize(Type type) {
|
||||
switch (type) {
|
||||
case FLOAT_32:
|
||||
return 4;
|
||||
|
@ -65,14 +68,14 @@ public class Floats {
|
|||
final static class FloatsWriter extends FixedStraightBytesImpl.Writer {
|
||||
|
||||
private final int size;
|
||||
private final IndexDocValuesArray template;
|
||||
private final DocValuesArray template;
|
||||
public FloatsWriter(Directory dir, String id, Counter bytesUsed,
|
||||
IOContext context, ValueType type) throws IOException {
|
||||
IOContext context, Type type) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
|
||||
size = typeToSize(type);
|
||||
this.bytesRef = new BytesRef(size);
|
||||
bytesRef.length = size;
|
||||
template = IndexDocValuesArray.TEMPLATES.get(type);
|
||||
template = DocValuesArray.TEMPLATES.get(type);
|
||||
assert template != null;
|
||||
}
|
||||
|
||||
|
@ -82,12 +85,12 @@ public class Floats {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, PerDocFieldValues docValues) throws IOException {
|
||||
add(docID, docValues.getFloat());
|
||||
public void add(int docID, DocValue docValue) throws IOException {
|
||||
add(docID, docValue.getFloat());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean tryBulkMerge(IndexDocValues docValues) {
|
||||
protected boolean tryBulkMerge(DocValues docValues) {
|
||||
// only bulk merge if value type is the same otherwise size differs
|
||||
return super.tryBulkMerge(docValues) && docValues.type() == template.type();
|
||||
}
|
||||
|
@ -100,11 +103,11 @@ public class Floats {
|
|||
}
|
||||
|
||||
final static class FloatsReader extends FixedStraightBytesImpl.FixedStraightReader {
|
||||
final IndexDocValuesArray arrayTemplate;
|
||||
FloatsReader(Directory dir, String id, int maxDoc, IOContext context, ValueType type)
|
||||
final DocValuesArray arrayTemplate;
|
||||
FloatsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, type);
|
||||
arrayTemplate = IndexDocValuesArray.TEMPLATES.get(type);
|
||||
arrayTemplate = DocValuesArray.TEMPLATES.get(type);
|
||||
assert size == 4 || size == 8;
|
||||
}
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,6 +19,9 @@ package org.apache.lucene.index.values;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -40,33 +43,33 @@ public final class Ints {
|
|||
}
|
||||
|
||||
public static Writer getWriter(Directory dir, String id, Counter bytesUsed,
|
||||
ValueType type, IOContext context) throws IOException {
|
||||
return type == ValueType.VAR_INTS ? new PackedIntValues.PackedIntsWriter(dir, id,
|
||||
Type type, IOContext context) throws IOException {
|
||||
return type == Type.VAR_INTS ? new PackedIntValues.PackedIntsWriter(dir, id,
|
||||
bytesUsed, context) : new IntsWriter(dir, id, bytesUsed, context, type);
|
||||
}
|
||||
|
||||
public static IndexDocValues getValues(Directory dir, String id, int numDocs,
|
||||
ValueType type, IOContext context) throws IOException {
|
||||
return type == ValueType.VAR_INTS ? new PackedIntValues.PackedIntsReader(dir, id,
|
||||
public static DocValues getValues(Directory dir, String id, int numDocs,
|
||||
Type type, IOContext context) throws IOException {
|
||||
return type == Type.VAR_INTS ? new PackedIntValues.PackedIntsReader(dir, id,
|
||||
numDocs, context) : new IntsReader(dir, id, numDocs, context, type);
|
||||
}
|
||||
|
||||
private static ValueType sizeToType(int size) {
|
||||
private static Type sizeToType(int size) {
|
||||
switch (size) {
|
||||
case 1:
|
||||
return ValueType.FIXED_INTS_8;
|
||||
return Type.FIXED_INTS_8;
|
||||
case 2:
|
||||
return ValueType.FIXED_INTS_16;
|
||||
return Type.FIXED_INTS_16;
|
||||
case 4:
|
||||
return ValueType.FIXED_INTS_32;
|
||||
return Type.FIXED_INTS_32;
|
||||
case 8:
|
||||
return ValueType.FIXED_INTS_64;
|
||||
return Type.FIXED_INTS_64;
|
||||
default:
|
||||
throw new IllegalStateException("illegal size " + size);
|
||||
}
|
||||
}
|
||||
|
||||
private static int typeToSize(ValueType type) {
|
||||
private static int typeToSize(Type type) {
|
||||
switch (type) {
|
||||
case FIXED_INTS_16:
|
||||
return 2;
|
||||
|
@ -83,20 +86,20 @@ public final class Ints {
|
|||
|
||||
|
||||
static class IntsWriter extends FixedStraightBytesImpl.Writer {
|
||||
private final IndexDocValuesArray template;
|
||||
private final DocValuesArray template;
|
||||
|
||||
public IntsWriter(Directory dir, String id, Counter bytesUsed,
|
||||
IOContext context, ValueType valueType) throws IOException {
|
||||
IOContext context, Type valueType) throws IOException {
|
||||
this(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, valueType);
|
||||
}
|
||||
|
||||
protected IntsWriter(Directory dir, String id, String codecName,
|
||||
int version, Counter bytesUsed, IOContext context, ValueType valueType) throws IOException {
|
||||
int version, Counter bytesUsed, IOContext context, Type valueType) throws IOException {
|
||||
super(dir, id, codecName, version, bytesUsed, context);
|
||||
size = typeToSize(valueType);
|
||||
this.bytesRef = new BytesRef(size);
|
||||
bytesRef.length = size;
|
||||
template = IndexDocValuesArray.TEMPLATES.get(valueType);
|
||||
template = DocValuesArray.TEMPLATES.get(valueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -106,8 +109,8 @@ public final class Ints {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, PerDocFieldValues docValues) throws IOException {
|
||||
add(docID, docValues.getInt());
|
||||
public void add(int docID, DocValue docValue) throws IOException {
|
||||
add(docID, docValue.getInt());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -117,20 +120,20 @@ public final class Ints {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected boolean tryBulkMerge(IndexDocValues docValues) {
|
||||
protected boolean tryBulkMerge(DocValues docValues) {
|
||||
// only bulk merge if value type is the same otherwise size differs
|
||||
return super.tryBulkMerge(docValues) && docValues.type() == template.type();
|
||||
}
|
||||
}
|
||||
|
||||
final static class IntsReader extends FixedStraightBytesImpl.FixedStraightReader {
|
||||
private final IndexDocValuesArray arrayTemplate;
|
||||
private final DocValuesArray arrayTemplate;
|
||||
|
||||
IntsReader(Directory dir, String id, int maxDoc, IOContext context, ValueType type)
|
||||
IntsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc,
|
||||
context, type);
|
||||
arrayTemplate = IndexDocValuesArray.TEMPLATES.get(type);
|
||||
arrayTemplate = DocValuesArray.TEMPLATES.get(type);
|
||||
assert arrayTemplate != null;
|
||||
assert type == sizeToType(size);
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -18,10 +18,13 @@ package org.apache.lucene.index.values;
|
|||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.values.FixedStraightBytesImpl.FixedBytesWriterBase;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.values.IndexDocValuesArray.LongValues;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.FixedStraightBytesImpl.FixedBytesWriterBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.DocValuesArray.LongValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -148,8 +151,8 @@ class PackedIntValues {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, PerDocFieldValues docValues) throws IOException {
|
||||
add(docID, docValues.getInt());
|
||||
public void add(int docID, DocValue docValue) throws IOException {
|
||||
add(docID, docValue.getInt());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -157,7 +160,7 @@ class PackedIntValues {
|
|||
* Opens all necessary files, but does not read any data in until you call
|
||||
* {@link #load}.
|
||||
*/
|
||||
static class PackedIntsReader extends IndexDocValues {
|
||||
static class PackedIntsReader extends DocValues {
|
||||
private final IndexInput datIn;
|
||||
private final byte type;
|
||||
private final int numDocs;
|
||||
|
@ -217,14 +220,14 @@ class PackedIntValues {
|
|||
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return ValueType.VAR_INTS;
|
||||
public Type type() {
|
||||
return Type.VAR_INTS;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource((IndexInput) datIn.clone(), 8, ValueType.FIXED_INTS_64) : new PackedIntsSource((IndexInput) datIn.clone(), true);
|
||||
return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource((IndexInput) datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource((IndexInput) datIn.clone(), true);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -235,7 +238,7 @@ class PackedIntValues {
|
|||
private final PackedInts.Reader values;
|
||||
|
||||
public PackedIntsSource(IndexInput dataIn, boolean direct) throws IOException {
|
||||
super(ValueType.VAR_INTS);
|
||||
super(Type.VAR_INTS);
|
||||
minValue = dataIn.readLong();
|
||||
defaultValue = dataIn.readLong();
|
||||
values = direct ? PackedInts.getDirectReader(dataIn) : PackedInts.getReader(dataIn);
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,9 +19,10 @@ package org.apache.lucene.index.values;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -92,7 +93,7 @@ class VarDerefBytesImpl {
|
|||
public static class VarDerefReader extends BytesReaderBase {
|
||||
private final long totalBytes;
|
||||
VarDerefReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, ValueType.BYTES_VAR_DEREF);
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, Type.BYTES_VAR_DEREF);
|
||||
totalBytes = idxIn.readLong();
|
||||
}
|
||||
|
||||
|
@ -114,7 +115,7 @@ class VarDerefBytesImpl {
|
|||
public VarDerefSource(IndexInput datIn, IndexInput idxIn, long totalBytes)
|
||||
throws IOException {
|
||||
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), totalBytes,
|
||||
ValueType.BYTES_VAR_DEREF);
|
||||
Type.BYTES_VAR_DEREF);
|
||||
addresses = PackedInts.getReader(idxIn);
|
||||
}
|
||||
|
||||
|
@ -129,7 +130,7 @@ class VarDerefBytesImpl {
|
|||
final static class DirectVarDerefSource extends DirectSource {
|
||||
private final PackedInts.Reader index;
|
||||
|
||||
DirectVarDerefSource(IndexInput data, IndexInput index, ValueType type)
|
||||
DirectVarDerefSource(IndexInput data, IndexInput index, Type type)
|
||||
throws IOException {
|
||||
super(data, type);
|
||||
this.index = PackedInts.getDirectReader(index);
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -21,13 +21,16 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.SortedBytesMergeUtils;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.SortedBytesMergeUtils.MergeContext;
|
||||
import org.apache.lucene.index.SortedBytesMergeUtils.SortedSourceSlice;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.SortedBytesMergeUtils.MergeContext;
|
||||
import org.apache.lucene.index.values.SortedBytesMergeUtils.SortedSourceSlice;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesSortedSourceBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -60,11 +63,11 @@ final class VarSortedBytesImpl {
|
|||
size = 0;
|
||||
}
|
||||
@Override
|
||||
public void merge(MergeState mergeState, IndexDocValues[] docValues)
|
||||
public void merge(MergeState mergeState, DocValues[] docValues)
|
||||
throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
MergeContext ctx = SortedBytesMergeUtils.init(ValueType.BYTES_VAR_SORTED, docValues, comp, mergeState);
|
||||
MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_VAR_SORTED, docValues, comp, mergeState);
|
||||
final List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState, docValues, ctx);
|
||||
IndexOutput datOut = getOrCreateDataOut();
|
||||
|
||||
|
@ -147,14 +150,14 @@ final class VarSortedBytesImpl {
|
|||
private final Comparator<BytesRef> comparator;
|
||||
|
||||
Reader(Directory dir, String id, int maxDoc,
|
||||
IOContext context, ValueType type, Comparator<BytesRef> comparator)
|
||||
IOContext context, Type type, Comparator<BytesRef> comparator)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, type);
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.lucene.index.values.IndexDocValues.Source load()
|
||||
public org.apache.lucene.index.DocValues.Source load()
|
||||
throws IOException {
|
||||
return new VarSortedSource(cloneData(), cloneIndex(), comparator);
|
||||
}
|
||||
|
@ -170,7 +173,7 @@ final class VarSortedBytesImpl {
|
|||
|
||||
VarSortedSource(IndexInput datIn, IndexInput idxIn,
|
||||
Comparator<BytesRef> comp) throws IOException {
|
||||
super(datIn, idxIn, comp, idxIn.readLong(), ValueType.BYTES_VAR_SORTED, true);
|
||||
super(datIn, idxIn, comp, idxIn.readLong(), Type.BYTES_VAR_SORTED, true);
|
||||
valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
|
||||
closeIndexInput();
|
||||
}
|
||||
|
@ -197,7 +200,7 @@ final class VarSortedBytesImpl {
|
|||
private final int valueCount;
|
||||
|
||||
DirectSortedSource(IndexInput datIn, IndexInput idxIn,
|
||||
Comparator<BytesRef> comparator, ValueType type) throws IOException {
|
||||
Comparator<BytesRef> comparator, Type type) throws IOException {
|
||||
super(type, comparator);
|
||||
idxIn.readLong();
|
||||
ordToOffsetIndex = PackedInts.getDirectReader(idxIn);
|
||||
|
@ -214,6 +217,11 @@ final class VarSortedBytesImpl {
|
|||
return (int) docToOrdIndex.get(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPackedDocToOrd() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PackedInts.Reader getDocToOrd() {
|
||||
return docToOrdIndex;
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,9 +19,10 @@ package org.apache.lucene.index.values;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -228,7 +229,7 @@ class VarStraightBytesImpl {
|
|||
private final int maxDoc;
|
||||
|
||||
VarStraightReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, ValueType.BYTES_VAR_STRAIGHT);
|
||||
super(dir, id, CODEC_NAME, VERSION_START, true, context, Type.BYTES_VAR_STRAIGHT);
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
|
@ -249,7 +250,7 @@ class VarStraightBytesImpl {
|
|||
|
||||
public VarStraightSource(IndexInput datIn, IndexInput idxIn) throws IOException {
|
||||
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), idxIn.readVLong(),
|
||||
ValueType.BYTES_VAR_STRAIGHT);
|
||||
Type.BYTES_VAR_STRAIGHT);
|
||||
addresses = PackedInts.getReader(idxIn);
|
||||
}
|
||||
|
||||
|
@ -265,7 +266,7 @@ class VarStraightBytesImpl {
|
|||
|
||||
private final PackedInts.Reader index;
|
||||
|
||||
DirectVarStraightSource(IndexInput data, IndexInput index, ValueType type)
|
||||
DirectVarStraightSource(IndexInput data, IndexInput index, Type type)
|
||||
throws IOException {
|
||||
super(data, type);
|
||||
index.readVLong();
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.values;
|
||||
package org.apache.lucene.index.codecs.lucene40.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,8 +19,9 @@ package org.apache.lucene.index.values;
|
|||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -41,6 +42,8 @@ import org.apache.lucene.util.Counter;
|
|||
*/
|
||||
public abstract class Writer extends DocValuesConsumer {
|
||||
protected Source currentMergeSource;
|
||||
protected final Counter bytesUsed;
|
||||
|
||||
/**
|
||||
* Creates a new {@link Writer}.
|
||||
*
|
||||
|
@ -50,7 +53,7 @@ public abstract class Writer extends DocValuesConsumer {
|
|||
* once {@link #finish(int)} has been called.
|
||||
*/
|
||||
protected Writer(Counter bytesUsed) {
|
||||
super(bytesUsed);
|
||||
this.bytesUsed = bytesUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -162,20 +165,20 @@ public abstract class Writer extends DocValuesConsumer {
|
|||
/**
|
||||
* Factory method to create a {@link Writer} instance for a given type. This
|
||||
* method returns default implementations for each of the different types
|
||||
* defined in the {@link ValueType} enumeration.
|
||||
* defined in the {@link Type} enumeration.
|
||||
*
|
||||
* @param type
|
||||
* the {@link ValueType} to create the {@link Writer} for
|
||||
* the {@link Type} to create the {@link Writer} for
|
||||
* @param id
|
||||
* the file name id used to create files within the writer.
|
||||
* @param directory
|
||||
* the {@link Directory} to create the files from.
|
||||
* @param bytesUsed
|
||||
* a byte-usage tracking reference
|
||||
* @return a new {@link Writer} instance for the given {@link ValueType}
|
||||
* @return a new {@link Writer} instance for the given {@link Type}
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Writer create(ValueType type, String id, Directory directory,
|
||||
public static Writer create(Type type, String id, Directory directory,
|
||||
Comparator<BytesRef> comp, Counter bytesUsed, IOContext context) throws IOException {
|
||||
if (comp == null) {
|
||||
comp = BytesRef.getUTF8SortedAsUnicodeComparator();
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.index.IndexFileNames;
|
|||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.codecs.DocValuesWriterBase;
|
||||
import org.apache.lucene.index.values.Writer;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Writer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
|
@ -53,7 +53,7 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
|||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
String filename = docValuesId(segmentInfo.name, fieldInfo.number);
|
||||
switch (fieldInfo.getDocValues()) {
|
||||
switch (fieldInfo.getDocValuesType()) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
|
|
|
@ -22,28 +22,28 @@ import java.util.Collection;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.DocValuesReaderBase;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Implementation of PerDocValues that uses separate files.
|
||||
* Implementation of PerDocProducer that uses separate files.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SepDocValuesProducer extends DocValuesReaderBase {
|
||||
private final TreeMap<String, IndexDocValues> docValues;
|
||||
private final TreeMap<String, DocValues> docValues;
|
||||
|
||||
/**
|
||||
* Creates a new {@link SepDocValuesProducer} instance and loads all
|
||||
* {@link IndexDocValues} instances for this segment and codec.
|
||||
* {@link DocValues} instances for this segment and codec.
|
||||
*/
|
||||
public SepDocValuesProducer(SegmentReadState state) throws IOException {
|
||||
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.context);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String,IndexDocValues> docValues() {
|
||||
protected Map<String,DocValues> docValues() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
|
|
|
@ -26,8 +26,8 @@ import org.apache.lucene.index.FieldInfos;
|
|||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.codecs.FieldInfosReader;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -98,12 +98,12 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
|
|||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch, DOCVALUES);
|
||||
String dvType = readString(DOCVALUES.length, scratch);
|
||||
final ValueType docValuesType;
|
||||
final DocValues.Type docValuesType;
|
||||
|
||||
if ("false".equals(dvType)) {
|
||||
docValuesType = null;
|
||||
} else {
|
||||
docValuesType = ValueType.valueOf(dvType);
|
||||
docValuesType = DocValues.Type.valueOf(dvType);
|
||||
}
|
||||
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
|
|
|
@ -100,7 +100,7 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
|
|||
if (!fi.hasDocValues()) {
|
||||
SimpleTextUtil.write(out, "false", scratch);
|
||||
} else {
|
||||
SimpleTextUtil.write(out, fi.getDocValues().toString(), scratch);
|
||||
SimpleTextUtil.write(out, fi.getDocValuesType().toString(), scratch);
|
||||
}
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
|
||||
|
|
|
@ -1,414 +0,0 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* {@link IndexDocValues} provides a dense per-document typed storage for fast
|
||||
* value access based on the lucene internal document id. {@link IndexDocValues}
|
||||
* exposes two distinct APIs:
|
||||
* <ul>
|
||||
* <li>via {@link #getSource()} providing RAM resident random access</li>
|
||||
* <li>via {@link #getDirectSource()} providing on disk random access</li>
|
||||
* </ul> {@link IndexDocValues} are exposed via
|
||||
* {@link IndexReader#perDocValues()} on a per-segment basis. For best
|
||||
* performance {@link IndexDocValues} should be consumed per-segment just like
|
||||
* IndexReader.
|
||||
* <p>
|
||||
* {@link IndexDocValues} are fully integrated into the {@link DocValuesFormat} API.
|
||||
*
|
||||
* @see ValueType for limitations and default implementation documentation
|
||||
* @see IndexDocValuesField for adding values to the index
|
||||
* @see DocValuesFormat#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
|
||||
* customization
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class IndexDocValues implements Closeable {
|
||||
|
||||
public static final IndexDocValues[] EMPTY_ARRAY = new IndexDocValues[0];
|
||||
|
||||
private volatile SourceCache cache = new SourceCache.DirectSourceCache();
|
||||
private final Object cacheLock = new Object();
|
||||
|
||||
/**
|
||||
* Loads a new {@link Source} instance for this {@link IndexDocValues} field
|
||||
* instance. Source instances returned from this method are not cached. It is
|
||||
* the callers responsibility to maintain the instance and release its
|
||||
* resources once the source is not needed anymore.
|
||||
* <p>
|
||||
* For managed {@link Source} instances see {@link #getSource()}.
|
||||
*
|
||||
* @see #getSource()
|
||||
* @see #setCache(SourceCache)
|
||||
*/
|
||||
public abstract Source load() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link Source} instance through the current {@link SourceCache}.
|
||||
* Iff no {@link Source} has been loaded into the cache so far the source will
|
||||
* be loaded through {@link #load()} and passed to the {@link SourceCache}.
|
||||
* The caller of this method should not close the obtained {@link Source}
|
||||
* instance unless it is not needed for the rest of its life time.
|
||||
* <p>
|
||||
* {@link Source} instances obtained from this method are closed / released
|
||||
* from the cache once this {@link IndexDocValues} instance is closed by the
|
||||
* {@link IndexReader}, {@link Fields} or {@link FieldsEnum} the
|
||||
* {@link IndexDocValues} was created from.
|
||||
*/
|
||||
public Source getSource() throws IOException {
|
||||
return cache.load(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a disk resident {@link Source} instance. Direct Sources are not
|
||||
* cached in the {@link SourceCache} and should not be shared between threads.
|
||||
*/
|
||||
public abstract Source getDirectSource() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType} of this {@link IndexDocValues} instance
|
||||
*/
|
||||
public abstract ValueType type();
|
||||
|
||||
/**
|
||||
* Closes this {@link IndexDocValues} instance. This method should only be called
|
||||
* by the creator of this {@link IndexDocValues} instance. API users should not
|
||||
* close {@link IndexDocValues} instances.
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
cache.close(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the size per value in bytes or <code>-1</code> iff size per value
|
||||
* is variable.
|
||||
*
|
||||
* @return the size per value in bytes or <code>-1</code> iff size per value
|
||||
* is variable.
|
||||
*/
|
||||
public int getValueSize() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link SourceCache} used by this {@link IndexDocValues} instance. This
|
||||
* method should be called before {@link #load()} is called. All {@link Source} instances in the currently used cache will be closed
|
||||
* before the new cache is installed.
|
||||
* <p>
|
||||
* Note: All instances previously obtained from {@link #load()} will be lost.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* if the given cache is <code>null</code>
|
||||
*
|
||||
*/
|
||||
public void setCache(SourceCache cache) {
|
||||
if (cache == null)
|
||||
throw new IllegalArgumentException("cache must not be null");
|
||||
synchronized (cacheLock) {
|
||||
SourceCache toClose = this.cache;
|
||||
this.cache = cache;
|
||||
toClose.close(this);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Source of per document values like long, double or {@link BytesRef}
|
||||
* depending on the {@link IndexDocValues} fields {@link ValueType}. Source
|
||||
* implementations provide random access semantics similar to array lookups
|
||||
* <p>
|
||||
* @see IndexDocValues#getSource()
|
||||
* @see IndexDocValues#getDirectSource()
|
||||
*/
|
||||
public static abstract class Source {
|
||||
|
||||
protected final ValueType type;
|
||||
|
||||
protected Source(ValueType type) {
|
||||
this.type = type;
|
||||
}
|
||||
/**
|
||||
* Returns a <tt>long</tt> for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>long</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>long</tt> values.
|
||||
*/
|
||||
public long getInt(int docID) {
|
||||
throw new UnsupportedOperationException("ints are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a <tt>double</tt> for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>double</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>double</tt> values.
|
||||
*/
|
||||
public double getFloat(int docID) {
|
||||
throw new UnsupportedOperationException("floats are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a {@link BytesRef} for the given document id or throws an
|
||||
* {@link UnsupportedOperationException} if this source doesn't support
|
||||
* <tt>byte[]</tt> values.
|
||||
* @throws IOException
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this source doesn't support <tt>byte[]</tt> values.
|
||||
*/
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
throw new UnsupportedOperationException("bytes are not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link ValueType} of this source.
|
||||
*
|
||||
* @return the {@link ValueType} of this source.
|
||||
*/
|
||||
public ValueType type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff this {@link Source} exposes an array via
|
||||
* {@link #getArray()} otherwise <code>false</code>.
|
||||
*
|
||||
* @return <code>true</code> iff this {@link Source} exposes an array via
|
||||
* {@link #getArray()} otherwise <code>false</code>.
|
||||
*/
|
||||
public boolean hasArray() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the internal array representation iff this {@link Source} uses an
|
||||
* array as its inner representation, otherwise <code>UOE</code>.
|
||||
*/
|
||||
public Object getArray() {
|
||||
throw new UnsupportedOperationException("getArray is not supported");
|
||||
}
|
||||
|
||||
/**
|
||||
* If this {@link Source} is sorted this method will return an instance of
|
||||
* {@link SortedSource} otherwise <code>UOE</code>
|
||||
*/
|
||||
public SortedSource asSortedSource() {
|
||||
throw new UnsupportedOperationException("asSortedSource is not supported");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A sorted variant of {@link Source} for <tt>byte[]</tt> values per document.
|
||||
* <p>
|
||||
*/
|
||||
public static abstract class SortedSource extends Source {
|
||||
|
||||
private final Comparator<BytesRef> comparator;
|
||||
|
||||
protected SortedSource(ValueType type, Comparator<BytesRef> comparator) {
|
||||
super(type);
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final int ord = ord(docID);
|
||||
if (ord < 0) {
|
||||
bytesRef.length = 0;
|
||||
} else {
|
||||
getByOrd(ord , bytesRef);
|
||||
}
|
||||
return bytesRef;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns ord for specified docID. Ord is dense, ie, starts at 0, then increments by 1
|
||||
* for the next (as defined by {@link Comparator} value.
|
||||
*/
|
||||
public abstract int ord(int docID);
|
||||
|
||||
/** Returns value for specified ord. */
|
||||
public abstract BytesRef getByOrd(int ord, BytesRef bytesRef);
|
||||
|
||||
/**
|
||||
* Returns the PackedInts.Reader impl that maps document to ord.
|
||||
*/
|
||||
public abstract PackedInts.Reader getDocToOrd();
|
||||
|
||||
/**
|
||||
* Returns the comparator used to order the BytesRefs.
|
||||
*/
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs a lookup by value.
|
||||
*
|
||||
* @param value
|
||||
* the value to look up
|
||||
* @param spare
|
||||
* a spare {@link BytesRef} instance used to compare internal
|
||||
* values to the given value. Must not be <code>null</code>
|
||||
* @return the given values ordinal if found or otherwise
|
||||
* <code>(-(ord)-1)</code>, defined as the ordinal of the first
|
||||
* element that is greater than the given value. This guarantees
|
||||
* that the return value will always be >= 0 if the given value
|
||||
* is found.
|
||||
*/
|
||||
public int getByValue(BytesRef value, BytesRef spare) {
|
||||
return binarySearch(value, spare, 0, getValueCount() - 1);
|
||||
}
|
||||
|
||||
private int binarySearch(BytesRef b, BytesRef bytesRef, int low,
|
||||
int high) {
|
||||
int mid = 0;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
getByOrd(mid, bytesRef);
|
||||
final int cmp = comparator.compare(bytesRef, b);
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
return mid;
|
||||
}
|
||||
}
|
||||
assert comparator.compare(bytesRef, b) != 0;
|
||||
return -(low + 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSource asSortedSource() {
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of unique values in this sorted source
|
||||
*/
|
||||
public abstract int getValueCount();
|
||||
}
|
||||
|
||||
/** Returns a Source that always returns default (missing)
|
||||
* values for all documents. */
|
||||
public static Source getDefaultSource(final ValueType type) {
|
||||
return new Source(type) {
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
return 0.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/** Returns a SortedSource that always returns default (missing)
|
||||
* values for all documents. */
|
||||
public static SortedSource getDefaultSortedSource(final ValueType type, final int size) {
|
||||
|
||||
final PackedInts.Reader docToOrd = new PackedInts.Reader() {
|
||||
@Override
|
||||
public long get(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBitsPerValue() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getArray() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
return new SortedSource(type, BytesRef.getUTF8SortedAsUnicodeComparator()) {
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int ord(int docID) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
|
||||
assert ord == 0;
|
||||
bytesRef.length = 0;
|
||||
return bytesRef;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PackedInts.Reader getDocToOrd() {
|
||||
return docToOrd;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getByValue(BytesRef value, BytesRef spare) {
|
||||
if (value.length == 0) {
|
||||
return 0;
|
||||
} else {
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return 1;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -1,211 +0,0 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
/**
|
||||
* A wrapper for compound IndexReader providing access to per segment
|
||||
* {@link IndexDocValues}
|
||||
*
|
||||
* @lucene.experimental
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class MultiIndexDocValues extends IndexDocValues {
|
||||
|
||||
public static class DocValuesIndex {
|
||||
public final static DocValuesIndex[] EMPTY_ARRAY = new DocValuesIndex[0];
|
||||
final int start;
|
||||
final int length;
|
||||
final IndexDocValues docValues;
|
||||
|
||||
public DocValuesIndex(IndexDocValues docValues, int start, int length) {
|
||||
this.docValues = docValues;
|
||||
this.start = start;
|
||||
this.length = length;
|
||||
}
|
||||
}
|
||||
|
||||
private DocValuesIndex[] docValuesIdx;
|
||||
private int[] starts;
|
||||
private ValueType type;
|
||||
private int valueSize;
|
||||
|
||||
public MultiIndexDocValues() {
|
||||
starts = new int[0];
|
||||
docValuesIdx = new DocValuesIndex[0];
|
||||
}
|
||||
|
||||
public MultiIndexDocValues(DocValuesIndex[] docValuesIdx) {
|
||||
reset(docValuesIdx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return new MultiSource(docValuesIdx, starts, false);
|
||||
}
|
||||
|
||||
public IndexDocValues reset(DocValuesIndex[] docValuesIdx) {
|
||||
final int[] start = new int[docValuesIdx.length];
|
||||
TypePromoter promoter = TypePromoter.getIdentityPromoter();
|
||||
for (int i = 0; i < docValuesIdx.length; i++) {
|
||||
start[i] = docValuesIdx[i].start;
|
||||
if (!(docValuesIdx[i].docValues instanceof EmptyDocValues)) {
|
||||
// only promote if not a dummy
|
||||
final TypePromoter incomingPromoter = TypePromoter.create(
|
||||
docValuesIdx[i].docValues.type(),
|
||||
docValuesIdx[i].docValues.getValueSize());
|
||||
promoter = promoter.promote(incomingPromoter);
|
||||
if (promoter == null) {
|
||||
throw new IllegalStateException("Can not promote " + incomingPromoter);
|
||||
}
|
||||
}
|
||||
}
|
||||
this.type = promoter.type();
|
||||
this.valueSize = promoter.getValueSize();
|
||||
this.starts = start;
|
||||
this.docValuesIdx = docValuesIdx;
|
||||
return this;
|
||||
}
|
||||
|
||||
public static class EmptyDocValues extends IndexDocValues {
|
||||
final int maxDoc;
|
||||
final Source emptySource;
|
||||
|
||||
public EmptyDocValues(int maxDoc, ValueType type) {
|
||||
this.maxDoc = maxDoc;
|
||||
this.emptySource = new EmptySource(type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return emptySource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return emptySource.type();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return emptySource;
|
||||
}
|
||||
}
|
||||
|
||||
private static class MultiSource extends Source {
|
||||
private int numDocs = 0;
|
||||
private int start = 0;
|
||||
private Source current;
|
||||
private final int[] starts;
|
||||
private final DocValuesIndex[] docValuesIdx;
|
||||
private boolean direct;
|
||||
|
||||
public MultiSource(DocValuesIndex[] docValuesIdx, int[] starts, boolean direct) {
|
||||
super(docValuesIdx[0].docValues.type());
|
||||
this.docValuesIdx = docValuesIdx;
|
||||
this.starts = starts;
|
||||
assert docValuesIdx.length != 0;
|
||||
this.direct = direct;
|
||||
}
|
||||
|
||||
public long getInt(int docID) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getInt(doc);
|
||||
}
|
||||
|
||||
private final int ensureSource(int docID) {
|
||||
if (docID >= start && docID < start+numDocs) {
|
||||
return docID - start;
|
||||
} else {
|
||||
final int idx = ReaderUtil.subIndex(docID, starts);
|
||||
assert idx >= 0 && idx < docValuesIdx.length : "idx was " + idx
|
||||
+ " for doc id: " + docID + " slices : " + Arrays.toString(starts);
|
||||
assert docValuesIdx[idx] != null;
|
||||
try {
|
||||
if (direct) {
|
||||
current = docValuesIdx[idx].docValues.getDirectSource();
|
||||
} else {
|
||||
current = docValuesIdx[idx].docValues.getSource();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("load failed", e); // TODO how should we
|
||||
// handle this
|
||||
}
|
||||
|
||||
start = docValuesIdx[idx].start;
|
||||
numDocs = docValuesIdx[idx].length;
|
||||
return docID - start;
|
||||
}
|
||||
}
|
||||
|
||||
public double getFloat(int docID) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getFloat(doc);
|
||||
}
|
||||
|
||||
public BytesRef getBytes(int docID, BytesRef bytesRef) {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getBytes(doc, bytesRef);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: this is dup of IndexDocValues.getDefaultSource()?
|
||||
private static class EmptySource extends Source {
|
||||
|
||||
public EmptySource(ValueType type) {
|
||||
super(type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
return 0d;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueSize() {
|
||||
return valueSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return new MultiSource(docValuesIdx, starts, true);
|
||||
}
|
||||
}
|
|
@ -1,93 +0,0 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
|
||||
/**
|
||||
* Abstract base class for {@link IndexDocValues} {@link Source} cache.
|
||||
* <p>
|
||||
* {@link Source} instances loaded via {@link IndexDocValues#load()} are entirely memory resident
|
||||
* and need to be maintained by the caller. Each call to
|
||||
* {@link IndexDocValues#load()} will cause an entire reload of
|
||||
* the underlying data. Source instances obtained from
|
||||
* {@link IndexDocValues#getSource()} and {@link IndexDocValues#getSource()}
|
||||
* respectively are maintained by a {@link SourceCache} that is closed (
|
||||
* {@link #close(IndexDocValues)}) once the {@link IndexReader} that created the
|
||||
* {@link IndexDocValues} instance is closed.
|
||||
* <p>
|
||||
* Unless {@link Source} instances are managed by another entity it is
|
||||
* recommended to use the cached variants to obtain a source instance.
|
||||
* <p>
|
||||
* Implementation of this API must be thread-safe.
|
||||
*
|
||||
* @see IndexDocValues#setCache(SourceCache)
|
||||
* @see IndexDocValues#getSource()
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class SourceCache {
|
||||
|
||||
/**
|
||||
* Atomically loads a {@link Source} into the cache from the given
|
||||
* {@link IndexDocValues} and returns it iff no other {@link Source} has already
|
||||
* been cached. Otherwise the cached source is returned.
|
||||
* <p>
|
||||
* This method will not return <code>null</code>
|
||||
*/
|
||||
public abstract Source load(IndexDocValues values) throws IOException;
|
||||
|
||||
/**
|
||||
* Atomically invalidates the cached {@link Source}
|
||||
* instances if any and empties the cache.
|
||||
*/
|
||||
public abstract void invalidate(IndexDocValues values);
|
||||
|
||||
/**
|
||||
* Atomically closes the cache and frees all resources.
|
||||
*/
|
||||
public synchronized void close(IndexDocValues values) {
|
||||
invalidate(values);
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple per {@link IndexDocValues} instance cache implementation that holds a
|
||||
* {@link Source} a member variable.
|
||||
* <p>
|
||||
* If a {@link DirectSourceCache} instance is closed or invalidated the cached
|
||||
* reference are simply set to <code>null</code>
|
||||
*/
|
||||
public static final class DirectSourceCache extends SourceCache {
|
||||
private Source ref;
|
||||
|
||||
public synchronized Source load(IndexDocValues values) throws IOException {
|
||||
if (ref == null) {
|
||||
ref = values.load();
|
||||
}
|
||||
return ref;
|
||||
}
|
||||
|
||||
public synchronized void invalidate(IndexDocValues values) {
|
||||
ref = null;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,220 +0,0 @@
|
|||
package org.apache.lucene.index.values;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* <code>ValueType</code> specifies the {@link IndexDocValues} type for a
|
||||
* certain field. A <code>ValueType</code> only defines the data type for a field
|
||||
* while the actual implementation used to encode and decode the values depends
|
||||
* on the the {@link DocValuesFormat#docsConsumer} and {@link DocValuesFormat#docsProducer} methods.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public enum ValueType {
|
||||
|
||||
/**
|
||||
* A variable bit signed integer value. By default this type uses
|
||||
* {@link PackedInts} to compress the values, as an offset
|
||||
* from the minimum value, as long as the value range
|
||||
* fits into 2<sup>63</sup>-1. Otherwise,
|
||||
* the default implementation falls back to fixed size 64bit
|
||||
* integers ({@link #FIXED_INTS_64}).
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
VAR_INTS,
|
||||
|
||||
/**
|
||||
* A 8 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>byte</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_8,
|
||||
|
||||
/**
|
||||
* A 16 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>short</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_16,
|
||||
|
||||
/**
|
||||
* A 32 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>int</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_32,
|
||||
|
||||
/**
|
||||
* A 64 bit signed integer value. {@link Source} instances of
|
||||
* this type return a <tt>long</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0</tt> as the default value without any
|
||||
* distinction between provided <tt>0</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FIXED_INTS_64,
|
||||
/**
|
||||
* A 32 bit floating point value. By default there is no compression
|
||||
* applied. To fit custom float values into less than 32bit either a custom
|
||||
* implementation is needed or values must be encoded into a
|
||||
* {@link #BYTES_FIXED_STRAIGHT} type. {@link Source} instances of
|
||||
* this type return a <tt>float</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0.0f</tt> as the default value without any
|
||||
* distinction between provided <tt>0.0f</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0.0f</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FLOAT_32,
|
||||
/**
|
||||
*
|
||||
* A 64 bit floating point value. By default there is no compression
|
||||
* applied. To fit custom float values into less than 64bit either a custom
|
||||
* implementation is needed or values must be encoded into a
|
||||
* {@link #BYTES_FIXED_STRAIGHT} type. {@link Source} instances of
|
||||
* this type return a <tt>double</tt> array from {@link Source#getArray()}
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0.0d</tt> as the default value without any
|
||||
* distinction between provided <tt>0.0d</tt> values during indexing. All
|
||||
* documents without an explicit value will use <tt>0.0d</tt> instead.
|
||||
* Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
FLOAT_64,
|
||||
|
||||
// TODO(simonw): -- shouldn't lucene decide/detect straight vs
|
||||
// deref, as well fixed vs var?
|
||||
/**
|
||||
* A fixed length straight byte[]. All values added to
|
||||
* such a field must be of the same length. All bytes are stored sequentially
|
||||
* for fast offset access.
|
||||
* <p>
|
||||
* NOTE: this type uses <tt>0 byte</tt> filled byte[] based on the length of the first seen
|
||||
* value as the default value without any distinction between explicitly
|
||||
* provided values during indexing. All documents without an explicit value
|
||||
* will use the default instead.Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_FIXED_STRAIGHT,
|
||||
|
||||
/**
|
||||
* A fixed length dereferenced byte[] variant. Fields with
|
||||
* this type only store distinct byte values and store an additional offset
|
||||
* pointer per document to dereference the shared byte[].
|
||||
* Use this type if your documents may share the same byte[].
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_FIXED_DEREF,
|
||||
|
||||
/**
|
||||
* Variable length straight stored byte[] variant. All bytes are
|
||||
* stored sequentially for compactness. Usage of this type via the
|
||||
* disk-resident API might yield performance degradation since no additional
|
||||
* index is used to advance by more than one document value at a time.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without an
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* byte[] reference. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_VAR_STRAIGHT,
|
||||
|
||||
/**
|
||||
* A variable length dereferenced byte[]. Just like
|
||||
* {@link #BYTES_FIXED_DEREF}, but allowing each
|
||||
* document's value to be a different length.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*/
|
||||
BYTES_VAR_DEREF,
|
||||
|
||||
|
||||
/**
|
||||
* A variable length pre-sorted byte[] variant. Just like
|
||||
* {@link #BYTES_FIXED_SORTED}, but allowing each
|
||||
* document's value to be a different length.
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference.Custom default values must be assigned explicitly.
|
||||
* </p>
|
||||
*
|
||||
* @see SortedSource
|
||||
*/
|
||||
BYTES_VAR_SORTED,
|
||||
|
||||
/**
|
||||
* A fixed length pre-sorted byte[] variant. Fields with this type only
|
||||
* store distinct byte values and store an additional offset pointer per
|
||||
* document to dereference the shared byte[]. The stored
|
||||
* byte[] is presorted, by default by unsigned byte order,
|
||||
* and allows access via document id, ordinal and by-value.
|
||||
* Use this type if your documents may share the same byte[].
|
||||
* <p>
|
||||
* NOTE: Fields of this type will not store values for documents without and
|
||||
* explicitly provided value. If a documents value is accessed while no
|
||||
* explicit value is stored the returned {@link BytesRef} will be a 0-length
|
||||
* reference. Custom default values must be assigned
|
||||
* explicitly.
|
||||
* </p>
|
||||
*
|
||||
* @see SortedSource
|
||||
*/
|
||||
BYTES_FIXED_SORTED
|
||||
|
||||
}
|
|
@ -21,11 +21,8 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.search.FieldCache.ByteParser;
|
||||
import org.apache.lucene.search.FieldCache.DocTerms;
|
||||
import org.apache.lucene.search.FieldCache.DocTermsIndex;
|
||||
|
@ -360,7 +357,7 @@ public abstract class FieldComparator<T> {
|
|||
public static final class FloatDocValuesComparator extends FieldComparator<Double> {
|
||||
private final double[] values;
|
||||
private final String field;
|
||||
private Source currentReaderValues;
|
||||
private DocValues.Source currentReaderValues;
|
||||
private double bottom;
|
||||
|
||||
FloatDocValuesComparator(int numHits, String field) {
|
||||
|
@ -400,11 +397,11 @@ public abstract class FieldComparator<T> {
|
|||
|
||||
@Override
|
||||
public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
|
||||
final IndexDocValues docValues = context.reader.docValues(field);
|
||||
final DocValues docValues = context.reader.docValues(field);
|
||||
if (docValues != null) {
|
||||
currentReaderValues = docValues.getSource();
|
||||
} else {
|
||||
currentReaderValues = IndexDocValues.getDefaultSource(ValueType.FLOAT_64);
|
||||
currentReaderValues = DocValues.getDefaultSource(DocValues.Type.FLOAT_64);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
@ -648,7 +645,7 @@ public abstract class FieldComparator<T> {
|
|||
/** Loads int index values and sorts by ascending value. */
|
||||
public static final class IntDocValuesComparator extends FieldComparator<Long> {
|
||||
private final long[] values;
|
||||
private Source currentReaderValues;
|
||||
private DocValues.Source currentReaderValues;
|
||||
private final String field;
|
||||
private long bottom;
|
||||
|
||||
|
@ -693,11 +690,11 @@ public abstract class FieldComparator<T> {
|
|||
|
||||
@Override
|
||||
public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
|
||||
IndexDocValues docValues = context.reader.docValues(field);
|
||||
DocValues docValues = context.reader.docValues(field);
|
||||
if (docValues != null) {
|
||||
currentReaderValues = docValues.getSource();
|
||||
} else {
|
||||
currentReaderValues = IndexDocValues.getDefaultSource(ValueType.FIXED_INTS_64);
|
||||
currentReaderValues = DocValues.getDefaultSource(DocValues.Type.FIXED_INTS_64);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
@ -1382,7 +1379,7 @@ public abstract class FieldComparator<T> {
|
|||
|
||||
/* Current reader's doc ord/values.
|
||||
@lucene.internal */
|
||||
SortedSource termsIndex;
|
||||
DocValues.SortedSource termsIndex;
|
||||
|
||||
/* Comparator for comparing by value.
|
||||
@lucene.internal */
|
||||
|
@ -1490,10 +1487,10 @@ public abstract class FieldComparator<T> {
|
|||
// Used per-segment when bit width of doc->ord is 8:
|
||||
private final class ByteOrdComparator extends PerSegmentComparator {
|
||||
private final byte[] readerOrds;
|
||||
private final SortedSource termsIndex;
|
||||
private final DocValues.SortedSource termsIndex;
|
||||
private final int docBase;
|
||||
|
||||
public ByteOrdComparator(byte[] readerOrds, SortedSource termsIndex, int docBase) {
|
||||
public ByteOrdComparator(byte[] readerOrds, DocValues.SortedSource termsIndex, int docBase) {
|
||||
this.readerOrds = readerOrds;
|
||||
this.termsIndex = termsIndex;
|
||||
this.docBase = docBase;
|
||||
|
@ -1535,10 +1532,10 @@ public abstract class FieldComparator<T> {
|
|||
// Used per-segment when bit width of doc->ord is 16:
|
||||
private final class ShortOrdComparator extends PerSegmentComparator {
|
||||
private final short[] readerOrds;
|
||||
private final SortedSource termsIndex;
|
||||
private final DocValues.SortedSource termsIndex;
|
||||
private final int docBase;
|
||||
|
||||
public ShortOrdComparator(short[] readerOrds, SortedSource termsIndex, int docBase) {
|
||||
public ShortOrdComparator(short[] readerOrds, DocValues.SortedSource termsIndex, int docBase) {
|
||||
this.readerOrds = readerOrds;
|
||||
this.termsIndex = termsIndex;
|
||||
this.docBase = docBase;
|
||||
|
@ -1580,10 +1577,10 @@ public abstract class FieldComparator<T> {
|
|||
// Used per-segment when bit width of doc->ord is 32:
|
||||
private final class IntOrdComparator extends PerSegmentComparator {
|
||||
private final int[] readerOrds;
|
||||
private final SortedSource termsIndex;
|
||||
private final DocValues.SortedSource termsIndex;
|
||||
private final int docBase;
|
||||
|
||||
public IntOrdComparator(int[] readerOrds, SortedSource termsIndex, int docBase) {
|
||||
public IntOrdComparator(int[] readerOrds, DocValues.SortedSource termsIndex, int docBase) {
|
||||
this.readerOrds = readerOrds;
|
||||
this.termsIndex = termsIndex;
|
||||
this.docBase = docBase;
|
||||
|
@ -1623,11 +1620,11 @@ public abstract class FieldComparator<T> {
|
|||
|
||||
// Used per-segment when bit width is not a native array
|
||||
// size (8, 16, 32):
|
||||
private final class AnyOrdComparator extends PerSegmentComparator {
|
||||
private final class AnyPackedDocToOrdComparator extends PerSegmentComparator {
|
||||
private final PackedInts.Reader readerOrds;
|
||||
private final int docBase;
|
||||
|
||||
public AnyOrdComparator(PackedInts.Reader readerOrds, int docBase) {
|
||||
public AnyPackedDocToOrdComparator(PackedInts.Reader readerOrds, int docBase) {
|
||||
this.readerOrds = readerOrds;
|
||||
this.docBase = docBase;
|
||||
}
|
||||
|
@ -1664,16 +1661,57 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
}
|
||||
|
||||
// Used per-segment when DV doesn't use packed ints for
|
||||
// docToOrds:
|
||||
private final class AnyOrdComparator extends PerSegmentComparator {
|
||||
private final int docBase;
|
||||
|
||||
public AnyOrdComparator(int docBase) {
|
||||
this.docBase = docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
assert bottomSlot != -1;
|
||||
if (bottomSameReader) {
|
||||
// ord is precisely comparable, even in the equal case
|
||||
return bottomOrd - termsIndex.ord(doc);
|
||||
} else {
|
||||
// ord is only approx comparable: if they are not
|
||||
// equal, we can use that; if they are equal, we
|
||||
// must fallback to compare by value
|
||||
final int order = termsIndex.ord(doc);
|
||||
final int cmp = bottomOrd - order;
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
}
|
||||
termsIndex.getByOrd(order, tempBR);
|
||||
return comp.compare(bottomValue, tempBR);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
final int ord = termsIndex.ord(doc);
|
||||
ords[slot] = ord;
|
||||
if (values[slot] == null) {
|
||||
values[slot] = new BytesRef();
|
||||
}
|
||||
termsIndex.getByOrd(ord, values[slot]);
|
||||
readerGen[slot] = currentReaderGen;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
|
||||
final int docBase = context.docBase;
|
||||
|
||||
final IndexDocValues dv = context.reader.docValues(field);
|
||||
final DocValues dv = context.reader.docValues(field);
|
||||
if (dv == null) {
|
||||
// This may mean entire segment had no docs with
|
||||
// this DV field; use default field value (empty
|
||||
// byte[]) in this case:
|
||||
termsIndex = IndexDocValues.getDefaultSortedSource(ValueType.BYTES_VAR_SORTED, context.reader.maxDoc());
|
||||
termsIndex = DocValues.getDefaultSortedSource(DocValues.Type.BYTES_VAR_SORTED, context.reader.maxDoc());
|
||||
} else {
|
||||
termsIndex = dv.getSource().asSortedSource();
|
||||
if (termsIndex == null) {
|
||||
|
@ -1687,24 +1725,30 @@ public abstract class FieldComparator<T> {
|
|||
comp = termsIndex.getComparator();
|
||||
|
||||
FieldComparator perSegComp = null;
|
||||
final PackedInts.Reader docToOrd = termsIndex.getDocToOrd();
|
||||
if (docToOrd.hasArray()) {
|
||||
final Object arr = docToOrd.getArray();
|
||||
assert arr != null;
|
||||
if (arr instanceof byte[]) {
|
||||
// 8 bit packed
|
||||
perSegComp = new ByteOrdComparator((byte[]) arr, termsIndex, docBase);
|
||||
} else if (arr instanceof short[]) {
|
||||
// 16 bit packed
|
||||
perSegComp = new ShortOrdComparator((short[]) arr, termsIndex, docBase);
|
||||
} else if (arr instanceof int[]) {
|
||||
// 32 bit packed
|
||||
perSegComp = new IntOrdComparator((int[]) arr, termsIndex, docBase);
|
||||
if (termsIndex.hasPackedDocToOrd()) {
|
||||
final PackedInts.Reader docToOrd = termsIndex.getDocToOrd();
|
||||
if (docToOrd.hasArray()) {
|
||||
final Object arr = docToOrd.getArray();
|
||||
assert arr != null;
|
||||
if (arr instanceof byte[]) {
|
||||
// 8 bit packed
|
||||
perSegComp = new ByteOrdComparator((byte[]) arr, termsIndex, docBase);
|
||||
} else if (arr instanceof short[]) {
|
||||
// 16 bit packed
|
||||
perSegComp = new ShortOrdComparator((short[]) arr, termsIndex, docBase);
|
||||
} else if (arr instanceof int[]) {
|
||||
// 32 bit packed
|
||||
perSegComp = new IntOrdComparator((int[]) arr, termsIndex, docBase);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (perSegComp == null) {
|
||||
perSegComp = new AnyOrdComparator(docToOrd, docBase);
|
||||
if (perSegComp == null) {
|
||||
perSegComp = new AnyPackedDocToOrdComparator(docToOrd, docBase);
|
||||
}
|
||||
} else {
|
||||
if (perSegComp == null) {
|
||||
perSegComp = new AnyOrdComparator(docBase);
|
||||
}
|
||||
}
|
||||
|
||||
currentReaderGen++;
|
||||
|
@ -1845,7 +1889,7 @@ public abstract class FieldComparator<T> {
|
|||
public static final class TermValDocValuesComparator extends FieldComparator<BytesRef> {
|
||||
|
||||
private BytesRef[] values;
|
||||
private Source docTerms;
|
||||
private DocValues.Source docTerms;
|
||||
private final String field;
|
||||
private BytesRef bottom;
|
||||
private final BytesRef tempBR = new BytesRef();
|
||||
|
@ -1878,11 +1922,11 @@ public abstract class FieldComparator<T> {
|
|||
|
||||
@Override
|
||||
public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
|
||||
final IndexDocValues dv = context.reader.docValues(field);
|
||||
final DocValues dv = context.reader.docValues(field);
|
||||
if (dv != null) {
|
||||
docTerms = dv.getSource();
|
||||
} else {
|
||||
docTerms = IndexDocValues.getDefaultSource(ValueType.BYTES_VAR_DEREF);
|
||||
docTerms = DocValues.getDefaultSource(DocValues.Type.BYTES_VAR_DEREF);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.search.similarities;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.document.IndexDocValuesField; // javadoc
|
||||
import org.apache.lucene.document.DocValuesField; // javadoc
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.IndexReader; // javadoc
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
|
@ -71,11 +71,11 @@ import org.apache.lucene.util.TermContext;
|
|||
* <p>
|
||||
* Because index-time boost is handled entirely at the application level anyway,
|
||||
* an application can alternatively store the index-time boost separately using an
|
||||
* {@link IndexDocValuesField}, and access this at query-time with
|
||||
* {@link DocValuesField}, and access this at query-time with
|
||||
* {@link IndexReader#docValues(String)}.
|
||||
* <p>
|
||||
* Finally, using index-time boosts (either via folding into the normalization byte or
|
||||
* via IndexDocValues), is an inefficient way to boost the scores of different fields if the
|
||||
* via DocValues), is an inefficient way to boost the scores of different fields if the
|
||||
* boost will be the same for every document, instead the Similarity can simply take a constant
|
||||
* boost parameter <i>C</i>, and the SimilarityProvider can return different instances with
|
||||
* different boosts depending upon field name.
|
||||
|
|
|
@ -24,11 +24,11 @@ import java.util.Random;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.IndexWriter; // javadoc
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -167,12 +167,12 @@ public class RandomIndexWriter implements Closeable {
|
|||
|
||||
private void randomPerDocFieldValues(Random random, Document doc) {
|
||||
|
||||
ValueType[] values = ValueType.values();
|
||||
ValueType type = values[random.nextInt(values.length)];
|
||||
DocValues.Type[] values = DocValues.Type.values();
|
||||
DocValues.Type type = values[random.nextInt(values.length)];
|
||||
String name = "random_" + type.name() + "" + docValuesFieldPrefix;
|
||||
if ("Lucene3x".equals(codec.getName()) || doc.getField(name) != null)
|
||||
return;
|
||||
IndexDocValuesField docValuesField = new IndexDocValuesField(name);
|
||||
DocValuesField docValuesField = new DocValuesField(name);
|
||||
switch (type) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.lucene.index.SegmentInfo;
|
|||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -43,7 +43,7 @@ public class MockSepDocValuesFormat extends DocValuesFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return new SepDocValuesProducer(state);
|
||||
}
|
||||
|
||||
|
|
|
@ -440,7 +440,7 @@ public class _TestUtil {
|
|||
/** Adds field info for a Document. */
|
||||
public static void add(Document doc, FieldInfos fieldInfos) {
|
||||
for (IndexableField field : doc) {
|
||||
fieldInfos.addOrUpdate(field.name(), field.fieldType(), false, field.docValuesType());
|
||||
fieldInfos.addOrUpdate(field.name(), field.fieldType());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
|
@ -47,7 +47,6 @@ import org.apache.lucene.index.codecs.lucene40.Lucene40SegmentInfosFormat;
|
|||
import org.apache.lucene.index.codecs.lucene40.Lucene40StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40TermVectorsFormat;
|
||||
import org.apache.lucene.index.codecs.pulsing.Pulsing40PostingsFormat;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
|
@ -1254,7 +1253,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
RandomIndexWriter w = new RandomIndexWriter(random, d1);
|
||||
Document doc = new Document();
|
||||
doc.add(newField("id", "1", StringField.TYPE_STORED));
|
||||
IndexDocValuesField dv = new IndexDocValuesField("dv");
|
||||
DocValuesField dv = new DocValuesField("dv");
|
||||
dv.setInt(1);
|
||||
doc.add(dv);
|
||||
w.addDocument(doc);
|
||||
|
@ -1265,7 +1264,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
w = new RandomIndexWriter(random, d2);
|
||||
doc = new Document();
|
||||
doc.add(newField("id", "2", StringField.TYPE_STORED));
|
||||
dv = new IndexDocValuesField("dv");
|
||||
dv = new DocValuesField("dv");
|
||||
dv.setInt(2);
|
||||
doc.add(dv);
|
||||
w.addDocument(doc);
|
||||
|
@ -1285,7 +1284,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
w.close();
|
||||
IndexReader sr = getOnlySegmentReader(r3);
|
||||
assertEquals(2, sr.numDocs());
|
||||
IndexDocValues docValues = sr.perDocValues().docValues("dv");
|
||||
DocValues docValues = sr.docValues("dv");
|
||||
assertNotNull(docValues);
|
||||
r3.close();
|
||||
d3.close();
|
||||
|
|
|
@ -18,17 +18,16 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -520,29 +519,15 @@ public class TestDuelingCodecs extends LuceneTestCase {
|
|||
* checks that docvalues across all fields are equivalent
|
||||
*/
|
||||
public void assertDocValues(IndexReader leftReader, IndexReader rightReader) throws Exception {
|
||||
PerDocValues leftPerDoc = MultiPerDocValues.getPerDocs(leftReader);
|
||||
PerDocValues rightPerDoc = MultiPerDocValues.getPerDocs(rightReader);
|
||||
|
||||
Fields leftFields = MultiFields.getFields(leftReader);
|
||||
Fields rightFields = MultiFields.getFields(rightReader);
|
||||
// Fields could be null if there are no postings,
|
||||
// but then it must be null for both
|
||||
if (leftFields == null || rightFields == null) {
|
||||
assertNull(info, leftFields);
|
||||
assertNull(info, rightFields);
|
||||
return;
|
||||
}
|
||||
|
||||
FieldsEnum fieldsEnum = leftFields.iterator();
|
||||
String field;
|
||||
while ((field = fieldsEnum.next()) != null) {
|
||||
IndexDocValues leftDocValues = leftPerDoc.docValues(field);
|
||||
IndexDocValues rightDocValues = rightPerDoc.docValues(field);
|
||||
if (leftDocValues == null || rightDocValues == null) {
|
||||
assertNull(info, leftDocValues);
|
||||
assertNull(info, rightDocValues);
|
||||
continue;
|
||||
}
|
||||
Set<String> leftValues = new HashSet<String>(leftReader.getFieldNames(FieldOption.DOC_VALUES));
|
||||
Set<String> rightValues = new HashSet<String>(rightReader.getFieldNames(FieldOption.DOC_VALUES));
|
||||
assertEquals(info, leftValues, rightValues);
|
||||
|
||||
for (String field : leftValues) {
|
||||
DocValues leftDocValues = MultiDocValues.getDocValues(leftReader, field);
|
||||
DocValues rightDocValues = MultiDocValues.getDocValues(rightReader, field);
|
||||
assertNotNull(info, leftDocValues);
|
||||
assertNotNull(info, rightDocValues);
|
||||
assertDocValuesSource(leftDocValues.getDirectSource(), rightDocValues.getDirectSource());
|
||||
assertDocValuesSource(leftDocValues.getSource(), rightDocValues.getSource());
|
||||
}
|
||||
|
@ -551,8 +536,8 @@ public class TestDuelingCodecs extends LuceneTestCase {
|
|||
/**
|
||||
* checks source API
|
||||
*/
|
||||
public void assertDocValuesSource(Source left, Source right) throws Exception {
|
||||
ValueType leftType = left.type();
|
||||
public void assertDocValuesSource(DocValues.Source left, DocValues.Source right) throws Exception {
|
||||
DocValues.Type leftType = left.type();
|
||||
assertEquals(info, leftType, right.type());
|
||||
switch(leftType) {
|
||||
case VAR_INTS:
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.index.codecs.FieldInfosWriter;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.junit.Ignore;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
@ -82,7 +81,6 @@ public class TestFieldInfos extends LuceneTestCase {
|
|||
|
||||
info = readIn.fieldInfo("textField2");
|
||||
assertTrue(info != null);
|
||||
assertTrue(info.storeTermVector == true);
|
||||
assertTrue(info.omitNorms == false);
|
||||
|
||||
info = readIn.fieldInfo("textField3");
|
||||
|
|
|
@ -28,8 +28,7 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.NumericField.DataType;
|
||||
import org.apache.lucene.document.NumericField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -157,12 +156,12 @@ public class TestIndexableField extends LuceneTestCase {
|
|||
|
||||
// TODO: randomly enable doc values
|
||||
@Override
|
||||
public PerDocFieldValues docValues() {
|
||||
public DocValue docValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType docValuesType() {
|
||||
public DocValues.Type docValueType() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -20,8 +20,14 @@ package org.apache.lucene.index.values;
|
|||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Bytes;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Floats;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Ints;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.Writer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Counter;
|
||||
|
@ -29,6 +35,7 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
// TODO: some of this should be under lucene40 codec tests? is talking to codec directly?f
|
||||
public class TestDocValues extends LuceneTestCase {
|
||||
private static final Comparator<BytesRef> COMP = BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
// TODO -- for sorted test, do our own Sort of the
|
||||
|
@ -76,12 +83,12 @@ public class TestDocValues extends LuceneTestCase {
|
|||
w.finish(maxDoc);
|
||||
assertEquals(0, trackBytes.get());
|
||||
|
||||
IndexDocValues r = Bytes.getValues(dir, "test", mode, fixedSize, maxDoc, COMP, newIOContext(random));
|
||||
DocValues r = Bytes.getValues(dir, "test", mode, fixedSize, maxDoc, COMP, newIOContext(random));
|
||||
|
||||
// Verify we can load source twice:
|
||||
for (int iter = 0; iter < 2; iter++) {
|
||||
Source s;
|
||||
IndexDocValues.SortedSource ss;
|
||||
DocValues.SortedSource ss;
|
||||
if (mode == Bytes.Mode.SORTED) {
|
||||
// default is unicode so we can simply pass null here
|
||||
s = ss = getSortedSource(r);
|
||||
|
@ -155,19 +162,19 @@ public class TestDocValues extends LuceneTestCase {
|
|||
{ Long.MIN_VALUE + 1, 1 }, { -1, Long.MAX_VALUE },
|
||||
{ Long.MIN_VALUE, -1 }, { 1, Long.MAX_VALUE },
|
||||
{ -1, Long.MAX_VALUE - 1 }, { Long.MIN_VALUE + 2, 1 }, };
|
||||
ValueType[] expectedTypes = new ValueType[] { ValueType.FIXED_INTS_64,
|
||||
ValueType.FIXED_INTS_64, ValueType.FIXED_INTS_64,
|
||||
ValueType.FIXED_INTS_64, ValueType.VAR_INTS, ValueType.VAR_INTS,
|
||||
ValueType.VAR_INTS, };
|
||||
Type[] expectedTypes = new Type[] { Type.FIXED_INTS_64,
|
||||
Type.FIXED_INTS_64, Type.FIXED_INTS_64,
|
||||
Type.FIXED_INTS_64, Type.VAR_INTS, Type.VAR_INTS,
|
||||
Type.VAR_INTS, };
|
||||
for (int i = 0; i < minMax.length; i++) {
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.VAR_INTS, newIOContext(random));
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.VAR_INTS, newIOContext(random));
|
||||
w.add(0, minMax[i][0]);
|
||||
w.add(1, minMax[i][1]);
|
||||
w.finish(2);
|
||||
assertEquals(0, trackBytes.get());
|
||||
IndexDocValues r = Ints.getValues(dir, "test", 2, ValueType.VAR_INTS, newIOContext(random));
|
||||
DocValues r = Ints.getValues(dir, "test", 2, Type.VAR_INTS, newIOContext(random));
|
||||
Source source = getSource(r);
|
||||
assertEquals(i + " with min: " + minMax[i][0] + " max: " + minMax[i][1],
|
||||
expectedTypes[i], source.type());
|
||||
|
@ -180,14 +187,14 @@ public class TestDocValues extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testVInts() throws IOException {
|
||||
testInts(ValueType.VAR_INTS, 63);
|
||||
testInts(Type.VAR_INTS, 63);
|
||||
}
|
||||
|
||||
public void testFixedInts() throws IOException {
|
||||
testInts(ValueType.FIXED_INTS_64, 63);
|
||||
testInts(ValueType.FIXED_INTS_32, 31);
|
||||
testInts(ValueType.FIXED_INTS_16, 15);
|
||||
testInts(ValueType.FIXED_INTS_8, 7);
|
||||
testInts(Type.FIXED_INTS_64, 63);
|
||||
testInts(Type.FIXED_INTS_32, 31);
|
||||
testInts(Type.FIXED_INTS_16, 15);
|
||||
testInts(Type.FIXED_INTS_8, 7);
|
||||
|
||||
}
|
||||
|
||||
|
@ -195,12 +202,12 @@ public class TestDocValues extends LuceneTestCase {
|
|||
byte[] sourceArray = new byte[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_8, newIOContext(random));
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_8, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, (long) sourceArray[i]);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
IndexDocValues r = Ints.getValues(dir, "test", sourceArray.length, ValueType.FIXED_INTS_8, newIOContext(random));
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_8, newIOContext(random));
|
||||
Source source = r.getSource();
|
||||
assertTrue(source.hasArray());
|
||||
byte[] loaded = ((byte[])source.getArray());
|
||||
|
@ -216,12 +223,12 @@ public class TestDocValues extends LuceneTestCase {
|
|||
short[] sourceArray = new short[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_16, newIOContext(random));
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_16, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, (long) sourceArray[i]);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
IndexDocValues r = Ints.getValues(dir, "test", sourceArray.length, ValueType.FIXED_INTS_16, newIOContext(random));
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_16, newIOContext(random));
|
||||
Source source = r.getSource();
|
||||
assertTrue(source.hasArray());
|
||||
short[] loaded = ((short[])source.getArray());
|
||||
|
@ -237,12 +244,12 @@ public class TestDocValues extends LuceneTestCase {
|
|||
long[] sourceArray = new long[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_64, newIOContext(random));
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_64, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, sourceArray[i]);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
IndexDocValues r = Ints.getValues(dir, "test", sourceArray.length, ValueType.FIXED_INTS_64, newIOContext(random));
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_64, newIOContext(random));
|
||||
Source source = r.getSource();
|
||||
assertTrue(source.hasArray());
|
||||
long[] loaded = ((long[])source.getArray());
|
||||
|
@ -258,12 +265,12 @@ public class TestDocValues extends LuceneTestCase {
|
|||
int[] sourceArray = new int[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, ValueType.FIXED_INTS_32, newIOContext(random));
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_32, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, (long) sourceArray[i]);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
IndexDocValues r = Ints.getValues(dir, "test", sourceArray.length, ValueType.FIXED_INTS_32, newIOContext(random));
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_32, newIOContext(random));
|
||||
Source source = r.getSource();
|
||||
assertTrue(source.hasArray());
|
||||
int[] loaded = ((int[])source.getArray());
|
||||
|
@ -279,12 +286,12 @@ public class TestDocValues extends LuceneTestCase {
|
|||
float[] sourceArray = new float[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), ValueType.FLOAT_32);
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_32);
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, sourceArray[i]);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
IndexDocValues r = Floats.getValues(dir, "test", 3, newIOContext(random), ValueType.FLOAT_32);
|
||||
DocValues r = Floats.getValues(dir, "test", 3, newIOContext(random), Type.FLOAT_32);
|
||||
Source source = r.getSource();
|
||||
assertTrue(source.hasArray());
|
||||
float[] loaded = ((float[])source.getArray());
|
||||
|
@ -300,12 +307,12 @@ public class TestDocValues extends LuceneTestCase {
|
|||
double[] sourceArray = new double[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), ValueType.FLOAT_64);
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_64);
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, sourceArray[i]);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
IndexDocValues r = Floats.getValues(dir, "test", 3, newIOContext(random), ValueType.FLOAT_64);
|
||||
DocValues r = Floats.getValues(dir, "test", 3, newIOContext(random), Type.FLOAT_64);
|
||||
Source source = r.getSource();
|
||||
assertTrue(source.hasArray());
|
||||
double[] loaded = ((double[])source.getArray());
|
||||
|
@ -317,7 +324,7 @@ public class TestDocValues extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
private void testInts(ValueType type, int maxBit) throws IOException {
|
||||
private void testInts(Type type, int maxBit) throws IOException {
|
||||
long maxV = 1;
|
||||
final int NUM_VALUES = 333 + random.nextInt(333);
|
||||
final long[] values = new long[NUM_VALUES];
|
||||
|
@ -334,7 +341,7 @@ public class TestDocValues extends LuceneTestCase {
|
|||
w.finish(NUM_VALUES + additionalDocs);
|
||||
assertEquals(0, trackBytes.get());
|
||||
|
||||
IndexDocValues r = Ints.getValues(dir, "test", NUM_VALUES + additionalDocs, type, newIOContext(random));
|
||||
DocValues r = Ints.getValues(dir, "test", NUM_VALUES + additionalDocs, type, newIOContext(random));
|
||||
for (int iter = 0; iter < 2; iter++) {
|
||||
Source s = getSource(r);
|
||||
assertEquals(type, s.type());
|
||||
|
@ -350,17 +357,17 @@ public class TestDocValues extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testFloats4() throws IOException {
|
||||
runTestFloats(ValueType.FLOAT_32, 0.00001);
|
||||
runTestFloats(Type.FLOAT_32, 0.00001);
|
||||
}
|
||||
|
||||
private void runTestFloats(ValueType type, double delta) throws IOException {
|
||||
private void runTestFloats(Type type, double delta) throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), type);
|
||||
final int NUM_VALUES = 777 + random.nextInt(777);;
|
||||
final double[] values = new double[NUM_VALUES];
|
||||
for (int i = 0; i < NUM_VALUES; i++) {
|
||||
final double v = type == ValueType.FLOAT_32 ? random.nextFloat() : random
|
||||
final double v = type == Type.FLOAT_32 ? random.nextFloat() : random
|
||||
.nextDouble();
|
||||
values[i] = v;
|
||||
w.add(i, v);
|
||||
|
@ -369,7 +376,7 @@ public class TestDocValues extends LuceneTestCase {
|
|||
w.finish(NUM_VALUES + additionalValues);
|
||||
assertEquals(0, trackBytes.get());
|
||||
|
||||
IndexDocValues r = Floats.getValues(dir, "test", NUM_VALUES + additionalValues, newIOContext(random), type);
|
||||
DocValues r = Floats.getValues(dir, "test", NUM_VALUES + additionalValues, newIOContext(random), type);
|
||||
for (int iter = 0; iter < 2; iter++) {
|
||||
Source s = getSource(r);
|
||||
for (int i = 0; i < NUM_VALUES; i++) {
|
||||
|
@ -381,11 +388,11 @@ public class TestDocValues extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testFloats8() throws IOException {
|
||||
runTestFloats(ValueType.FLOAT_64, 0.0);
|
||||
runTestFloats(Type.FLOAT_64, 0.0);
|
||||
}
|
||||
|
||||
|
||||
private Source getSource(IndexDocValues values) throws IOException {
|
||||
private Source getSource(DocValues values) throws IOException {
|
||||
// getSource uses cache internally
|
||||
switch(random.nextInt(5)) {
|
||||
case 3:
|
||||
|
@ -399,7 +406,7 @@ public class TestDocValues extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private SortedSource getSortedSource(IndexDocValues values) throws IOException {
|
||||
private SortedSource getSortedSource(DocValues values) throws IOException {
|
||||
return getSource(values).asSortedSource();
|
||||
}
|
||||
|
||||
|
|
|
@ -27,21 +27,23 @@ import java.util.List;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LogDocMergePolicy;
|
||||
import org.apache.lucene.index.LogMergePolicy;
|
||||
import org.apache.lucene.index.MultiPerDocValues;
|
||||
import org.apache.lucene.index.MultiDocValues;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.codecs.PerDocProducer;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
|
@ -76,7 +78,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
IndexWriter writer = new IndexWriter(dir, writerConfig(false));
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Document doc = new Document();
|
||||
IndexDocValuesField valuesField = new IndexDocValuesField("docId");
|
||||
DocValuesField valuesField = new DocValuesField("docId");
|
||||
valuesField.setInt(i);
|
||||
doc.add(valuesField);
|
||||
doc.add(new TextField("docId", "" + i));
|
||||
|
@ -102,7 +104,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
TopDocs search = searcher.search(query, 10);
|
||||
assertEquals(5, search.totalHits);
|
||||
ScoreDoc[] scoreDocs = search.scoreDocs;
|
||||
IndexDocValues docValues = MultiPerDocValues.getPerDocs(reader).docValues("docId");
|
||||
DocValues docValues = MultiDocValues.getDocValues(reader, "docId");
|
||||
Source source = docValues.getSource();
|
||||
for (int i = 0; i < scoreDocs.length; i++) {
|
||||
assertEquals(i, scoreDocs[i].doc);
|
||||
|
@ -130,10 +132,10 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
|
||||
public void testAddIndexes() throws IOException {
|
||||
int valuesPerIndex = 10;
|
||||
List<ValueType> values = Arrays.asList(ValueType.values());
|
||||
List<Type> values = Arrays.asList(Type.values());
|
||||
Collections.shuffle(values, random);
|
||||
ValueType first = values.get(0);
|
||||
ValueType second = values.get(1);
|
||||
Type first = values.get(0);
|
||||
Type second = values.get(1);
|
||||
// index first index
|
||||
Directory d_1 = newDirectory();
|
||||
IndexWriter w_1 = new IndexWriter(d_1, writerConfig(random.nextBoolean()));
|
||||
|
@ -256,11 +258,11 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
Directory d = newDirectory();
|
||||
IndexWriter w = new IndexWriter(d, cfg);
|
||||
final int numValues = 50 + atLeast(10);
|
||||
final List<ValueType> numVariantList = new ArrayList<ValueType>(NUMERICS);
|
||||
final List<Type> numVariantList = new ArrayList<Type>(NUMERICS);
|
||||
|
||||
// run in random order to test if fill works correctly during merges
|
||||
Collections.shuffle(numVariantList, random);
|
||||
for (ValueType val : numVariantList) {
|
||||
for (Type val : numVariantList) {
|
||||
FixedBitSet deleted = indexValues(w, numValues, val, numVariantList,
|
||||
withDeletions, 7);
|
||||
List<Closeable> closeables = new ArrayList<Closeable>();
|
||||
|
@ -277,7 +279,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
case FIXED_INTS_32:
|
||||
case FIXED_INTS_64:
|
||||
case VAR_INTS: {
|
||||
IndexDocValues intsReader = getDocValues(r, val.name());
|
||||
DocValues intsReader = getDocValues(r, val.name());
|
||||
assertNotNull(intsReader);
|
||||
|
||||
Source ints = getSource(intsReader);
|
||||
|
@ -298,7 +300,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
break;
|
||||
case FLOAT_32:
|
||||
case FLOAT_64: {
|
||||
IndexDocValues floatReader = getDocValues(r, val.name());
|
||||
DocValues floatReader = getDocValues(r, val.name());
|
||||
assertNotNull(floatReader);
|
||||
Source floats = getSource(floatReader);
|
||||
for (int i = 0; i < base; i++) {
|
||||
|
@ -333,11 +335,11 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
throws CorruptIndexException, LockObtainFailedException, IOException {
|
||||
final Directory d = newDirectory();
|
||||
IndexWriter w = new IndexWriter(d, cfg);
|
||||
final List<ValueType> byteVariantList = new ArrayList<ValueType>(BYTES);
|
||||
final List<Type> byteVariantList = new ArrayList<Type>(BYTES);
|
||||
// run in random order to test if fill works correctly during merges
|
||||
Collections.shuffle(byteVariantList, random);
|
||||
final int numValues = 50 + atLeast(10);
|
||||
for (ValueType byteIndexValue : byteVariantList) {
|
||||
for (Type byteIndexValue : byteVariantList) {
|
||||
List<Closeable> closeables = new ArrayList<Closeable>();
|
||||
final int bytesSize = 1 + atLeast(50);
|
||||
FixedBitSet deleted = indexValues(w, numValues, byteIndexValue,
|
||||
|
@ -346,7 +348,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
assertEquals(0, r.numDeletedDocs());
|
||||
final int numRemainingValues = numValues - deleted.cardinality();
|
||||
final int base = r.numDocs() - numRemainingValues;
|
||||
IndexDocValues bytesReader = getDocValues(r, byteIndexValue.name());
|
||||
DocValues bytesReader = getDocValues(r, byteIndexValue.name());
|
||||
assertNotNull("field " + byteIndexValue.name()
|
||||
+ " returned null reader - maybe merged failed", bytesReader);
|
||||
Source bytes = getSource(bytesReader);
|
||||
|
@ -416,27 +418,11 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
d.close();
|
||||
}
|
||||
|
||||
private IndexDocValues getDocValues(IndexReader reader, String field)
|
||||
throws IOException {
|
||||
boolean singleSeg = reader.getSequentialSubReaders().length == 1;
|
||||
PerDocValues perDoc = singleSeg ? reader.getSequentialSubReaders()[0].perDocValues()
|
||||
: MultiPerDocValues.getPerDocs(reader);
|
||||
switch (random.nextInt(singleSeg ? 3 : 2)) { // case 2 only if single seg
|
||||
case 0:
|
||||
return perDoc.docValues(field);
|
||||
case 1:
|
||||
IndexDocValues docValues = perDoc.docValues(field);
|
||||
if (docValues != null) {
|
||||
return docValues;
|
||||
}
|
||||
throw new RuntimeException("no such field " + field);
|
||||
case 2:// this only works if we are on a single seg index!
|
||||
return reader.getSequentialSubReaders()[0].docValues(field);
|
||||
}
|
||||
throw new RuntimeException();
|
||||
private DocValues getDocValues(IndexReader reader, String field) throws IOException {
|
||||
return MultiDocValues.getDocValues(reader, field);
|
||||
}
|
||||
|
||||
private Source getSource(IndexDocValues values) throws IOException {
|
||||
private Source getSource(DocValues values) throws IOException {
|
||||
// getSource uses cache internally
|
||||
switch(random.nextInt(5)) {
|
||||
case 3:
|
||||
|
@ -451,24 +437,24 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
}
|
||||
|
||||
|
||||
private static EnumSet<ValueType> BYTES = EnumSet.of(ValueType.BYTES_FIXED_DEREF,
|
||||
ValueType.BYTES_FIXED_STRAIGHT, ValueType.BYTES_VAR_DEREF,
|
||||
ValueType.BYTES_VAR_STRAIGHT, ValueType.BYTES_FIXED_SORTED, ValueType.BYTES_VAR_SORTED);
|
||||
private static EnumSet<Type> BYTES = EnumSet.of(Type.BYTES_FIXED_DEREF,
|
||||
Type.BYTES_FIXED_STRAIGHT, Type.BYTES_VAR_DEREF,
|
||||
Type.BYTES_VAR_STRAIGHT, Type.BYTES_FIXED_SORTED, Type.BYTES_VAR_SORTED);
|
||||
|
||||
private static EnumSet<ValueType> NUMERICS = EnumSet.of(ValueType.VAR_INTS,
|
||||
ValueType.FIXED_INTS_16, ValueType.FIXED_INTS_32,
|
||||
ValueType.FIXED_INTS_64,
|
||||
ValueType.FIXED_INTS_8,
|
||||
ValueType.FLOAT_32,
|
||||
ValueType.FLOAT_64);
|
||||
private static EnumSet<Type> NUMERICS = EnumSet.of(Type.VAR_INTS,
|
||||
Type.FIXED_INTS_16, Type.FIXED_INTS_32,
|
||||
Type.FIXED_INTS_64,
|
||||
Type.FIXED_INTS_8,
|
||||
Type.FLOAT_32,
|
||||
Type.FLOAT_64);
|
||||
|
||||
private FixedBitSet indexValues(IndexWriter w, int numValues, ValueType value,
|
||||
List<ValueType> valueVarList, boolean withDeletions, int bytesSize)
|
||||
private FixedBitSet indexValues(IndexWriter w, int numValues, Type value,
|
||||
List<Type> valueVarList, boolean withDeletions, int bytesSize)
|
||||
throws CorruptIndexException, IOException {
|
||||
final boolean isNumeric = NUMERICS.contains(value);
|
||||
FixedBitSet deleted = new FixedBitSet(numValues);
|
||||
Document doc = new Document();
|
||||
IndexDocValuesField valField = new IndexDocValuesField(value.name());
|
||||
DocValuesField valField = new DocValuesField(value.name());
|
||||
doc.add(valField);
|
||||
final BytesRef bytesRef = new BytesRef();
|
||||
|
||||
|
@ -522,7 +508,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
|
||||
if (i % 7 == 0) {
|
||||
if (withDeletions && random.nextBoolean()) {
|
||||
ValueType val = valueVarList.get(random.nextInt(1 + valueVarList
|
||||
Type val = valueVarList.get(random.nextInt(1 + valueVarList
|
||||
.indexOf(value)));
|
||||
final int randInt = val == value ? random.nextInt(1 + i) : random
|
||||
.nextInt(numValues);
|
||||
|
@ -545,11 +531,11 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
return deleted;
|
||||
}
|
||||
|
||||
public void testMultiValuedIndexDocValuesField() throws Exception {
|
||||
public void testMultiValuedDocValuesField() throws Exception {
|
||||
Directory d = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random, d);
|
||||
Document doc = new Document();
|
||||
IndexDocValuesField f = new IndexDocValuesField("field");
|
||||
DocValuesField f = new DocValuesField("field");
|
||||
f.setInt(17);
|
||||
// Index doc values are single-valued so we should not
|
||||
// be able to add same field more than once:
|
||||
|
@ -568,7 +554,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
w.forceMerge(1);
|
||||
IndexReader r = w.getReader();
|
||||
w.close();
|
||||
assertEquals(17, r.getSequentialSubReaders()[0].perDocValues().docValues("field").load().getInt(0));
|
||||
assertEquals(17, r.getSequentialSubReaders()[0].docValues("field").load().getInt(0));
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
@ -577,12 +563,12 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
Directory d = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random, d);
|
||||
Document doc = new Document();
|
||||
IndexDocValuesField f = new IndexDocValuesField("field");
|
||||
DocValuesField f = new DocValuesField("field");
|
||||
f.setInt(17);
|
||||
// Index doc values are single-valued so we should not
|
||||
// be able to add same field more than once:
|
||||
doc.add(f);
|
||||
IndexDocValuesField f2 = new IndexDocValuesField("field");
|
||||
DocValuesField f2 = new DocValuesField("field");
|
||||
f2.setFloat(22.0);
|
||||
doc.add(f2);
|
||||
try {
|
||||
|
@ -598,7 +584,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
w.forceMerge(1);
|
||||
IndexReader r = w.getReader();
|
||||
w.close();
|
||||
assertEquals(17, r.getSequentialSubReaders()[0].perDocValues().docValues("field").load().getInt(0));
|
||||
assertEquals(17, r.getSequentialSubReaders()[0].docValues("field").load().getInt(0));
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
|
|
@ -7,17 +7,20 @@ import java.util.Random;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReader.ReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.SlowMultiReaderWrapper;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.index.codecs.lucene40.values.BytesRefUtils;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -46,22 +49,22 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
assumeFalse("cannot work with preflex codec", Codec.getDefault().getName().equals("Lucene3x"));
|
||||
}
|
||||
|
||||
private static EnumSet<ValueType> INTEGERS = EnumSet.of(ValueType.VAR_INTS,
|
||||
ValueType.FIXED_INTS_16, ValueType.FIXED_INTS_32,
|
||||
ValueType.FIXED_INTS_64, ValueType.FIXED_INTS_8);
|
||||
private static EnumSet<Type> INTEGERS = EnumSet.of(Type.VAR_INTS,
|
||||
Type.FIXED_INTS_16, Type.FIXED_INTS_32,
|
||||
Type.FIXED_INTS_64, Type.FIXED_INTS_8);
|
||||
|
||||
private static EnumSet<ValueType> FLOATS = EnumSet.of(ValueType.FLOAT_32,
|
||||
ValueType.FLOAT_64);
|
||||
private static EnumSet<Type> FLOATS = EnumSet.of(Type.FLOAT_32,
|
||||
Type.FLOAT_64);
|
||||
|
||||
private static EnumSet<ValueType> UNSORTED_BYTES = EnumSet.of(
|
||||
ValueType.BYTES_FIXED_DEREF, ValueType.BYTES_FIXED_STRAIGHT,
|
||||
ValueType.BYTES_VAR_STRAIGHT, ValueType.BYTES_VAR_DEREF);
|
||||
private static EnumSet<Type> UNSORTED_BYTES = EnumSet.of(
|
||||
Type.BYTES_FIXED_DEREF, Type.BYTES_FIXED_STRAIGHT,
|
||||
Type.BYTES_VAR_STRAIGHT, Type.BYTES_VAR_DEREF);
|
||||
|
||||
private static EnumSet<ValueType> SORTED_BYTES = EnumSet.of(
|
||||
ValueType.BYTES_FIXED_SORTED, ValueType.BYTES_VAR_SORTED);
|
||||
private static EnumSet<Type> SORTED_BYTES = EnumSet.of(
|
||||
Type.BYTES_FIXED_SORTED, Type.BYTES_VAR_SORTED);
|
||||
|
||||
public ValueType randomValueType(EnumSet<ValueType> typeEnum, Random random) {
|
||||
ValueType[] array = typeEnum.toArray(new ValueType[0]);
|
||||
public Type randomValueType(EnumSet<Type> typeEnum, Random random) {
|
||||
Type[] array = typeEnum.toArray(new Type[0]);
|
||||
return array[random.nextInt(array.length)];
|
||||
}
|
||||
|
||||
|
@ -69,7 +72,7 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
Int, Float, Byte
|
||||
}
|
||||
|
||||
private void runTest(EnumSet<ValueType> types, TestType type)
|
||||
private void runTest(EnumSet<Type> types, TestType type)
|
||||
throws CorruptIndexException, IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir,
|
||||
|
@ -78,11 +81,11 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
int num_2 = atLeast(200);
|
||||
int num_3 = atLeast(200);
|
||||
long[] values = new long[num_1 + num_2 + num_3];
|
||||
index(writer, new IndexDocValuesField("promote"),
|
||||
index(writer, new DocValuesField("promote"),
|
||||
randomValueType(types, random), values, 0, num_1);
|
||||
writer.commit();
|
||||
|
||||
index(writer, new IndexDocValuesField("promote"),
|
||||
index(writer, new DocValuesField("promote"),
|
||||
randomValueType(types, random), values, num_1, num_2);
|
||||
writer.commit();
|
||||
|
||||
|
@ -93,7 +96,7 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
Directory dir_2 = newDirectory() ;
|
||||
IndexWriter writer_2 = new IndexWriter(dir_2,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
index(writer_2, new IndexDocValuesField("promote"),
|
||||
index(writer_2, new DocValuesField("promote"),
|
||||
randomValueType(types, random), values, num_1 + num_2, num_3);
|
||||
writer_2.commit();
|
||||
writer_2.close();
|
||||
|
@ -109,7 +112,7 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
}
|
||||
dir_2.close();
|
||||
} else {
|
||||
index(writer, new IndexDocValuesField("promote"),
|
||||
index(writer, new DocValuesField("promote"),
|
||||
randomValueType(types, random), values, num_1 + num_2, num_3);
|
||||
}
|
||||
|
||||
|
@ -126,7 +129,7 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
assertEquals(1, reader.getSequentialSubReaders().length);
|
||||
ReaderContext topReaderContext = reader.getTopReaderContext();
|
||||
ReaderContext[] children = topReaderContext.children();
|
||||
IndexDocValues docValues = children[0].reader.docValues("promote");
|
||||
DocValues docValues = children[0].reader.docValues("promote");
|
||||
assertEquals(1, children.length);
|
||||
Source directSource = docValues.getDirectSource();
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
|
@ -171,8 +174,8 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
reader.close();
|
||||
}
|
||||
|
||||
public void index(IndexWriter writer, IndexDocValuesField valField,
|
||||
ValueType valueType, long[] values, int offset, int num)
|
||||
public void index(IndexWriter writer, DocValuesField valField,
|
||||
Type valueType, long[] values, int offset, int num)
|
||||
throws CorruptIndexException, IOException {
|
||||
BytesRef ref = new BytesRef(new byte[] { 1, 2, 3, 4 });
|
||||
for (int i = offset; i < offset + num; i++) {
|
||||
|
@ -266,7 +269,7 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
int num_1 = atLeast(200);
|
||||
int num_2 = atLeast(200);
|
||||
long[] values = new long[num_1 + num_2];
|
||||
index(writer, new IndexDocValuesField("promote"),
|
||||
index(writer, new DocValuesField("promote"),
|
||||
randomValueType(INTEGERS, random), values, 0, num_1);
|
||||
writer.commit();
|
||||
|
||||
|
@ -275,7 +278,7 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
Directory dir_2 = newDirectory() ;
|
||||
IndexWriter writer_2 = new IndexWriter(dir_2,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
index(writer_2, new IndexDocValuesField("promote"),
|
||||
index(writer_2, new DocValuesField("promote"),
|
||||
randomValueType(random.nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random), values, num_1, num_2);
|
||||
writer_2.commit();
|
||||
writer_2.close();
|
||||
|
@ -289,7 +292,7 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
}
|
||||
dir_2.close();
|
||||
} else {
|
||||
index(writer, new IndexDocValuesField("promote"),
|
||||
index(writer, new DocValuesField("promote"),
|
||||
randomValueType(random.nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random), values, num_1, num_2);
|
||||
writer.commit();
|
||||
}
|
||||
|
@ -306,10 +309,10 @@ public class TestTypePromotion extends LuceneTestCase {
|
|||
assertEquals(1, reader.getSequentialSubReaders().length);
|
||||
ReaderContext topReaderContext = reader.getTopReaderContext();
|
||||
ReaderContext[] children = topReaderContext.children();
|
||||
IndexDocValues docValues = children[0].reader.docValues("promote");
|
||||
DocValues docValues = children[0].reader.docValues("promote");
|
||||
assertNotNull(docValues);
|
||||
assertValues(TestType.Byte, dir, values);
|
||||
assertEquals(ValueType.BYTES_VAR_STRAIGHT, docValues.type());
|
||||
assertEquals(Type.BYTES_VAR_STRAIGHT, docValues.type());
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -21,15 +21,15 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
|
@ -48,7 +48,7 @@ public class TestDocValuesScoring extends LuceneTestCase {
|
|||
private static final float SCORE_EPSILON = 0.001f; /* for comparing floats */
|
||||
|
||||
public void testSimple() throws Exception {
|
||||
assumeFalse("PreFlex codec cannot work with IndexDocValues!",
|
||||
assumeFalse("PreFlex codec cannot work with DocValues!",
|
||||
"Lucene3x".equals(Codec.getDefault().getName()));
|
||||
|
||||
Directory dir = newDirectory();
|
||||
|
@ -56,7 +56,7 @@ public class TestDocValuesScoring extends LuceneTestCase {
|
|||
Document doc = new Document();
|
||||
Field field = newField("foo", "", TextField.TYPE_UNSTORED);
|
||||
doc.add(field);
|
||||
IndexDocValuesField dvField = new IndexDocValuesField("foo_boost");
|
||||
DocValuesField dvField = new DocValuesField("foo_boost");
|
||||
doc.add(dvField);
|
||||
Field field2 = newField("bar", "", TextField.TYPE_UNSTORED);
|
||||
doc.add(field2);
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.IndexDocValuesField;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
|
@ -42,7 +42,7 @@ import org.apache.lucene.index.RandomIndexWriter;
|
|||
import org.apache.lucene.index.SlowMultiReaderWrapper;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.FieldValueHitQueue.Entry;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -121,18 +121,18 @@ public class TestSort extends LuceneTestCase {
|
|||
dirs.add(indexStore);
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random, indexStore, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
|
||||
final ValueType stringDVType;
|
||||
final DocValues.Type stringDVType;
|
||||
if (dvStringSorted) {
|
||||
// Index sorted
|
||||
stringDVType = random.nextBoolean() ? ValueType.BYTES_VAR_SORTED : ValueType.BYTES_FIXED_SORTED;
|
||||
stringDVType = random.nextBoolean() ? DocValues.Type.BYTES_VAR_SORTED : DocValues.Type.BYTES_FIXED_SORTED;
|
||||
} else {
|
||||
// Index non-sorted
|
||||
if (random.nextBoolean()) {
|
||||
// Fixed
|
||||
stringDVType = random.nextBoolean() ? ValueType.BYTES_FIXED_STRAIGHT : ValueType.BYTES_FIXED_DEREF;
|
||||
stringDVType = random.nextBoolean() ? DocValues.Type.BYTES_FIXED_STRAIGHT : DocValues.Type.BYTES_FIXED_DEREF;
|
||||
} else {
|
||||
// Var
|
||||
stringDVType = random.nextBoolean() ? ValueType.BYTES_VAR_STRAIGHT : ValueType.BYTES_VAR_DEREF;
|
||||
stringDVType = random.nextBoolean() ? DocValues.Type.BYTES_VAR_STRAIGHT : DocValues.Type.BYTES_VAR_DEREF;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -148,21 +148,21 @@ public class TestSort extends LuceneTestCase {
|
|||
if (data[i][2] != null) {
|
||||
Field f = new StringField ("int", data[i][2]);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, ValueType.VAR_INTS);
|
||||
f = DocValuesField.build(f, DocValues.Type.VAR_INTS);
|
||||
}
|
||||
doc.add(f);
|
||||
}
|
||||
if (data[i][3] != null) {
|
||||
Field f = new StringField ("float", data[i][3]);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, ValueType.FLOAT_32);
|
||||
f = DocValuesField.build(f, DocValues.Type.FLOAT_32);
|
||||
}
|
||||
doc.add(f);
|
||||
}
|
||||
if (data[i][4] != null) {
|
||||
Field f = new StringField ("string", data[i][4]);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, stringDVType);
|
||||
f = DocValuesField.build(f, stringDVType);
|
||||
}
|
||||
doc.add(f);
|
||||
}
|
||||
|
@ -172,7 +172,7 @@ public class TestSort extends LuceneTestCase {
|
|||
if (data[i][8] != null) {
|
||||
Field f = new StringField ("double", data[i][8]);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, ValueType.FLOAT_64);
|
||||
f = DocValuesField.build(f, DocValues.Type.FLOAT_64);
|
||||
}
|
||||
doc.add(f);
|
||||
}
|
||||
|
@ -219,13 +219,13 @@ public class TestSort extends LuceneTestCase {
|
|||
//doc.add (new Field ("contents", Integer.toString(i), Field.Store.NO, Field.Index.ANALYZED));
|
||||
Field f = new StringField("string", num);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, ValueType.BYTES_VAR_SORTED);
|
||||
f = DocValuesField.build(f, DocValues.Type.BYTES_VAR_SORTED);
|
||||
}
|
||||
doc.add (f);
|
||||
String num2 = getRandomCharString(getRandomNumber(1, 4), 48, 50);
|
||||
f = new StringField ("string2", num2);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, ValueType.BYTES_VAR_SORTED);
|
||||
f = DocValuesField.build(f, DocValues.Type.BYTES_VAR_SORTED);
|
||||
}
|
||||
doc.add (f);
|
||||
doc.add (new Field ("tracer2", num2, onlyStored));
|
||||
|
@ -238,13 +238,13 @@ public class TestSort extends LuceneTestCase {
|
|||
//doc.add (new Field ("contents", Integer.toString(i), Field.Store.NO, Field.Index.ANALYZED));
|
||||
f = new StringField("string_fixed", numFixed);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, ValueType.BYTES_FIXED_SORTED);
|
||||
f = DocValuesField.build(f, DocValues.Type.BYTES_FIXED_SORTED);
|
||||
}
|
||||
doc.add (f);
|
||||
String num2Fixed = getRandomCharString(fixedLen2, 48, 52);
|
||||
f = new StringField ("string2_fixed", num2Fixed);
|
||||
if (supportsDocValues) {
|
||||
f = IndexDocValuesField.build(f, ValueType.BYTES_FIXED_SORTED);
|
||||
f = DocValuesField.build(f, DocValues.Type.BYTES_FIXED_SORTED);
|
||||
}
|
||||
doc.add (f);
|
||||
doc.add (new Field ("tracer2_fixed", num2Fixed, onlyStored));
|
||||
|
|
|
@ -17,4 +17,4 @@ LUCENE-3483: Move Function grouping collectors from Solr to
|
|||
|
||||
New features
|
||||
|
||||
LUCENE-3496: Support grouping by IndexDocValues. (Martijn van Groningen)
|
||||
LUCENE-3496: Support grouping by DocValues. (Martijn van Groningen)
|
|
@ -17,9 +17,9 @@ package org.apache.lucene.search.grouping.dv;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -40,13 +40,13 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
|
||||
final String groupField;
|
||||
final boolean diskResident;
|
||||
final ValueType valueType;
|
||||
final DocValues.Type valueType;
|
||||
final BytesRef scratchBytesRef = new BytesRef();
|
||||
|
||||
IndexReader.AtomicReaderContext readerContext;
|
||||
Scorer scorer;
|
||||
|
||||
DVAllGroupHeadsCollector(String groupField, ValueType valueType, int numberOfSorts, boolean diskResident) {
|
||||
DVAllGroupHeadsCollector(String groupField, DocValues.Type valueType, int numberOfSorts, boolean diskResident) {
|
||||
super(numberOfSorts);
|
||||
this.groupField = groupField;
|
||||
this.valueType = valueType;
|
||||
|
@ -59,12 +59,12 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
*
|
||||
* @param groupField The field to group by
|
||||
* @param sortWithinGroup The sort within each group
|
||||
* @param type The {@link ValueType} which is used to select a concrete implementation.
|
||||
* @param type The {@link Type} which is used to select a concrete implementation.
|
||||
* @param diskResident Whether the values to group by should be disk resident
|
||||
* @return an <code>AbstractAllGroupHeadsCollector</code> instance based on the supplied arguments
|
||||
* @throws IOException If I/O related errors occur
|
||||
*/
|
||||
public static AbstractAllGroupHeadsCollector create(String groupField, Sort sortWithinGroup, ValueType type, boolean diskResident) throws IOException {
|
||||
public static AbstractAllGroupHeadsCollector create(String groupField, Sort sortWithinGroup, DocValues.Type type, boolean diskResident) throws IOException {
|
||||
switch (type) {
|
||||
case VAR_INTS:
|
||||
case FIXED_INTS_8:
|
||||
|
@ -126,8 +126,8 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
|
||||
this.readerContext = readerContext;
|
||||
|
||||
final IndexDocValues dv = readerContext.reader.docValues(groupField);
|
||||
final IndexDocValues.Source dvSource;
|
||||
final DocValues dv = readerContext.reader.docValues(groupField);
|
||||
final DocValues.Source dvSource;
|
||||
if (dv != null) {
|
||||
dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
|
||||
} else {
|
||||
|
@ -141,14 +141,14 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
*
|
||||
* @param source The idv source to be used by concrete implementations
|
||||
*/
|
||||
protected abstract void setDocValuesSources(IndexDocValues.Source source);
|
||||
protected abstract void setDocValuesSources(DocValues.Source source);
|
||||
|
||||
/**
|
||||
* @return The default source when no doc values are available.
|
||||
* @param readerContext The current reader context
|
||||
*/
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSource(valueType);
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSource(valueType);
|
||||
}
|
||||
|
||||
// A general impl that works for any group sort.
|
||||
|
@ -157,7 +157,7 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
private final Sort sortWithinGroup;
|
||||
private final Map<Comparable, GroupHead> groups;
|
||||
|
||||
GeneralAllGroupHeadsCollector(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
GeneralAllGroupHeadsCollector(String groupField, DocValues.Type valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
super(groupField, valueType, sortWithinGroup.getSort().length, diskResident);
|
||||
this.sortWithinGroup = sortWithinGroup;
|
||||
groups = new HashMap<Comparable, GroupHead>();
|
||||
|
@ -211,9 +211,9 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
|
||||
static class SortedBR extends GeneralAllGroupHeadsCollector {
|
||||
|
||||
private IndexDocValues.SortedSource source;
|
||||
private DocValues.SortedSource source;
|
||||
|
||||
SortedBR(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
SortedBR(String groupField, DocValues.Type valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
super(groupField, valueType, sortWithinGroup, diskResident);
|
||||
}
|
||||
|
||||
|
@ -225,21 +225,21 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
return BytesRef.deepCopyOf((BytesRef) value);
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.source = source.asSortedSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
}
|
||||
}
|
||||
|
||||
static class BR extends GeneralAllGroupHeadsCollector {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
BR(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
BR(String groupField, DocValues.Type valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
super(groupField, valueType, sortWithinGroup, diskResident);
|
||||
}
|
||||
|
||||
|
@ -251,7 +251,7 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
return BytesRef.deepCopyOf((BytesRef) value);
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
|
@ -259,9 +259,9 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
|
||||
static class Lng extends GeneralAllGroupHeadsCollector {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Lng(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
Lng(String groupField, DocValues.Type valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
super(groupField, valueType, sortWithinGroup, diskResident);
|
||||
}
|
||||
|
||||
|
@ -273,16 +273,16 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
return value;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
}
|
||||
|
||||
static class Dbl extends GeneralAllGroupHeadsCollector {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Dbl(String groupField, ValueType valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
Dbl(String groupField, DocValues.Type valueType, Sort sortWithinGroup, boolean diskResident) throws IOException {
|
||||
super(groupField, valueType, sortWithinGroup, diskResident);
|
||||
}
|
||||
|
||||
|
@ -294,7 +294,7 @@ public abstract class DVAllGroupHeadsCollector<GH extends AbstractAllGroupHeadsC
|
|||
return value;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
|
|
|
@ -17,9 +17,9 @@ package org.apache.lucene.search.grouping.dv;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
|
||||
import org.apache.lucene.search.grouping.SentinelIntSet;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -29,7 +29,7 @@ import java.util.*;
|
|||
|
||||
/**
|
||||
* Implementation of {@link AbstractAllGroupsCollector} that groups documents based on
|
||||
* {@link IndexDocValues} fields.
|
||||
* {@link DocValues} fields.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
@ -39,20 +39,20 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
|
||||
/**
|
||||
* Expert: Constructs a {@link DVAllGroupsCollector}.
|
||||
* Selects and constructs the most optimal all groups collector implementation for grouping by {@link IndexDocValues}.
|
||||
* Selects and constructs the most optimal all groups collector implementation for grouping by {@link DocValues}.
|
||||
*
|
||||
*
|
||||
* @param groupField The field to group by
|
||||
* @param type The {@link ValueType} which is used to select a concrete implementation.
|
||||
* @param type The {@link Type} which is used to select a concrete implementation.
|
||||
* @param diskResident Whether the values to group by should be disk resident
|
||||
* @param initialSize The initial allocation size of the
|
||||
* internal int set and group list
|
||||
* which should roughly match the total
|
||||
* number of expected unique groups. Be aware that the
|
||||
* heap usage is 4 bytes * initialSize. Not all concrete implementions use this!
|
||||
* @return the most optimal all groups collector implementation for grouping by {@link IndexDocValues}
|
||||
* @return the most optimal all groups collector implementation for grouping by {@link DocValues}
|
||||
*/
|
||||
public static DVAllGroupsCollector create(String groupField, ValueType type, boolean diskResident, int initialSize) {
|
||||
public static DVAllGroupsCollector create(String groupField, DocValues.Type type, boolean diskResident, int initialSize) {
|
||||
switch (type) {
|
||||
case VAR_INTS:
|
||||
case FIXED_INTS_8:
|
||||
|
@ -78,25 +78,25 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
|
||||
/**
|
||||
* Constructs a {@link DVAllGroupsCollector}.
|
||||
* Selects and constructs the most optimal all groups collector implementation for grouping by {@link IndexDocValues}.
|
||||
* Selects and constructs the most optimal all groups collector implementation for grouping by {@link DocValues}.
|
||||
* If implementations require an initial allocation size then this will be set to 128.
|
||||
*
|
||||
*
|
||||
* @param groupField The field to group by
|
||||
* @param type The {@link ValueType} which is used to select a concrete implementation.
|
||||
* @param type The {@link Type} which is used to select a concrete implementation.
|
||||
* @param diskResident Wether the values to group by should be disk resident
|
||||
* @return the most optimal all groups collector implementation for grouping by {@link IndexDocValues}
|
||||
* @return the most optimal all groups collector implementation for grouping by {@link DocValues}
|
||||
*/
|
||||
public static DVAllGroupsCollector create(String groupField, ValueType type, boolean diskResident) {
|
||||
public static DVAllGroupsCollector create(String groupField, DocValues.Type type, boolean diskResident) {
|
||||
return create(groupField, type, diskResident, DEFAULT_INITIAL_SIZE);
|
||||
}
|
||||
|
||||
final String groupField;
|
||||
final ValueType valueType;
|
||||
final DocValues.Type valueType;
|
||||
final boolean diskResident;
|
||||
final Collection<GROUP_VALUE_TYPE> groups;
|
||||
|
||||
DVAllGroupsCollector(String groupField, ValueType valueType, boolean diskResident, Collection<GROUP_VALUE_TYPE> groups) {
|
||||
DVAllGroupsCollector(String groupField, DocValues.Type valueType, boolean diskResident, Collection<GROUP_VALUE_TYPE> groups) {
|
||||
this.groupField = groupField;
|
||||
this.valueType = valueType;
|
||||
this.diskResident = diskResident;
|
||||
|
@ -105,8 +105,8 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
|
||||
@Override
|
||||
public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
|
||||
final IndexDocValues dv = readerContext.reader.docValues(groupField);
|
||||
final IndexDocValues.Source dvSource;
|
||||
final DocValues dv = readerContext.reader.docValues(groupField);
|
||||
final DocValues.Source dvSource;
|
||||
if (dv != null) {
|
||||
dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
|
||||
} else {
|
||||
|
@ -121,21 +121,21 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
* @param source The idv source to be used by concrete implementations
|
||||
* @param readerContext The current reader context
|
||||
*/
|
||||
protected abstract void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext);
|
||||
protected abstract void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext);
|
||||
|
||||
/**
|
||||
* @return The default source when no doc values are available.
|
||||
* @param readerContext The current reader context
|
||||
*/
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSource(valueType);
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSource(valueType);
|
||||
}
|
||||
|
||||
static class Lng extends DVAllGroupsCollector<Long> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Lng(String groupField, ValueType valueType, boolean diskResident) {
|
||||
Lng(String groupField, DocValues.Type valueType, boolean diskResident) {
|
||||
super(groupField, valueType, diskResident, new TreeSet<Long>());
|
||||
}
|
||||
|
||||
|
@ -150,7 +150,7 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
return groups;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
|
@ -158,9 +158,9 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
|
||||
static class Dbl extends DVAllGroupsCollector<Double> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Dbl(String groupField, ValueType valueType, boolean diskResident) {
|
||||
Dbl(String groupField, DocValues.Type valueType, boolean diskResident) {
|
||||
super(groupField, valueType, diskResident, new TreeSet<Double>());
|
||||
}
|
||||
|
||||
|
@ -175,7 +175,7 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
return groups;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
|
@ -185,9 +185,9 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
|
||||
private final BytesRef spare = new BytesRef();
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
BR(String groupField, ValueType valueType, boolean diskResident) {
|
||||
BR(String groupField, DocValues.Type valueType, boolean diskResident) {
|
||||
super(groupField, valueType, diskResident, new TreeSet<BytesRef>());
|
||||
}
|
||||
|
||||
|
@ -202,7 +202,7 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
return groups;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
|
@ -213,9 +213,9 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
private final SentinelIntSet ordSet;
|
||||
private final BytesRef spare = new BytesRef();
|
||||
|
||||
private IndexDocValues.SortedSource source;
|
||||
private DocValues.SortedSource source;
|
||||
|
||||
SortedBR(String groupField, ValueType valueType, boolean diskResident, int initialSize) {
|
||||
SortedBR(String groupField, DocValues.Type valueType, boolean diskResident, int initialSize) {
|
||||
super(groupField, valueType, diskResident, new ArrayList<BytesRef>(initialSize));
|
||||
ordSet = new SentinelIntSet(initialSize, -1);
|
||||
}
|
||||
|
@ -233,7 +233,7 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
return groups;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source.asSortedSource();
|
||||
|
||||
ordSet.clear();
|
||||
|
@ -246,8 +246,8 @@ public abstract class DVAllGroupsCollector<GROUP_VALUE_TYPE> extends AbstractAll
|
|||
}
|
||||
|
||||
@Override
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,9 +17,9 @@ package org.apache.lucene.search.grouping.dv;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -35,9 +35,9 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
|
||||
final String groupField;
|
||||
final boolean diskResident;
|
||||
final ValueType valueType;
|
||||
final DocValues.Type valueType;
|
||||
|
||||
public static DVFirstPassGroupingCollector create(Sort groupSort, int topNGroups, String groupField, ValueType type, boolean diskResident) throws IOException {
|
||||
public static DVFirstPassGroupingCollector create(Sort groupSort, int topNGroups, String groupField, DocValues.Type type, boolean diskResident) throws IOException {
|
||||
switch (type) {
|
||||
case VAR_INTS:
|
||||
case FIXED_INTS_8:
|
||||
|
@ -61,7 +61,7 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
}
|
||||
}
|
||||
|
||||
DVFirstPassGroupingCollector(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType valueType) throws IOException {
|
||||
DVFirstPassGroupingCollector(Sort groupSort, int topNGroups, String groupField, boolean diskResident, DocValues.Type valueType) throws IOException {
|
||||
super(groupSort, topNGroups);
|
||||
this.groupField = groupField;
|
||||
this.diskResident = diskResident;
|
||||
|
@ -72,8 +72,8 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
|
||||
super.setNextReader(readerContext);
|
||||
|
||||
final IndexDocValues dv = readerContext.reader.docValues(groupField);
|
||||
final IndexDocValues.Source dvSource;
|
||||
final DocValues dv = readerContext.reader.docValues(groupField);
|
||||
final DocValues.Source dvSource;
|
||||
if (dv != null) {
|
||||
dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
|
||||
} else {
|
||||
|
@ -87,21 +87,21 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
*
|
||||
* @param source The idv source to be used by concrete implementations
|
||||
*/
|
||||
protected abstract void setDocValuesSources(IndexDocValues.Source source);
|
||||
protected abstract void setDocValuesSources(DocValues.Source source);
|
||||
|
||||
/**
|
||||
* @return The default source when no doc values are available.
|
||||
* @param readerContext The current reader context
|
||||
*/
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSource(valueType);
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSource(valueType);
|
||||
}
|
||||
|
||||
static class Lng extends DVFirstPassGroupingCollector<Long> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Lng(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
|
||||
Lng(Sort groupSort, int topNGroups, String groupField, boolean diskResident, DocValues.Type type) throws IOException {
|
||||
super(groupSort, topNGroups, groupField, diskResident, type);
|
||||
}
|
||||
|
||||
|
@ -113,16 +113,16 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
return groupValue;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
}
|
||||
|
||||
static class Dbl extends DVFirstPassGroupingCollector<Double> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Dbl(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
|
||||
Dbl(Sort groupSort, int topNGroups, String groupField, boolean diskResident, DocValues.Type type) throws IOException {
|
||||
super(groupSort, topNGroups, groupField, diskResident, type);
|
||||
}
|
||||
|
||||
|
@ -134,17 +134,17 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
return groupValue;
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
}
|
||||
|
||||
static class BR extends DVFirstPassGroupingCollector<BytesRef> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
private final BytesRef spare = new BytesRef();
|
||||
|
||||
BR(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
|
||||
BR(Sort groupSort, int topNGroups, String groupField, boolean diskResident, DocValues.Type type) throws IOException {
|
||||
super(groupSort, topNGroups, groupField, diskResident, type);
|
||||
}
|
||||
|
||||
|
@ -162,17 +162,17 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
}
|
||||
|
||||
static class SortedBR extends DVFirstPassGroupingCollector<BytesRef> {
|
||||
|
||||
private IndexDocValues.SortedSource sortedSource;
|
||||
private DocValues.SortedSource sortedSource;
|
||||
private final BytesRef spare = new BytesRef();
|
||||
|
||||
SortedBR(Sort groupSort, int topNGroups, String groupField, boolean diskResident, ValueType type) throws IOException {
|
||||
SortedBR(Sort groupSort, int topNGroups, String groupField, boolean diskResident, DocValues.Type type) throws IOException {
|
||||
super(groupSort, topNGroups, groupField, diskResident, type);
|
||||
}
|
||||
|
||||
|
@ -192,13 +192,13 @@ public abstract class DVFirstPassGroupingCollector<GROUP_VALUE_TYPE> extends Abs
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void setDocValuesSources(IndexDocValues.Source source) {
|
||||
protected void setDocValuesSources(DocValues.Source source) {
|
||||
this.sortedSource = source.asSortedSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -17,9 +17,9 @@ package org.apache.lucene.search.grouping.dv;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
|
||||
import org.apache.lucene.search.grouping.SearchGroup;
|
||||
|
@ -38,11 +38,11 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
|
||||
/**
|
||||
* Constructs a {@link DVSecondPassGroupingCollector}.
|
||||
* Selects and constructs the most optimal second pass collector implementation for grouping by {@link IndexDocValues}.
|
||||
* Selects and constructs the most optimal second pass collector implementation for grouping by {@link DocValues}.
|
||||
*
|
||||
* @param groupField The field to group by
|
||||
* @param diskResident Whether the values to group by should be disk resident
|
||||
* @param type The {@link org.apache.lucene.index.values.ValueType} which is used to select a concrete implementation.
|
||||
* @param type The {@link Type} which is used to select a concrete implementation.
|
||||
* @param searchGroups The groups from the first phase search
|
||||
* @param groupSort The sort used for the groups
|
||||
* @param withinGroupSort The sort used for documents inside a group
|
||||
|
@ -50,13 +50,13 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
* @param getScores Whether to include scores for the documents inside a group
|
||||
* @param getMaxScores Whether to keep track of the higest score per group
|
||||
* @param fillSortFields Whether to include the sort values
|
||||
* @return the most optimal second pass collector implementation for grouping by {@link IndexDocValues}
|
||||
* @return the most optimal second pass collector implementation for grouping by {@link DocValues}
|
||||
* @throws IOException If I/O related errors occur
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public static DVSecondPassGroupingCollector create(String groupField,
|
||||
boolean diskResident,
|
||||
ValueType type,
|
||||
DocValues.Type type,
|
||||
Collection<SearchGroup> searchGroups,
|
||||
Sort groupSort,
|
||||
Sort withinGroupSort,
|
||||
|
@ -92,10 +92,10 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
}
|
||||
|
||||
final String groupField;
|
||||
final ValueType valueType;
|
||||
final DocValues.Type valueType;
|
||||
final boolean diskResident;
|
||||
|
||||
DVSecondPassGroupingCollector(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<GROUP_VALUE>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
DVSecondPassGroupingCollector(String groupField, DocValues.Type valueType, boolean diskResident, Collection<SearchGroup<GROUP_VALUE>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
super(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
this.groupField = groupField;
|
||||
this.valueType = valueType;
|
||||
|
@ -106,8 +106,8 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
|
||||
super.setNextReader(readerContext);
|
||||
|
||||
final IndexDocValues dv = readerContext.reader.docValues(groupField);
|
||||
final IndexDocValues.Source dvSource;
|
||||
final DocValues dv = readerContext.reader.docValues(groupField);
|
||||
final DocValues.Source dvSource;
|
||||
if (dv != null) {
|
||||
dvSource = diskResident ? dv.getDirectSource() : dv.getSource();
|
||||
} else {
|
||||
|
@ -122,21 +122,21 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
* @param source The idv source to be used by concrete implementations
|
||||
* @param readerContext The current reader context
|
||||
*/
|
||||
protected abstract void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext);
|
||||
protected abstract void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext);
|
||||
|
||||
/**
|
||||
* @return The default source when no doc values are available.
|
||||
* @param readerContext The current reader context
|
||||
*/
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSource(valueType);
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSource(valueType);
|
||||
}
|
||||
|
||||
static class Lng extends DVSecondPassGroupingCollector<Long> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Lng(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<Long>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
Lng(String groupField, DocValues.Type valueType, boolean diskResident, Collection<SearchGroup<Long>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
}
|
||||
|
||||
|
@ -144,16 +144,16 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
return groupMap.get(source.getInt(doc));
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source;
|
||||
}
|
||||
}
|
||||
|
||||
static class Dbl extends DVSecondPassGroupingCollector<Double> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
|
||||
Dbl(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<Double>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
Dbl(String groupField, DocValues.Type valueType, boolean diskResident, Collection<SearchGroup<Double>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
}
|
||||
|
||||
|
@ -161,17 +161,17 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
return groupMap.get(source.getFloat(doc));
|
||||
}
|
||||
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source;
|
||||
}
|
||||
}
|
||||
|
||||
static class BR extends DVSecondPassGroupingCollector<BytesRef> {
|
||||
|
||||
private IndexDocValues.Source source;
|
||||
private DocValues.Source source;
|
||||
private final BytesRef spare = new BytesRef();
|
||||
|
||||
BR(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<BytesRef>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
BR(String groupField, DocValues.Type valueType, boolean diskResident, Collection<SearchGroup<BytesRef>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
}
|
||||
|
||||
|
@ -180,7 +180,7 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
|
@ -188,12 +188,12 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
|
||||
static class SortedBR extends DVSecondPassGroupingCollector<BytesRef> {
|
||||
|
||||
private IndexDocValues.SortedSource source;
|
||||
private DocValues.SortedSource source;
|
||||
private final BytesRef spare = new BytesRef();
|
||||
private final SentinelIntSet ordSet;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
SortedBR(String groupField, ValueType valueType, boolean diskResident, Collection<SearchGroup<BytesRef>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
SortedBR(String groupField, DocValues.Type valueType, boolean diskResident, Collection<SearchGroup<BytesRef>> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException {
|
||||
super(groupField, valueType, diskResident, searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
ordSet = new SentinelIntSet(groupMap.size(), -1);
|
||||
groupDocs = (SearchGroupDocs<BytesRef>[]) new SearchGroupDocs[ordSet.keys.length];
|
||||
|
@ -209,7 +209,7 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void setDocValuesSources(IndexDocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
protected void setDocValuesSources(DocValues.Source source, IndexReader.AtomicReaderContext readerContext) {
|
||||
this.source = source.asSortedSource();
|
||||
|
||||
ordSet.clear();
|
||||
|
@ -222,8 +222,8 @@ public abstract class DVSecondPassGroupingCollector<GROUP_VALUE> extends Abstrac
|
|||
}
|
||||
|
||||
@Override
|
||||
protected IndexDocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return IndexDocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
protected DocValues.Source getDefaultSource(IndexReader.AtomicReaderContext readerContext) {
|
||||
return DocValues.getDefaultSortedSource(valueType, readerContext.reader.maxDoc());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.search.grouping.function;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
|
@ -45,7 +45,7 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
|
|||
private final Map<MutableValue, GroupHead> groups;
|
||||
private final Sort sortWithinGroup;
|
||||
|
||||
private DocValues.ValueFiller filler;
|
||||
private FunctionValues.ValueFiller filler;
|
||||
private MutableValue mval;
|
||||
private IndexReader.AtomicReaderContext readerContext;
|
||||
private Scorer scorer;
|
||||
|
@ -105,7 +105,7 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
|
|||
|
||||
public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
|
||||
this.readerContext = context;
|
||||
DocValues docValues = groupBy.getValues(vsContext, context);
|
||||
FunctionValues docValues = groupBy.getValues(vsContext, context);
|
||||
filler = docValues.getValueFiller();
|
||||
mval = filler.getValue();
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.search.grouping.function;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
|
@ -36,7 +36,7 @@ import java.util.TreeSet;
|
|||
* the most relevant document of a group.
|
||||
*
|
||||
* <p/>
|
||||
* Implementation detail: Uses {@link ValueSource} and {@link DocValues} to retrieve the
|
||||
* Implementation detail: Uses {@link ValueSource} and {@link FunctionValues} to retrieve the
|
||||
* field values to group by.
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
@ -47,7 +47,7 @@ public class FunctionAllGroupsCollector extends AbstractAllGroupsCollector<Mutab
|
|||
private final ValueSource groupBy;
|
||||
private final SortedSet<MutableValue> groups = new TreeSet<MutableValue>();
|
||||
|
||||
private DocValues.ValueFiller filler;
|
||||
private FunctionValues.ValueFiller filler;
|
||||
private MutableValue mval;
|
||||
|
||||
/**
|
||||
|
@ -79,7 +79,7 @@ public class FunctionAllGroupsCollector extends AbstractAllGroupsCollector<Mutab
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
public void setNextReader(IndexReader.AtomicReaderContext context) throws IOException {
|
||||
DocValues docValues = groupBy.getValues(vsContext, context);
|
||||
FunctionValues docValues = groupBy.getValues(vsContext, context);
|
||||
filler = docValues.getValueFiller();
|
||||
mval = filler.getValue();
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.search.grouping.function;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
|
||||
|
@ -38,8 +38,8 @@ public class FunctionFirstPassGroupingCollector extends AbstractFirstPassGroupin
|
|||
private final ValueSource groupByVS;
|
||||
private final Map vsContext;
|
||||
|
||||
private DocValues docValues;
|
||||
private DocValues.ValueFiller filler;
|
||||
private FunctionValues docValues;
|
||||
private FunctionValues.ValueFiller filler;
|
||||
private MutableValue mval;
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.search.grouping.function;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
|
||||
|
@ -41,7 +41,7 @@ public class FunctionSecondPassGroupingCollector extends AbstractSecondPassGroup
|
|||
private final ValueSource groupByVS;
|
||||
private final Map vsContext;
|
||||
|
||||
private DocValues.ValueFiller filler;
|
||||
private FunctionValues.ValueFiller filler;
|
||||
private MutableValue mval;
|
||||
|
||||
/**
|
||||
|
@ -77,7 +77,7 @@ public class FunctionSecondPassGroupingCollector extends AbstractSecondPassGroup
|
|||
*/
|
||||
public void setNextReader(IndexReader.AtomicReaderContext readerContext) throws IOException {
|
||||
super.setNextReader(readerContext);
|
||||
DocValues docValues = groupByVS.getValues(vsContext, readerContext);
|
||||
FunctionValues docValues = groupByVS.getValues(vsContext, readerContext);
|
||||
filler = docValues.getValueFiller();
|
||||
mval = filler.getValue();
|
||||
}
|
||||
|
|
|
@ -186,14 +186,14 @@ fields, <code>FieldCache</code>, etc.).
|
|||
</p>
|
||||
|
||||
<p>
|
||||
There are also IndexDocValues based implementations available for the group collectors. There are factory methods
|
||||
available for creating idv based instances. A typical example using idv based grouping collectors:
|
||||
There are also DocValues based implementations available for the group collectors. There are factory methods
|
||||
available for creating dv based instances. A typical example using dv based grouping collectors:
|
||||
</p>
|
||||
|
||||
<pre class="prettyprint">
|
||||
boolean diskResident = true; // Whether values should fetched directly from disk by passing the Java heap space.
|
||||
AbstractFirstPassGroupingCollector c1 = DVFirstPassGroupingCollector.create(
|
||||
groupSort, groupOffset+topNGroups, "author", ValueType.BYTES_VAR_SORTED, diskResident
|
||||
groupSort, groupOffset+topNGroups, "author", DocValues.Type.BYTES_VAR_SORTED, diskResident
|
||||
);
|
||||
|
||||
s.search(new TermQuery(new Term("content", searchTerm)), c1);
|
||||
|
@ -209,7 +209,7 @@ fields, <code>FieldCache</code>, etc.).
|
|||
boolean getMaxScores = true;
|
||||
boolean fillFields = true;
|
||||
AbstractSecondPassGroupingCollector<BytesRef> c2 = DVSecondPassGroupingCollector.create(
|
||||
"author", diskResident, ValueType.BYTES_VAR_SORTED, topGroups, groupSort, docSort,
|
||||
"author", diskResident, DocValues.Type.BYTES_VAR_SORTED, topGroups, groupSort, docSort,
|
||||
docOffset+docsPerGroup, getScores, getMaxScores, fillFields
|
||||
);
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.lucene.index.IndexReader;
|
|||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SlowMultiReaderWrapper;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
|
||||
import org.apache.lucene.search.*;
|
||||
|
@ -41,8 +41,8 @@ import java.util.*;
|
|||
|
||||
public class AllGroupHeadsCollectorTest extends LuceneTestCase {
|
||||
|
||||
private static final ValueType[] vts = new ValueType[]{
|
||||
ValueType.BYTES_VAR_DEREF, ValueType.BYTES_VAR_STRAIGHT, ValueType.BYTES_VAR_SORTED
|
||||
private static final Type[] vts = new Type[]{
|
||||
Type.BYTES_VAR_DEREF, Type.BYTES_VAR_STRAIGHT, Type.BYTES_VAR_SORTED
|
||||
};
|
||||
|
||||
public void testBasic() throws Exception {
|
||||
|
@ -54,7 +54,7 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
|
|||
newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
boolean canUseIDV = !"Lucene3x".equals(w.w.getConfig().getCodec().getName());
|
||||
ValueType valueType = vts[random.nextInt(vts.length)];
|
||||
Type valueType = vts[random.nextInt(vts.length)];
|
||||
|
||||
// 0
|
||||
Document doc = new Document();
|
||||
|
@ -203,15 +203,15 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
|
|||
new MockAnalyzer(random)));
|
||||
boolean preFlex = "Lucene3x".equals(w.w.getConfig().getCodec().getName());
|
||||
boolean canUseIDV = !preFlex;
|
||||
ValueType valueType = vts[random.nextInt(vts.length)];
|
||||
Type valueType = vts[random.nextInt(vts.length)];
|
||||
|
||||
Document doc = new Document();
|
||||
Document docNoGroup = new Document();
|
||||
Field group = newField("group", "", StringField.TYPE_UNSTORED);
|
||||
doc.add(group);
|
||||
IndexDocValuesField valuesField = null;
|
||||
DocValuesField valuesField = null;
|
||||
if (canUseIDV) {
|
||||
valuesField = new IndexDocValuesField("group");
|
||||
valuesField = new DocValuesField("group");
|
||||
doc.add(valuesField);
|
||||
}
|
||||
Field sort1 = newField("sort1", "", StringField.TYPE_UNSTORED);
|
||||
|
@ -505,7 +505,7 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
|
|||
};
|
||||
}
|
||||
|
||||
private AbstractAllGroupHeadsCollector createRandomCollector(String groupField, Sort sortWithinGroup, boolean canUseIDV, ValueType valueType) throws IOException {
|
||||
private AbstractAllGroupHeadsCollector createRandomCollector(String groupField, Sort sortWithinGroup, boolean canUseIDV, Type valueType) throws IOException {
|
||||
AbstractAllGroupHeadsCollector collector;
|
||||
if (random.nextBoolean()) {
|
||||
ValueSource vs = new BytesRefFieldSource(groupField);
|
||||
|
@ -524,10 +524,10 @@ public class AllGroupHeadsCollectorTest extends LuceneTestCase {
|
|||
return collector;
|
||||
}
|
||||
|
||||
private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV, ValueType valueType) {
|
||||
private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV, Type valueType) {
|
||||
doc.add(new Field(groupField, value, TextField.TYPE_STORED));
|
||||
if (canUseIDV) {
|
||||
IndexDocValuesField valuesField = new IndexDocValuesField(groupField);
|
||||
DocValuesField valuesField = new DocValuesField(groupField);
|
||||
valuesField.setBytes(new BytesRef(value), valueType);
|
||||
doc.add(valuesField);
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.document.*;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -123,8 +123,8 @@ public class AllGroupsCollectorTest extends LuceneTestCase {
|
|||
private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV) {
|
||||
doc.add(new Field(groupField, value, TextField.TYPE_STORED));
|
||||
if (canUseIDV) {
|
||||
IndexDocValuesField valuesField = new IndexDocValuesField(groupField);
|
||||
valuesField.setBytes(new BytesRef(value), ValueType.BYTES_VAR_SORTED);
|
||||
DocValuesField valuesField = new DocValuesField(groupField);
|
||||
valuesField.setBytes(new BytesRef(value), Type.BYTES_VAR_SORTED);
|
||||
doc.add(valuesField);
|
||||
}
|
||||
}
|
||||
|
@ -133,7 +133,7 @@ public class AllGroupsCollectorTest extends LuceneTestCase {
|
|||
AbstractAllGroupsCollector selected;
|
||||
if (random.nextBoolean() && canUseIDV) {
|
||||
boolean diskResident = random.nextBoolean();
|
||||
selected = DVAllGroupsCollector.create(groupField, ValueType.BYTES_VAR_SORTED, diskResident);
|
||||
selected = DVAllGroupsCollector.create(groupField, Type.BYTES_VAR_SORTED, diskResident);
|
||||
} else if (random.nextBoolean()) {
|
||||
selected = new TermAllGroupsCollector(groupField);
|
||||
} else {
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.index.IndexReader;
|
|||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SlowMultiReaderWrapper;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.valuesource.BytesRefFieldSource;
|
||||
import org.apache.lucene.search.*;
|
||||
|
@ -171,8 +171,8 @@ public class TestGrouping extends LuceneTestCase {
|
|||
private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV) {
|
||||
doc.add(new Field(groupField, value, TextField.TYPE_STORED));
|
||||
if (canUseIDV) {
|
||||
IndexDocValuesField valuesField = new IndexDocValuesField(groupField);
|
||||
valuesField.setBytes(new BytesRef(value), ValueType.BYTES_VAR_SORTED);
|
||||
DocValuesField valuesField = new DocValuesField(groupField);
|
||||
valuesField.setBytes(new BytesRef(value), Type.BYTES_VAR_SORTED);
|
||||
doc.add(valuesField);
|
||||
}
|
||||
}
|
||||
|
@ -181,7 +181,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
AbstractFirstPassGroupingCollector selected;
|
||||
if (canUseIDV && random.nextBoolean()) {
|
||||
boolean diskResident = random.nextBoolean();
|
||||
selected = DVFirstPassGroupingCollector.create(groupSort, topDocs, groupField, ValueType.BYTES_VAR_SORTED, diskResident);
|
||||
selected = DVFirstPassGroupingCollector.create(groupSort, topDocs, groupField, Type.BYTES_VAR_SORTED, diskResident);
|
||||
} else if (random.nextBoolean()) {
|
||||
ValueSource vs = new BytesRefFieldSource(groupField);
|
||||
selected = new FunctionFirstPassGroupingCollector(vs, new HashMap(), groupSort, topDocs);
|
||||
|
@ -197,7 +197,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
private AbstractFirstPassGroupingCollector createFirstPassCollector(String groupField, Sort groupSort, int topDocs, AbstractFirstPassGroupingCollector firstPassGroupingCollector) throws IOException {
|
||||
if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
|
||||
boolean diskResident = random.nextBoolean();
|
||||
return DVFirstPassGroupingCollector.create(groupSort, topDocs, groupField, ValueType.BYTES_VAR_SORTED, diskResident);
|
||||
return DVFirstPassGroupingCollector.create(groupSort, topDocs, groupField, Type.BYTES_VAR_SORTED, diskResident);
|
||||
} else if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
|
||||
ValueSource vs = new BytesRefFieldSource(groupField);
|
||||
return new FunctionFirstPassGroupingCollector(vs, new HashMap(), groupSort, topDocs);
|
||||
|
@ -220,7 +220,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
|
||||
boolean diskResident = random.nextBoolean();
|
||||
Collection<SearchGroup> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
|
||||
return DVSecondPassGroupingCollector.create(groupField, diskResident, ValueType.BYTES_VAR_SORTED, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
return DVSecondPassGroupingCollector.create(groupField, diskResident, Type.BYTES_VAR_SORTED, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
} else if (TermFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
|
||||
Collection<SearchGroup<BytesRef>> searchGroups = firstPassGroupingCollector.getTopGroups(groupOffset, fillSortFields);
|
||||
return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
|
||||
|
@ -244,7 +244,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
boolean fillSortFields) throws IOException {
|
||||
if (DVFirstPassGroupingCollector.class.isAssignableFrom(firstPassGroupingCollector.getClass())) {
|
||||
boolean diskResident = random.nextBoolean();
|
||||
return DVSecondPassGroupingCollector.create(groupField, diskResident, ValueType.BYTES_VAR_SORTED, (Collection) searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
return DVSecondPassGroupingCollector.create(groupField, diskResident, Type.BYTES_VAR_SORTED, (Collection) searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
|
||||
} else if (firstPassGroupingCollector.getClass().isAssignableFrom(TermFirstPassGroupingCollector.class)) {
|
||||
return new TermSecondPassGroupingCollector(groupField, searchGroups, groupSort, sortWithinGroup, maxDocsPerGroup , getScores, getMaxScores, fillSortFields);
|
||||
} else {
|
||||
|
@ -274,7 +274,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
return new TermAllGroupsCollector(groupField);
|
||||
} else if (firstPassGroupingCollector.getClass().isAssignableFrom(DVFirstPassGroupingCollector.class)) {
|
||||
boolean diskResident = random.nextBoolean();
|
||||
return DVAllGroupsCollector.create(groupField, ValueType.BYTES_VAR_SORTED, diskResident);
|
||||
return DVAllGroupsCollector.create(groupField, Type.BYTES_VAR_SORTED, diskResident);
|
||||
} else {
|
||||
ValueSource vs = new BytesRefFieldSource(groupField);
|
||||
return new FunctionAllGroupsCollector(vs, new HashMap());
|
||||
|
@ -705,7 +705,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
|
||||
Document doc = new Document();
|
||||
Document docNoGroup = new Document();
|
||||
IndexDocValuesField idvGroupField = new IndexDocValuesField("group");
|
||||
DocValuesField idvGroupField = new DocValuesField("group");
|
||||
if (canUseIDV) {
|
||||
doc.add(idvGroupField);
|
||||
}
|
||||
|
@ -747,7 +747,7 @@ public class TestGrouping extends LuceneTestCase {
|
|||
if (groupDoc.group != null) {
|
||||
group.setValue(groupDoc.group.utf8ToString());
|
||||
if (canUseIDV) {
|
||||
idvGroupField.setBytes(BytesRef.deepCopyOf(groupDoc.group), ValueType.BYTES_VAR_SORTED);
|
||||
idvGroupField.setBytes(BytesRef.deepCopyOf(groupDoc.group), Type.BYTES_VAR_SORTED);
|
||||
}
|
||||
}
|
||||
sort1.setValue(groupDoc.sort1.utf8ToString());
|
||||
|
|
|
@ -108,7 +108,7 @@ public class BoostedQuery extends Query {
|
|||
if (!subQueryExpl.isMatch()) {
|
||||
return subQueryExpl;
|
||||
}
|
||||
DocValues vals = boostVal.getValues(fcontext, readerContext);
|
||||
FunctionValues vals = boostVal.getValues(fcontext, readerContext);
|
||||
float sc = subQueryExpl.getValue() * vals.floatVal(doc);
|
||||
Explanation res = new ComplexExplanation(
|
||||
true, sc, BoostedQuery.this.toString() + ", product of:");
|
||||
|
@ -123,7 +123,7 @@ public class BoostedQuery extends Query {
|
|||
private final BoostedQuery.BoostedWeight weight;
|
||||
private final float qWeight;
|
||||
private final Scorer scorer;
|
||||
private final DocValues vals;
|
||||
private final FunctionValues vals;
|
||||
private final AtomicReaderContext readerContext;
|
||||
|
||||
private CustomScorer(AtomicReaderContext readerContext, BoostedQuery.BoostedWeight w, float qWeight,
|
||||
|
|
|
@ -106,7 +106,7 @@ public class FunctionQuery extends Query {
|
|||
final int maxDoc;
|
||||
final float qWeight;
|
||||
int doc=-1;
|
||||
final DocValues vals;
|
||||
final FunctionValues vals;
|
||||
final Bits liveDocs;
|
||||
|
||||
public AllScorer(AtomicReaderContext context, Bits acceptDocs, FunctionWeight w, float qWeight) throws IOException {
|
||||
|
|
|
@ -30,13 +30,13 @@ import org.apache.lucene.util.mutable.MutableValueFloat;
|
|||
*
|
||||
*/
|
||||
|
||||
// DocValues is distinct from ValueSource because
|
||||
// FunctionValues is distinct from ValueSource because
|
||||
// there needs to be an object created at query evaluation time that
|
||||
// is not referenced by the query itself because:
|
||||
// - Query objects should be MT safe
|
||||
// - For caching, Query objects are often used as keys... you don't
|
||||
// want the Query carrying around big objects
|
||||
public abstract class DocValues {
|
||||
public abstract class FunctionValues {
|
||||
|
||||
public byte byteVal(int doc) { throw new UnsupportedOperationException(); }
|
||||
public short shortVal(int doc) { throw new UnsupportedOperationException(); }
|
||||
|
@ -65,7 +65,7 @@ public abstract class DocValues {
|
|||
|
||||
/** Native Java Object representation of the value */
|
||||
public Object objectVal(int doc) {
|
||||
// most DocValues are functions, so by default return a Float()
|
||||
// most FunctionValues are functions, so by default return a Float()
|
||||
return floatVal(doc);
|
||||
}
|
||||
|
|
@ -33,7 +33,7 @@ import java.util.IdentityHashMap;
|
|||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Instantiates {@link DocValues} for a particular reader.
|
||||
* Instantiates {@link FunctionValues} for a particular reader.
|
||||
* <br>
|
||||
* Often used when creating a {@link FunctionQuery}.
|
||||
*
|
||||
|
@ -45,7 +45,7 @@ public abstract class ValueSource implements Serializable {
|
|||
* Gets the values for this reader and the context that was previously
|
||||
* passed to createWeight()
|
||||
*/
|
||||
public abstract DocValues getValues(Map context, AtomicReaderContext readerContext) throws IOException;
|
||||
public abstract FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException;
|
||||
|
||||
@Override
|
||||
public abstract boolean equals(Object o);
|
||||
|
@ -129,12 +129,12 @@ public abstract class ValueSource implements Serializable {
|
|||
|
||||
/**
|
||||
* Implement a {@link org.apache.lucene.search.FieldComparator} that works
|
||||
* off of the {@link DocValues} for a ValueSource
|
||||
* off of the {@link FunctionValues} for a ValueSource
|
||||
* instead of the normal Lucene FieldComparator that works off of a FieldCache.
|
||||
*/
|
||||
class ValueSourceComparator extends FieldComparator<Double> {
|
||||
private final double[] values;
|
||||
private DocValues docVals;
|
||||
private FunctionValues docVals;
|
||||
private double bottom;
|
||||
private Map fcontext;
|
||||
|
||||
|
|
|
@ -28,11 +28,11 @@ public class ValueSourceScorer extends Scorer {
|
|||
protected IndexReader reader;
|
||||
private int doc = -1;
|
||||
protected final int maxDoc;
|
||||
protected final DocValues values;
|
||||
protected final FunctionValues values;
|
||||
protected boolean checkDeletes;
|
||||
private final Bits liveDocs;
|
||||
|
||||
protected ValueSourceScorer(IndexReader reader, DocValues values) {
|
||||
protected ValueSourceScorer(IndexReader reader, FunctionValues values) {
|
||||
super(null);
|
||||
this.reader = reader;
|
||||
this.maxDoc = reader.maxDoc();
|
||||
|
|
|
@ -1,12 +1,12 @@
|
|||
package org.apache.lucene.queries.function.docvalues;
|
||||
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueBool;
|
||||
|
||||
|
||||
public abstract class BoolDocValues extends DocValues {
|
||||
public abstract class BoolDocValues extends FunctionValues {
|
||||
protected final ValueSource vs;
|
||||
|
||||
public BoolDocValues(ValueSource vs) {
|
||||
|
|
|
@ -1,11 +1,11 @@
|
|||
package org.apache.lucene.queries.function.docvalues;
|
||||
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueDouble;
|
||||
|
||||
public abstract class DoubleDocValues extends DocValues {
|
||||
public abstract class DoubleDocValues extends FunctionValues {
|
||||
protected final ValueSource vs;
|
||||
|
||||
public DoubleDocValues(ValueSource vs) {
|
||||
|
|
|
@ -1,11 +1,11 @@
|
|||
package org.apache.lucene.queries.function.docvalues;
|
||||
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueFloat;
|
||||
|
||||
public abstract class FloatDocValues extends DocValues {
|
||||
public abstract class FloatDocValues extends FunctionValues {
|
||||
protected final ValueSource vs;
|
||||
|
||||
public FloatDocValues(ValueSource vs) {
|
||||
|
|
|
@ -1,12 +1,12 @@
|
|||
package org.apache.lucene.queries.function.docvalues;
|
||||
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueInt;
|
||||
|
||||
|
||||
public abstract class IntDocValues extends DocValues {
|
||||
public abstract class IntDocValues extends FunctionValues {
|
||||
protected final ValueSource vs;
|
||||
|
||||
public IntDocValues(ValueSource vs) {
|
||||
|
|
|
@ -1,12 +1,12 @@
|
|||
package org.apache.lucene.queries.function.docvalues;
|
||||
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueLong;
|
||||
|
||||
|
||||
public abstract class LongDocValues extends DocValues {
|
||||
public abstract class LongDocValues extends FunctionValues {
|
||||
protected final ValueSource vs;
|
||||
|
||||
public LongDocValues(ValueSource vs) {
|
||||
|
|
|
@ -1,11 +1,11 @@
|
|||
package org.apache.lucene.queries.function.docvalues;
|
||||
|
||||
import org.apache.lucene.queries.function.DocValues;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueStr;
|
||||
|
||||
public abstract class StrDocValues extends DocValues {
|
||||
public abstract class StrDocValues extends FunctionValues {
|
||||
protected final ValueSource vs;
|
||||
|
||||
public StrDocValues(ValueSource vs) {
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue