mirror of https://github.com/apache/lucene.git
LUCENE-5969: current state for dv/norms
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1628382 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ffca49423d
commit
23a0e2fa3a
|
@ -65,12 +65,29 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
private final Map<String,Accountable> instanceInfo = new HashMap<>();
|
private final Map<String,Accountable> instanceInfo = new HashMap<>();
|
||||||
|
|
||||||
private final AtomicLong ramBytesUsed;
|
private final AtomicLong ramBytesUsed;
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
Lucene40DocValuesReader(Lucene40DocValuesReader original) throws IOException {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
dir = original.dir;
|
||||||
|
state = original.state;
|
||||||
|
legacyKey = original.legacyKey;
|
||||||
|
numericInstances.putAll(original.numericInstances);
|
||||||
|
binaryInstances.putAll(original.binaryInstances);
|
||||||
|
sortedInstances.putAll(original.sortedInstances);
|
||||||
|
instanceInfo.putAll(original.instanceInfo);
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
Lucene40DocValuesReader(SegmentReadState state, String filename, String legacyKey) throws IOException {
|
Lucene40DocValuesReader(SegmentReadState state, String filename, String legacyKey) throws IOException {
|
||||||
this.state = state;
|
this.state = state;
|
||||||
this.legacyKey = legacyKey;
|
this.legacyKey = legacyKey;
|
||||||
this.dir = new CompoundFileDirectory(state.directory, filename, state.context, false);
|
this.dir = new CompoundFileDirectory(state.directory, filename, state.context, false);
|
||||||
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOf(getClass()));
|
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOf(getClass()));
|
||||||
|
merging = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -115,7 +132,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
IOUtils.closeWhileHandlingException(input);
|
IOUtils.closeWhileHandlingException(input);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
numericInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
numericInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -132,8 +151,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
values[i] = input.readLong();
|
values[i] = input.readLong();
|
||||||
}
|
}
|
||||||
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
|
||||||
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -144,8 +165,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
final long minValue = input.readLong();
|
final long minValue = input.readLong();
|
||||||
final long defaultValue = input.readLong();
|
final long defaultValue = input.readLong();
|
||||||
final PackedInts.Reader reader = PackedInts.getReader(input);
|
final PackedInts.Reader reader = PackedInts.getReader(input);
|
||||||
instanceInfo.put(field.name, reader);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
instanceInfo.put(field.name, reader);
|
||||||
|
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -174,8 +197,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
final byte values[] = new byte[maxDoc];
|
final byte values[] = new byte[maxDoc];
|
||||||
input.readBytes(values, 0, values.length);
|
input.readBytes(values, 0, values.length);
|
||||||
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("byte array", bytesUsed));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
instanceInfo.put(field.name, Accountables.namedAccountable("byte array", bytesUsed));
|
||||||
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -198,8 +223,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
values[i] = input.readShort();
|
values[i] = input.readShort();
|
||||||
}
|
}
|
||||||
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("short array", bytesUsed));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
instanceInfo.put(field.name, Accountables.namedAccountable("short array", bytesUsed));
|
||||||
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -222,8 +249,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
values[i] = input.readInt();
|
values[i] = input.readInt();
|
||||||
}
|
}
|
||||||
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("int array", bytesUsed));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
instanceInfo.put(field.name, Accountables.namedAccountable("int array", bytesUsed));
|
||||||
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -246,8 +275,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
values[i] = input.readLong();
|
values[i] = input.readLong();
|
||||||
}
|
}
|
||||||
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
instanceInfo.put(field.name, Accountables.namedAccountable("long array", bytesUsed));
|
||||||
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -270,8 +301,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
values[i] = input.readInt();
|
values[i] = input.readInt();
|
||||||
}
|
}
|
||||||
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("float array", bytesUsed));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
instanceInfo.put(field.name, Accountables.namedAccountable("float array", bytesUsed));
|
||||||
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -294,8 +327,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
values[i] = input.readLong();
|
values[i] = input.readLong();
|
||||||
}
|
}
|
||||||
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
long bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("double array", bytesUsed));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
instanceInfo.put(field.name, Accountables.namedAccountable("double array", bytesUsed));
|
||||||
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -324,7 +359,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
default:
|
default:
|
||||||
throw new AssertionError();
|
throw new AssertionError();
|
||||||
}
|
}
|
||||||
binaryInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
binaryInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -343,8 +380,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
||||||
CodecUtil.checkEOF(input);
|
CodecUtil.checkEOF(input);
|
||||||
success = true;
|
success = true;
|
||||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
|
if (!merging) {
|
||||||
instanceInfo.put(field.name, bytesReader);
|
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
|
||||||
|
instanceInfo.put(field.name, bytesReader);
|
||||||
|
}
|
||||||
return new BinaryDocValues() {
|
return new BinaryDocValues() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -387,8 +426,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
CodecUtil.checkEOF(index);
|
CodecUtil.checkEOF(index);
|
||||||
success = true;
|
success = true;
|
||||||
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
if (!merging) {
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("variable straight", bytesUsed));
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
instanceInfo.put(field.name, Accountables.namedAccountable("variable straight", bytesUsed));
|
||||||
|
}
|
||||||
return new BinaryDocValues() {
|
return new BinaryDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public BytesRef get(int docID) {
|
public BytesRef get(int docID) {
|
||||||
|
@ -433,8 +474,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
CodecUtil.checkEOF(data);
|
CodecUtil.checkEOF(data);
|
||||||
CodecUtil.checkEOF(index);
|
CodecUtil.checkEOF(index);
|
||||||
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
if (!merging) {
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("fixed deref", bytesUsed));
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
instanceInfo.put(field.name, Accountables.namedAccountable("fixed deref", bytesUsed));
|
||||||
|
}
|
||||||
success = true;
|
success = true;
|
||||||
return new BinaryDocValues() {
|
return new BinaryDocValues() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -478,8 +521,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
CodecUtil.checkEOF(data);
|
CodecUtil.checkEOF(data);
|
||||||
CodecUtil.checkEOF(index);
|
CodecUtil.checkEOF(index);
|
||||||
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
if (!merging) {
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("variable deref", bytesUsed));
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
instanceInfo.put(field.name, Accountables.namedAccountable("variable deref", bytesUsed));
|
||||||
|
}
|
||||||
success = true;
|
success = true;
|
||||||
return new BinaryDocValues() {
|
return new BinaryDocValues() {
|
||||||
|
|
||||||
|
@ -542,7 +587,9 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
IOUtils.closeWhileHandlingException(data, index);
|
IOUtils.closeWhileHandlingException(data, index);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
sortedInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
sortedInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -563,8 +610,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
||||||
final PackedInts.Reader reader = PackedInts.getReader(index);
|
final PackedInts.Reader reader = PackedInts.getReader(index);
|
||||||
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
long bytesUsed = bytesReader.ramBytesUsed() + reader.ramBytesUsed();
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
if (!merging) {
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("fixed sorted", bytesUsed));
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
instanceInfo.put(field.name, Accountables.namedAccountable("fixed sorted", bytesUsed));
|
||||||
|
}
|
||||||
|
|
||||||
return correctBuggyOrds(new SortedDocValues() {
|
return correctBuggyOrds(new SortedDocValues() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -603,8 +652,10 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
|
|
||||||
final int valueCount = addressReader.size() - 1;
|
final int valueCount = addressReader.size() - 1;
|
||||||
long bytesUsed = bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed();
|
long bytesUsed = bytesReader.ramBytesUsed() + addressReader.ramBytesUsed() + ordsReader.ramBytesUsed();
|
||||||
ramBytesUsed.addAndGet(bytesUsed);
|
if (!merging) {
|
||||||
instanceInfo.put(field.name, Accountables.namedAccountable("var sorted", bytesUsed));
|
ramBytesUsed.addAndGet(bytesUsed);
|
||||||
|
instanceInfo.put(field.name, Accountables.namedAccountable("var sorted", bytesUsed));
|
||||||
|
}
|
||||||
|
|
||||||
return correctBuggyOrds(new SortedDocValues() {
|
return correctBuggyOrds(new SortedDocValues() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -690,6 +741,11 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
|
||||||
public void checkIntegrity() throws IOException {
|
public void checkIntegrity() throws IOException {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene40DocValuesReader(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName();
|
return getClass().getSimpleName();
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene40;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.lucene.codecs.DocValuesProducer;
|
||||||
import org.apache.lucene.codecs.NormsProducer;
|
import org.apache.lucene.codecs.NormsProducer;
|
||||||
import org.apache.lucene.index.FieldInfo;
|
import org.apache.lucene.index.FieldInfo;
|
||||||
import org.apache.lucene.index.NumericDocValues;
|
import org.apache.lucene.index.NumericDocValues;
|
||||||
|
@ -31,7 +32,12 @@ import org.apache.lucene.util.Accountable;
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
final class Lucene40NormsReader extends NormsProducer {
|
final class Lucene40NormsReader extends NormsProducer {
|
||||||
private final Lucene40DocValuesReader impl;
|
private final DocValuesProducer impl;
|
||||||
|
|
||||||
|
// clone for merge
|
||||||
|
Lucene40NormsReader(DocValuesProducer impl) throws IOException {
|
||||||
|
this.impl = impl.getMergeInstance();
|
||||||
|
}
|
||||||
|
|
||||||
Lucene40NormsReader(SegmentReadState state, String filename) throws IOException {
|
Lucene40NormsReader(SegmentReadState state, String filename) throws IOException {
|
||||||
impl = new Lucene40DocValuesReader(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
|
impl = new Lucene40DocValuesReader(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
|
||||||
|
@ -62,6 +68,11 @@ final class Lucene40NormsReader extends NormsProducer {
|
||||||
impl.checkIntegrity();
|
impl.checkIntegrity();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NormsProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene40NormsReader(impl);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(" + impl + ")";
|
return getClass().getSimpleName() + "(" + impl + ")";
|
||||||
|
|
|
@ -106,9 +106,34 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
static final int VERSION_GCD_COMPRESSION = 1;
|
static final int VERSION_GCD_COMPRESSION = 1;
|
||||||
static final int VERSION_CHECKSUM = 2;
|
static final int VERSION_CHECKSUM = 2;
|
||||||
static final int VERSION_CURRENT = VERSION_CHECKSUM;
|
static final int VERSION_CURRENT = VERSION_CHECKSUM;
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
Lucene42DocValuesProducer(Lucene42DocValuesProducer original) throws IOException {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
numerics = original.numerics;
|
||||||
|
binaries = original.binaries;
|
||||||
|
fsts = original.fsts;
|
||||||
|
data = original.data.clone();
|
||||||
|
version = original.version;
|
||||||
|
numEntries = original.numEntries;
|
||||||
|
|
||||||
|
numericInstances.putAll(original.numericInstances);
|
||||||
|
binaryInstances.putAll(original.binaryInstances);
|
||||||
|
fstInstances.putAll(original.fstInstances);
|
||||||
|
numericInfo.putAll(original.numericInfo);
|
||||||
|
binaryInfo.putAll(original.binaryInfo);
|
||||||
|
addressInfo.putAll(original.addressInfo);
|
||||||
|
|
||||||
|
maxDoc = original.maxDoc;
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
Lucene42DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
Lucene42DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
maxDoc = state.segmentInfo.getDocCount();
|
maxDoc = state.segmentInfo.getDocCount();
|
||||||
|
merging = false;
|
||||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
// read in the entries from the metadata file.
|
// read in the entries from the metadata file.
|
||||||
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
||||||
|
@ -223,7 +248,9 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
NumericDocValues instance = numericInstances.get(field.name);
|
NumericDocValues instance = numericInstances.get(field.name);
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
instance = loadNumeric(field);
|
instance = loadNumeric(field);
|
||||||
numericInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
numericInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -271,8 +298,10 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
final int formatID = data.readVInt();
|
final int formatID = data.readVInt();
|
||||||
final int bitsPerValue = data.readVInt();
|
final int bitsPerValue = data.readVInt();
|
||||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
|
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
|
||||||
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
|
if (!merging) {
|
||||||
numericInfo.put(field.name, ordsReader);
|
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
|
||||||
|
numericInfo.put(field.name, ordsReader);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -282,14 +311,18 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
case DELTA_COMPRESSED:
|
case DELTA_COMPRESSED:
|
||||||
final int blockSize = data.readVInt();
|
final int blockSize = data.readVInt();
|
||||||
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
|
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
|
||||||
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
if (!merging) {
|
||||||
numericInfo.put(field.name, reader);
|
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
||||||
|
numericInfo.put(field.name, reader);
|
||||||
|
}
|
||||||
return reader;
|
return reader;
|
||||||
case UNCOMPRESSED:
|
case UNCOMPRESSED:
|
||||||
final byte bytes[] = new byte[maxDoc];
|
final byte bytes[] = new byte[maxDoc];
|
||||||
data.readBytes(bytes, 0, bytes.length);
|
data.readBytes(bytes, 0, bytes.length);
|
||||||
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
|
if (!merging) {
|
||||||
numericInfo.put(field.name, Accountables.namedAccountable("byte array", maxDoc));
|
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
|
||||||
|
numericInfo.put(field.name, Accountables.namedAccountable("byte array", maxDoc));
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -301,8 +334,10 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
final long mult = data.readLong();
|
final long mult = data.readLong();
|
||||||
final int quotientBlockSize = data.readVInt();
|
final int quotientBlockSize = data.readVInt();
|
||||||
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
|
final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
|
||||||
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
|
if (!merging) {
|
||||||
numericInfo.put(field.name, quotientReader);
|
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
|
||||||
|
numericInfo.put(field.name, quotientReader);
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -319,7 +354,9 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
BinaryDocValues instance = binaryInstances.get(field.name);
|
BinaryDocValues instance = binaryInstances.get(field.name);
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
instance = loadBinary(field);
|
instance = loadBinary(field);
|
||||||
binaryInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
binaryInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -330,10 +367,14 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
PagedBytes bytes = new PagedBytes(16);
|
PagedBytes bytes = new PagedBytes(16);
|
||||||
bytes.copy(data, entry.numBytes);
|
bytes.copy(data, entry.numBytes);
|
||||||
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
final PagedBytes.Reader bytesReader = bytes.freeze(true);
|
||||||
binaryInfo.put(field.name, bytesReader);
|
if (!merging) {
|
||||||
|
binaryInfo.put(field.name, bytesReader);
|
||||||
|
}
|
||||||
if (entry.minLength == entry.maxLength) {
|
if (entry.minLength == entry.maxLength) {
|
||||||
final int fixedLength = entry.minLength;
|
final int fixedLength = entry.minLength;
|
||||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
|
if (!merging) {
|
||||||
|
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed());
|
||||||
|
}
|
||||||
return new BinaryDocValues() {
|
return new BinaryDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public BytesRef get(int docID) {
|
public BytesRef get(int docID) {
|
||||||
|
@ -344,8 +385,10 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
};
|
};
|
||||||
} else {
|
} else {
|
||||||
final MonotonicBlockPackedReader addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
|
final MonotonicBlockPackedReader addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
|
||||||
addressInfo.put(field.name, addresses);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addresses.ramBytesUsed());
|
addressInfo.put(field.name, addresses);
|
||||||
|
ramBytesUsed.addAndGet(bytesReader.ramBytesUsed() + addresses.ramBytesUsed());
|
||||||
|
}
|
||||||
return new BinaryDocValues() {
|
return new BinaryDocValues() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -369,8 +412,10 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
||||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
if (!merging) {
|
||||||
fstInstances.put(field.name, instance);
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
fstInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final NumericDocValues docToOrd = getNumeric(field);
|
final NumericDocValues docToOrd = getNumeric(field);
|
||||||
|
@ -446,8 +491,10 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
||||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
if (!merging) {
|
||||||
fstInstances.put(field.name, instance);
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
fstInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final BinaryDocValues docToOrds = getBinary(field);
|
final BinaryDocValues docToOrds = getBinary(field);
|
||||||
|
@ -538,6 +585,11 @@ final class Lucene42DocValuesProducer extends DocValuesProducer {
|
||||||
throw new IllegalStateException("Lucene 4.2 does not support SortedNumeric: how did you pull this off?");
|
throw new IllegalStateException("Lucene 4.2 does not support SortedNumeric: how did you pull this off?");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene42DocValuesProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
data.close();
|
data.close();
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene42;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.lucene.codecs.DocValuesProducer;
|
||||||
import org.apache.lucene.codecs.NormsProducer;
|
import org.apache.lucene.codecs.NormsProducer;
|
||||||
import org.apache.lucene.index.FieldInfo;
|
import org.apache.lucene.index.FieldInfo;
|
||||||
import org.apache.lucene.index.NumericDocValues;
|
import org.apache.lucene.index.NumericDocValues;
|
||||||
|
@ -31,7 +32,12 @@ import org.apache.lucene.util.Accountable;
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
final class Lucene42NormsProducer extends NormsProducer {
|
final class Lucene42NormsProducer extends NormsProducer {
|
||||||
private final Lucene42DocValuesProducer impl;
|
private final DocValuesProducer impl;
|
||||||
|
|
||||||
|
// clone for merge
|
||||||
|
Lucene42NormsProducer(DocValuesProducer impl) throws IOException {
|
||||||
|
this.impl = impl.getMergeInstance();
|
||||||
|
}
|
||||||
|
|
||||||
Lucene42NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
Lucene42NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
impl = new Lucene42DocValuesProducer(state, dataCodec, dataExtension, metaCodec, metaExtension);
|
impl = new Lucene42DocValuesProducer(state, dataCodec, dataExtension, metaCodec, metaExtension);
|
||||||
|
@ -62,6 +68,11 @@ final class Lucene42NormsProducer extends NormsProducer {
|
||||||
impl.close();
|
impl.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NormsProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene42NormsProducer(impl);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(" + impl + ")";
|
return getClass().getSimpleName() + "(" + impl + ")";
|
||||||
|
|
|
@ -97,6 +97,27 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
|
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
|
||||||
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
|
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
Lucene45DocValuesProducer(Lucene45DocValuesProducer original) throws IOException {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
numerics = original.numerics;
|
||||||
|
binaries = original.binaries;
|
||||||
|
sortedSets = original.sortedSets;
|
||||||
|
ords = original.ords;
|
||||||
|
ordIndexes = original.ordIndexes;
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
data = original.data.clone();
|
||||||
|
maxDoc = original.maxDoc;
|
||||||
|
version = original.version;
|
||||||
|
numFields = original.numFields;
|
||||||
|
lenientFieldInfoCheck = original.lenientFieldInfoCheck;
|
||||||
|
addressInstances.putAll(original.addressInstances);
|
||||||
|
ordIndexInstances.putAll(original.ordIndexInstances);
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
/** expert: instantiates a new reader */
|
/** expert: instantiates a new reader */
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
protected Lucene45DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
protected Lucene45DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
|
@ -106,6 +127,7 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
// read in the entries from the metadata file.
|
// read in the entries from the metadata file.
|
||||||
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
||||||
this.maxDoc = state.segmentInfo.getDocCount();
|
this.maxDoc = state.segmentInfo.getDocCount();
|
||||||
|
merging = false;
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
version = CodecUtil.checkHeader(in, metaCodec,
|
version = CodecUtil.checkHeader(in, metaCodec,
|
||||||
|
@ -442,8 +464,10 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
if (addrInstance == null) {
|
if (addrInstance == null) {
|
||||||
data.seek(bytes.addressesOffset);
|
data.seek(bytes.addressesOffset);
|
||||||
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
|
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
|
||||||
addressInstances.put(field.number, addrInstance);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
addressInstances.put(field.number, addrInstance);
|
||||||
|
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
addresses = addrInstance;
|
addresses = addrInstance;
|
||||||
return addresses;
|
return addresses;
|
||||||
|
@ -489,8 +513,10 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
size = 1L + bytes.count / interval;
|
size = 1L + bytes.count / interval;
|
||||||
}
|
}
|
||||||
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
||||||
addressInstances.put(field.number, addrInstance);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
addressInstances.put(field.number, addrInstance);
|
||||||
|
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
addresses = addrInstance;
|
addresses = addrInstance;
|
||||||
return addresses;
|
return addresses;
|
||||||
|
@ -559,8 +585,10 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
if (ordIndexInstance == null) {
|
if (ordIndexInstance == null) {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
|
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
|
||||||
ordIndexInstances.put(field.number, ordIndexInstance);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
ordIndexInstances.put(field.number, ordIndexInstance);
|
||||||
|
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
ordIndex = ordIndexInstance;
|
ordIndex = ordIndexInstance;
|
||||||
return ordIndex;
|
return ordIndex;
|
||||||
|
@ -694,6 +722,11 @@ class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene45DocValuesProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
data.close();
|
data.close();
|
||||||
|
|
|
@ -89,12 +89,34 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
|
private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
|
||||||
private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
|
private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
Lucene49DocValuesProducer(Lucene49DocValuesProducer original) throws IOException {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
numerics = original.numerics;
|
||||||
|
binaries = original.binaries;
|
||||||
|
sortedSets = original.sortedSets;
|
||||||
|
sortedNumerics = original.sortedNumerics;
|
||||||
|
ords = original.ords;
|
||||||
|
ordIndexes = original.ordIndexes;
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed());
|
||||||
|
data = original.data.clone();
|
||||||
|
numFields = original.numFields;
|
||||||
|
maxDoc = original.maxDoc;
|
||||||
|
version = original.version;
|
||||||
|
addressInstances.putAll(original.addressInstances);
|
||||||
|
ordIndexInstances.putAll(original.ordIndexInstances);
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
/** expert: instantiates a new reader */
|
/** expert: instantiates a new reader */
|
||||||
Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
// read in the entries from the metadata file.
|
// read in the entries from the metadata file.
|
||||||
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
||||||
this.maxDoc = state.segmentInfo.getDocCount();
|
this.maxDoc = state.segmentInfo.getDocCount();
|
||||||
|
merging = false;
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
version = CodecUtil.checkHeader(in, metaCodec,
|
version = CodecUtil.checkHeader(in, metaCodec,
|
||||||
|
@ -447,8 +469,10 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
if (addrInstance == null) {
|
if (addrInstance == null) {
|
||||||
data.seek(bytes.addressesOffset);
|
data.seek(bytes.addressesOffset);
|
||||||
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
|
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
|
||||||
addressInstances.put(field.name, addrInstance);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
addressInstances.put(field.name, addrInstance);
|
||||||
|
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
addresses = addrInstance;
|
addresses = addrInstance;
|
||||||
return addresses;
|
return addresses;
|
||||||
|
@ -493,8 +517,10 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
size = 1L + bytes.count / interval;
|
size = 1L + bytes.count / interval;
|
||||||
}
|
}
|
||||||
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
||||||
addressInstances.put(field.name, addrInstance);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
addressInstances.put(field.name, addrInstance);
|
||||||
|
ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
addresses = addrInstance;
|
addresses = addrInstance;
|
||||||
return addresses;
|
return addresses;
|
||||||
|
@ -560,8 +586,10 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
if (ordIndexInstance == null) {
|
if (ordIndexInstance == null) {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
|
ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
|
||||||
ordIndexInstances.put(field.name, ordIndexInstance);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
ordIndexInstances.put(field.name, ordIndexInstance);
|
||||||
|
ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
ordIndex = ordIndexInstance;
|
ordIndex = ordIndexInstance;
|
||||||
return ordIndex;
|
return ordIndex;
|
||||||
|
@ -733,6 +761,11 @@ class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
|
||||||
data.close();
|
data.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene49DocValuesProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
/** metadata entry for a numeric docvalues field */
|
/** metadata entry for a numeric docvalues field */
|
||||||
static class NumericEntry {
|
static class NumericEntry {
|
||||||
private NumericEntry() {}
|
private NumericEntry() {}
|
||||||
|
|
|
@ -0,0 +1,54 @@
|
||||||
|
package org.apache.lucene.codecs.lucene49;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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.NormsConsumer;
|
||||||
|
import org.apache.lucene.codecs.NormsFormat;
|
||||||
|
import org.apache.lucene.codecs.NormsProducer;
|
||||||
|
import org.apache.lucene.index.SegmentReadState;
|
||||||
|
import org.apache.lucene.index.SegmentWriteState;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Lucene 4.9 Score normalization format.
|
||||||
|
* @deprecated only for reading 4.9/4.10 indexes
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
public class Lucene49NormsFormat extends NormsFormat {
|
||||||
|
|
||||||
|
/** Sole Constructor */
|
||||||
|
public Lucene49NormsFormat() {}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||||
|
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final NormsProducer normsProducer(SegmentReadState state) throws IOException {
|
||||||
|
return new Lucene49NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||||
|
}
|
||||||
|
|
||||||
|
static final String DATA_CODEC = "Lucene49NormsData";
|
||||||
|
static final String DATA_EXTENSION = "nvd";
|
||||||
|
static final String METADATA_CODEC = "Lucene49NormsMetadata";
|
||||||
|
static final String METADATA_EXTENSION = "nvm";
|
||||||
|
static final int VERSION_START = 0;
|
||||||
|
static final int VERSION_CURRENT = VERSION_START;
|
||||||
|
}
|
|
@ -42,15 +42,19 @@ import org.apache.lucene.util.packed.PackedInts;
|
||||||
|
|
||||||
import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_START;
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_START;
|
||||||
import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_CURRENT;
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_CURRENT;
|
||||||
import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.CONST_COMPRESSED;
|
|
||||||
import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.DELTA_COMPRESSED;
|
|
||||||
import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.TABLE_COMPRESSED;
|
|
||||||
import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.UNCOMPRESSED;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reader for {@link Lucene49NormsFormat}
|
* Reader for 4.9 norms
|
||||||
|
* @deprecated only for reading 4.9/4.10 indexes
|
||||||
*/
|
*/
|
||||||
class Lucene49NormsProducer extends NormsProducer {
|
@Deprecated
|
||||||
|
final class Lucene49NormsProducer extends NormsProducer {
|
||||||
|
static final byte DELTA_COMPRESSED = 0;
|
||||||
|
static final byte TABLE_COMPRESSED = 1;
|
||||||
|
static final byte CONST_COMPRESSED = 2;
|
||||||
|
static final byte UNCOMPRESSED = 3;
|
||||||
|
static final int BLOCK_SIZE = 16384;
|
||||||
|
|
||||||
// metadata maps (just file pointers and minimal stuff)
|
// metadata maps (just file pointers and minimal stuff)
|
||||||
private final Map<String,NormsEntry> norms = new HashMap<>();
|
private final Map<String,NormsEntry> norms = new HashMap<>();
|
||||||
private final IndexInput data;
|
private final IndexInput data;
|
||||||
|
@ -62,9 +66,25 @@ class Lucene49NormsProducer extends NormsProducer {
|
||||||
private final int maxDoc;
|
private final int maxDoc;
|
||||||
private final AtomicLong ramBytesUsed;
|
private final AtomicLong ramBytesUsed;
|
||||||
private final AtomicInteger activeCount = new AtomicInteger();
|
private final AtomicInteger activeCount = new AtomicInteger();
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
Lucene49NormsProducer(Lucene49NormsProducer original) {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
norms.putAll(original.norms);
|
||||||
|
data = original.data.clone();
|
||||||
|
instances.putAll(original.instances);
|
||||||
|
instancesInfo.putAll(original.instancesInfo);
|
||||||
|
maxDoc = original.maxDoc;
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
activeCount.set(original.activeCount.get());
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
Lucene49NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
Lucene49NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
maxDoc = state.segmentInfo.getDocCount();
|
maxDoc = state.segmentInfo.getDocCount();
|
||||||
|
merging = false;
|
||||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
|
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
|
||||||
int version = -1;
|
int version = -1;
|
||||||
|
@ -136,8 +156,10 @@ class Lucene49NormsProducer extends NormsProducer {
|
||||||
NumericDocValues instance = instances.get(field.name);
|
NumericDocValues instance = instances.get(field.name);
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
instance = loadNorms(field);
|
instance = loadNorms(field);
|
||||||
instances.put(field.name, instance);
|
if (!merging) {
|
||||||
activeCount.incrementAndGet();
|
instances.put(field.name, instance);
|
||||||
|
activeCount.incrementAndGet();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -161,8 +183,10 @@ class Lucene49NormsProducer extends NormsProducer {
|
||||||
NormsEntry entry = norms.get(field.name);
|
NormsEntry entry = norms.get(field.name);
|
||||||
switch(entry.format) {
|
switch(entry.format) {
|
||||||
case CONST_COMPRESSED:
|
case CONST_COMPRESSED:
|
||||||
instancesInfo.put(field.name, Accountables.namedAccountable("constant", 8));
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(8);
|
instancesInfo.put(field.name, Accountables.namedAccountable("constant", 8));
|
||||||
|
ramBytesUsed.addAndGet(8);
|
||||||
|
}
|
||||||
final long v = entry.offset;
|
final long v = entry.offset;
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -174,8 +198,10 @@ class Lucene49NormsProducer extends NormsProducer {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
final byte bytes[] = new byte[maxDoc];
|
final byte bytes[] = new byte[maxDoc];
|
||||||
data.readBytes(bytes, 0, bytes.length);
|
data.readBytes(bytes, 0, bytes.length);
|
||||||
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
|
if (!merging) {
|
||||||
instancesInfo.put(field.name, Accountables.namedAccountable("byte array", maxDoc));
|
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
|
||||||
|
instancesInfo.put(field.name, Accountables.namedAccountable("byte array", maxDoc));
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -187,8 +213,10 @@ class Lucene49NormsProducer extends NormsProducer {
|
||||||
int packedIntsVersion = data.readVInt();
|
int packedIntsVersion = data.readVInt();
|
||||||
int blockSize = data.readVInt();
|
int blockSize = data.readVInt();
|
||||||
final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, maxDoc, false);
|
final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, maxDoc, false);
|
||||||
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
if (!merging) {
|
||||||
instancesInfo.put(field.name, Accountables.namedAccountable("delta compressed", reader));
|
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
||||||
|
instancesInfo.put(field.name, Accountables.namedAccountable("delta compressed", reader));
|
||||||
|
}
|
||||||
return reader;
|
return reader;
|
||||||
case TABLE_COMPRESSED:
|
case TABLE_COMPRESSED:
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
|
@ -204,8 +232,10 @@ class Lucene49NormsProducer extends NormsProducer {
|
||||||
final int formatID = data.readVInt();
|
final int formatID = data.readVInt();
|
||||||
final int bitsPerValue = data.readVInt();
|
final int bitsPerValue = data.readVInt();
|
||||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedVersion, maxDoc, bitsPerValue);
|
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedVersion, maxDoc, bitsPerValue);
|
||||||
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
|
if (!merging) {
|
||||||
instancesInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader));
|
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
|
||||||
|
instancesInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader));
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -222,6 +252,11 @@ class Lucene49NormsProducer extends NormsProducer {
|
||||||
data.close();
|
data.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized NormsProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene49NormsProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
static class NormsEntry {
|
static class NormsEntry {
|
||||||
byte format;
|
byte format;
|
||||||
long offset;
|
long offset;
|
|
@ -17,6 +17,13 @@ package org.apache.lucene.codecs.lucene49;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_CURRENT;
|
||||||
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsProducer.BLOCK_SIZE;
|
||||||
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsProducer.CONST_COMPRESSED;
|
||||||
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsProducer.DELTA_COMPRESSED;
|
||||||
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsProducer.TABLE_COMPRESSED;
|
||||||
|
import static org.apache.lucene.codecs.lucene49.Lucene49NormsProducer.UNCOMPRESSED;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -32,18 +39,12 @@ import org.apache.lucene.util.IOUtils;
|
||||||
import org.apache.lucene.util.packed.BlockPackedWriter;
|
import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||||
import org.apache.lucene.util.packed.PackedInts;
|
import org.apache.lucene.util.packed.PackedInts;
|
||||||
|
|
||||||
import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_CURRENT;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Writer for {@link Lucene49NormsFormat}
|
* Writer for 4.9 norms
|
||||||
|
* @deprecated for test purposes only
|
||||||
*/
|
*/
|
||||||
class Lucene49NormsConsumer extends NormsConsumer {
|
@Deprecated
|
||||||
static final byte DELTA_COMPRESSED = 0;
|
final class Lucene49NormsConsumer extends NormsConsumer {
|
||||||
static final byte TABLE_COMPRESSED = 1;
|
|
||||||
static final byte CONST_COMPRESSED = 2;
|
|
||||||
static final byte UNCOMPRESSED = 3;
|
|
||||||
static final int BLOCK_SIZE = 16384;
|
|
||||||
|
|
||||||
IndexOutput data, meta;
|
IndexOutput data, meta;
|
||||||
final int maxDoc;
|
final int maxDoc;
|
||||||
|
|
|
@ -40,7 +40,7 @@ public final class Lucene49RWCodec extends Lucene49Codec {
|
||||||
return docValues;
|
return docValues;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final NormsFormat norms = new Lucene49NormsFormat();
|
private static final NormsFormat norms = new Lucene49RWNormsFormat();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public NormsFormat normsFormat() {
|
public NormsFormat normsFormat() {
|
||||||
|
|
|
@ -0,0 +1,35 @@
|
||||||
|
package org.apache.lucene.codecs.lucene49;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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.NormsConsumer;
|
||||||
|
import org.apache.lucene.index.SegmentWriteState;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read-Write version of 4.9 norms format for testing
|
||||||
|
* @deprecated for test purposes only
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
public class Lucene49RWNormsFormat extends Lucene49NormsFormat {
|
||||||
|
@Override
|
||||||
|
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||||
|
return new Lucene49NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene49;
|
||||||
|
|
||||||
import org.apache.lucene.codecs.Codec;
|
import org.apache.lucene.codecs.Codec;
|
||||||
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
|
||||||
import org.junit.BeforeClass;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests Lucene49DocValuesFormat
|
* Tests Lucene49DocValuesFormat
|
||||||
|
|
|
@ -19,13 +19,12 @@ package org.apache.lucene.codecs.lucene49;
|
||||||
|
|
||||||
import org.apache.lucene.codecs.Codec;
|
import org.apache.lucene.codecs.Codec;
|
||||||
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
||||||
import org.apache.lucene.util.TestUtil;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests Lucene49NormsFormat
|
* Tests Lucene49NormsFormat
|
||||||
*/
|
*/
|
||||||
public class TestLucene49NormsFormat extends BaseNormsFormatTestCase {
|
public class TestLucene49NormsFormat extends BaseNormsFormatTestCase {
|
||||||
private final Codec codec = TestUtil.getDefaultCodec();
|
private final Codec codec = new Lucene49RWCodec();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Codec getCodec() {
|
protected Codec getCodec() {
|
|
@ -76,6 +76,8 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
private final AtomicLong ramBytesUsed;
|
private final AtomicLong ramBytesUsed;
|
||||||
private final int version;
|
private final int version;
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
static final byte NUMBER = 0;
|
static final byte NUMBER = 0;
|
||||||
static final byte BYTES = 1;
|
static final byte BYTES = 1;
|
||||||
static final byte SORTED = 2;
|
static final byte SORTED = 2;
|
||||||
|
@ -86,9 +88,34 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
|
|
||||||
static final int VERSION_START = 2;
|
static final int VERSION_START = 2;
|
||||||
static final int VERSION_CURRENT = VERSION_START;
|
static final int VERSION_CURRENT = VERSION_START;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
DirectDocValuesProducer(DirectDocValuesProducer original) throws IOException {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
numerics.putAll(original.numerics);
|
||||||
|
binaries.putAll(original.binaries);
|
||||||
|
sorteds.putAll(original.sorteds);
|
||||||
|
sortedSets.putAll(original.sortedSets);
|
||||||
|
sortedNumerics.putAll(original.sortedNumerics);
|
||||||
|
data = original.data.clone();
|
||||||
|
|
||||||
|
numericInstances.putAll(original.numericInstances);
|
||||||
|
binaryInstances.putAll(original.binaryInstances);
|
||||||
|
sortedInstances.putAll(original.sortedInstances);
|
||||||
|
sortedSetInstances.putAll(original.sortedSetInstances);
|
||||||
|
sortedNumericInstances.putAll(original.sortedNumericInstances);
|
||||||
|
docsWithFieldInstances.putAll(original.docsWithFieldInstances);
|
||||||
|
|
||||||
|
numEntries = original.numEntries;
|
||||||
|
maxDoc = original.maxDoc;
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
version = original.version;
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
DirectDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
DirectDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
maxDoc = state.segmentInfo.getDocCount();
|
maxDoc = state.segmentInfo.getDocCount();
|
||||||
|
merging = false;
|
||||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
// read in the entries from the metadata file.
|
// read in the entries from the metadata file.
|
||||||
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
||||||
|
@ -261,7 +288,10 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
// Lazy load
|
// Lazy load
|
||||||
instance = loadNumeric(numerics.get(field.name));
|
instance = loadNumeric(numerics.get(field.name));
|
||||||
numericInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
numericInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance.numerics;
|
return instance.numerics;
|
||||||
}
|
}
|
||||||
|
@ -275,7 +305,6 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
final byte[] values = new byte[entry.count];
|
final byte[] values = new byte[entry.count];
|
||||||
data.readBytes(values, 0, entry.count);
|
data.readBytes(values, 0, entry.count);
|
||||||
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
ramBytesUsed.addAndGet(ret.bytesUsed);
|
|
||||||
ret.numerics = new NumericDocValues() {
|
ret.numerics = new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int idx) {
|
public long get(int idx) {
|
||||||
|
@ -292,7 +321,6 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
values[i] = data.readShort();
|
values[i] = data.readShort();
|
||||||
}
|
}
|
||||||
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
ramBytesUsed.addAndGet(ret.bytesUsed);
|
|
||||||
ret.numerics = new NumericDocValues() {
|
ret.numerics = new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int idx) {
|
public long get(int idx) {
|
||||||
|
@ -309,7 +337,6 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
values[i] = data.readInt();
|
values[i] = data.readInt();
|
||||||
}
|
}
|
||||||
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
ramBytesUsed.addAndGet(ret.bytesUsed);
|
|
||||||
ret.numerics = new NumericDocValues() {
|
ret.numerics = new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int idx) {
|
public long get(int idx) {
|
||||||
|
@ -326,7 +353,6 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
values[i] = data.readLong();
|
values[i] = data.readLong();
|
||||||
}
|
}
|
||||||
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
ret.bytesUsed = RamUsageEstimator.sizeOf(values);
|
||||||
ramBytesUsed.addAndGet(ret.bytesUsed);
|
|
||||||
ret.numerics = new NumericDocValues() {
|
ret.numerics = new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int idx) {
|
public long get(int idx) {
|
||||||
|
@ -347,7 +373,10 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
// Lazy load
|
// Lazy load
|
||||||
instance = loadBinary(binaries.get(field.name));
|
instance = loadBinary(binaries.get(field.name));
|
||||||
binaryInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
binaryInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
final byte[] bytes = instance.bytes;
|
final byte[] bytes = instance.bytes;
|
||||||
final int[] address = instance.address;
|
final int[] address = instance.address;
|
||||||
|
@ -377,8 +406,6 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
}
|
}
|
||||||
address[entry.count] = data.readInt();
|
address[entry.count] = data.readInt();
|
||||||
|
|
||||||
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes) + RamUsageEstimator.sizeOf(address));
|
|
||||||
|
|
||||||
BinaryRawValues values = new BinaryRawValues();
|
BinaryRawValues values = new BinaryRawValues();
|
||||||
values.bytes = bytes;
|
values.bytes = bytes;
|
||||||
values.address = address;
|
values.address = address;
|
||||||
|
@ -394,7 +421,10 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
// Lazy load
|
// Lazy load
|
||||||
instance = loadSorted(field);
|
instance = loadSorted(field);
|
||||||
sortedInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
sortedInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return newSortedInstance(instance.docToOrd.numerics, getBinary(field), entry.values.count);
|
return newSortedInstance(instance.docToOrd.numerics, getBinary(field), entry.values.count);
|
||||||
|
@ -439,7 +469,10 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
// Lazy load
|
// Lazy load
|
||||||
instance = loadSortedNumeric(entry);
|
instance = loadSortedNumeric(entry);
|
||||||
sortedNumericInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
sortedNumericInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (entry.docToAddress == null) {
|
if (entry.docToAddress == null) {
|
||||||
|
@ -489,7 +522,10 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
// Lazy load
|
// Lazy load
|
||||||
instance = loadSortedSet(entry);
|
instance = loadSortedSet(entry);
|
||||||
sortedSetInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
sortedSetInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (instance.docToOrdAddress == null) {
|
if (instance.docToOrdAddress == null) {
|
||||||
|
@ -573,7 +609,10 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
bits[i] = data.readLong();
|
bits[i] = data.readLong();
|
||||||
}
|
}
|
||||||
instance = new FixedBitSet(bits, maxDoc);
|
instance = new FixedBitSet(bits, maxDoc);
|
||||||
docsWithFieldInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
docsWithFieldInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
|
@ -600,6 +639,11 @@ class DirectDocValuesProducer extends DocValuesProducer {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new DirectDocValuesProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
data.close();
|
data.close();
|
||||||
|
|
|
@ -93,6 +93,8 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
private final AtomicLong ramBytesUsed;
|
private final AtomicLong ramBytesUsed;
|
||||||
private final int version;
|
private final int version;
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
static final byte NUMBER = 0;
|
static final byte NUMBER = 0;
|
||||||
static final byte BYTES = 1;
|
static final byte BYTES = 1;
|
||||||
static final byte FST = 2;
|
static final byte FST = 2;
|
||||||
|
@ -110,9 +112,35 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
|
|
||||||
static final int VERSION_START = 3;
|
static final int VERSION_START = 3;
|
||||||
static final int VERSION_CURRENT = VERSION_START;
|
static final int VERSION_CURRENT = VERSION_START;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
MemoryDocValuesProducer(MemoryDocValuesProducer original) throws IOException {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
numerics.putAll(original.numerics);
|
||||||
|
binaries.putAll(original.binaries);
|
||||||
|
fsts.putAll(original.fsts);
|
||||||
|
sortedSets.putAll(original.sortedSets);
|
||||||
|
sortedNumerics.putAll(original.sortedNumerics);
|
||||||
|
data = original.data.clone();
|
||||||
|
|
||||||
|
numericInstances.putAll(original.numericInstances);
|
||||||
|
pagedBytesInstances.putAll(original.pagedBytesInstances);
|
||||||
|
fstInstances.putAll(original.fstInstances);
|
||||||
|
docsWithFieldInstances.putAll(original.docsWithFieldInstances);
|
||||||
|
addresses.putAll(original.addresses);
|
||||||
|
|
||||||
|
numericInfo.putAll(original.numericInfo);
|
||||||
|
|
||||||
|
numEntries = original.numEntries;
|
||||||
|
maxDoc = original.maxDoc;
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
version = original.version;
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
maxDoc = state.segmentInfo.getDocCount();
|
maxDoc = state.segmentInfo.getDocCount();
|
||||||
|
merging = false;
|
||||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
// read in the entries from the metadata file.
|
// read in the entries from the metadata file.
|
||||||
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
|
||||||
|
@ -257,7 +285,9 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
NumericDocValues instance = numericInstances.get(field.name);
|
NumericDocValues instance = numericInstances.get(field.name);
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
instance = loadNumeric(field);
|
instance = loadNumeric(field);
|
||||||
numericInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
numericInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -283,6 +313,11 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
CodecUtil.checksumEntireFile(data);
|
CodecUtil.checksumEntireFile(data);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new MemoryDocValuesProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(entries=" + numEntries + ")";
|
return getClass().getSimpleName() + "(entries=" + numEntries + ")";
|
||||||
|
@ -304,8 +339,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
final int formatID = data.readVInt();
|
final int formatID = data.readVInt();
|
||||||
final int bitsPerValue = data.readVInt();
|
final int bitsPerValue = data.readVInt();
|
||||||
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, (int)entry.count, bitsPerValue);
|
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, (int)entry.count, bitsPerValue);
|
||||||
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
|
if (!merging) {
|
||||||
numericInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader));
|
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
|
||||||
|
numericInfo.put(field.name, Accountables.namedAccountable("table compressed", ordsReader));
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -317,8 +354,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
final int formatIDDelta = data.readVInt();
|
final int formatIDDelta = data.readVInt();
|
||||||
final int bitsPerValueDelta = data.readVInt();
|
final int bitsPerValueDelta = data.readVInt();
|
||||||
final PackedInts.Reader deltaReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDDelta), entry.packedIntsVersion, (int)entry.count, bitsPerValueDelta);
|
final PackedInts.Reader deltaReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDDelta), entry.packedIntsVersion, (int)entry.count, bitsPerValueDelta);
|
||||||
ramBytesUsed.addAndGet(deltaReader.ramBytesUsed());
|
if (!merging) {
|
||||||
numericInfo.put(field.name, Accountables.namedAccountable("delta compressed", deltaReader));
|
ramBytesUsed.addAndGet(deltaReader.ramBytesUsed());
|
||||||
|
numericInfo.put(field.name, Accountables.namedAccountable("delta compressed", deltaReader));
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -328,8 +367,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
case BLOCK_COMPRESSED:
|
case BLOCK_COMPRESSED:
|
||||||
final int blockSize = data.readVInt();
|
final int blockSize = data.readVInt();
|
||||||
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, entry.count, false);
|
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, entry.count, false);
|
||||||
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
if (!merging) {
|
||||||
numericInfo.put(field.name, Accountables.namedAccountable("block compressed", reader));
|
ramBytesUsed.addAndGet(reader.ramBytesUsed());
|
||||||
|
numericInfo.put(field.name, Accountables.namedAccountable("block compressed", reader));
|
||||||
|
}
|
||||||
return reader;
|
return reader;
|
||||||
case GCD_COMPRESSED:
|
case GCD_COMPRESSED:
|
||||||
final long min = data.readLong();
|
final long min = data.readLong();
|
||||||
|
@ -337,8 +378,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
final int formatIDGCD = data.readVInt();
|
final int formatIDGCD = data.readVInt();
|
||||||
final int bitsPerValueGCD = data.readVInt();
|
final int bitsPerValueGCD = data.readVInt();
|
||||||
final PackedInts.Reader quotientReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDGCD), entry.packedIntsVersion, (int)entry.count, bitsPerValueGCD);
|
final PackedInts.Reader quotientReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDGCD), entry.packedIntsVersion, (int)entry.count, bitsPerValueGCD);
|
||||||
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
|
if (!merging) {
|
||||||
numericInfo.put(field.name, Accountables.namedAccountable("gcd compressed", quotientReader));
|
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
|
||||||
|
numericInfo.put(field.name, Accountables.namedAccountable("gcd compressed", quotientReader));
|
||||||
|
}
|
||||||
return new NumericDocValues() {
|
return new NumericDocValues() {
|
||||||
@Override
|
@Override
|
||||||
public long get(int docID) {
|
public long get(int docID) {
|
||||||
|
@ -359,7 +402,9 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
instance = pagedBytesInstances.get(field.name);
|
instance = pagedBytesInstances.get(field.name);
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
instance = loadBinary(field);
|
instance = loadBinary(field);
|
||||||
pagedBytesInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
pagedBytesInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final PagedBytes.Reader bytesReader = instance.reader;
|
final PagedBytes.Reader bytesReader = instance.reader;
|
||||||
|
@ -399,11 +444,15 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
PagedBytes bytes = new PagedBytes(16);
|
PagedBytes bytes = new PagedBytes(16);
|
||||||
bytes.copy(data, entry.numBytes);
|
bytes.copy(data, entry.numBytes);
|
||||||
bytesAndAddresses.reader = bytes.freeze(true);
|
bytesAndAddresses.reader = bytes.freeze(true);
|
||||||
ramBytesUsed.addAndGet(bytesAndAddresses.reader.ramBytesUsed());
|
if (!merging) {
|
||||||
|
ramBytesUsed.addAndGet(bytesAndAddresses.reader.ramBytesUsed());
|
||||||
|
}
|
||||||
if (entry.minLength != entry.maxLength) {
|
if (entry.minLength != entry.maxLength) {
|
||||||
data.seek(data.getFilePointer() + entry.missingBytes);
|
data.seek(data.getFilePointer() + entry.missingBytes);
|
||||||
bytesAndAddresses.addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
|
bytesAndAddresses.addresses = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
|
||||||
ramBytesUsed.addAndGet(bytesAndAddresses.addresses.ramBytesUsed());
|
if (!merging) {
|
||||||
|
ramBytesUsed.addAndGet(bytesAndAddresses.addresses.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return bytesAndAddresses;
|
return bytesAndAddresses;
|
||||||
}
|
}
|
||||||
|
@ -420,8 +469,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
||||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
if (!merging) {
|
||||||
fstInstances.put(field.name, instance);
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
fstInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final NumericDocValues docToOrd = getNumeric(field);
|
final NumericDocValues docToOrd = getNumeric(field);
|
||||||
|
@ -498,7 +549,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
if (res == null) {
|
if (res == null) {
|
||||||
data.seek(entry.addressOffset);
|
data.seek(entry.addressOffset);
|
||||||
res = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.valueCount, false);
|
res = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.valueCount, false);
|
||||||
addresses.put(field.name, res);
|
if (!merging) {
|
||||||
|
addresses.put(field.name, res);
|
||||||
|
ramBytesUsed.addAndGet(res.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
addr = res;
|
addr = res;
|
||||||
}
|
}
|
||||||
|
@ -567,8 +621,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
instance = new FST<>(data, PositiveIntOutputs.getSingleton());
|
||||||
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
if (!merging) {
|
||||||
fstInstances.put(field.name, instance);
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
fstInstances.put(field.name, instance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final BinaryDocValues docToOrds = getBinary(field);
|
final BinaryDocValues docToOrds = getBinary(field);
|
||||||
|
@ -659,7 +715,10 @@ class MemoryDocValuesProducer extends DocValuesProducer {
|
||||||
bits[i] = data.readLong();
|
bits[i] = data.readLong();
|
||||||
}
|
}
|
||||||
instance = new FixedBitSet(bits, maxDoc);
|
instance = new FixedBitSet(bits, maxDoc);
|
||||||
docsWithFieldInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
|
docsWithFieldInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
|
|
|
@ -92,10 +92,33 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
|
||||||
private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
|
private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
|
||||||
private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
|
private final Map<String,ReverseTermsIndex> reverseIndexInstances = new HashMap<>();
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
Lucene410DocValuesProducer(Lucene410DocValuesProducer original) throws IOException {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
numerics.putAll(original.numerics);
|
||||||
|
binaries.putAll(original.binaries);
|
||||||
|
sortedSets.putAll(original.sortedSets);
|
||||||
|
sortedNumerics.putAll(original.sortedNumerics);
|
||||||
|
ords.putAll(original.ords);
|
||||||
|
ordIndexes.putAll(original.ordIndexes);
|
||||||
|
numFields = original.numFields;
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
data = original.data.clone();
|
||||||
|
maxDoc = original.maxDoc;
|
||||||
|
|
||||||
|
addressInstances.putAll(original.addressInstances);
|
||||||
|
ordIndexInstances.putAll(original.ordIndexInstances);
|
||||||
|
reverseIndexInstances.putAll(original.reverseIndexInstances);
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
/** expert: instantiates a new reader */
|
/** expert: instantiates a new reader */
|
||||||
Lucene410DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
Lucene410DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
this.maxDoc = state.segmentInfo.getDocCount();
|
this.maxDoc = state.segmentInfo.getDocCount();
|
||||||
|
merging = false;
|
||||||
|
|
||||||
int version = -1;
|
int version = -1;
|
||||||
int numFields = -1;
|
int numFields = -1;
|
||||||
|
@ -441,8 +464,10 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
|
||||||
if (addresses == null) {
|
if (addresses == null) {
|
||||||
data.seek(bytes.addressesOffset);
|
data.seek(bytes.addressesOffset);
|
||||||
addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
|
addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
|
||||||
addressInstances.put(field.name, addresses);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
addressInstances.put(field.name, addresses);
|
||||||
|
ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return addresses;
|
return addresses;
|
||||||
}
|
}
|
||||||
|
@ -479,8 +504,10 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
|
||||||
data.seek(bytes.addressesOffset);
|
data.seek(bytes.addressesOffset);
|
||||||
final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
|
final long size = (bytes.count + INTERVAL_MASK) >>> INTERVAL_SHIFT;
|
||||||
addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
addresses = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
|
||||||
addressInstances.put(field.name, addresses);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
addressInstances.put(field.name, addresses);
|
||||||
|
ramBytesUsed.addAndGet(addresses.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return addresses;
|
return addresses;
|
||||||
}
|
}
|
||||||
|
@ -497,8 +524,10 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
|
||||||
PagedBytes pagedBytes = new PagedBytes(15);
|
PagedBytes pagedBytes = new PagedBytes(15);
|
||||||
pagedBytes.copy(data, dataSize);
|
pagedBytes.copy(data, dataSize);
|
||||||
index.terms = pagedBytes.freeze(true);
|
index.terms = pagedBytes.freeze(true);
|
||||||
reverseIndexInstances.put(field.name, index);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(index.ramBytesUsed());
|
reverseIndexInstances.put(field.name, index);
|
||||||
|
ramBytesUsed.addAndGet(index.ramBytesUsed());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return index;
|
return index;
|
||||||
}
|
}
|
||||||
|
@ -560,8 +589,10 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
data.seek(entry.offset);
|
data.seek(entry.offset);
|
||||||
instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
|
instance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
|
||||||
ordIndexInstances.put(field.name, instance);
|
if (!merging) {
|
||||||
ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
ordIndexInstances.put(field.name, instance);
|
||||||
|
ramBytesUsed.addAndGet(instance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return instance;
|
return instance;
|
||||||
}
|
}
|
||||||
|
@ -725,6 +756,11 @@ class Lucene410DocValuesProducer extends DocValuesProducer implements Closeable
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene410DocValuesProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
data.close();
|
data.close();
|
||||||
|
|
|
@ -1,25 +0,0 @@
|
||||||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
|
||||||
<!--
|
|
||||||
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.
|
|
||||||
-->
|
|
||||||
<html>
|
|
||||||
<head>
|
|
||||||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
|
||||||
</head>
|
|
||||||
<body>
|
|
||||||
Lucene 4.9 file format.
|
|
||||||
</body>
|
|
||||||
</html>
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.codecs.PostingsFormat;
|
||||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||||
import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
|
|
||||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||||
|
|
||||||
|
@ -123,7 +122,7 @@ public class Lucene50Codec extends Codec {
|
||||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||||
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
|
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
|
||||||
|
|
||||||
private final NormsFormat normsFormat = new Lucene49NormsFormat();
|
private final NormsFormat normsFormat = new Lucene50NormsFormat();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public final NormsFormat normsFormat() {
|
public final NormsFormat normsFormat() {
|
||||||
|
|
|
@ -0,0 +1,319 @@
|
||||||
|
package org.apache.lucene.codecs.lucene50;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.lucene.codecs.CodecUtil;
|
||||||
|
import org.apache.lucene.codecs.NormsConsumer;
|
||||||
|
import org.apache.lucene.index.FieldInfo;
|
||||||
|
import org.apache.lucene.index.IndexFileNames;
|
||||||
|
import org.apache.lucene.index.SegmentWriteState;
|
||||||
|
import org.apache.lucene.store.IndexOutput;
|
||||||
|
import org.apache.lucene.util.FilterIterator;
|
||||||
|
import org.apache.lucene.util.IOUtils;
|
||||||
|
import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||||
|
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
|
||||||
|
import org.apache.lucene.util.packed.PackedInts;
|
||||||
|
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
|
||||||
|
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Writer for {@link Lucene50NormsFormat}
|
||||||
|
*/
|
||||||
|
class Lucene50NormsConsumer extends NormsConsumer {
|
||||||
|
static final byte DELTA_COMPRESSED = 0;
|
||||||
|
static final byte TABLE_COMPRESSED = 1;
|
||||||
|
static final byte CONST_COMPRESSED = 2;
|
||||||
|
static final byte UNCOMPRESSED = 3;
|
||||||
|
static final byte INDIRECT = 4;
|
||||||
|
static final int BLOCK_SIZE = 1 << 14;
|
||||||
|
|
||||||
|
// threshold for indirect encoding, computed as 1 - 1/log2(maxint)
|
||||||
|
// norms are only read for matching postings... so this is the threshold
|
||||||
|
// where n log n operations < maxdoc (e.g. it performs similar to other fields)
|
||||||
|
static final float INDIRECT_THRESHOLD = 1 - 1 / 31F;
|
||||||
|
|
||||||
|
IndexOutput data, meta;
|
||||||
|
|
||||||
|
Lucene50NormsConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
|
boolean success = false;
|
||||||
|
try {
|
||||||
|
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||||
|
data = state.directory.createOutput(dataName, state.context);
|
||||||
|
CodecUtil.writeSegmentHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId());
|
||||||
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
|
meta = state.directory.createOutput(metaName, state.context);
|
||||||
|
CodecUtil.writeSegmentHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId());
|
||||||
|
success = true;
|
||||||
|
} finally {
|
||||||
|
if (!success) {
|
||||||
|
IOUtils.closeWhileHandlingException(this);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// we explicitly use only certain bits per value and a specified format, so we statically check this will work
|
||||||
|
static {
|
||||||
|
assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(1);
|
||||||
|
assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(2);
|
||||||
|
assert PackedInts.Format.PACKED_SINGLE_BLOCK.isSupported(4);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
|
||||||
|
meta.writeVInt(field.number);
|
||||||
|
long minValue = Long.MAX_VALUE;
|
||||||
|
long maxValue = Long.MIN_VALUE;
|
||||||
|
// TODO: more efficient?
|
||||||
|
NormMap uniqueValues = new NormMap();
|
||||||
|
|
||||||
|
int count = 0;
|
||||||
|
int missingCount = 0;
|
||||||
|
|
||||||
|
for (Number nv : values) {
|
||||||
|
if (nv == null) {
|
||||||
|
throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
|
||||||
|
}
|
||||||
|
final long v = nv.longValue();
|
||||||
|
if (v == 0) {
|
||||||
|
missingCount++;
|
||||||
|
}
|
||||||
|
|
||||||
|
minValue = Math.min(minValue, v);
|
||||||
|
maxValue = Math.max(maxValue, v);
|
||||||
|
|
||||||
|
if (uniqueValues != null) {
|
||||||
|
if (uniqueValues.add(v)) {
|
||||||
|
if (uniqueValues.size > 256) {
|
||||||
|
uniqueValues = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
count++;
|
||||||
|
}
|
||||||
|
if (uniqueValues != null && uniqueValues.size == 1) {
|
||||||
|
// 0 bpv
|
||||||
|
addConstant(minValue);
|
||||||
|
} else if (count > 256 && missingCount > count * INDIRECT_THRESHOLD) {
|
||||||
|
// sparse encoding
|
||||||
|
addIndirect(field, values, count, missingCount);
|
||||||
|
} else if (uniqueValues != null) {
|
||||||
|
// small number of unique values: this is the typical case:
|
||||||
|
FormatAndBits compression = fastestFormatAndBits(uniqueValues.size-1);
|
||||||
|
|
||||||
|
if (compression.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
|
||||||
|
addUncompressed(values, count);
|
||||||
|
} else {
|
||||||
|
addTableCompressed(values, compression, count, uniqueValues);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
addDeltaCompressed(values, count);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private FormatAndBits fastestFormatAndBits(int max) {
|
||||||
|
// we only use bpv=1,2,4,8
|
||||||
|
PackedInts.Format format = PackedInts.Format.PACKED_SINGLE_BLOCK;
|
||||||
|
int bitsPerValue = PackedInts.bitsRequired(max);
|
||||||
|
if (bitsPerValue == 3) {
|
||||||
|
bitsPerValue = 4;
|
||||||
|
} else if (bitsPerValue > 4) {
|
||||||
|
bitsPerValue = 8;
|
||||||
|
}
|
||||||
|
return new FormatAndBits(format, bitsPerValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addConstant(long constant) throws IOException {
|
||||||
|
meta.writeVInt(0);
|
||||||
|
meta.writeByte(CONST_COMPRESSED);
|
||||||
|
meta.writeLong(constant);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addUncompressed(Iterable<Number> values, int count) throws IOException {
|
||||||
|
meta.writeVInt(count);
|
||||||
|
meta.writeByte(UNCOMPRESSED); // uncompressed byte[]
|
||||||
|
meta.writeLong(data.getFilePointer());
|
||||||
|
for (Number nv : values) {
|
||||||
|
data.writeByte((byte) nv.longValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addTableCompressed(Iterable<Number> values, FormatAndBits compression, int count, NormMap uniqueValues) throws IOException {
|
||||||
|
meta.writeVInt(count);
|
||||||
|
meta.writeByte(TABLE_COMPRESSED); // table-compressed
|
||||||
|
meta.writeLong(data.getFilePointer());
|
||||||
|
data.writeVInt(PackedInts.VERSION_CURRENT);
|
||||||
|
|
||||||
|
long[] decode = uniqueValues.getDecodeTable();
|
||||||
|
// upgrade to power of two sized array
|
||||||
|
int size = 1 << compression.bitsPerValue;
|
||||||
|
data.writeVInt(size);
|
||||||
|
for (int i = 0; i < decode.length; i++) {
|
||||||
|
data.writeLong(decode[i]);
|
||||||
|
}
|
||||||
|
for (int i = decode.length; i < size; i++) {
|
||||||
|
data.writeLong(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
data.writeVInt(compression.format.getId());
|
||||||
|
data.writeVInt(compression.bitsPerValue);
|
||||||
|
|
||||||
|
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, compression.format, count, compression.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
|
||||||
|
for(Number nv : values) {
|
||||||
|
writer.add(uniqueValues.getOrd(nv.longValue()));
|
||||||
|
}
|
||||||
|
writer.finish();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addDeltaCompressed(Iterable<Number> values, int count) throws IOException {
|
||||||
|
meta.writeVInt(count);
|
||||||
|
meta.writeByte(DELTA_COMPRESSED); // delta-compressed
|
||||||
|
meta.writeLong(data.getFilePointer());
|
||||||
|
data.writeVInt(PackedInts.VERSION_CURRENT);
|
||||||
|
data.writeVInt(BLOCK_SIZE);
|
||||||
|
|
||||||
|
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
|
||||||
|
for (Number nv : values) {
|
||||||
|
writer.add(nv.longValue());
|
||||||
|
}
|
||||||
|
writer.finish();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, int missingCount) throws IOException {
|
||||||
|
meta.writeVInt(count - missingCount);
|
||||||
|
meta.writeByte(INDIRECT);
|
||||||
|
meta.writeLong(data.getFilePointer());
|
||||||
|
data.writeVInt(PackedInts.VERSION_CURRENT);
|
||||||
|
data.writeVInt(BLOCK_SIZE);
|
||||||
|
|
||||||
|
// write docs with value
|
||||||
|
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
|
||||||
|
int doc = 0;
|
||||||
|
for (Number n : values) {
|
||||||
|
long v = n.longValue();
|
||||||
|
if (v != 0) {
|
||||||
|
writer.add(doc);
|
||||||
|
}
|
||||||
|
doc++;
|
||||||
|
}
|
||||||
|
writer.finish();
|
||||||
|
|
||||||
|
// write actual values
|
||||||
|
addNormsField(field, new Iterable<Number>() {
|
||||||
|
@Override
|
||||||
|
public Iterator<Number> iterator() {
|
||||||
|
return new FilterIterator<Number,Number>(values.iterator()) {
|
||||||
|
@Override
|
||||||
|
protected boolean predicateFunction(Number value) {
|
||||||
|
return value.longValue() != 0;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
boolean success = false;
|
||||||
|
try {
|
||||||
|
if (meta != null) {
|
||||||
|
meta.writeVInt(-1); // write EOF marker
|
||||||
|
CodecUtil.writeFooter(meta); // write checksum
|
||||||
|
}
|
||||||
|
if (data != null) {
|
||||||
|
CodecUtil.writeFooter(data); // write checksum
|
||||||
|
}
|
||||||
|
success = true;
|
||||||
|
} finally {
|
||||||
|
if (success) {
|
||||||
|
IOUtils.close(data, meta);
|
||||||
|
} else {
|
||||||
|
IOUtils.closeWhileHandlingException(data, meta);
|
||||||
|
}
|
||||||
|
meta = data = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// specialized deduplication of long->ord for norms: 99.99999% of the time this will be a single-byte range.
|
||||||
|
static class NormMap {
|
||||||
|
// we use short: at most we will add 257 values to this map before its rejected as too big above.
|
||||||
|
final short[] singleByteRange = new short[256];
|
||||||
|
final Map<Long,Short> other = new HashMap<Long,Short>();
|
||||||
|
int size;
|
||||||
|
|
||||||
|
{
|
||||||
|
Arrays.fill(singleByteRange, (short)-1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** adds an item to the mapping. returns true if actually added */
|
||||||
|
public boolean add(long l) {
|
||||||
|
assert size <= 256; // once we add > 256 values, we nullify the map in addNumericField and don't use this strategy
|
||||||
|
if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) {
|
||||||
|
int index = (int) (l + 128);
|
||||||
|
short previous = singleByteRange[index];
|
||||||
|
if (previous < 0) {
|
||||||
|
singleByteRange[index] = (short) size;
|
||||||
|
size++;
|
||||||
|
return true;
|
||||||
|
} else {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (!other.containsKey(l)) {
|
||||||
|
other.put(l, (short)size);
|
||||||
|
size++;
|
||||||
|
return true;
|
||||||
|
} else {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** gets the ordinal for a previously added item */
|
||||||
|
public int getOrd(long l) {
|
||||||
|
if (l >= Byte.MIN_VALUE && l <= Byte.MAX_VALUE) {
|
||||||
|
int index = (int) (l + 128);
|
||||||
|
return singleByteRange[index];
|
||||||
|
} else {
|
||||||
|
// NPE if something is screwed up
|
||||||
|
return other.get(l);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** retrieves the ordinal table for previously added items */
|
||||||
|
public long[] getDecodeTable() {
|
||||||
|
long decode[] = new long[size];
|
||||||
|
for (int i = 0; i < singleByteRange.length; i++) {
|
||||||
|
short s = singleByteRange[i];
|
||||||
|
if (s >= 0) {
|
||||||
|
decode[s] = i - 128;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for (Map.Entry<Long,Short> entry : other.entrySet()) {
|
||||||
|
decode[entry.getValue()] = entry.getKey();
|
||||||
|
}
|
||||||
|
return decode;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,4 +1,4 @@
|
||||||
package org.apache.lucene.codecs.lucene49;
|
package org.apache.lucene.codecs.lucene50;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
@ -31,7 +31,7 @@ import org.apache.lucene.util.packed.BlockPackedWriter;
|
||||||
import org.apache.lucene.util.packed.PackedInts;
|
import org.apache.lucene.util.packed.PackedInts;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Lucene 4.9 Score normalization format.
|
* Lucene 5.0 Score normalization format.
|
||||||
* <p>
|
* <p>
|
||||||
* Encodes normalization values with these strategies:
|
* Encodes normalization values with these strategies:
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -47,6 +47,9 @@ import org.apache.lucene.util.packed.PackedInts;
|
||||||
* <li>Delta-compressed: per-document integers written as deltas from the minimum value,
|
* <li>Delta-compressed: per-document integers written as deltas from the minimum value,
|
||||||
* compressed with bitpacking. For more information, see {@link BlockPackedWriter}.
|
* compressed with bitpacking. For more information, see {@link BlockPackedWriter}.
|
||||||
* This is only used when norms of larger than one byte are present.
|
* This is only used when norms of larger than one byte are present.
|
||||||
|
* <li>Indirect: when norms are extremely sparse, missing values are omitted.
|
||||||
|
* Access to an individual value is slower, but missing norm values are never accessed
|
||||||
|
* by search code.
|
||||||
* </ul>
|
* </ul>
|
||||||
* <p>
|
* <p>
|
||||||
* Files:
|
* Files:
|
||||||
|
@ -78,6 +81,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
||||||
* a lookup table of unique values is written, followed by the ordinal for each document.
|
* a lookup table of unique values is written, followed by the ordinal for each document.
|
||||||
* <li>2 --> constant. When there is a single value for the entire field.
|
* <li>2 --> constant. When there is a single value for the entire field.
|
||||||
* <li>3 --> uncompressed: Values written as a simple byte[].
|
* <li>3 --> uncompressed: Values written as a simple byte[].
|
||||||
|
* <li>4 --> indirect. Only documents with a value are written with a sparse encoding.
|
||||||
* </ul>
|
* </ul>
|
||||||
* <li><a name="nvd" id="nvd"></a>
|
* <li><a name="nvd" id="nvd"></a>
|
||||||
* <p>The Norms data or .nvd file.</p>
|
* <p>The Norms data or .nvd file.</p>
|
||||||
|
@ -97,24 +101,24 @@ import org.apache.lucene.util.packed.PackedInts;
|
||||||
* </ol>
|
* </ol>
|
||||||
* @lucene.experimental
|
* @lucene.experimental
|
||||||
*/
|
*/
|
||||||
public class Lucene49NormsFormat extends NormsFormat {
|
public class Lucene50NormsFormat extends NormsFormat {
|
||||||
|
|
||||||
/** Sole Constructor */
|
/** Sole Constructor */
|
||||||
public Lucene49NormsFormat() {}
|
public Lucene50NormsFormat() {}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
|
||||||
return new Lucene49NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
return new Lucene50NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
|
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
|
||||||
return new Lucene49NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
return new Lucene50NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final String DATA_CODEC = "Lucene49NormsData";
|
private static final String DATA_CODEC = "Lucene50NormsData";
|
||||||
private static final String DATA_EXTENSION = "nvd";
|
private static final String DATA_EXTENSION = "nvd";
|
||||||
private static final String METADATA_CODEC = "Lucene49NormsMetadata";
|
private static final String METADATA_CODEC = "Lucene50NormsMetadata";
|
||||||
private static final String METADATA_EXTENSION = "nvm";
|
private static final String METADATA_EXTENSION = "nvm";
|
||||||
static final int VERSION_START = 0;
|
static final int VERSION_START = 0;
|
||||||
static final int VERSION_CURRENT = VERSION_START;
|
static final int VERSION_CURRENT = VERSION_START;
|
|
@ -0,0 +1,320 @@
|
||||||
|
package org.apache.lucene.codecs.lucene50;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
import org.apache.lucene.codecs.CodecUtil;
|
||||||
|
import org.apache.lucene.codecs.NormsProducer;
|
||||||
|
import org.apache.lucene.index.CorruptIndexException;
|
||||||
|
import org.apache.lucene.index.FieldInfo;
|
||||||
|
import org.apache.lucene.index.FieldInfos;
|
||||||
|
import org.apache.lucene.index.IndexFileNames;
|
||||||
|
import org.apache.lucene.index.NumericDocValues;
|
||||||
|
import org.apache.lucene.index.SegmentReadState;
|
||||||
|
import org.apache.lucene.store.ChecksumIndexInput;
|
||||||
|
import org.apache.lucene.store.IndexInput;
|
||||||
|
import org.apache.lucene.util.Accountable;
|
||||||
|
import org.apache.lucene.util.Accountables;
|
||||||
|
import org.apache.lucene.util.IOUtils;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
|
import org.apache.lucene.util.packed.BlockPackedReader;
|
||||||
|
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
|
||||||
|
import org.apache.lucene.util.packed.PackedInts;
|
||||||
|
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_START;
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsFormat.VERSION_CURRENT;
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.CONST_COMPRESSED;
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.DELTA_COMPRESSED;
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
|
||||||
|
import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reader for {@link Lucene50NormsFormat}
|
||||||
|
*/
|
||||||
|
class Lucene50NormsProducer extends NormsProducer {
|
||||||
|
// metadata maps (just file pointers and minimal stuff)
|
||||||
|
private final Map<String,NormsEntry> norms = new HashMap<>();
|
||||||
|
private final IndexInput data;
|
||||||
|
|
||||||
|
// ram instances we have already loaded
|
||||||
|
final Map<String,NumericDocValues> instances = new HashMap<>();
|
||||||
|
final Map<String,Accountable> instancesInfo = new HashMap<>();
|
||||||
|
|
||||||
|
private final AtomicLong ramBytesUsed;
|
||||||
|
private final AtomicInteger activeCount = new AtomicInteger();
|
||||||
|
|
||||||
|
private final boolean merging;
|
||||||
|
|
||||||
|
// clone for merge: when merging we don't do any instances.put()s
|
||||||
|
Lucene50NormsProducer(Lucene50NormsProducer original) {
|
||||||
|
assert Thread.holdsLock(original);
|
||||||
|
norms.putAll(original.norms);
|
||||||
|
data = original.data.clone();
|
||||||
|
instances.putAll(original.instances);
|
||||||
|
instancesInfo.putAll(original.instancesInfo);
|
||||||
|
ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
|
||||||
|
activeCount.set(original.activeCount.get());
|
||||||
|
merging = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
|
||||||
|
merging = false;
|
||||||
|
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
|
||||||
|
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
|
||||||
|
int version = -1;
|
||||||
|
|
||||||
|
// read in the entries from the metadata file.
|
||||||
|
try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
|
||||||
|
Throwable priorE = null;
|
||||||
|
try {
|
||||||
|
version = CodecUtil.checkSegmentHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId());
|
||||||
|
readFields(in, state.fieldInfos);
|
||||||
|
} catch (Throwable exception) {
|
||||||
|
priorE = exception;
|
||||||
|
} finally {
|
||||||
|
CodecUtil.checkFooter(in, priorE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
|
||||||
|
this.data = state.directory.openInput(dataName, state.context);
|
||||||
|
boolean success = false;
|
||||||
|
try {
|
||||||
|
final int version2 = CodecUtil.checkSegmentHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId());
|
||||||
|
if (version != version2) {
|
||||||
|
throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data);
|
||||||
|
}
|
||||||
|
|
||||||
|
// NOTE: data file is too costly to verify checksum against all the bytes on open,
|
||||||
|
// but for now we at least verify proper structure of the checksum footer: which looks
|
||||||
|
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
|
||||||
|
// such as file truncation.
|
||||||
|
CodecUtil.retrieveChecksum(data);
|
||||||
|
|
||||||
|
success = true;
|
||||||
|
} finally {
|
||||||
|
if (!success) {
|
||||||
|
IOUtils.closeWhileHandlingException(this.data);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
|
||||||
|
int fieldNumber = meta.readVInt();
|
||||||
|
while (fieldNumber != -1) {
|
||||||
|
FieldInfo info = infos.fieldInfo(fieldNumber);
|
||||||
|
if (info == null) {
|
||||||
|
throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta);
|
||||||
|
} else if (!info.hasNorms()) {
|
||||||
|
throw new CorruptIndexException("Invalid field: " + info.name, meta);
|
||||||
|
}
|
||||||
|
NormsEntry entry = readEntry(info, meta);
|
||||||
|
norms.put(info.name, entry);
|
||||||
|
fieldNumber = meta.readVInt();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private NormsEntry readEntry(FieldInfo info, IndexInput meta) throws IOException {
|
||||||
|
NormsEntry entry = new NormsEntry();
|
||||||
|
entry.count = meta.readVInt();
|
||||||
|
entry.format = meta.readByte();
|
||||||
|
entry.offset = meta.readLong();
|
||||||
|
switch(entry.format) {
|
||||||
|
case CONST_COMPRESSED:
|
||||||
|
case UNCOMPRESSED:
|
||||||
|
case TABLE_COMPRESSED:
|
||||||
|
case DELTA_COMPRESSED:
|
||||||
|
break;
|
||||||
|
case INDIRECT:
|
||||||
|
if (meta.readVInt() != info.number) {
|
||||||
|
throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
|
||||||
|
}
|
||||||
|
entry.nested = readEntry(info, meta);
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new CorruptIndexException("Unknown format: " + entry.format, meta);
|
||||||
|
}
|
||||||
|
return entry;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized NumericDocValues getNorms(FieldInfo field) throws IOException {
|
||||||
|
NumericDocValues instance = instances.get(field.name);
|
||||||
|
if (instance == null) {
|
||||||
|
LoadedNorms loaded = loadNorms(norms.get(field.name));
|
||||||
|
instance = loaded.norms;
|
||||||
|
if (!merging) {
|
||||||
|
instances.put(field.name, instance);
|
||||||
|
activeCount.incrementAndGet();
|
||||||
|
ramBytesUsed.addAndGet(loaded.ramBytesUsed);
|
||||||
|
instancesInfo.put(field.name, loaded.info);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return instance;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long ramBytesUsed() {
|
||||||
|
return ramBytesUsed.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized Iterable<? extends Accountable> getChildResources() {
|
||||||
|
return Accountables.namedAccountables("field", instancesInfo);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void checkIntegrity() throws IOException {
|
||||||
|
CodecUtil.checksumEntireFile(data);
|
||||||
|
}
|
||||||
|
|
||||||
|
private LoadedNorms loadNorms(NormsEntry entry) throws IOException {
|
||||||
|
LoadedNorms instance = new LoadedNorms();
|
||||||
|
switch(entry.format) {
|
||||||
|
case CONST_COMPRESSED: {
|
||||||
|
final long v = entry.offset;
|
||||||
|
instance.info = Accountables.namedAccountable("constant", 8);
|
||||||
|
instance.ramBytesUsed = 8;
|
||||||
|
instance.norms = new NumericDocValues() {
|
||||||
|
@Override
|
||||||
|
public long get(int docID) {
|
||||||
|
return v;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case UNCOMPRESSED: {
|
||||||
|
data.seek(entry.offset);
|
||||||
|
final byte bytes[] = new byte[entry.count];
|
||||||
|
data.readBytes(bytes, 0, bytes.length);
|
||||||
|
instance.info = Accountables.namedAccountable("byte array", bytes.length);
|
||||||
|
instance.ramBytesUsed = RamUsageEstimator.sizeOf(bytes);
|
||||||
|
instance.norms = new NumericDocValues() {
|
||||||
|
@Override
|
||||||
|
public long get(int docID) {
|
||||||
|
return bytes[docID];
|
||||||
|
}
|
||||||
|
};
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case DELTA_COMPRESSED: {
|
||||||
|
data.seek(entry.offset);
|
||||||
|
int packedIntsVersion = data.readVInt();
|
||||||
|
int blockSize = data.readVInt();
|
||||||
|
final BlockPackedReader reader = new BlockPackedReader(data, packedIntsVersion, blockSize, entry.count, false);
|
||||||
|
instance.info = Accountables.namedAccountable("delta compressed", reader);
|
||||||
|
instance.ramBytesUsed = reader.ramBytesUsed();
|
||||||
|
instance.norms = reader;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case TABLE_COMPRESSED: {
|
||||||
|
data.seek(entry.offset);
|
||||||
|
int packedIntsVersion = data.readVInt();
|
||||||
|
int size = data.readVInt();
|
||||||
|
if (size > 256) {
|
||||||
|
throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, got=" + size, data);
|
||||||
|
}
|
||||||
|
final long decode[] = new long[size];
|
||||||
|
for (int i = 0; i < decode.length; i++) {
|
||||||
|
decode[i] = data.readLong();
|
||||||
|
}
|
||||||
|
final int formatID = data.readVInt();
|
||||||
|
final int bitsPerValue = data.readVInt();
|
||||||
|
final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), packedIntsVersion, entry.count, bitsPerValue);
|
||||||
|
instance.info = Accountables.namedAccountable("table compressed", ordsReader);
|
||||||
|
instance.ramBytesUsed = RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed();
|
||||||
|
instance.norms = new NumericDocValues() {
|
||||||
|
@Override
|
||||||
|
public long get(int docID) {
|
||||||
|
return decode[(int)ordsReader.get(docID)];
|
||||||
|
}
|
||||||
|
};
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case INDIRECT: {
|
||||||
|
data.seek(entry.offset);
|
||||||
|
int packedIntsVersion = data.readVInt();
|
||||||
|
int blockSize = data.readVInt();
|
||||||
|
final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
|
||||||
|
LoadedNorms nestedInstance = loadNorms(entry.nested);
|
||||||
|
instance.ramBytesUsed = live.ramBytesUsed() + nestedInstance.ramBytesUsed;
|
||||||
|
instance.info = Accountables.namedAccountable("indirect -> " + nestedInstance.info, instance.ramBytesUsed);
|
||||||
|
final NumericDocValues values = nestedInstance.norms;
|
||||||
|
final int upperBound = entry.count-1;
|
||||||
|
instance.norms = new NumericDocValues() {
|
||||||
|
@Override
|
||||||
|
public long get(int docID) {
|
||||||
|
int low = 0;
|
||||||
|
int high = upperBound;
|
||||||
|
while (low <= high) {
|
||||||
|
int mid = (low + high) >>> 1;
|
||||||
|
long doc = live.get(mid);
|
||||||
|
|
||||||
|
if (doc < docID) {
|
||||||
|
low = mid + 1;
|
||||||
|
} else if (doc > docID) {
|
||||||
|
high = mid - 1;
|
||||||
|
} else {
|
||||||
|
return values.get(mid);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
throw new AssertionError();
|
||||||
|
}
|
||||||
|
return instance;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
data.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
static class NormsEntry {
|
||||||
|
byte format;
|
||||||
|
long offset;
|
||||||
|
int count;
|
||||||
|
NormsEntry nested;
|
||||||
|
}
|
||||||
|
|
||||||
|
static class LoadedNorms {
|
||||||
|
NumericDocValues norms;
|
||||||
|
long ramBytesUsed;
|
||||||
|
Accountable info;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized NormsProducer getMergeInstance() throws IOException {
|
||||||
|
return new Lucene50NormsProducer(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return getClass().getSimpleName() + "(fields=" + norms.size() + ",active=" + activeCount.get() + ")";
|
||||||
|
}
|
||||||
|
}
|
|
@ -225,6 +225,24 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
||||||
|
|
||||||
private final Map<String,DocValuesProducer> fields = new TreeMap<>();
|
private final Map<String,DocValuesProducer> fields = new TreeMap<>();
|
||||||
private final Map<String,DocValuesProducer> formats = new HashMap<>();
|
private final Map<String,DocValuesProducer> formats = new HashMap<>();
|
||||||
|
|
||||||
|
// clone for merge
|
||||||
|
FieldsReader(FieldsReader other) throws IOException {
|
||||||
|
Map<DocValuesProducer,DocValuesProducer> oldToNew = new IdentityHashMap<>();
|
||||||
|
// First clone all formats
|
||||||
|
for(Map.Entry<String,DocValuesProducer> ent : other.formats.entrySet()) {
|
||||||
|
DocValuesProducer values = ent.getValue().getMergeInstance();
|
||||||
|
formats.put(ent.getKey(), values);
|
||||||
|
oldToNew.put(ent.getValue(), values);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Then rebuild fields:
|
||||||
|
for(Map.Entry<String,DocValuesProducer> ent : other.fields.entrySet()) {
|
||||||
|
DocValuesProducer producer = oldToNew.get(ent.getValue());
|
||||||
|
assert producer != null;
|
||||||
|
fields.put(ent.getKey(), producer);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public FieldsReader(final SegmentReadState readState) throws IOException {
|
public FieldsReader(final SegmentReadState readState) throws IOException {
|
||||||
|
|
||||||
|
@ -259,24 +277,6 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private FieldsReader(FieldsReader other) {
|
|
||||||
|
|
||||||
Map<DocValuesProducer,DocValuesProducer> oldToNew = new IdentityHashMap<>();
|
|
||||||
// First clone all formats
|
|
||||||
for(Map.Entry<String,DocValuesProducer> ent : other.formats.entrySet()) {
|
|
||||||
DocValuesProducer values = ent.getValue();
|
|
||||||
formats.put(ent.getKey(), values);
|
|
||||||
oldToNew.put(ent.getValue(), values);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Then rebuild fields:
|
|
||||||
for(Map.Entry<String,DocValuesProducer> ent : other.fields.entrySet()) {
|
|
||||||
DocValuesProducer producer = oldToNew.get(ent.getValue());
|
|
||||||
assert producer != null;
|
|
||||||
fields.put(ent.getKey(), producer);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
|
||||||
DocValuesProducer producer = fields.get(field.name);
|
DocValuesProducer producer = fields.get(field.name);
|
||||||
|
@ -318,11 +318,6 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
||||||
IOUtils.close(formats.values());
|
IOUtils.close(formats.values());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public DocValuesProducer clone() {
|
|
||||||
return new FieldsReader(this);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long ramBytesUsed() {
|
public long ramBytesUsed() {
|
||||||
long size = 0;
|
long size = 0;
|
||||||
|
@ -345,6 +340,11 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new FieldsReader(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "PerFieldDocValues(formats=" + formats.size() + ")";
|
return "PerFieldDocValues(formats=" + formats.size() + ")";
|
||||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.IdentityHashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -215,6 +216,24 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
||||||
|
|
||||||
private final Map<String,FieldsProducer> fields = new TreeMap<>();
|
private final Map<String,FieldsProducer> fields = new TreeMap<>();
|
||||||
private final Map<String,FieldsProducer> formats = new HashMap<>();
|
private final Map<String,FieldsProducer> formats = new HashMap<>();
|
||||||
|
|
||||||
|
// clone for merge
|
||||||
|
FieldsReader(FieldsReader other) throws IOException {
|
||||||
|
Map<FieldsProducer,FieldsProducer> oldToNew = new IdentityHashMap<>();
|
||||||
|
// First clone all formats
|
||||||
|
for(Map.Entry<String,FieldsProducer> ent : other.formats.entrySet()) {
|
||||||
|
FieldsProducer values = ent.getValue().getMergeInstance();
|
||||||
|
formats.put(ent.getKey(), values);
|
||||||
|
oldToNew.put(ent.getValue(), values);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Then rebuild fields:
|
||||||
|
for(Map.Entry<String,FieldsProducer> ent : other.fields.entrySet()) {
|
||||||
|
FieldsProducer producer = oldToNew.get(ent.getValue());
|
||||||
|
assert producer != null;
|
||||||
|
fields.put(ent.getKey(), producer);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public FieldsReader(final SegmentReadState readState) throws IOException {
|
public FieldsReader(final SegmentReadState readState) throws IOException {
|
||||||
|
|
||||||
|
@ -293,6 +312,11 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FieldsProducer getMergeInstance() throws IOException {
|
||||||
|
return new FieldsReader(this);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "PerFieldPostings(formats=" + formats.size() + ")";
|
return "PerFieldPostings(formats=" + formats.size() + ")";
|
||||||
|
|
|
@ -0,0 +1,34 @@
|
||||||
|
package org.apache.lucene.codecs.lucene50;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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.codecs.Codec;
|
||||||
|
import org.apache.lucene.index.BaseNormsFormatTestCase;
|
||||||
|
import org.apache.lucene.util.TestUtil;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests Lucene49NormsFormat
|
||||||
|
*/
|
||||||
|
public class TestLucene50NormsFormat extends BaseNormsFormatTestCase {
|
||||||
|
private final Codec codec = TestUtil.getDefaultCodec();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Codec getCodec() {
|
||||||
|
return codec;
|
||||||
|
}
|
||||||
|
}
|
|
@ -300,6 +300,11 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
|
||||||
in.checkIntegrity();
|
in.checkIntegrity();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DocValuesProducer getMergeInstance() throws IOException {
|
||||||
|
return new AssertingDocValuesProducer(in.getMergeInstance(), maxDoc);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
||||||
|
|
|
@ -123,6 +123,11 @@ public class AssertingNormsFormat extends NormsFormat {
|
||||||
in.checkIntegrity();
|
in.checkIntegrity();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NormsProducer getMergeInstance() throws IOException {
|
||||||
|
return new AssertingNormsProducer(in.getMergeInstance(), maxDoc);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
||||||
|
|
|
@ -110,6 +110,11 @@ public final class AssertingPostingsFormat extends PostingsFormat {
|
||||||
in.checkIntegrity();
|
in.checkIntegrity();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FieldsProducer getMergeInstance() throws IOException {
|
||||||
|
return new AssertingFieldsProducer(in.getMergeInstance());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
||||||
|
|
|
@ -96,6 +96,11 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
||||||
in.checkIntegrity();
|
in.checkIntegrity();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public StoredFieldsReader getMergeInstance() throws IOException {
|
||||||
|
return new AssertingStoredFieldsReader(in.getMergeInstance(), maxDoc);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
||||||
|
|
|
@ -95,6 +95,11 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
|
||||||
in.checkIntegrity();
|
in.checkIntegrity();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TermVectorsReader getMergeInstance() throws IOException {
|
||||||
|
return new AssertingTermVectorsReader(in.getMergeInstance());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
return getClass().getSimpleName() + "(" + in.toString() + ")";
|
||||||
|
|
|
@ -139,6 +139,19 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testSparse() throws Exception {
|
||||||
|
int iterations = atLeast(1);
|
||||||
|
final Random r = random();
|
||||||
|
for (int i = 0; i < iterations; i++) {
|
||||||
|
doTestNormsVersusStoredFields(new LongProducer() {
|
||||||
|
@Override
|
||||||
|
long next() {
|
||||||
|
return r.nextInt(100) == 0 ? TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE) : 0;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception {
|
private void doTestNormsVersusStoredFields(LongProducer longs) throws Exception {
|
||||||
int numDocs = atLeast(500);
|
int numDocs = atLeast(500);
|
||||||
long norms[] = new long[numDocs];
|
long norms[] = new long[numDocs];
|
||||||
|
|
Loading…
Reference in New Issue