mirror of https://github.com/apache/lucene.git
LUCENE-3074: simple text doc values
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1297920 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0b348028ed
commit
786d3bcf14
|
@ -0,0 +1,513 @@
|
|||
package org.apache.lucene.codecs;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumMap;
|
||||
import java.util.Map;
|
||||
|
||||
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;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
* @lucene.internal
|
||||
*/
|
||||
public abstract class DocValuesArraySource extends Source {
|
||||
|
||||
private static final Map<Type, DocValuesArraySource> TEMPLATES;
|
||||
|
||||
static {
|
||||
EnumMap<Type, DocValuesArraySource> templates = new EnumMap<Type, DocValuesArraySource>(
|
||||
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);
|
||||
}
|
||||
|
||||
public static DocValuesArraySource forType(Type type) {
|
||||
return TEMPLATES.get(type);
|
||||
}
|
||||
|
||||
protected final int bytesPerValue;
|
||||
|
||||
DocValuesArraySource(int bytesPerValue, Type type) {
|
||||
super(type);
|
||||
this.bytesPerValue = bytesPerValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract BytesRef getBytes(int docID, BytesRef ref);
|
||||
|
||||
|
||||
public abstract DocValuesArraySource newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException;
|
||||
|
||||
public abstract DocValuesArraySource newFromArray(Object array);
|
||||
|
||||
@Override
|
||||
public final boolean hasArray() {
|
||||
return true;
|
||||
}
|
||||
|
||||
public void toBytes(long value, BytesRef bytesRef) {
|
||||
copyLong(bytesRef, value);
|
||||
}
|
||||
|
||||
public void toBytes(double value, BytesRef bytesRef) {
|
||||
copyLong(bytesRef, Double.doubleToRawLongBits(value));
|
||||
}
|
||||
|
||||
final static class ByteValues extends DocValuesArraySource {
|
||||
private final byte[] values;
|
||||
|
||||
ByteValues() {
|
||||
super(1, Type.FIXED_INTS_8);
|
||||
values = new byte[0];
|
||||
}
|
||||
private ByteValues(byte[] array) {
|
||||
super(1, Type.FIXED_INTS_8);
|
||||
values = array;
|
||||
}
|
||||
|
||||
private ByteValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(1, Type.FIXED_INTS_8);
|
||||
values = new byte[numDocs];
|
||||
input.readBytes(values, 0, values.length, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new ByteValues(input, numDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromArray(Object array) {
|
||||
assert array instanceof byte[];
|
||||
return new ByteValues((byte[]) array);
|
||||
}
|
||||
|
||||
public void toBytes(long value, BytesRef bytesRef) {
|
||||
if (bytesRef.bytes.length == 0) {
|
||||
bytesRef.bytes = new byte[1];
|
||||
}
|
||||
bytesRef.bytes[0] = (byte) (0xFFL & value);
|
||||
bytesRef.offset = 0;
|
||||
bytesRef.length = 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
toBytes(getInt(docID), ref);
|
||||
return ref;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class ShortValues extends DocValuesArraySource {
|
||||
private final short[] values;
|
||||
|
||||
ShortValues() {
|
||||
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
|
||||
values = new short[0];
|
||||
}
|
||||
|
||||
private ShortValues(short[] array) {
|
||||
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
|
||||
values = array;
|
||||
}
|
||||
|
||||
private ShortValues(IndexInput input, int numDocs) throws IOException {
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public short[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new ShortValues(input, numDocs);
|
||||
}
|
||||
|
||||
public void toBytes(long value, BytesRef bytesRef) {
|
||||
copyShort(bytesRef, (short) (0xFFFFL & value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromArray(Object array) {
|
||||
assert array instanceof short[];
|
||||
return new ShortValues((short[]) array);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
toBytes(getInt(docID), ref);
|
||||
return ref;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class IntValues extends DocValuesArraySource {
|
||||
private final int[] values;
|
||||
|
||||
IntValues() {
|
||||
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, Type.FIXED_INTS_32);
|
||||
values = new int[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readInt();
|
||||
}
|
||||
}
|
||||
|
||||
private IntValues(int[] array) {
|
||||
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
|
||||
values = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return 0xFFFFFFFF & values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new IntValues(input, numDocs);
|
||||
}
|
||||
|
||||
public void toBytes(long value, BytesRef bytesRef) {
|
||||
copyInt(bytesRef, (int) (0xFFFFFFFF & value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromArray(Object array) {
|
||||
assert array instanceof int[];
|
||||
return new IntValues((int[]) array);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
toBytes(getInt(docID), ref);
|
||||
return ref;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class LongValues extends DocValuesArraySource {
|
||||
private final long[] values;
|
||||
|
||||
LongValues() {
|
||||
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, Type.FIXED_INTS_64);
|
||||
values = new long[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readLong();
|
||||
}
|
||||
}
|
||||
|
||||
private LongValues(long[] array) {
|
||||
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
|
||||
values = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new LongValues(input, numDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromArray(Object array) {
|
||||
assert array instanceof long[];
|
||||
return new LongValues((long[])array);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
toBytes(getInt(docID), ref);
|
||||
return ref;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class FloatValues extends DocValuesArraySource {
|
||||
private final float[] values;
|
||||
|
||||
FloatValues() {
|
||||
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, Type.FLOAT_32);
|
||||
values = new float[numDocs];
|
||||
/*
|
||||
* we always read BIG_ENDIAN here since the writer serialized plain bytes
|
||||
* we can simply read the ints / longs back in using readInt / readLong
|
||||
*/
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = Float.intBitsToFloat(input.readInt());
|
||||
}
|
||||
}
|
||||
|
||||
private FloatValues(float[] array) {
|
||||
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
|
||||
values = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void toBytes(double value, BytesRef bytesRef) {
|
||||
copyInt(bytesRef, Float.floatToRawIntBits((float)value));
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new FloatValues(input, numDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromArray(Object array) {
|
||||
assert array instanceof float[];
|
||||
return new FloatValues((float[]) array);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
toBytes(getFloat(docID), ref);
|
||||
return ref;
|
||||
}
|
||||
};
|
||||
|
||||
final static class DoubleValues extends DocValuesArraySource {
|
||||
private final double[] values;
|
||||
|
||||
DoubleValues() {
|
||||
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, Type.FLOAT_64);
|
||||
values = new double[numDocs];
|
||||
/*
|
||||
* we always read BIG_ENDIAN here since the writer serialized plain bytes
|
||||
* we can simply read the ints / longs back in using readInt / readLong
|
||||
*/
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = Double.longBitsToDouble(input.readLong());
|
||||
}
|
||||
}
|
||||
|
||||
private DoubleValues(double[] array) {
|
||||
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
|
||||
values = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new DoubleValues(input, numDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArraySource newFromArray(Object array) {
|
||||
assert array instanceof double[];
|
||||
return new DoubleValues((double[]) array);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
toBytes(getFloat(docID), ref);
|
||||
return ref;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
/**
|
||||
* Copies the given long value and encodes it as 8 byte Big-Endian.
|
||||
* <p>
|
||||
* NOTE: this method resets the offset to 0, length to 8 and resizes the
|
||||
* reference array if needed.
|
||||
*/
|
||||
public static void copyLong(BytesRef ref, long value) {
|
||||
if (ref.bytes.length < 8) {
|
||||
ref.bytes = new byte[8];
|
||||
}
|
||||
copyInternal(ref, (int) (value >> 32), ref.offset = 0);
|
||||
copyInternal(ref, (int) value, 4);
|
||||
ref.length = 8;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the given int value and encodes it as 4 byte Big-Endian.
|
||||
* <p>
|
||||
* NOTE: this method resets the offset to 0, length to 4 and resizes the
|
||||
* reference array if needed.
|
||||
*/
|
||||
public static void copyInt(BytesRef ref, int value) {
|
||||
if (ref.bytes.length < 4) {
|
||||
ref.bytes = new byte[4];
|
||||
}
|
||||
copyInternal(ref, value, ref.offset = 0);
|
||||
ref.length = 4;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the given short value and encodes it as a 2 byte Big-Endian.
|
||||
* <p>
|
||||
* NOTE: this method resets the offset to 0, length to 2 and resizes the
|
||||
* reference array if needed.
|
||||
*/
|
||||
public static void copyShort(BytesRef ref, short value) {
|
||||
if (ref.bytes.length < 2) {
|
||||
ref.bytes = new byte[2];
|
||||
}
|
||||
ref.offset = 0;
|
||||
ref.bytes[ref.offset] = (byte) (value >> 8);
|
||||
ref.bytes[ref.offset + 1] = (byte) (value);
|
||||
ref.length = 2;
|
||||
}
|
||||
|
||||
private static void copyInternal(BytesRef ref, int value, int startOffset) {
|
||||
ref.bytes[startOffset] = (byte) (value >> 24);
|
||||
ref.bytes[startOffset + 1] = (byte) (value >> 16);
|
||||
ref.bytes[startOffset + 2] = (byte) (value >> 8);
|
||||
ref.bytes[startOffset + 3] = (byte) (value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts 2 consecutive bytes from the current offset to a short. Bytes are
|
||||
* interpreted as Big-Endian (most significant bit first)
|
||||
* <p>
|
||||
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
|
||||
*/
|
||||
public static short asShort(BytesRef b) {
|
||||
return (short) (0xFFFF & ((b.bytes[b.offset] & 0xFF) << 8) | (b.bytes[b.offset + 1] & 0xFF));
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts 4 consecutive bytes from the current offset to an int. Bytes are
|
||||
* interpreted as Big-Endian (most significant bit first)
|
||||
* <p>
|
||||
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
|
||||
*/
|
||||
public static int asInt(BytesRef b) {
|
||||
return asIntInternal(b, b.offset);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts 8 consecutive bytes from the current offset to a long. Bytes are
|
||||
* interpreted as Big-Endian (most significant bit first)
|
||||
* <p>
|
||||
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
|
||||
*/
|
||||
public static long asLong(BytesRef b) {
|
||||
return (((long) asIntInternal(b, b.offset) << 32) | asIntInternal(b,
|
||||
b.offset + 4) & 0xFFFFFFFFL);
|
||||
}
|
||||
|
||||
private static int asIntInternal(BytesRef b, int pos) {
|
||||
return ((b.bytes[pos++] & 0xFF) << 24) | ((b.bytes[pos++] & 0xFF) << 16)
|
||||
| ((b.bytes[pos++] & 0xFF) << 8) | (b.bytes[pos] & 0xFF);
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.document.DocValuesField;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -40,6 +41,7 @@ public abstract class DocValuesConsumer {
|
|||
|
||||
protected final BytesRef spare = new BytesRef();
|
||||
|
||||
protected abstract Type getType();
|
||||
/**
|
||||
* Adds the given {@link IndexableField} instance to this
|
||||
* {@link DocValuesConsumer}
|
||||
|
@ -110,7 +112,7 @@ public abstract class DocValuesConsumer {
|
|||
final Source source = reader.getDirectSource();
|
||||
assert source != null;
|
||||
int docID = docBase;
|
||||
final DocValues.Type type = reader.type();
|
||||
final Type type = getType();
|
||||
final Field scratchField;
|
||||
switch(type) {
|
||||
case VAR_INTS:
|
||||
|
@ -160,7 +162,7 @@ public abstract class DocValuesConsumer {
|
|||
*/
|
||||
protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc)
|
||||
throws IOException {
|
||||
switch(source.type()) {
|
||||
switch(getType()) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.lucene40.values;
|
||||
package org.apache.lucene.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -24,10 +24,6 @@ import java.util.Comparator;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.codecs.lucene40.values.Bytes;
|
||||
import org.apache.lucene.codecs.lucene40.values.Floats;
|
||||
import org.apache.lucene.codecs.lucene40.values.Ints;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
@ -40,7 +36,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* Abstract base class for PerDocProducer implementations
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class DocValuesReaderBase extends PerDocProducer {
|
||||
public abstract class PerDocProducerBase extends PerDocProducer {
|
||||
|
||||
protected abstract void closeInternal(Collection<? extends Closeable> closeables) throws IOException;
|
||||
protected abstract Map<String, DocValues> docValues();
|
||||
|
@ -70,9 +66,7 @@ public abstract class DocValuesReaderBase extends PerDocProducer {
|
|||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (canLoad(fieldInfo)) {
|
||||
final String field = fieldInfo.name;
|
||||
// TODO can we have a compound file per segment and codec for
|
||||
// docvalues?
|
||||
final String id = DocValuesWriterBase.docValuesId(segment,
|
||||
final String id = docValuesId(segment,
|
||||
fieldInfo.number);
|
||||
values.put(field,
|
||||
loadDocValues(docCount, dir, id, getDocValuesType(fieldInfo), context));
|
||||
|
@ -100,6 +94,10 @@ public abstract class DocValuesReaderBase extends PerDocProducer {
|
|||
return infos.anyDocValuesFields();
|
||||
}
|
||||
|
||||
public static String docValuesId(String segmentsName, int fieldId) {
|
||||
return segmentsName + "_" + fieldId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads a {@link DocValues} instance depending on the given {@link Type}.
|
||||
* Codecs that use different implementations for a certain {@link Type} can
|
||||
|
@ -119,33 +117,6 @@ public abstract class DocValuesReaderBase extends PerDocProducer {
|
|||
* @throws IllegalArgumentException
|
||||
* if the given {@link Type} is not supported
|
||||
*/
|
||||
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:
|
||||
case FIXED_INTS_64:
|
||||
case FIXED_INTS_8:
|
||||
case VAR_INTS:
|
||||
return Ints.getValues(dir, id, docCount, type, context);
|
||||
case FLOAT_32:
|
||||
return Floats.getValues(dir, id, docCount, context, type);
|
||||
case FLOAT_64:
|
||||
return Floats.getValues(dir, id, docCount, context, type);
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
|
||||
case BYTES_FIXED_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
|
||||
case BYTES_FIXED_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
|
||||
case BYTES_VAR_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
|
||||
case BYTES_VAR_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
|
||||
default:
|
||||
throw new IllegalStateException("unrecognized index values mode " + type);
|
||||
}
|
||||
}
|
||||
protected abstract DocValues loadDocValues(int docCount, Directory dir, String id,
|
||||
DocValues.Type type, IOContext context) throws IOException;
|
||||
}
|
|
@ -24,19 +24,24 @@ import java.util.Collection;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.codecs.lucene40.values.DocValuesReaderBase;
|
||||
import org.apache.lucene.codecs.PerDocProducerBase;
|
||||
import org.apache.lucene.codecs.lucene40.values.Bytes;
|
||||
import org.apache.lucene.codecs.lucene40.values.Floats;
|
||||
import org.apache.lucene.codecs.lucene40.values.Ints;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Default PerDocProducer implementation that uses compound file.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Lucene40DocValuesProducer extends DocValuesReaderBase {
|
||||
public class Lucene40DocValuesProducer extends PerDocProducerBase {
|
||||
protected final TreeMap<String,DocValues> docValues;
|
||||
private final Directory cfs;
|
||||
/**
|
||||
|
@ -71,4 +76,35 @@ public class Lucene40DocValuesProducer extends DocValuesReaderBase {
|
|||
IOUtils.close(closeables);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DocValues loadDocValues(int docCount, Directory dir, String id,
|
||||
Type type, IOContext context) throws IOException {
|
||||
switch (type) {
|
||||
case FIXED_INTS_16:
|
||||
case FIXED_INTS_32:
|
||||
case FIXED_INTS_64:
|
||||
case FIXED_INTS_8:
|
||||
case VAR_INTS:
|
||||
return Ints.getValues(dir, id, docCount, type, context);
|
||||
case FLOAT_32:
|
||||
return Floats.getValues(dir, id, docCount, context, type);
|
||||
case FLOAT_64:
|
||||
return Floats.getValues(dir, id, docCount, context, type);
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
|
||||
case BYTES_FIXED_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
|
||||
case BYTES_FIXED_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
|
||||
case BYTES_VAR_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
|
||||
case BYTES_VAR_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
|
||||
default:
|
||||
throw new IllegalStateException("unrecognized index values mode " + type);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.Comparator;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
|
@ -64,7 +63,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
final class Bytes {
|
||||
public final class Bytes {
|
||||
|
||||
static final String DV_SEGMENT_SUFFIX = "dv";
|
||||
|
||||
|
@ -242,8 +241,8 @@ final class Bytes {
|
|||
private final IOContext context;
|
||||
|
||||
protected BytesWriterBase(Directory dir, String id, String codecName,
|
||||
int version, Counter bytesUsed, IOContext context) throws IOException {
|
||||
super(bytesUsed);
|
||||
int version, Counter bytesUsed, IOContext context, Type type) throws IOException {
|
||||
super(bytesUsed, type);
|
||||
this.id = id;
|
||||
this.dir = dir;
|
||||
this.codecName = codecName;
|
||||
|
@ -292,25 +291,11 @@ final class Bytes {
|
|||
}
|
||||
return idxOut;
|
||||
}
|
||||
/**
|
||||
* Must be called only with increasing docIDs. It's OK for some docIDs to be
|
||||
* skipped; they will be filled with 0 bytes.
|
||||
*/
|
||||
protected
|
||||
abstract void add(int docID, BytesRef bytes) throws IOException;
|
||||
|
||||
|
||||
@Override
|
||||
public abstract void finish(int docCount) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc) throws IOException {
|
||||
add(docID, source.getBytes(sourceDoc, bytesRef));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField docValue) throws IOException {
|
||||
add(docID, docValue.binaryValue());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -393,22 +378,22 @@ final class Bytes {
|
|||
protected long maxBytes = 0;
|
||||
|
||||
protected DerefBytesWriterBase(Directory dir, String id, String codecName,
|
||||
int codecVersion, Counter bytesUsed, IOContext context)
|
||||
int codecVersion, Counter bytesUsed, IOContext context, Type type)
|
||||
throws IOException {
|
||||
this(dir, id, codecName, codecVersion, new DirectTrackingAllocator(
|
||||
ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, false);
|
||||
ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, false, type);
|
||||
}
|
||||
|
||||
protected DerefBytesWriterBase(Directory dir, String id, String codecName,
|
||||
int codecVersion, Counter bytesUsed, IOContext context, boolean fasterButMoreRam)
|
||||
int codecVersion, Counter bytesUsed, IOContext context, boolean fasterButMoreRam, Type type)
|
||||
throws IOException {
|
||||
this(dir, id, codecName, codecVersion, new DirectTrackingAllocator(
|
||||
ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, fasterButMoreRam);
|
||||
ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, fasterButMoreRam,type);
|
||||
}
|
||||
|
||||
protected DerefBytesWriterBase(Directory dir, String id, String codecName, int codecVersion, Allocator allocator,
|
||||
Counter bytesUsed, IOContext context, boolean fasterButMoreRam) throws IOException {
|
||||
super(dir, id, codecName, codecVersion, bytesUsed, context);
|
||||
Counter bytesUsed, IOContext context, boolean fasterButMoreRam, Type type) throws IOException {
|
||||
super(dir, id, codecName, codecVersion, bytesUsed, context, type);
|
||||
hash = new BytesRefHash(new ByteBlockPool(allocator),
|
||||
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
|
||||
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
|
||||
|
@ -430,7 +415,9 @@ final class Bytes {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void add(int docID, BytesRef bytes) throws IOException {
|
||||
public void add(int docID, IndexableField value) throws IOException {
|
||||
BytesRef bytes = value.binaryValue();
|
||||
assert bytes != null;
|
||||
if (bytes.length == 0) { // default value - skip it
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -1,120 +0,0 @@
|
|||
package org.apache.lucene.codecs.lucene40.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.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Package private BytesRefUtils - can move this into the o.a.l.utils package if
|
||||
* needed.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
final class BytesRefUtils {
|
||||
|
||||
private BytesRefUtils() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the given long value and encodes it as 8 byte Big-Endian.
|
||||
* <p>
|
||||
* NOTE: this method resets the offset to 0, length to 8 and resizes the
|
||||
* reference array if needed.
|
||||
*/
|
||||
public static void copyLong(BytesRef ref, long value) {
|
||||
if (ref.bytes.length < 8) {
|
||||
ref.bytes = new byte[8];
|
||||
}
|
||||
copyInternal(ref, (int) (value >> 32), ref.offset = 0);
|
||||
copyInternal(ref, (int) value, 4);
|
||||
ref.length = 8;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the given int value and encodes it as 4 byte Big-Endian.
|
||||
* <p>
|
||||
* NOTE: this method resets the offset to 0, length to 4 and resizes the
|
||||
* reference array if needed.
|
||||
*/
|
||||
public static void copyInt(BytesRef ref, int value) {
|
||||
if (ref.bytes.length < 4) {
|
||||
ref.bytes = new byte[4];
|
||||
}
|
||||
copyInternal(ref, value, ref.offset = 0);
|
||||
ref.length = 4;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the given short value and encodes it as a 2 byte Big-Endian.
|
||||
* <p>
|
||||
* NOTE: this method resets the offset to 0, length to 2 and resizes the
|
||||
* reference array if needed.
|
||||
*/
|
||||
public static void copyShort(BytesRef ref, short value) {
|
||||
if (ref.bytes.length < 2) {
|
||||
ref.bytes = new byte[2];
|
||||
}
|
||||
ref.bytes[ref.offset] = (byte) (value >> 8);
|
||||
ref.bytes[ref.offset + 1] = (byte) (value);
|
||||
ref.length = 2;
|
||||
}
|
||||
|
||||
private static void copyInternal(BytesRef ref, int value, int startOffset) {
|
||||
ref.bytes[startOffset] = (byte) (value >> 24);
|
||||
ref.bytes[startOffset + 1] = (byte) (value >> 16);
|
||||
ref.bytes[startOffset + 2] = (byte) (value >> 8);
|
||||
ref.bytes[startOffset + 3] = (byte) (value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts 2 consecutive bytes from the current offset to a short. Bytes are
|
||||
* interpreted as Big-Endian (most significant bit first)
|
||||
* <p>
|
||||
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
|
||||
*/
|
||||
public static short asShort(BytesRef b) {
|
||||
return (short) (0xFFFF & ((b.bytes[b.offset] & 0xFF) << 8) | (b.bytes[b.offset + 1] & 0xFF));
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts 4 consecutive bytes from the current offset to an int. Bytes are
|
||||
* interpreted as Big-Endian (most significant bit first)
|
||||
* <p>
|
||||
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
|
||||
*/
|
||||
public static int asInt(BytesRef b) {
|
||||
return asIntInternal(b, b.offset);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts 8 consecutive bytes from the current offset to a long. Bytes are
|
||||
* interpreted as Big-Endian (most significant bit first)
|
||||
* <p>
|
||||
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
|
||||
*/
|
||||
public static long asLong(BytesRef b) {
|
||||
return (((long) asIntInternal(b, b.offset) << 32) | asIntInternal(b,
|
||||
b.offset + 4) & 0xFFFFFFFFL);
|
||||
}
|
||||
|
||||
private static int asIntInternal(BytesRef b, int pos) {
|
||||
return ((b.bytes[pos++] & 0xFF) << 24) | ((b.bytes[pos++] & 0xFF) << 16)
|
||||
| ((b.bytes[pos++] & 0xFF) << 8) | (b.bytes[pos] & 0xFF);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,306 +0,0 @@
|
|||
package org.apache.lucene.codecs.lucene40.values;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumMap;
|
||||
import java.util.Map;
|
||||
|
||||
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;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
abstract class DocValuesArray extends Source {
|
||||
|
||||
static final Map<Type, DocValuesArray> TEMPLATES;
|
||||
|
||||
static {
|
||||
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;
|
||||
|
||||
DocValuesArray(int bytesPerValue, Type type) {
|
||||
super(type);
|
||||
this.bytesPerValue = bytesPerValue;
|
||||
}
|
||||
|
||||
public abstract DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException;
|
||||
|
||||
@Override
|
||||
public final boolean hasArray() {
|
||||
return true;
|
||||
}
|
||||
|
||||
void toBytes(long value, BytesRef bytesRef) {
|
||||
BytesRefUtils.copyLong(bytesRef, value);
|
||||
}
|
||||
|
||||
void toBytes(double value, BytesRef bytesRef) {
|
||||
BytesRefUtils.copyLong(bytesRef, Double.doubleToRawLongBits(value));
|
||||
}
|
||||
|
||||
final static class ByteValues extends DocValuesArray {
|
||||
private final byte[] values;
|
||||
|
||||
ByteValues() {
|
||||
super(1, Type.FIXED_INTS_8);
|
||||
values = new byte[0];
|
||||
}
|
||||
|
||||
private ByteValues(IndexInput input, int numDocs) throws IOException {
|
||||
super(1, Type.FIXED_INTS_8);
|
||||
values = new byte[numDocs];
|
||||
input.readBytes(values, 0, values.length, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new ByteValues(input, numDocs);
|
||||
}
|
||||
|
||||
void toBytes(long value, BytesRef bytesRef) {
|
||||
bytesRef.bytes[0] = (byte) (0xFFL & value);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class ShortValues extends DocValuesArray {
|
||||
private final short[] values;
|
||||
|
||||
ShortValues() {
|
||||
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, Type.FIXED_INTS_16);
|
||||
values = new short[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readShort();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public short[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new ShortValues(input, numDocs);
|
||||
}
|
||||
|
||||
void toBytes(long value, BytesRef bytesRef) {
|
||||
BytesRefUtils.copyShort(bytesRef, (short) (0xFFFFL & value));
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class IntValues extends DocValuesArray {
|
||||
private final int[] values;
|
||||
|
||||
IntValues() {
|
||||
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, Type.FIXED_INTS_32);
|
||||
values = new int[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readInt();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return 0xFFFFFFFF & values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new IntValues(input, numDocs);
|
||||
}
|
||||
|
||||
void toBytes(long value, BytesRef bytesRef) {
|
||||
BytesRefUtils.copyInt(bytesRef, (int) (0xFFFFFFFF & value));
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class LongValues extends DocValuesArray {
|
||||
private final long[] values;
|
||||
|
||||
LongValues() {
|
||||
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, Type.FIXED_INTS_64);
|
||||
values = new long[numDocs];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = input.readLong();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new LongValues(input, numDocs);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
final static class FloatValues extends DocValuesArray {
|
||||
private final float[] values;
|
||||
|
||||
FloatValues() {
|
||||
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, Type.FLOAT_32);
|
||||
values = new float[numDocs];
|
||||
/*
|
||||
* we always read BIG_ENDIAN here since the writer serialized plain bytes
|
||||
* we can simply read the ints / longs back in using readInt / readLong
|
||||
*/
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = Float.intBitsToFloat(input.readInt());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public float[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
void toBytes(double value, BytesRef bytesRef) {
|
||||
BytesRefUtils.copyInt(bytesRef, Float.floatToRawIntBits((float)value));
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new FloatValues(input, numDocs);
|
||||
}
|
||||
};
|
||||
|
||||
final static class DoubleValues extends DocValuesArray {
|
||||
private final double[] values;
|
||||
|
||||
DoubleValues() {
|
||||
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, Type.FLOAT_64);
|
||||
values = new double[numDocs];
|
||||
/*
|
||||
* we always read BIG_ENDIAN here since the writer serialized plain bytes
|
||||
* we can simply read the ints / longs back in using readInt / readLong
|
||||
*/
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = Double.longBitsToDouble(input.readLong());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public double[] getArray() {
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat(int docID) {
|
||||
assert docID >= 0 && docID < values.length;
|
||||
return values[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesArray newFromInput(IndexInput input, int numDocs)
|
||||
throws IOException {
|
||||
return new DoubleValues(input, numDocs);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.PerDocProducerBase;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.lucene40.values.Writer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
|
@ -81,14 +82,10 @@ public abstract class DocValuesWriterBase extends PerDocConsumer {
|
|||
@Override
|
||||
public DocValuesConsumer addValuesField(Type valueType, FieldInfo field) throws IOException {
|
||||
return Writer.create(valueType,
|
||||
docValuesId(segmentName, field.number),
|
||||
PerDocProducerBase.docValuesId(segmentName, field.number),
|
||||
getDirectory(), getComparator(), bytesUsed, context, fasterButMoreRam);
|
||||
}
|
||||
|
||||
public static String docValuesId(String segmentsName, int fieldId) {
|
||||
return segmentsName + "_" + fieldId;
|
||||
}
|
||||
|
||||
|
||||
public Comparator<BytesRef> getComparator() throws IOException {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
|
|
|
@ -46,7 +46,7 @@ class FixedDerefBytesImpl {
|
|||
public static class Writer extends DerefBytesWriterBase {
|
||||
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.BYTES_FIXED_DEREF);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -58,7 +58,7 @@ class FixedSortedBytesImpl {
|
|||
|
||||
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
|
||||
Counter bytesUsed, IOContext context, boolean fasterButMoreRam) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam);
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam, Type.BYTES_FIXED_SORTED);
|
||||
this.comp = comp;
|
||||
}
|
||||
|
||||
|
|
|
@ -22,10 +22,12 @@ import java.io.IOException;
|
|||
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
|
||||
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -52,6 +54,7 @@ class FixedStraightBytesImpl {
|
|||
static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
static abstract class FixedBytesWriterBase extends BytesWriterBase {
|
||||
protected final DocValuesField bytesSpareField = new DocValuesField("", new BytesRef(), Type.BYTES_FIXED_STRAIGHT);
|
||||
protected int lastDocID = -1;
|
||||
// start at -1 if the first added value is > 0
|
||||
protected int size = -1;
|
||||
|
@ -60,13 +63,20 @@ class FixedStraightBytesImpl {
|
|||
|
||||
protected FixedBytesWriterBase(Directory dir, String id, String codecName,
|
||||
int version, Counter bytesUsed, IOContext context) throws IOException {
|
||||
super(dir, id, codecName, version, bytesUsed, context);
|
||||
this(dir, id, codecName, version, bytesUsed, context, Type.BYTES_FIXED_STRAIGHT);
|
||||
}
|
||||
|
||||
protected FixedBytesWriterBase(Directory dir, String id, String codecName,
|
||||
int version, Counter bytesUsed, IOContext context, Type type) throws IOException {
|
||||
super(dir, id, codecName, version, bytesUsed, context, type);
|
||||
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
|
||||
pool.nextBuffer();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void add(int docID, BytesRef bytes) throws IOException {
|
||||
public void add(int docID, IndexableField value) throws IOException {
|
||||
final BytesRef bytes = value.binaryValue();
|
||||
assert bytes != null;
|
||||
assert lastDocID < docID;
|
||||
|
||||
if (size == -1) {
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene40.values;
|
|||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesArraySource;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
|
@ -39,7 +40,7 @@ import org.apache.lucene.util.IOUtils;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class Floats {
|
||||
public class Floats {
|
||||
|
||||
protected static final String CODEC_NAME = "Floats";
|
||||
protected static final int VERSION_START = 0;
|
||||
|
@ -69,33 +70,30 @@ class Floats {
|
|||
final static class FloatsWriter extends FixedStraightBytesImpl.Writer {
|
||||
|
||||
private final int size;
|
||||
private final DocValuesArray template;
|
||||
private final DocValuesArraySource template;
|
||||
public FloatsWriter(Directory dir, String id, Counter bytesUsed,
|
||||
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 = DocValuesArray.TEMPLATES.get(type);
|
||||
template = DocValuesArraySource.forType(type);
|
||||
assert template != null;
|
||||
}
|
||||
|
||||
protected void add(int docID, double v) throws IOException {
|
||||
template.toBytes(v, bytesRef);
|
||||
add(docID, bytesRef);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField docValue) throws IOException {
|
||||
add(docID, docValue.numericValue().doubleValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
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();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField value) throws IOException {
|
||||
template.toBytes(value.numericValue().doubleValue(), bytesRef);
|
||||
bytesSpareField.setBytesValue(bytesRef);
|
||||
super.add(docID, bytesSpareField);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setMergeBytes(Source source, int sourceDoc) {
|
||||
final double value = source.getFloat(sourceDoc);
|
||||
|
@ -104,11 +102,11 @@ class Floats {
|
|||
}
|
||||
|
||||
final static class FloatsReader extends FixedStraightBytesImpl.FixedStraightReader {
|
||||
final DocValuesArray arrayTemplate;
|
||||
final DocValuesArraySource 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 = DocValuesArray.TEMPLATES.get(type);
|
||||
arrayTemplate = DocValuesArraySource.forType(type);
|
||||
assert size == 4 || size == 8: "wrong size=" + size + " type=" + type + " id=" + id;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene40.values;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesArraySource;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
|
@ -36,7 +37,7 @@ import org.apache.lucene.util.IOUtils;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
final class Ints {
|
||||
public final class Ints {
|
||||
protected static final String CODEC_NAME = "Ints";
|
||||
protected static final int VERSION_START = 0;
|
||||
protected static final int VERSION_CURRENT = VERSION_START;
|
||||
|
@ -88,7 +89,7 @@ final class Ints {
|
|||
|
||||
|
||||
static class IntsWriter extends FixedStraightBytesImpl.Writer {
|
||||
private final DocValuesArray template;
|
||||
private final DocValuesArraySource template;
|
||||
|
||||
public IntsWriter(Directory dir, String id, Counter bytesUsed,
|
||||
IOContext context, Type valueType) throws IOException {
|
||||
|
@ -101,17 +102,7 @@ final class Ints {
|
|||
size = typeToSize(valueType);
|
||||
this.bytesRef = new BytesRef(size);
|
||||
bytesRef.length = size;
|
||||
template = DocValuesArray.TEMPLATES.get(valueType);
|
||||
}
|
||||
|
||||
protected void add(int docID, long v) throws IOException {
|
||||
template.toBytes(v, bytesRef);
|
||||
add(docID, bytesRef);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField docValue) throws IOException {
|
||||
add(docID, docValue.numericValue().longValue());
|
||||
template = DocValuesArraySource.forType(valueType);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -120,6 +111,13 @@ final class Ints {
|
|||
template.toBytes(value, bytesRef);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField value) throws IOException {
|
||||
template.toBytes(value.numericValue().longValue(), bytesRef);
|
||||
bytesSpareField.setBytesValue(bytesRef);
|
||||
super.add(docID, bytesSpareField);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean tryBulkMerge(DocValues docValues) {
|
||||
// only bulk merge if value type is the same otherwise size differs
|
||||
|
@ -128,13 +126,13 @@ final class Ints {
|
|||
}
|
||||
|
||||
final static class IntsReader extends FixedStraightBytesImpl.FixedStraightReader {
|
||||
private final DocValuesArray arrayTemplate;
|
||||
private final DocValuesArraySource arrayTemplate;
|
||||
|
||||
IntsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc,
|
||||
context, type);
|
||||
arrayTemplate = DocValuesArray.TEMPLATES.get(type);
|
||||
arrayTemplate = DocValuesArraySource.forType(type);
|
||||
assert arrayTemplate != null;
|
||||
assert type == sizeToType(size);
|
||||
}
|
||||
|
|
|
@ -18,9 +18,8 @@ package org.apache.lucene.codecs.lucene40.values;
|
|||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.lucene40.values.DocValuesArray.LongValues;
|
||||
import org.apache.lucene.codecs.DocValuesArraySource;
|
||||
import org.apache.lucene.codecs.lucene40.values.FixedStraightBytesImpl.FixedBytesWriterBase;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
@ -59,27 +58,10 @@ class PackedIntValues {
|
|||
|
||||
protected PackedIntsWriter(Directory dir, String id, Counter bytesUsed,
|
||||
IOContext context) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.VAR_INTS);
|
||||
bytesRef = new BytesRef(8);
|
||||
}
|
||||
|
||||
protected void add(int docID, long v) throws IOException {
|
||||
assert lastDocId < docID;
|
||||
if (!started) {
|
||||
started = true;
|
||||
minValue = maxValue = v;
|
||||
} else {
|
||||
if (v < minValue) {
|
||||
minValue = v;
|
||||
} else if (v > maxValue) {
|
||||
maxValue = v;
|
||||
}
|
||||
}
|
||||
lastDocId = docID;
|
||||
BytesRefUtils.copyLong(bytesRef, v);
|
||||
add(docID, bytesRef);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
boolean success = false;
|
||||
|
@ -112,13 +94,6 @@ class PackedIntValues {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc) throws IOException {
|
||||
assert docID > lastDocId : "docID: " + docID
|
||||
+ " must be greater than the last added doc id: " + lastDocId;
|
||||
add(docID, source.getInt(sourceDoc));
|
||||
}
|
||||
|
||||
private void writePackedInts(IndexOutput datOut, int docCount) throws IOException {
|
||||
datOut.writeLong(minValue);
|
||||
|
||||
|
@ -149,10 +124,25 @@ class PackedIntValues {
|
|||
}
|
||||
w.finish();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField docValue) throws IOException {
|
||||
add(docID, docValue.numericValue().longValue());
|
||||
final long v = docValue.numericValue().longValue();
|
||||
assert lastDocId < docID;
|
||||
if (!started) {
|
||||
started = true;
|
||||
minValue = maxValue = v;
|
||||
} else {
|
||||
if (v < minValue) {
|
||||
minValue = v;
|
||||
} else if (v > maxValue) {
|
||||
maxValue = v;
|
||||
}
|
||||
}
|
||||
lastDocId = docID;
|
||||
DocValuesArraySource.copyLong(bytesRef, v);
|
||||
bytesSpareField.setBytesValue(bytesRef);
|
||||
super.add(docID, bytesSpareField);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -164,7 +154,7 @@ class PackedIntValues {
|
|||
private final IndexInput datIn;
|
||||
private final byte type;
|
||||
private final int numDocs;
|
||||
private final LongValues values;
|
||||
private final DocValuesArraySource values;
|
||||
|
||||
protected PackedIntsReader(Directory dir, String id, int numDocs,
|
||||
IOContext context) throws IOException {
|
||||
|
@ -176,7 +166,7 @@ class PackedIntValues {
|
|||
try {
|
||||
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
|
||||
type = datIn.readByte();
|
||||
values = type == FIXED_64 ? new LongValues() : null;
|
||||
values = type == FIXED_64 ? DocValuesArraySource.forType(Type.FIXED_INTS_64) : null;
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -247,7 +237,7 @@ class PackedIntValues {
|
|||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.grow(8);
|
||||
BytesRefUtils.copyLong(ref, getInt(docID));
|
||||
DocValuesArraySource.copyLong(ref, getInt(docID));
|
||||
return ref;
|
||||
}
|
||||
|
||||
|
|
|
@ -57,7 +57,7 @@ class VarDerefBytesImpl {
|
|||
static class Writer extends DerefBytesWriterBase {
|
||||
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_DEREF);
|
||||
size = 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -59,7 +59,7 @@ final class VarSortedBytesImpl {
|
|||
|
||||
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
|
||||
Counter bytesUsed, IOContext context, boolean fasterButMoreRam) throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam);
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam, Type.BYTES_VAR_SORTED);
|
||||
this.comp = comp;
|
||||
size = 0;
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.document.Field;
|
|||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -63,7 +64,7 @@ class VarStraightBytesImpl {
|
|||
private boolean merge = false;
|
||||
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
|
||||
throws IOException {
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
|
||||
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_STRAIGHT);
|
||||
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
|
||||
docToAddress = new long[1];
|
||||
pool.nextBuffer(); // init
|
||||
|
@ -84,7 +85,9 @@ class VarStraightBytesImpl {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void add(int docID, BytesRef bytes) throws IOException {
|
||||
public void add(int docID, IndexableField value) throws IOException {
|
||||
final BytesRef bytes = value.binaryValue();
|
||||
assert bytes != null;
|
||||
assert !merge;
|
||||
if (bytes.length == 0) {
|
||||
return; // default
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.util.Counter;
|
|||
*/
|
||||
abstract class Writer extends DocValuesConsumer {
|
||||
protected final Counter bytesUsed;
|
||||
protected Type type;
|
||||
|
||||
/**
|
||||
* Creates a new {@link Writer}.
|
||||
|
@ -49,9 +50,19 @@ abstract class Writer extends DocValuesConsumer {
|
|||
* internally allocated memory. All tracked bytes must be released
|
||||
* once {@link #finish(int)} has been called.
|
||||
*/
|
||||
protected Writer(Counter bytesUsed) {
|
||||
protected Writer(Counter bytesUsed, Type type) {
|
||||
this.bytesUsed = bytesUsed;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
protected Type getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Factory method to create a {@link Writer} instance for a given type. This
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.PerDocProducerBase;
|
||||
import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
|
@ -58,7 +59,7 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
|||
private static void files(Directory dir,FieldInfos fieldInfos, String segmentName, Set<String> files) {
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
String filename = docValuesId(segmentName, fieldInfo.number);
|
||||
String filename = PerDocProducerBase.docValuesId(segmentName, fieldInfo.number);
|
||||
switch (fieldInfo.getDocValuesType()) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_VAR_DEREF:
|
||||
|
|
|
@ -22,16 +22,22 @@ import java.util.Collection;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.codecs.lucene40.values.DocValuesReaderBase;
|
||||
import org.apache.lucene.codecs.PerDocProducerBase;
|
||||
import org.apache.lucene.codecs.lucene40.values.Bytes;
|
||||
import org.apache.lucene.codecs.lucene40.values.Floats;
|
||||
import org.apache.lucene.codecs.lucene40.values.Ints;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Implementation of PerDocProducer that uses separate files.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SepDocValuesProducer extends DocValuesReaderBase {
|
||||
public class SepDocValuesProducer extends PerDocProducerBase {
|
||||
private final TreeMap<String, DocValues> docValues;
|
||||
|
||||
/**
|
||||
|
@ -51,4 +57,35 @@ public class SepDocValuesProducer extends DocValuesReaderBase {
|
|||
protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
|
||||
IOUtils.close(closeables);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DocValues loadDocValues(int docCount, Directory dir, String id,
|
||||
Type type, IOContext context) throws IOException {
|
||||
switch (type) {
|
||||
case FIXED_INTS_16:
|
||||
case FIXED_INTS_32:
|
||||
case FIXED_INTS_64:
|
||||
case FIXED_INTS_8:
|
||||
case VAR_INTS:
|
||||
return Ints.getValues(dir, id, docCount, type, context);
|
||||
case FLOAT_32:
|
||||
return Floats.getValues(dir, id, docCount, context, type);
|
||||
case FLOAT_64:
|
||||
return Floats.getValues(dir, id, docCount, context, type);
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
|
||||
case BYTES_FIXED_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
|
||||
case BYTES_FIXED_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
|
||||
case BYTES_VAR_DEREF:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
|
||||
case BYTES_VAR_SORTED:
|
||||
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
|
||||
default:
|
||||
throw new IllegalStateException("unrecognized index values mode " + type);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.codecs.PostingsFormat;
|
|||
import org.apache.lucene.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
|
||||
|
||||
/**
|
||||
* plain text index format.
|
||||
|
@ -41,7 +40,7 @@ public final class SimpleTextCodec extends Codec {
|
|||
private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
|
||||
// TODO: need a plain-text impl
|
||||
private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
|
||||
private final DocValuesFormat docValues = new SimpleTextDocValuesFormat();
|
||||
// TODO: need a plain-text impl (using the above)
|
||||
private final NormsFormat normsFormat = new SimpleTextNormsFormat();
|
||||
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
|
||||
|
|
|
@ -0,0 +1,288 @@
|
|||
package org.apache.lucene.codecs.simpletext;
|
||||
/**
|
||||
* 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.codecs.DocValuesArraySource;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextDocValuesConsumer extends DocValuesConsumer {
|
||||
|
||||
static final BytesRef ZERO_DOUBLE = new BytesRef(Double.toString(0d));
|
||||
static final BytesRef ZERO_INT = new BytesRef(Integer.toString(0));
|
||||
static final BytesRef HEADER = new BytesRef("SimpleTextDocValues");
|
||||
|
||||
static final BytesRef END = new BytesRef("END");
|
||||
static final BytesRef VALUE_SIZE = new BytesRef("valuesize ");
|
||||
static final BytesRef DOC = new BytesRef(" doc ");
|
||||
static final BytesRef VALUE = new BytesRef(" value ");
|
||||
protected BytesRef scratch = new BytesRef();
|
||||
protected int maxDocId = -1;
|
||||
protected final String segment;
|
||||
protected final Directory dir;
|
||||
protected final IOContext ctx;
|
||||
protected final Type type;
|
||||
protected final BytesRefHash hash;
|
||||
private int[] ords;
|
||||
private int fixedSize = Integer.MIN_VALUE;
|
||||
private BytesRef zeroBytes;
|
||||
private final String segmentSuffix;
|
||||
|
||||
|
||||
public SimpleTextDocValuesConsumer(String segment, Directory dir,
|
||||
IOContext ctx, Type type, String segmentSuffix) {
|
||||
this.ctx = ctx;
|
||||
this.dir = dir;
|
||||
this.segment = segment;
|
||||
this.type = type;
|
||||
hash = new BytesRefHash();
|
||||
ords = new int[0];
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField value) throws IOException {
|
||||
assert docID >= 0;
|
||||
int ord = -1;
|
||||
int vSize = -1;
|
||||
switch (type) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
vSize = value.binaryValue().length;
|
||||
ord = hash.add(value.binaryValue());
|
||||
break;
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
vSize = -1;
|
||||
try {
|
||||
ord = hash.add(value.binaryValue());
|
||||
} catch (NullPointerException e) {
|
||||
System.err.println();
|
||||
}
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
vSize = 2;
|
||||
scratch.grow(2);
|
||||
DocValuesArraySource.copyShort(scratch, value.numericValue().shortValue());
|
||||
ord = hash.add(scratch);
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
vSize = 4;
|
||||
scratch.grow(4);
|
||||
DocValuesArraySource.copyInt(scratch, value.numericValue().intValue());
|
||||
ord = hash.add(scratch);
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
vSize = 1;
|
||||
scratch.grow(1);
|
||||
scratch.bytes[scratch.offset] = value.numericValue().byteValue();
|
||||
scratch.length = 1;
|
||||
ord = hash.add(scratch);
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
vSize = 8;
|
||||
case VAR_INTS:
|
||||
scratch.grow(8);
|
||||
DocValuesArraySource.copyLong(scratch, value.numericValue().longValue());
|
||||
ord = hash.add(scratch);
|
||||
break;
|
||||
case FLOAT_32:
|
||||
vSize = 4;
|
||||
scratch.grow(4);
|
||||
DocValuesArraySource.copyInt(scratch,
|
||||
Float.floatToRawIntBits(value.numericValue().floatValue()));
|
||||
ord = hash.add(scratch);
|
||||
break;
|
||||
case FLOAT_64:
|
||||
vSize = 8;
|
||||
scratch.grow(8);
|
||||
DocValuesArraySource.copyLong(scratch,
|
||||
Double.doubleToRawLongBits(value.numericValue().doubleValue()));
|
||||
ord = hash.add(scratch);
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
if (fixedSize == Integer.MIN_VALUE) {
|
||||
assert maxDocId == -1;
|
||||
fixedSize = vSize;
|
||||
} else {
|
||||
if (fixedSize != vSize) {
|
||||
throw new IllegalArgumentException("value size must be " + fixedSize + " but was: " + vSize);
|
||||
}
|
||||
}
|
||||
maxDocId = Math.max(docID, maxDocId);
|
||||
ords = grow(ords, docID);
|
||||
|
||||
ords[docID] = (ord < 0 ? (-ord)-1 : ord) + 1;
|
||||
}
|
||||
|
||||
protected BytesRef getHeader() {
|
||||
return HEADER;
|
||||
}
|
||||
|
||||
private int[] grow(int[] array, int upto) {
|
||||
if (array.length <= upto) {
|
||||
return ArrayUtil.grow(array, 1 + upto);
|
||||
}
|
||||
return array;
|
||||
}
|
||||
|
||||
private void prepareFlush(int docCount) {
|
||||
assert ords != null;
|
||||
ords = grow(ords, docCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
final String fileName = IndexFileNames.segmentFileName(segment, "",
|
||||
segmentSuffix);
|
||||
IndexOutput output = dir.createOutput(fileName, ctx);
|
||||
boolean success = false;
|
||||
BytesRef spare = new BytesRef();
|
||||
try {
|
||||
SimpleTextUtil.write(output, getHeader());
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
SimpleTextUtil.write(output, VALUE_SIZE);
|
||||
SimpleTextUtil.write(output, Integer.toString(this.fixedSize), scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
prepareFlush(docCount);
|
||||
for (int i = 0; i < docCount; i++) {
|
||||
SimpleTextUtil.write(output, DOC);
|
||||
SimpleTextUtil.write(output, Integer.toString(i), scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
SimpleTextUtil.write(output, VALUE);
|
||||
writeDoc(output, i, spare);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
}
|
||||
SimpleTextUtil.write(output, END);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
success = true;
|
||||
} finally {
|
||||
hash.close();
|
||||
if (success) {
|
||||
IOUtils.close(output);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(output);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeDoc(IndexOutput output, int docId, BytesRef spare) throws IOException {
|
||||
int ord = ords[docId] - 1;
|
||||
if (ord != -1) {
|
||||
assert ord >= 0;
|
||||
hash.get(ord, spare);
|
||||
|
||||
switch (type) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
SimpleTextUtil.write(output, spare);
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
SimpleTextUtil.write(output,
|
||||
Short.toString(DocValuesArraySource.asShort(spare)), scratch);
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
SimpleTextUtil.write(output,
|
||||
Integer.toString(DocValuesArraySource.asInt(spare)), scratch);
|
||||
break;
|
||||
case VAR_INTS:
|
||||
case FIXED_INTS_64:
|
||||
SimpleTextUtil.write(output,
|
||||
Long.toString(DocValuesArraySource.asLong(spare)), scratch);
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
assert spare.length == 1 : spare.length;
|
||||
SimpleTextUtil.write(output,
|
||||
Integer.toString(spare.bytes[spare.offset]), scratch);
|
||||
break;
|
||||
case FLOAT_32:
|
||||
float valueFloat = Float.intBitsToFloat(DocValuesArraySource.asInt(spare));
|
||||
SimpleTextUtil.write(output, Float.toString(valueFloat), scratch);
|
||||
break;
|
||||
case FLOAT_64:
|
||||
double valueDouble = Double.longBitsToDouble(DocValuesArraySource
|
||||
.asLong(spare));
|
||||
SimpleTextUtil.write(output, Double.toString(valueDouble), scratch);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("unsupported type: " + type);
|
||||
}
|
||||
} else {
|
||||
switch (type) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
if(zeroBytes == null) {
|
||||
assert fixedSize > 0;
|
||||
zeroBytes = new BytesRef(new byte[fixedSize]);
|
||||
}
|
||||
SimpleTextUtil.write(output, zeroBytes);
|
||||
break;
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
scratch.length = 0;
|
||||
SimpleTextUtil.write(output, scratch);
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
case FIXED_INTS_32:
|
||||
case FIXED_INTS_64:
|
||||
case FIXED_INTS_8:
|
||||
case VAR_INTS:
|
||||
SimpleTextUtil.write(output, ZERO_INT);
|
||||
break;
|
||||
case FLOAT_32:
|
||||
case FLOAT_64:
|
||||
SimpleTextUtil.write(output, ZERO_DOUBLE);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("unsupported type: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* 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.Set;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextDocValuesFormat extends DocValuesFormat {
|
||||
private static final String DOC_VALUES_SEG_SUFFIX = "dv";
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new SimpleTextPerDocConsumer(state, DOC_VALUES_SEG_SUFFIX);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return new SimpleTextPerDocProducer(state, BytesRef.getUTF8SortedAsUnicodeComparator(), DOC_VALUES_SEG_SUFFIX);
|
||||
}
|
||||
|
||||
static String docValuesId(String segmentsName, int fieldId) {
|
||||
return segmentsName + "_" + fieldId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(SegmentInfo info, Set<String> files)
|
||||
throws IOException {
|
||||
SimpleTextPerDocConsumer.files(info, files, DOC_VALUES_SEG_SUFFIX);
|
||||
}
|
||||
}
|
|
@ -1,294 +0,0 @@
|
|||
package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* 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.Set;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Writes plain-text norms
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextNormsConsumer extends PerDocConsumer {
|
||||
|
||||
/** Extension of norms file */
|
||||
static final String NORMS_EXTENSION = "len";
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
final static BytesRef FIELD = new BytesRef("field ");
|
||||
final static BytesRef DOC = new BytesRef(" doc ");
|
||||
final static BytesRef NORM = new BytesRef(" norm ");
|
||||
|
||||
private NormsWriter writer;
|
||||
|
||||
private final Directory directory;
|
||||
|
||||
private final String segment;
|
||||
|
||||
private final IOContext context;
|
||||
|
||||
public SimpleTextNormsConsumer(Directory directory, String segment,
|
||||
IOContext context) throws IOException {
|
||||
this.directory = directory;
|
||||
this.segment = segment;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (writer != null) {
|
||||
boolean success = false;
|
||||
try {
|
||||
writer.finish();
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(writer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(writer);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
|
||||
throws IOException {
|
||||
return reader.normValues(info.name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean canMerge(FieldInfo info) {
|
||||
return info.normsPresent();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getDocValuesType(FieldInfo info) {
|
||||
return info.getNormType();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
|
||||
throws IOException {
|
||||
if (type != Type.FIXED_INTS_8) {
|
||||
throw new UnsupportedOperationException("Codec only supports single byte norm values. Type give: " + type);
|
||||
}
|
||||
return new SimpleTextNormsDocValuesConsumer(fieldInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
if (writer != null) {
|
||||
try {
|
||||
writer.abort();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private class SimpleTextNormsDocValuesConsumer extends DocValuesConsumer {
|
||||
// Holds all docID/norm pairs we've seen
|
||||
int[] docIDs = new int[1];
|
||||
byte[] norms = new byte[1];
|
||||
int upto;
|
||||
private final FieldInfo fi;
|
||||
|
||||
public SimpleTextNormsDocValuesConsumer(FieldInfo fieldInfo) {
|
||||
fi = fieldInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, IndexableField docValue) throws IOException {
|
||||
add(docID, docValue.numericValue().longValue());
|
||||
}
|
||||
|
||||
public void add(int docID, long value) {
|
||||
if (docIDs.length <= upto) {
|
||||
assert docIDs.length == upto;
|
||||
docIDs = ArrayUtil.grow(docIDs, 1 + upto);
|
||||
}
|
||||
if (norms.length <= upto) {
|
||||
assert norms.length == upto;
|
||||
norms = ArrayUtil.grow(norms, 1 + upto);
|
||||
}
|
||||
norms[upto] = (byte) value;
|
||||
|
||||
docIDs[upto] = docID;
|
||||
upto++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
final NormsWriter normsWriter = getNormsWriter();
|
||||
boolean success = false;
|
||||
try {
|
||||
int uptoDoc = 0;
|
||||
normsWriter.setNumTotalDocs(docCount);
|
||||
if (upto > 0) {
|
||||
normsWriter.startField(fi);
|
||||
int docID = 0;
|
||||
for (; docID < docCount; docID++) {
|
||||
if (uptoDoc < upto && docIDs[uptoDoc] == docID) {
|
||||
normsWriter.writeNorm(norms[uptoDoc]);
|
||||
uptoDoc++;
|
||||
} else {
|
||||
normsWriter.writeNorm((byte) 0);
|
||||
}
|
||||
}
|
||||
// we should have consumed every norm
|
||||
assert uptoDoc == upto;
|
||||
|
||||
} else {
|
||||
// Fill entire field with default norm:
|
||||
normsWriter.startField(fi);
|
||||
for (; upto < docCount; upto++)
|
||||
normsWriter.writeNorm((byte) 0);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
normsWriter.abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public NormsWriter getNormsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
writer = new NormsWriter(directory, segment, context);
|
||||
}
|
||||
return writer;
|
||||
}
|
||||
|
||||
private static class NormsWriter implements Closeable{
|
||||
|
||||
private final IndexOutput output;
|
||||
private int numTotalDocs = 0;
|
||||
private int docid = 0;
|
||||
|
||||
private final BytesRef scratch = new BytesRef();
|
||||
|
||||
|
||||
public NormsWriter(Directory directory, String segment, IOContext context)
|
||||
throws IOException {
|
||||
final String normsFileName = IndexFileNames.segmentFileName(segment, "",
|
||||
NORMS_EXTENSION);
|
||||
output = directory.createOutput(normsFileName, context);
|
||||
|
||||
}
|
||||
|
||||
public void startField(FieldInfo info) throws IOException {
|
||||
assert info.omitNorms == false;
|
||||
docid = 0;
|
||||
write(FIELD);
|
||||
write(info.name);
|
||||
newLine();
|
||||
}
|
||||
|
||||
public void writeNorm(byte norm) throws IOException {
|
||||
write(DOC);
|
||||
write(Integer.toString(docid));
|
||||
newLine();
|
||||
|
||||
write(NORM);
|
||||
write(norm);
|
||||
newLine();
|
||||
docid++;
|
||||
}
|
||||
|
||||
public void finish(int numDocs) throws IOException {
|
||||
if (docid != numDocs) {
|
||||
throw new RuntimeException(
|
||||
"mergeNorms produced an invalid result: docCount is " + numDocs
|
||||
+ " but only saw " + docid + " file=" + output.toString()
|
||||
+ "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
write(END);
|
||||
newLine();
|
||||
}
|
||||
|
||||
private void write(String s) throws IOException {
|
||||
SimpleTextUtil.write(output, s, scratch);
|
||||
}
|
||||
|
||||
private void write(BytesRef bytes) throws IOException {
|
||||
SimpleTextUtil.write(output, bytes);
|
||||
}
|
||||
|
||||
private void write(byte b) throws IOException {
|
||||
scratch.grow(1);
|
||||
scratch.bytes[scratch.offset] = b;
|
||||
scratch.length = 1;
|
||||
SimpleTextUtil.write(output, scratch);
|
||||
}
|
||||
|
||||
private void newLine() throws IOException {
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
}
|
||||
|
||||
public void setNumTotalDocs(int numTotalDocs) {
|
||||
assert this.numTotalDocs == 0 || numTotalDocs == this.numTotalDocs;
|
||||
this.numTotalDocs = numTotalDocs;
|
||||
}
|
||||
|
||||
public void abort() throws IOException {
|
||||
close();
|
||||
}
|
||||
|
||||
public void finish() throws IOException {
|
||||
finish(numTotalDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
output.close();
|
||||
}
|
||||
}
|
||||
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = info.getFieldInfos();
|
||||
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.normsPresent()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "",
|
||||
NORMS_EXTENSION));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,35 +18,123 @@ package org.apache.lucene.codecs.simpletext;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* plain-text norms format
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextNormsFormat extends NormsFormat {
|
||||
private static final String NORMS_SEG_SUFFIX = "len";
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new SimpleTextNormsConsumer(state.directory, state.segmentName, state.context);
|
||||
return new SimpleTextNormsPerDocConsumer(state, NORMS_SEG_SUFFIX);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return new SimpleTextNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context);
|
||||
return new SimpleTextNormsPerDocProducer(state,
|
||||
BytesRef.getUTF8SortedAsUnicodeComparator(), NORMS_SEG_SUFFIX);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextNormsConsumer.files(info, files);
|
||||
}
|
||||
SimpleTextNormsPerDocConsumer.files(info, files);
|
||||
}
|
||||
|
||||
public static class SimpleTextNormsPerDocProducer extends
|
||||
SimpleTextPerDocProducer {
|
||||
|
||||
public SimpleTextNormsPerDocProducer(SegmentReadState state,
|
||||
Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
|
||||
super(state, comp, segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean canLoad(FieldInfo info) {
|
||||
return info.normsPresent();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getDocValuesType(FieldInfo info) {
|
||||
return info.getNormType();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean anyDocValuesFields(FieldInfos infos) {
|
||||
return infos.hasNorms();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class SimpleTextNormsPerDocConsumer extends
|
||||
SimpleTextPerDocConsumer {
|
||||
|
||||
public SimpleTextNormsPerDocConsumer(PerDocWriteState state,
|
||||
String segmentSuffix) throws IOException {
|
||||
super(state, segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
|
||||
throws IOException {
|
||||
return reader.normValues(info.name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean canMerge(FieldInfo info) {
|
||||
return info.normsPresent();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getDocValuesType(FieldInfo info) {
|
||||
return info.getNormType();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
Set<String> files = new HashSet<String>();
|
||||
filesInternal(state.fieldInfos, state.segmentName, files, segmentSuffix);
|
||||
IOUtils.deleteFilesIgnoringExceptions(state.directory,
|
||||
files.toArray(new String[0]));
|
||||
}
|
||||
|
||||
public static void files(SegmentInfo segmentInfo, Set<String> files)
|
||||
throws IOException {
|
||||
filesInternal(segmentInfo.getFieldInfos(), segmentInfo.name, files,
|
||||
NORMS_SEG_SUFFIX);
|
||||
}
|
||||
|
||||
public static void filesInternal(FieldInfos fieldInfos, String segmentName,
|
||||
Set<String> files, String segmentSuffix) {
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.normsPresent()) {
|
||||
String id = docValuesId(segmentName, fieldInfo.number);
|
||||
files.add(IndexFileNames.segmentFileName(id, "",
|
||||
segmentSuffix));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,175 +0,0 @@
|
|||
package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* 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 static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.DOC;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.END;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.FIELD;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.NORM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.NORMS_EXTENSION;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
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.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Reads plain-text norms
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextNormsProducer extends PerDocProducer {
|
||||
|
||||
Map<String,NormsDocValues> norms = new HashMap<String,NormsDocValues>();
|
||||
|
||||
public SimpleTextNormsProducer(Directory directory, SegmentInfo si, FieldInfos fields, IOContext context) throws IOException {
|
||||
if (fields.hasNorms()) {
|
||||
readNorms(directory.openInput(IndexFileNames.segmentFileName(si.name, "", NORMS_EXTENSION), context), si.docCount);
|
||||
}
|
||||
}
|
||||
|
||||
// we read in all the norms up front into a hashmap
|
||||
private void readNorms(IndexInput in, int maxDoc) throws IOException {
|
||||
BytesRef scratch = new BytesRef();
|
||||
boolean success = false;
|
||||
try {
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
while (!scratch.equals(END)) {
|
||||
assert StringHelper.startsWith(scratch, FIELD);
|
||||
final String fieldName = readString(FIELD.length, scratch);
|
||||
byte bytes[] = new byte[maxDoc];
|
||||
for (int i = 0; i < bytes.length; i++) {
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, DOC);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, NORM);
|
||||
bytes[i] = scratch.bytes[scratch.offset + NORM.length];
|
||||
}
|
||||
norms.put(fieldName, new NormsDocValues(new Norm(bytes)));
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, FIELD) || scratch.equals(END);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
norms = null;
|
||||
}
|
||||
|
||||
static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = info.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.normsPresent()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", SimpleTextNormsConsumer.NORMS_EXTENSION));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private String readString(int offset, BytesRef scratch) {
|
||||
return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
return norms.get(field);
|
||||
}
|
||||
|
||||
private class NormsDocValues extends DocValues {
|
||||
private final Source source;
|
||||
public NormsDocValues(Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return getSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return Type.FIXED_INTS_8;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueSize() {
|
||||
return 1;
|
||||
}
|
||||
}
|
||||
|
||||
static final class Norm extends Source {
|
||||
protected Norm(byte[] bytes) {
|
||||
super(Type.FIXED_INTS_8);
|
||||
this.bytes = bytes;
|
||||
}
|
||||
final byte bytes[];
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.bytes = bytes;
|
||||
ref.offset = docID;
|
||||
ref.length = 1;
|
||||
return ref;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
return bytes[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getArray() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
package org.apache.lucene.codecs.simpletext;
|
||||
/**
|
||||
* 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.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class SimpleTextPerDocConsumer extends PerDocConsumer {
|
||||
|
||||
protected final PerDocWriteState state;
|
||||
protected final String segmentSuffix;
|
||||
public SimpleTextPerDocConsumer(PerDocWriteState state, String segmentSuffix)
|
||||
throws IOException {
|
||||
this.state = state;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(Type type, FieldInfo field)
|
||||
throws IOException {
|
||||
return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentName,
|
||||
field.number), state.directory, state.context, type, segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
Set<String> files = new HashSet<String>();
|
||||
files(state.directory, state.fieldInfos, state.segmentName, files, segmentSuffix);
|
||||
IOUtils.deleteFilesIgnoringExceptions(state.directory,
|
||||
files.toArray(new String[0]));
|
||||
}
|
||||
|
||||
|
||||
static void files(SegmentInfo info, Set<String> files, String segmentSuffix) throws IOException {
|
||||
files(info.dir, info.getFieldInfos(), info.name, files, segmentSuffix);
|
||||
}
|
||||
|
||||
static String docValuesId(String segmentsName, int fieldId) {
|
||||
return segmentsName + "_" + fieldId;
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
private static void files(Directory dir, FieldInfos fieldInfos,
|
||||
String segmentName, Set<String> files, String segmentSuffix) {
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
String filename = docValuesId(segmentName, fieldInfo.number);
|
||||
files.add(IndexFileNames.segmentFileName(filename, "",
|
||||
segmentSuffix));
|
||||
try {
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
|
||||
segmentSuffix));
|
||||
} catch (IOException e) {
|
||||
// don't throw checked exception - dir is only used in assert
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,431 @@
|
|||
package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* 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 static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.DOC;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.END;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.HEADER;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.VALUE_SIZE;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesArraySource;
|
||||
import org.apache.lucene.codecs.PerDocProducerBase;
|
||||
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.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.packed.PackedInts.Reader;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextPerDocProducer extends PerDocProducerBase {
|
||||
protected final TreeMap<String, DocValues> docValues;
|
||||
private Comparator<BytesRef> comp;
|
||||
private final String segmentSuffix;
|
||||
|
||||
/**
|
||||
* Creates a new {@link Lucene40DocValuesProducer} instance and loads all
|
||||
* {@link DocValues} instances for this segment and codec.
|
||||
*/
|
||||
public SimpleTextPerDocProducer(SegmentReadState state,
|
||||
Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
|
||||
this.comp = comp;
|
||||
this.segmentSuffix = segmentSuffix;
|
||||
if (anyDocValuesFields(state.fieldInfos)) {
|
||||
docValues = load(state.fieldInfos, state.segmentInfo.name,
|
||||
state.segmentInfo.docCount, state.dir, state.context);
|
||||
} else {
|
||||
docValues = new TreeMap<String, DocValues>();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, DocValues> docValues() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
protected DocValues loadDocValues(int docCount, Directory dir, String id,
|
||||
DocValues.Type type, IOContext context) throws IOException {
|
||||
return new SimpleTextDocValues(dir, context, type, id, docCount, comp, segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void closeInternal(Collection<? extends Closeable> closeables)
|
||||
throws IOException {
|
||||
IOUtils.close(closeables);
|
||||
}
|
||||
|
||||
private static class SimpleTextDocValues extends DocValues {
|
||||
|
||||
private int docCount;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
super.close();
|
||||
} finally {
|
||||
IOUtils.close(input);
|
||||
}
|
||||
}
|
||||
|
||||
private Type type;
|
||||
private Comparator<BytesRef> comp;
|
||||
private int valueSize;
|
||||
private final IndexInput input;
|
||||
|
||||
public SimpleTextDocValues(Directory dir, IOContext ctx, Type type,
|
||||
String id, int docCount, Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
|
||||
this.type = type;
|
||||
this.docCount = docCount;
|
||||
this.comp = comp;
|
||||
final String fileName = IndexFileNames.segmentFileName(id, "", segmentSuffix);
|
||||
boolean success = false;
|
||||
IndexInput in = null;
|
||||
try {
|
||||
in = dir.openInput(fileName, ctx);
|
||||
valueSize = readHeader(in);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
}
|
||||
}
|
||||
input = in;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
boolean success = false;
|
||||
IndexInput in = (IndexInput) input.clone();
|
||||
try {
|
||||
Source source = null;
|
||||
switch (type) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
source = read(in, new ValueReader(type, docCount, comp));
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
case FIXED_INTS_32:
|
||||
case VAR_INTS:
|
||||
case FIXED_INTS_64:
|
||||
case FIXED_INTS_8:
|
||||
case FLOAT_32:
|
||||
case FLOAT_64:
|
||||
source = read(in, new ValueReader(type, docCount, null));
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("unknown type: " + type);
|
||||
}
|
||||
assert source != null;
|
||||
success = true;
|
||||
return source;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
} else {
|
||||
IOUtils.close(in);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int readHeader(IndexInput in) throws IOException {
|
||||
BytesRef scratch = new BytesRef();
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, HEADER);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, VALUE_SIZE);
|
||||
return Integer.parseInt(readString(scratch.offset + VALUE_SIZE.length,
|
||||
scratch));
|
||||
}
|
||||
|
||||
private Source read(IndexInput in, ValueReader reader) throws IOException {
|
||||
BytesRef scratch = new BytesRef();
|
||||
for (int i = 0; i < docCount; i++) {
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
|
||||
assert StringHelper.startsWith(scratch, DOC) : scratch.utf8ToString();
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, VALUE);
|
||||
reader.fromString(i, scratch, scratch.offset + VALUE.length);
|
||||
}
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert scratch.equals(END);
|
||||
return reader.getSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return this.getSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueSize() {
|
||||
return valueSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static String readString(int offset, BytesRef scratch) {
|
||||
return new String(scratch.bytes, scratch.offset + offset, scratch.length
|
||||
- offset, IOUtils.CHARSET_UTF_8);
|
||||
}
|
||||
|
||||
private static final class ValueReader {
|
||||
private final Type type;
|
||||
private byte[] bytes;
|
||||
private short[] shorts;
|
||||
private int[] ints;
|
||||
private long[] longs;
|
||||
private float[] floats;
|
||||
private double[] doubles;
|
||||
private Source source;
|
||||
private BytesRefHash hash;
|
||||
private BytesRef scratch;
|
||||
|
||||
public ValueReader(Type type, int maxDocs, Comparator<BytesRef> comp) {
|
||||
super();
|
||||
this.type = type;
|
||||
Source docValuesArray = null;
|
||||
switch (type) {
|
||||
case FIXED_INTS_16:
|
||||
shorts = new short[maxDocs];
|
||||
docValuesArray = DocValuesArraySource.forType(type)
|
||||
.newFromArray(shorts);
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
ints = new int[maxDocs];
|
||||
docValuesArray = DocValuesArraySource.forType(type).newFromArray(ints);
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
longs = new long[maxDocs];
|
||||
docValuesArray = DocValuesArraySource.forType(type)
|
||||
.newFromArray(longs);
|
||||
break;
|
||||
case VAR_INTS:
|
||||
longs = new long[maxDocs];
|
||||
docValuesArray = new VarIntsArraySource(type, longs);
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
bytes = new byte[maxDocs];
|
||||
docValuesArray = DocValuesArraySource.forType(type).newFromArray(bytes);
|
||||
break;
|
||||
case FLOAT_32:
|
||||
floats = new float[maxDocs];
|
||||
docValuesArray = DocValuesArraySource.forType(type)
|
||||
.newFromArray(floats);
|
||||
break;
|
||||
case FLOAT_64:
|
||||
doubles = new double[maxDocs];
|
||||
docValuesArray = DocValuesArraySource.forType(type).newFromArray(
|
||||
doubles);
|
||||
break;
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
assert comp != null;
|
||||
hash = new BytesRefHash();
|
||||
BytesSource bytesSource = new BytesSource(type, comp, maxDocs, hash);
|
||||
ints = bytesSource.docIdToEntry;
|
||||
source = bytesSource;
|
||||
scratch = new BytesRef();
|
||||
break;
|
||||
|
||||
}
|
||||
if (docValuesArray != null) {
|
||||
assert source == null;
|
||||
this.source = docValuesArray;
|
||||
}
|
||||
}
|
||||
|
||||
public void fromString(int ord, BytesRef ref, int offset) {
|
||||
switch (type) {
|
||||
case FIXED_INTS_16:
|
||||
assert shorts != null;
|
||||
shorts[ord] = Short.parseShort(readString(offset, ref));
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
assert ints != null;
|
||||
ints[ord] = Integer.parseInt(readString(offset, ref));
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
case VAR_INTS:
|
||||
assert longs != null;
|
||||
longs[ord] = Long.parseLong(readString(offset, ref));
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
assert bytes != null;
|
||||
bytes[ord] = (byte) Integer.parseInt(readString(offset, ref));
|
||||
break;
|
||||
case FLOAT_32:
|
||||
assert floats != null;
|
||||
floats[ord] = Float.parseFloat(readString(offset, ref));
|
||||
break;
|
||||
case FLOAT_64:
|
||||
assert doubles != null;
|
||||
doubles[ord] = Double.parseDouble(readString(offset, ref));
|
||||
break;
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
scratch.bytes = ref.bytes;
|
||||
scratch.length = ref.length - offset;
|
||||
scratch.offset = ref.offset + offset;
|
||||
int key = hash.add(scratch);
|
||||
ints[ord] = key < 0 ? (-key) - 1 : key;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
public Source getSource() {
|
||||
if (source instanceof BytesSource) {
|
||||
((BytesSource) source).maybeSort();
|
||||
}
|
||||
return source;
|
||||
}
|
||||
}
|
||||
|
||||
private static final class BytesSource extends SortedSource {
|
||||
|
||||
private final BytesRefHash hash;
|
||||
int[] docIdToEntry;
|
||||
int[] sortedEntries;
|
||||
int[] adresses;
|
||||
private final boolean isSorted;
|
||||
|
||||
protected BytesSource(Type type, Comparator<BytesRef> comp, int maxDoc,
|
||||
BytesRefHash hash) {
|
||||
super(type, comp);
|
||||
docIdToEntry = new int[maxDoc];
|
||||
this.hash = hash;
|
||||
isSorted = type == Type.BYTES_FIXED_SORTED
|
||||
|| type == Type.BYTES_VAR_SORTED;
|
||||
}
|
||||
|
||||
void maybeSort() {
|
||||
if (isSorted) {
|
||||
adresses = new int[hash.size()];
|
||||
sortedEntries = hash.sort(getComparator());
|
||||
for (int i = 0; i < adresses.length; i++) {
|
||||
int entry = sortedEntries[i];
|
||||
adresses[entry] = i;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
if (isSorted) {
|
||||
return hash.get(sortedEntries[ord(docID)], ref);
|
||||
} else {
|
||||
return hash.get(docIdToEntry[docID], ref);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSource asSortedSource() {
|
||||
if (isSorted) {
|
||||
return this;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int ord(int docID) {
|
||||
assert isSorted;
|
||||
try {
|
||||
return adresses[docIdToEntry[docID]];
|
||||
} catch (Exception e) {
|
||||
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
|
||||
assert isSorted;
|
||||
return hash.get(sortedEntries[ord], bytesRef);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Reader getDocToOrd() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCount() {
|
||||
return hash.size();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class VarIntsArraySource extends Source {
|
||||
|
||||
private final long[] array;
|
||||
|
||||
protected VarIntsArraySource(Type type, long[] array) {
|
||||
super(type);
|
||||
this.array = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt(int docID) {
|
||||
return array[docID];
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
DocValuesArraySource.copyLong(ref, getInt(docID));
|
||||
return ref;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -54,7 +54,7 @@ public interface IndexableField {
|
|||
/** Non-null if this field has a Reader value */
|
||||
public Reader readerValue();
|
||||
|
||||
/** Non-null if this field hasa numeric value */
|
||||
/** Non-null if this field has a numeric value */
|
||||
public Number numericValue();
|
||||
|
||||
/**
|
||||
|
|
|
@ -148,8 +148,8 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
|
||||
Directory target = newDirectory();
|
||||
IndexWriter w = new IndexWriter(target, writerConfig(random.nextBoolean()));
|
||||
IndexReader r_1 = IndexReader.open(w_1, true);
|
||||
IndexReader r_2 = IndexReader.open(w_2, true);
|
||||
DirectoryReader r_1 = DirectoryReader.open(w_1, true);
|
||||
DirectoryReader r_2 = DirectoryReader.open(w_2, true);
|
||||
if (random.nextBoolean()) {
|
||||
w.addIndexes(d_1, d_2);
|
||||
} else {
|
||||
|
@ -163,7 +163,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
|
||||
// check values
|
||||
|
||||
IndexReader merged = IndexReader.open(w, true);
|
||||
DirectoryReader merged = DirectoryReader.open(w, true);
|
||||
Source source_1 = getSource(getDocValues(r_1, first.name()));
|
||||
Source source_2 = getSource(getDocValues(r_2, second.name()));
|
||||
Source source_1_merged = getSource(getDocValues(merged, first.name()));
|
||||
|
@ -260,7 +260,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
FixedBitSet deleted = indexValues(w, numValues, val, numVariantList,
|
||||
withDeletions, 7);
|
||||
List<Closeable> closeables = new ArrayList<Closeable>();
|
||||
IndexReader r = IndexReader.open(w, true);
|
||||
DirectoryReader r = DirectoryReader.open(w, true);
|
||||
final int numRemainingValues = numValues - deleted.cardinality();
|
||||
final int base = r.numDocs() - numRemainingValues;
|
||||
// for FIXED_INTS_8 we use value mod 128 - to enable testing in
|
||||
|
@ -338,7 +338,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
final int bytesSize = 1 + atLeast(50);
|
||||
FixedBitSet deleted = indexValues(w, numValues, byteIndexValue,
|
||||
byteVariantList, withDeletions, bytesSize);
|
||||
final IndexReader r = IndexReader.open(w, withDeletions);
|
||||
final DirectoryReader r = DirectoryReader.open(w, withDeletions);
|
||||
assertEquals(0, r.numDeletedDocs());
|
||||
final int numRemainingValues = numValues - deleted.cardinality();
|
||||
final int base = r.numDocs() - numRemainingValues;
|
||||
|
@ -422,12 +422,16 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
for (Type val : numVariantList) {
|
||||
indexValues(w, numValues, val, numVariantList,
|
||||
false, 7);
|
||||
IndexReader r = IndexReader.open(w, true);
|
||||
DirectoryReader r = DirectoryReader.open(w, true);
|
||||
if (val == Type.VAR_INTS) {
|
||||
DocValues docValues = getDocValues(r, val.name());
|
||||
}
|
||||
DocValues docValues = getDocValues(r, val.name());
|
||||
assertNotNull(docValues);
|
||||
// make sure we don't get a direct source since they don't support getArray()
|
||||
if (val == Type.VAR_INTS) {
|
||||
}
|
||||
Source source = docValues.getSource();
|
||||
|
||||
switch (source.type()) {
|
||||
case FIXED_INTS_8:
|
||||
{
|
||||
|
@ -465,7 +469,8 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
break;
|
||||
case VAR_INTS:
|
||||
case VAR_INTS:
|
||||
System.out.println(source.hasArray());
|
||||
assertFalse(source.hasArray());
|
||||
break;
|
||||
case FLOAT_32:
|
||||
|
@ -503,7 +508,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
final int numValues = 50 + atLeast(10);
|
||||
// only single byte fixed straight supports getArray()
|
||||
indexValues(w, numValues, Type.BYTES_FIXED_STRAIGHT, null, false, 1);
|
||||
IndexReader r = IndexReader.open(w, true);
|
||||
DirectoryReader r = DirectoryReader.open(w, true);
|
||||
DocValues docValues = getDocValues(r, Type.BYTES_FIXED_STRAIGHT.name());
|
||||
assertNotNull(docValues);
|
||||
// make sure we don't get a direct source since they don't support
|
||||
|
@ -513,12 +518,13 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
switch (source.type()) {
|
||||
case BYTES_FIXED_STRAIGHT: {
|
||||
BytesRef ref = new BytesRef();
|
||||
assertTrue(source.hasArray());
|
||||
byte[] values = (byte[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
source.getBytes(i, ref);
|
||||
assertEquals(1, ref.length);
|
||||
assertEquals(values[i], ref.bytes[ref.offset]);
|
||||
if (source.hasArray()) {
|
||||
byte[] values = (byte[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
source.getBytes(i, ref);
|
||||
assertEquals(1, ref.length);
|
||||
assertEquals(values[i], ref.bytes[ref.offset]);
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
|
@ -925,4 +931,4 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -163,6 +163,11 @@ class PreFlexRWNormsConsumer extends PerDocConsumer {
|
|||
docIDs[upto] = docID;
|
||||
upto++;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getType() {
|
||||
return Type.FIXED_INTS_8;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue