LUCENE-5894: refactor bulk merge logic

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1619392 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2014-08-21 13:46:36 +00:00
parent 8af2ef5345
commit f979dee252
53 changed files with 1483 additions and 1110 deletions

View File

@ -21,7 +21,11 @@ import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -36,12 +40,12 @@ public class SimpleTextNormsFormat extends NormsFormat {
private static final String NORMS_SEG_EXTENSION = "len";
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new SimpleTextNormsConsumer(state);
}
@Override
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
return new SimpleTextNormsProducer(state);
}
@ -52,11 +56,33 @@ public class SimpleTextNormsFormat extends NormsFormat {
*
* @lucene.experimental
*/
public static class SimpleTextNormsProducer extends SimpleTextDocValuesReader {
public static class SimpleTextNormsProducer extends NormsProducer {
private final SimpleTextDocValuesReader impl;
public SimpleTextNormsProducer(SegmentReadState state) throws IOException {
// All we do is change the extension from .dat -> .len;
// otherwise this is a normal simple doc values file:
super(state, NORMS_SEG_EXTENSION);
impl = new SimpleTextDocValuesReader(state, NORMS_SEG_EXTENSION);
}
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
return impl.getNumeric(field);
}
@Override
public void close() throws IOException {
impl.close();
}
@Override
public long ramBytesUsed() {
return impl.ramBytesUsed();
}
@Override
public void checkIntegrity() throws IOException {
impl.checkIntegrity();
}
}
@ -67,11 +93,23 @@ public class SimpleTextNormsFormat extends NormsFormat {
*
* @lucene.experimental
*/
public static class SimpleTextNormsConsumer extends SimpleTextDocValuesWriter {
public static class SimpleTextNormsConsumer extends NormsConsumer {
private final SimpleTextDocValuesWriter impl;
public SimpleTextNormsConsumer(SegmentWriteState state) throws IOException {
// All we do is change the extension from .dat -> .len;
// otherwise this is a normal simple doc values file:
super(state, NORMS_SEG_EXTENSION);
impl = new SimpleTextDocValuesWriter(state, NORMS_SEG_EXTENSION);
}
@Override
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
impl.addNumericField(field, values);
}
@Override
public void close() throws IOException {
impl.close();
}
}
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
@ -28,7 +29,9 @@ import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
@ -51,13 +54,14 @@ import org.apache.lucene.util.packed.PackedInts;
* The lifecycle is:
* <ol>
* <li>DocValuesConsumer is created by
* {@link DocValuesFormat#fieldsConsumer(SegmentWriteState)} or
* {@link NormsFormat#normsConsumer(SegmentWriteState)}.
* <li>{@link #addNumericField}, {@link #addBinaryField},
* or {@link #addSortedField} are called for each Numeric,
* Binary, or Sorted docvalues field. The API is a "pull" rather
* than "push", and the implementation is free to iterate over the
* values multiple times ({@link Iterable#iterator()}).
* {@link #addSortedField}, {@link #addSortedSetField},
* or {@link #addSortedNumericField} are called for each Numeric,
* Binary, Sorted, SortedSet, or SortedNumeric docvalues field.
* The API is a "pull" rather than "push", and the implementation
* is free to iterate over the values multiple times
* ({@link Iterable#iterator()}).
* <li>After all fields are added, the consumer is {@link #close}d.
* </ol>
*
@ -118,6 +122,83 @@ public abstract class DocValuesConsumer implements Closeable {
*/
public abstract void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException;
/** Merges in the fields from the readers in
* <code>mergeState</code>. The default implementation
* calls {@link #mergeNumericField}, {@link #mergeBinaryField},
* {@link #mergeSortedField}, {@link #mergeSortedSetField},
* or {@link #mergeSortedNumericField} for each field,
* depending on its type.
* Implementations can override this method
* for more sophisticated merging (bulk-byte copying, etc). */
public void merge(MergeState mergeState) throws IOException {
for (FieldInfo field : mergeState.fieldInfos) {
DocValuesType type = field.getDocValuesType();
if (type != null) {
if (type == DocValuesType.NUMERIC) {
List<NumericDocValues> toMerge = new ArrayList<>();
List<Bits> docsWithField = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
NumericDocValues values = reader.getNumericDocValues(field.name);
Bits bits = reader.getDocsWithField(field.name);
if (values == null) {
values = DocValues.emptyNumeric();
bits = new Bits.MatchNoBits(reader.maxDoc());
}
toMerge.add(values);
docsWithField.add(bits);
}
mergeNumericField(field, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.BINARY) {
List<BinaryDocValues> toMerge = new ArrayList<>();
List<Bits> docsWithField = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
BinaryDocValues values = reader.getBinaryDocValues(field.name);
Bits bits = reader.getDocsWithField(field.name);
if (values == null) {
values = DocValues.emptyBinary();
bits = new Bits.MatchNoBits(reader.maxDoc());
}
toMerge.add(values);
docsWithField.add(bits);
}
mergeBinaryField(field, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.SORTED) {
List<SortedDocValues> toMerge = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
SortedDocValues values = reader.getSortedDocValues(field.name);
if (values == null) {
values = DocValues.emptySorted();
}
toMerge.add(values);
}
mergeSortedField(field, mergeState, toMerge);
} else if (type == DocValuesType.SORTED_SET) {
List<SortedSetDocValues> toMerge = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
SortedSetDocValues values = reader.getSortedSetDocValues(field.name);
if (values == null) {
values = DocValues.emptySortedSet();
}
toMerge.add(values);
}
mergeSortedSetField(field, mergeState, toMerge);
} else if (type == DocValuesType.SORTED_NUMERIC) {
List<SortedNumericDocValues> toMerge = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
SortedNumericDocValues values = reader.getSortedNumericDocValues(field.name);
if (values == null) {
values = DocValues.emptySortedNumeric(reader.maxDoc());
}
toMerge.add(values);
}
mergeSortedNumericField(field, mergeState, toMerge);
} else {
throw new AssertionError("type=" + type);
}
}
}
}
/**
* Merges the numeric docvalues from <code>toMerge</code>.
* <p>

View File

@ -29,8 +29,8 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
/** Abstract API that produces numeric, binary and
* sorted docvalues.
/** Abstract API that produces numeric, binary, sorted, sortedset,
* and sortednumeric docvalues.
*
* @lucene.experimental
*/

View File

@ -19,10 +19,15 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.FieldInfo; // javadocs
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.SegmentWriteState; // javadocs
import org.apache.lucene.index.MappedMultiFields;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.ReaderSlice;
/**
* Abstract API that consumes terms, doc, freq, prox, offset and
@ -74,6 +79,34 @@ public abstract class FieldsConsumer implements Closeable {
*/
public abstract void write(Fields fields) throws IOException;
/** Merges in the fields from the readers in
* <code>mergeState</code>. The default implementation skips
* and maps around deleted documents, and calls {@link #write(Fields)}.
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public void merge(MergeState mergeState) throws IOException {
final List<Fields> fields = new ArrayList<>();
final List<ReaderSlice> slices = new ArrayList<>();
int docBase = 0;
for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
final AtomicReader reader = mergeState.readers.get(readerIndex);
final Fields f = reader.fields();
final int maxDoc = reader.maxDoc();
if (f != null) {
slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
fields.add(f);
}
docBase += maxDoc;
}
Fields mergedFields = new MappedMultiFields(mergeState,
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderSlice.EMPTY_ARRAY)));
write(mergedFields);
}
// NOTE: strange but necessary so javadocs linting is happy:
@Override
public abstract void close() throws IOException;

View File

@ -0,0 +1,163 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.Bits;
/**
* Abstract API that consumes normalization values.
* Concrete implementations of this
* actually do "something" with the norms (write it into
* the index in a specific format).
* <p>
* The lifecycle is:
* <ol>
* <li>NormsConsumer is created by
* {@link NormsFormat#normsConsumer(SegmentWriteState)}.
* <li>{@link #addNormsField} is called for each field with
* normalization values. The API is a "pull" rather
* than "push", and the implementation is free to iterate over the
* values multiple times ({@link Iterable#iterator()}).
* <li>After all fields are added, the consumer is {@link #close}d.
* </ol>
*
* @lucene.experimental
*/
public abstract class NormsConsumer implements Closeable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected NormsConsumer() {}
/**
* Writes normalization values for a field.
* @param field field information
* @param values Iterable of numeric values (one for each document).
* @throws IOException if an I/O error occurred.
*/
public abstract void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException;
/** Merges in the fields from the readers in
* <code>mergeState</code>. The default implementation
* calls {@link #mergeNormsField} for each field,
* filling segments with missing norms for the field with zeros.
* Implementations can override this method
* for more sophisticated merging (bulk-byte copying, etc). */
public void merge(MergeState mergeState) throws IOException {
for (FieldInfo field : mergeState.fieldInfos) {
if (field.hasNorms()) {
List<NumericDocValues> toMerge = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
NumericDocValues norms = reader.getNormValues(field.name);
if (norms == null) {
norms = DocValues.emptyNumeric();
}
toMerge.add(norms);
}
mergeNormsField(field, mergeState, toMerge);
}
}
}
/**
* Merges the norms from <code>toMerge</code>.
* <p>
* The default implementation calls {@link #addNormsField}, passing
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeNormsField(final FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge) throws IOException {
// TODO: try to share code with default merge of DVConsumer by passing MatchAllBits ?
addNormsField(fieldInfo,
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
AtomicReader currentReader;
NumericDocValues currentValues;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
public boolean hasNext() {
return nextIsSet || setNext();
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public Number next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
assert nextIsSet;
nextIsSet = false;
return nextValue;
}
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentReader = mergeState.readers.get(readerUpto);
currentValues = toMerge.get(readerUpto);
currentLiveDocs = currentReader.getLiveDocs();
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
nextValue = currentValues.get(docIDUpto);
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
});
}
}

View File

@ -31,12 +31,12 @@ public abstract class NormsFormat {
protected NormsFormat() {
}
/** Returns a {@link DocValuesConsumer} to write norms to the
/** Returns a {@link NormsConsumer} to write norms to the
* index. */
public abstract DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException;
public abstract NormsConsumer normsConsumer(SegmentWriteState state) throws IOException;
/**
* Returns a {@link DocValuesProducer} to read norms from the index.
* Returns a {@link NormsProducer} to read norms from the index.
* <p>
* NOTE: by the time this call returns, it must hold open any files it will
* need to use; else, those files may be deleted. Additionally, required files
@ -45,5 +45,5 @@ public abstract class NormsFormat {
* the implementation. IOExceptions are expected and will automatically cause
* a retry of the segment opening logic with the newly revised segments.
*/
public abstract DocValuesProducer normsProducer(SegmentReadState state) throws IOException;
public abstract NormsProducer normsProducer(SegmentReadState state) throws IOException;
}

View File

@ -0,0 +1,50 @@
package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.util.Accountable;
/*
* 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.
*/
/** Abstract API that produces field normalization values
*
* @lucene.experimental
*/
public abstract class NormsProducer implements Closeable, Accountable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected NormsProducer() {}
/** Returns {@link NumericDocValues} for this field.
* The returned instance need not be thread-safe: it will only be
* used by a single thread. */
public abstract NumericDocValues getNorms(FieldInfo field) throws IOException;
/**
* Checks consistency of this producer
* <p>
* Note that this may be costly in terms of I/O, e.g.
* may involve computing a checksum value against large data files.
* @lucene.internal
*/
public abstract void checkIntegrity() throws IOException;
}

View File

@ -31,8 +31,8 @@ import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_CHECKSUM;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_CURRENT;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_START;
import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.FIELDS_EXTENSION;
import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_EXTENSION;
import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_INDEX_EXTENSION;
import java.io.EOFException;
import java.io.IOException;

View File

@ -17,9 +17,6 @@ package org.apache.lucene.codecs.compressing;
* limitations under the License.
*/
import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.FIELDS_EXTENSION;
import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION;
import java.io.IOException;
import java.util.Arrays;
@ -54,6 +51,12 @@ import org.apache.lucene.util.packed.PackedInts;
*/
public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
/** Extension of stored fields file */
public static final String FIELDS_EXTENSION = "fdt";
/** Extension of stored fields index file */
public static final String FIELDS_INDEX_EXTENSION = "fdx";
// hard limit on the maximum number of documents per chunk
static final int MAX_DOCUMENTS_PER_CHUNK = 128;
@ -332,8 +335,10 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
int docCount = 0;
int idx = 0;
MatchingReaders matching = new MatchingReaders(mergeState);
for (AtomicReader reader : mergeState.readers) {
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
final SegmentReader matchingSegmentReader = matching.matchingSegmentReaders[idx++];
CompressingStoredFieldsReader matchingFieldsReader = null;
if (matchingSegmentReader != null) {
final StoredFieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();

View File

@ -730,8 +730,10 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
int docCount = 0;
int idx = 0;
MatchingReaders matching = new MatchingReaders(mergeState);
for (AtomicReader reader : mergeState.readers) {
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
final SegmentReader matchingSegmentReader = matching.matchingSegmentReaders[idx++];
CompressingTermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) {
final TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();

View File

@ -0,0 +1,86 @@
package org.apache.lucene.codecs.compressing;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentReader;
/**
* Computes which segments have identical field name->number mappings,
* which allows stored fields and term vectors in this codec to be bulk-merged.
*/
class MatchingReaders {
/** {@link SegmentReader}s that have identical field
* name/number mapping, so their stored fields and term
* vectors may be bulk merged. */
final SegmentReader[] matchingSegmentReaders;
/** How many {@link #matchingSegmentReaders} are set. */
final int count;
MatchingReaders(MergeState mergeState) {
// If the i'th reader is a SegmentReader and has
// identical fieldName -> number mapping, then this
// array will be non-null at position i:
int numReaders = mergeState.readers.size();
int matchedCount = 0;
matchingSegmentReaders = new SegmentReader[numReaders];
// If this reader is a SegmentReader, and all of its
// field name -> number mappings match the "merged"
// FieldInfos, then we can do a bulk copy of the
// stored fields:
for (int i = 0; i < numReaders; i++) {
AtomicReader reader = mergeState.readers.get(i);
// TODO: we may be able to broaden this to
// non-SegmentReaders, since FieldInfos is now
// required? But... this'd also require exposing
// bulk-copy (TVs and stored fields) API in foreign
// readers..
if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader;
boolean same = true;
FieldInfos segmentFieldInfos = segmentReader.getFieldInfos();
for (FieldInfo fi : segmentFieldInfos) {
FieldInfo other = mergeState.fieldInfos.fieldInfo(fi.number);
if (other == null || !other.name.equals(fi.name)) {
same = false;
break;
}
}
if (same) {
matchingSegmentReaders[i] = segmentReader;
matchedCount++;
}
}
}
this.count = matchedCount;
if (mergeState.infoStream.isEnabled("SM")) {
mergeState.infoStream.message("SM", "merge store matchedCount=" + count + " vs " + mergeState.readers.size());
if (count != mergeState.readers.size()) {
mergeState.infoStream.message("SM", "" + (mergeState.readers.size() - count) + " non-bulk merges");
}
}
}
}

View File

@ -62,12 +62,12 @@ public class Lucene40Codec extends Codec {
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
public StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}

View File

@ -19,9 +19,9 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -49,15 +49,15 @@ public class Lucene40NormsFormat extends NormsFormat {
public Lucene40NormsFormat() {}
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
String filename = IndexFileNames.segmentFileName(state.segmentInfo.name,
"nrm",
IndexFileNames.COMPOUND_FILE_EXTENSION);
return new Lucene40DocValuesReader(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
return new Lucene40NormsReader(state, filename);
}
}

View File

@ -0,0 +1,59 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
/**
* Reads 4.0/4.1 norms.
* Implemented the same as docvalues, but with a different filename.
* @deprecated Only for reading old 4.0 and 4.1 segments
*/
@Deprecated
class Lucene40NormsReader extends NormsProducer {
private final Lucene40DocValuesReader impl;
public Lucene40NormsReader(SegmentReadState state, String filename) throws IOException {
impl = new Lucene40DocValuesReader(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
}
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
return impl.getNumeric(field);
}
@Override
public void close() throws IOException {
impl.close();
}
@Override
public long ramBytesUsed() {
return impl.ramBytesUsed();
}
@Override
public void checkIntegrity() throws IOException {
impl.checkIntegrity();
}
}

View File

@ -94,6 +94,6 @@ public class Lucene40StoredFieldsFormat extends StoredFieldsFormat {
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si,
IOContext context) throws IOException {
return new Lucene40StoredFieldsWriter(directory, si.name, context);
throw new UnsupportedOperationException("this codec can only be used for reading");
}
}

View File

@ -37,8 +37,6 @@ import org.apache.lucene.util.RamUsageEstimator;
import java.io.Closeable;
import java.nio.charset.StandardCharsets;
import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.*;
/**
* Class responsible for access to stored document fields.
* <p/>
@ -49,6 +47,38 @@ import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.*;
*/
public final class Lucene40StoredFieldsReader extends StoredFieldsReader implements Cloneable, Closeable {
// NOTE: bit 0 is free here! You can steal it!
static final int FIELD_IS_BINARY = 1 << 1;
// the old bit 1 << 2 was compressed, is now left out
private static final int _NUMERIC_BIT_SHIFT = 3;
static final int FIELD_IS_NUMERIC_MASK = 0x07 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_INT = 1 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_LONG = 2 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_FLOAT = 3 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_DOUBLE = 4 << _NUMERIC_BIT_SHIFT;
// the next possible bits are: 1 << 6; 1 << 7
// currently unused: static final int FIELD_IS_NUMERIC_SHORT = 5 << _NUMERIC_BIT_SHIFT;
// currently unused: static final int FIELD_IS_NUMERIC_BYTE = 6 << _NUMERIC_BIT_SHIFT;
static final String CODEC_NAME_IDX = "Lucene40StoredFieldsIndex";
static final String CODEC_NAME_DAT = "Lucene40StoredFieldsData";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final long HEADER_LENGTH_IDX = CodecUtil.headerLength(CODEC_NAME_IDX);
static final long HEADER_LENGTH_DAT = CodecUtil.headerLength(CODEC_NAME_DAT);
/** Extension of stored fields file */
public static final String FIELDS_EXTENSION = "fdt";
/** Extension of stored fields index file */
public static final String FIELDS_INDEX_EXTENSION = "fdx";
private static final long RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene40StoredFieldsReader.class);
private final FieldInfos fieldInfos;
@ -224,32 +254,6 @@ public final class Lucene40StoredFieldsReader extends StoredFieldsReader impleme
}
}
/** Returns the length in bytes of each raw document in a
* contiguous range of length numDocs starting with
* startDocID. Returns the IndexInput (the fieldStream),
* already seeked to the starting point for startDocID.*/
public final IndexInput rawDocs(int[] lengths, int startDocID, int numDocs) throws IOException {
seekIndex(startDocID);
long startOffset = indexStream.readLong();
long lastOffset = startOffset;
int count = 0;
while (count < numDocs) {
final long offset;
final int docID = startDocID + count + 1;
assert docID <= numTotalDocs;
if (docID < numTotalDocs)
offset = indexStream.readLong();
else
offset = fieldsStream.length();
lengths[count++] = (int) (offset-lastOffset);
lastOffset = offset;
}
fieldsStream.seek(startOffset);
return fieldsStream;
}
@Override
public long ramBytesUsed() {
return RAM_BYTES_USED;

View File

@ -1,356 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/**
* Copyright 2004 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
* the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.StoredDocument;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* Class responsible for writing stored document fields.
* <p/>
* It uses &lt;segment&gt;.fdt and &lt;segment&gt;.fdx; files.
*
* @see Lucene40StoredFieldsFormat
* @lucene.experimental
*/
public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
// NOTE: bit 0 is free here! You can steal it!
static final int FIELD_IS_BINARY = 1 << 1;
// the old bit 1 << 2 was compressed, is now left out
private static final int _NUMERIC_BIT_SHIFT = 3;
static final int FIELD_IS_NUMERIC_MASK = 0x07 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_INT = 1 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_LONG = 2 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_FLOAT = 3 << _NUMERIC_BIT_SHIFT;
static final int FIELD_IS_NUMERIC_DOUBLE = 4 << _NUMERIC_BIT_SHIFT;
// the next possible bits are: 1 << 6; 1 << 7
// currently unused: static final int FIELD_IS_NUMERIC_SHORT = 5 << _NUMERIC_BIT_SHIFT;
// currently unused: static final int FIELD_IS_NUMERIC_BYTE = 6 << _NUMERIC_BIT_SHIFT;
static final String CODEC_NAME_IDX = "Lucene40StoredFieldsIndex";
static final String CODEC_NAME_DAT = "Lucene40StoredFieldsData";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final long HEADER_LENGTH_IDX = CodecUtil.headerLength(CODEC_NAME_IDX);
static final long HEADER_LENGTH_DAT = CodecUtil.headerLength(CODEC_NAME_DAT);
/** Extension of stored fields file */
public static final String FIELDS_EXTENSION = "fdt";
/** Extension of stored fields index file */
public static final String FIELDS_INDEX_EXTENSION = "fdx";
private final Directory directory;
private final String segment;
private IndexOutput fieldsStream;
private IndexOutput indexStream;
private final RAMOutputStream fieldsBuffer = new RAMOutputStream();
/** Sole constructor. */
public Lucene40StoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
assert directory != null;
this.directory = directory;
this.segment = segment;
boolean success = false;
try {
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION), context);
CodecUtil.writeHeader(fieldsStream, CODEC_NAME_DAT, VERSION_CURRENT);
CodecUtil.writeHeader(indexStream, CODEC_NAME_IDX, VERSION_CURRENT);
assert HEADER_LENGTH_DAT == fieldsStream.getFilePointer();
assert HEADER_LENGTH_IDX == indexStream.getFilePointer();
success = true;
} finally {
if (!success) {
abort();
}
}
}
int numStoredFields;
// Writes the contents of buffer into the fields stream
// and adds a new entry for this document into the index
// stream. This assumes the buffer was already written
// in the correct fields format.
@Override
public void startDocument() throws IOException {
indexStream.writeLong(fieldsStream.getFilePointer());
}
@Override
public void finishDocument() throws IOException {
fieldsStream.writeVInt(numStoredFields);
fieldsBuffer.writeTo(fieldsStream);
fieldsBuffer.reset();
numStoredFields = 0;
}
@Override
public void close() throws IOException {
try {
IOUtils.close(fieldsStream, indexStream);
} finally {
fieldsStream = indexStream = null;
}
}
@Override
public void abort() {
try {
close();
} catch (Throwable ignored) {}
IOUtils.deleteFilesIgnoringExceptions(directory,
IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION),
IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
}
@Override
public void writeField(FieldInfo info, StorableField field) throws IOException {
numStoredFields++;
fieldsBuffer.writeVInt(info.number);
int bits = 0;
final BytesRef bytes;
final String string;
// TODO: maybe a field should serialize itself?
// this way we don't bake into indexer all these
// specific encodings for different fields? and apps
// can customize...
Number number = field.numericValue();
if (number != null) {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
bits |= FIELD_IS_NUMERIC_INT;
} else if (number instanceof Long) {
bits |= FIELD_IS_NUMERIC_LONG;
} else if (number instanceof Float) {
bits |= FIELD_IS_NUMERIC_FLOAT;
} else if (number instanceof Double) {
bits |= FIELD_IS_NUMERIC_DOUBLE;
} else {
throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
}
string = null;
bytes = null;
} else {
bytes = field.binaryValue();
if (bytes != null) {
bits |= FIELD_IS_BINARY;
string = null;
} else {
string = field.stringValue();
if (string == null) {
throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
}
}
}
fieldsBuffer.writeByte((byte) bits);
if (bytes != null) {
fieldsBuffer.writeVInt(bytes.length);
fieldsBuffer.writeBytes(bytes.bytes, bytes.offset, bytes.length);
} else if (string != null) {
fieldsBuffer.writeString(field.stringValue());
} else {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
fieldsBuffer.writeInt(number.intValue());
} else if (number instanceof Long) {
fieldsBuffer.writeLong(number.longValue());
} else if (number instanceof Float) {
fieldsBuffer.writeInt(Float.floatToIntBits(number.floatValue()));
} else if (number instanceof Double) {
fieldsBuffer.writeLong(Double.doubleToLongBits(number.doubleValue()));
} else {
throw new AssertionError("Cannot get here");
}
}
}
/** Bulk write a contiguous series of documents. The
* lengths array is the length (in bytes) of each raw
* document. The stream IndexInput is the
* fieldsStream from which we should bulk-copy all
* bytes. */
public void addRawDocuments(IndexInput stream, int[] lengths, int numDocs) throws IOException {
long position = fieldsStream.getFilePointer();
long start = position;
for(int i=0;i<numDocs;i++) {
indexStream.writeLong(position);
position += lengths[i];
}
fieldsStream.copyBytes(stream, position-start);
assert fieldsStream.getFilePointer() == position;
}
@Override
public void finish(FieldInfos fis, int numDocs) {
long indexFP = indexStream.getFilePointer();
if (HEADER_LENGTH_IDX+((long) numDocs)*8 != indexFP)
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
// we detect that the bug has struck, here, and
// throw an exception to prevent the corruption from
// entering the index. See LUCENE-1282 for
// details.
throw new RuntimeException("fdx size mismatch: docCount is " + numDocs + " but fdx file size is " + indexFP + " (wrote numDocs=" + ((indexFP-HEADER_LENGTH_IDX)/8.0) + " file=" + indexStream.toString() + "; now aborting this merge to prevent index corruption");
}
@Override
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
// Used for bulk-reading raw bytes for stored fields
int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
int idx = 0;
for (AtomicReader reader : mergeState.readers) {
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
Lucene40StoredFieldsReader matchingFieldsReader = null;
if (matchingSegmentReader != null) {
final StoredFieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
// we can only bulk-copy if the matching reader is also a Lucene40FieldsReader
if (fieldsReader != null && fieldsReader instanceof Lucene40StoredFieldsReader) {
matchingFieldsReader = (Lucene40StoredFieldsReader) fieldsReader;
}
}
if (reader.getLiveDocs() != null) {
docCount += copyFieldsWithDeletions(mergeState,
reader, matchingFieldsReader, rawDocLengths);
} else {
docCount += copyFieldsNoDeletions(mergeState,
reader, matchingFieldsReader, rawDocLengths);
}
}
finish(mergeState.fieldInfos, docCount);
return docCount;
}
/** Maximum number of contiguous documents to bulk-copy
when merging stored fields */
private final static int MAX_RAW_MERGE_DOCS = 4192;
private int copyFieldsWithDeletions(MergeState mergeState, final AtomicReader reader,
final Lucene40StoredFieldsReader matchingFieldsReader, int rawDocLengths[])
throws IOException {
int docCount = 0;
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
assert liveDocs != null;
if (matchingFieldsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
for (int j = 0; j < maxDoc;) {
if (!liveDocs.get(j)) {
// skip deleted docs
++j;
continue;
}
// We can optimize this case (doing a bulk byte copy) since the field
// numbers are identical
int start = j, numDocs = 0;
do {
j++;
numDocs++;
if (j >= maxDoc) break;
if (!liveDocs.get(j)) {
j++;
break;
}
} while(numDocs < MAX_RAW_MERGE_DOCS);
IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, start, numDocs);
addRawDocuments(stream, rawDocLengths, numDocs);
docCount += numDocs;
mergeState.checkAbort.work(300 * numDocs);
}
} else {
for (int j = 0; j < maxDoc; j++) {
if (!liveDocs.get(j)) {
// skip deleted docs
continue;
}
// TODO: this could be more efficient using
// FieldVisitor instead of loading/writing entire
// doc; ie we just have to renumber the field number
// on the fly?
// NOTE: it's very important to first assign to doc then pass it to
// fieldsWriter.addDocument; see LUCENE-1282
StoredDocument doc = reader.document(j);
addDocument(doc, mergeState.fieldInfos);
docCount++;
mergeState.checkAbort.work(300);
}
}
return docCount;
}
private int copyFieldsNoDeletions(MergeState mergeState, final AtomicReader reader,
final Lucene40StoredFieldsReader matchingFieldsReader, int rawDocLengths[])
throws IOException {
final int maxDoc = reader.maxDoc();
int docCount = 0;
if (matchingFieldsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
while (docCount < maxDoc) {
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, docCount, len);
addRawDocuments(stream, rawDocLengths, len);
docCount += len;
mergeState.checkAbort.work(300 * len);
}
} else {
for (; docCount < maxDoc; docCount++) {
// NOTE: it's very important to first assign to doc then pass it to
// fieldsWriter.addDocument; see LUCENE-1282
StoredDocument doc = reader.document(docCount);
addDocument(doc, mergeState.fieldInfos);
mergeState.checkAbort.work(300);
}
}
return docCount;
}
}

View File

@ -126,6 +126,6 @@ public class Lucene40TermVectorsFormat extends TermVectorsFormat {
@Override
public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
return new Lucene40TermVectorsWriter(directory, segmentInfo.name, context);
throw new UnsupportedOperationException("this codec can only be used for reading");
}
}

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene40;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
@ -146,66 +145,11 @@ public class Lucene40TermVectorsReader extends TermVectorsReader implements Clos
}
}
// Used for bulk copy when merging
IndexInput getTvdStream() {
return tvd;
}
// Used for bulk copy when merging
IndexInput getTvfStream() {
return tvf;
}
// Not private to avoid synthetic access$NNN methods
void seekTvx(final int docNum) throws IOException {
tvx.seek(docNum * 16L + HEADER_LENGTH_INDEX);
}
/** Retrieve the length (in bytes) of the tvd and tvf
* entries for the next numDocs starting with
* startDocID. This is used for bulk copying when
* merging segments, if the field numbers are
* congruent. Once this returns, the tvf & tvd streams
* are seeked to the startDocID. */
final void rawDocs(int[] tvdLengths, int[] tvfLengths, int startDocID, int numDocs) throws IOException {
if (tvx == null) {
Arrays.fill(tvdLengths, 0);
Arrays.fill(tvfLengths, 0);
return;
}
seekTvx(startDocID);
long tvdPosition = tvx.readLong();
tvd.seek(tvdPosition);
long tvfPosition = tvx.readLong();
tvf.seek(tvfPosition);
long lastTvdPosition = tvdPosition;
long lastTvfPosition = tvfPosition;
int count = 0;
while (count < numDocs) {
final int docID = startDocID + count + 1;
assert docID <= numTotalDocs;
if (docID < numTotalDocs) {
tvdPosition = tvx.readLong();
tvfPosition = tvx.readLong();
} else {
tvdPosition = tvd.length();
tvfPosition = tvf.length();
assert count == numDocs-1;
}
tvdLengths[count] = (int) (tvdPosition-lastTvdPosition);
tvfLengths[count] = (int) (tvfPosition-lastTvfPosition);
count++;
lastTvdPosition = tvdPosition;
lastTvfPosition = tvfPosition;
}
}
@Override
public void close() throws IOException {
IOUtils.close(tvx, tvd, tvf);

View File

@ -86,7 +86,7 @@ public class Lucene41Codec extends Codec {
}
@Override
public final TermVectorsFormat termVectorsFormat() {
public TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}

View File

@ -20,11 +20,11 @@ package org.apache.lucene.codecs.lucene42;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
@ -137,7 +137,7 @@ public class Lucene42Codec extends Codec {
private final NormsFormat normsFormat = new Lucene42NormsFormat() {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};

View File

@ -21,7 +21,9 @@ import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
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;
import org.apache.lucene.util.packed.PackedInts;
@ -66,13 +68,13 @@ public class Lucene42NormsFormat extends NormsFormat {
}
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
return new Lucene42NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
static final String DATA_CODEC = "Lucene41NormsData";

View File

@ -0,0 +1,59 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
/**
* Reads 4.2-4.8 norms.
* Implemented the same as docvalues, but with a different filename.
* @deprecated Only for reading old segments
*/
@Deprecated
class Lucene42NormsProducer extends NormsProducer {
private final Lucene42DocValuesProducer impl;
Lucene42NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
impl = new Lucene42DocValuesProducer(state, dataCodec, dataExtension, metaCodec, metaExtension);
}
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
return impl.getNumeric(field);
}
@Override
public void checkIntegrity() throws IOException {
impl.checkIntegrity();
}
@Override
public long ramBytesUsed() {
return impl.ramBytesUsed();
}
@Override
public void close() throws IOException {
impl.close();
}
}

View File

@ -20,11 +20,11 @@ package org.apache.lucene.codecs.lucene45;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
@ -140,7 +140,7 @@ public class Lucene45Codec extends Codec {
private final NormsFormat normsFormat = new Lucene42NormsFormat() {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};

View File

@ -20,11 +20,11 @@ package org.apache.lucene.codecs.lucene46;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
@ -137,7 +137,7 @@ public class Lucene46Codec extends Codec {
private final NormsFormat normsFormat = new Lucene42NormsFormat() {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};

View File

@ -20,11 +20,11 @@ package org.apache.lucene.codecs.lucene49;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
@ -32,7 +32,6 @@ import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.lucene46.Lucene46FieldInfosFormat;
import org.apache.lucene.codecs.lucene46.Lucene46SegmentInfoFormat;
@ -137,7 +136,7 @@ public class Lucene49Codec extends Codec {
private final NormsFormat normsFormat = new Lucene49NormsFormat() {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
};

View File

@ -23,12 +23,11 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
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.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
@ -38,7 +37,7 @@ import static org.apache.lucene.codecs.lucene49.Lucene49NormsFormat.VERSION_CURR
/**
* Writer for {@link Lucene49NormsFormat}
*/
class Lucene49NormsConsumer extends DocValuesConsumer {
class Lucene49NormsConsumer extends NormsConsumer {
static final byte DELTA_COMPRESSED = 0;
static final byte TABLE_COMPRESSED = 1;
static final byte CONST_COMPRESSED = 2;
@ -74,7 +73,7 @@ class Lucene49NormsConsumer extends DocValuesConsumer {
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
meta.writeVInt(field.number);
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
@ -187,26 +186,6 @@ class Lucene49NormsConsumer extends DocValuesConsumer {
}
}
@Override
public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
throw new UnsupportedOperationException();
}
// 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.

View File

@ -20,9 +20,9 @@ package org.apache.lucene.codecs.lucene49;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
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;
import org.apache.lucene.store.DataOutput;
@ -103,12 +103,12 @@ public class Lucene49NormsFormat extends NormsFormat {
public Lucene49NormsFormat() {}
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new Lucene49NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
@Override
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
return new Lucene49NormsProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}

View File

@ -23,20 +23,15 @@ import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
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.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.BlockPackedReader;
@ -52,7 +47,7 @@ import static org.apache.lucene.codecs.lucene49.Lucene49NormsConsumer.UNCOMPRESS
/**
* Reader for {@link Lucene49NormsFormat}
*/
class Lucene49NormsProducer extends DocValuesProducer {
class Lucene49NormsProducer extends NormsProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer,NormsEntry> norms = new HashMap<>();
private final IndexInput data;
@ -134,7 +129,7 @@ class Lucene49NormsProducer extends DocValuesProducer {
}
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
public synchronized NumericDocValues getNorms(FieldInfo field) throws IOException {
NumericDocValues instance = instances.get(field.number);
if (instance == null) {
instance = loadNorms(field);
@ -208,31 +203,6 @@ class Lucene49NormsProducer extends DocValuesProducer {
}
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
throw new IllegalStateException();
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
throw new IllegalStateException();
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
throw new IllegalStateException();
}
@Override
public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
throw new IllegalStateException();
}
@Override
public Bits getDocsWithField(FieldInfo field) throws IOException {
throw new IllegalStateException();
}
@Override
public void close() throws IOException {
data.close();

View File

@ -26,6 +26,7 @@ import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.document.FieldType;
@ -171,7 +172,7 @@ final class DefaultIndexingChain extends DocConsumer {
private void writeNorms(SegmentWriteState state) throws IOException {
boolean success = false;
DocValuesConsumer normsConsumer = null;
NormsConsumer normsConsumer = null;
try {
if (state.fieldInfos.hasNorms()) {
NormsFormat normsFormat = state.segmentInfo.getCodec().normsFormat();
@ -405,7 +406,7 @@ final class DefaultIndexingChain extends DocConsumer {
case NUMERIC:
if (fp.docValuesWriter == null) {
fp.docValuesWriter = new NumericDocValuesWriter(fp.fieldInfo, bytesUsed, true);
fp.docValuesWriter = new NumericDocValuesWriter(fp.fieldInfo, bytesUsed);
}
((NumericDocValuesWriter) fp.docValuesWriter).addValue(docID, field.numericValue().longValue());
break;
@ -520,7 +521,7 @@ final class DefaultIndexingChain extends DocConsumer {
PerField next;
// Lazy init'd:
NumericDocValuesWriter norms;
NormValuesWriter norms;
// reused
TokenStream tokenStream;
@ -547,7 +548,7 @@ final class DefaultIndexingChain extends DocConsumer {
if (fieldInfo.omitsNorms() == false) {
if (norms == null) {
fieldInfo.setNormValueType(FieldInfo.DocValuesType.NUMERIC);
norms = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed, false);
norms = new NormValuesWriter(fieldInfo, docState.docWriter.bytesUsed);
}
norms.addValue(docState.docID, similarity.computeNorm(invertState));
}

View File

@ -27,11 +27,14 @@ import static org.apache.lucene.index.FilterAtomicReader.FilterTermsEnum;
/** A {@link Fields} implementation that merges multiple
* Fields into one, and maps around deleted documents.
* This is used for merging. */
class MappedMultiFields extends FilterFields {
* This is used for merging.
* @lucene.internal
*/
public class MappedMultiFields extends FilterFields {
final MergeState mergeState;
/** Create a new MappedMultiFields for merging, based on the supplied
* mergestate and merged view of terms. */
public MappedMultiFields(MergeState mergeState, MultiFields multiFields) {
super(multiFields);
this.mergeState = mergeState;

View File

@ -156,17 +156,6 @@ public class MergeState {
* @lucene.internal */
public int checkAbortCount;
// TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging)
// but is this really so expensive to compute again in different components, versus once in SM?
/** {@link SegmentReader}s that have identical field
* name/number mapping, so their stored fields and term
* vectors may be bulk merged. */
public SegmentReader[] matchingSegmentReaders;
/** How many {@link #matchingSegmentReaders} are set. */
public int matchedCount;
/** Sole constructor. */
MergeState(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, CheckAbort checkAbort) {
this.readers = readers;

View File

@ -0,0 +1,120 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** Buffers up pending long per doc, then flushes when
* segment flushes. */
class NormValuesWriter {
private final static long MISSING = 0L;
private PackedLongValues.Builder pending;
private final Counter iwBytesUsed;
private long bytesUsed;
private final FieldInfo fieldInfo;
public NormValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
bytesUsed = pending.ramBytesUsed();
this.fieldInfo = fieldInfo;
this.iwBytesUsed = iwBytesUsed;
iwBytesUsed.addAndGet(bytesUsed);
}
public void addValue(int docID, long value) {
// Fill in any holes:
for (int i = (int)pending.size(); i < docID; ++i) {
pending.add(MISSING);
}
pending.add(value);
updateBytesUsed();
}
private void updateBytesUsed() {
final long newBytesUsed = pending.ramBytesUsed();
iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
bytesUsed = newBytesUsed;
}
public void finish(int maxDoc) {
}
public void flush(SegmentWriteState state, NormsConsumer normsConsumer) throws IOException {
final int maxDoc = state.segmentInfo.getDocCount();
final PackedLongValues values = pending.build();
normsConsumer.addNormsField(fieldInfo,
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new NumericIterator(maxDoc, values);
}
});
}
// iterates over the values we have in ram
private static class NumericIterator implements Iterator<Number> {
final PackedLongValues.Iterator iter;
final int size;
final int maxDoc;
int upto;
NumericIterator(int maxDoc, PackedLongValues values) {
this.maxDoc = maxDoc;
this.iter = values.iterator();
this.size = (int) values.size();
}
@Override
public boolean hasNext() {
return upto < maxDoc;
}
@Override
public Number next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
Long value;
if (upto < size) {
value = iter.next();
} else {
value = MISSING;
}
upto++;
return value;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
}

View File

@ -40,9 +40,9 @@ class NumericDocValuesWriter extends DocValuesWriter {
private FixedBitSet docsWithField;
private final FieldInfo fieldInfo;
public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed, boolean trackDocsWithField) {
public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
pending = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
docsWithField = trackDocsWithField ? new FixedBitSet(64) : null;
docsWithField = new FixedBitSet(64);
bytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
this.fieldInfo = fieldInfo;
this.iwBytesUsed = iwBytesUsed;
@ -60,17 +60,15 @@ class NumericDocValuesWriter extends DocValuesWriter {
}
pending.add(value);
if (docsWithField != null) {
docsWithField = FixedBitSet.ensureCapacity(docsWithField, docID);
docsWithField.set(docID);
}
updateBytesUsed();
}
private long docsWithFieldBytesUsed() {
// size of the long[] + some overhead
return docsWithField == null ? 0 : RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
return RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
}
private void updateBytesUsed() {
@ -126,13 +124,13 @@ class NumericDocValuesWriter extends DocValuesWriter {
Long value;
if (upto < size) {
long v = iter.next();
if (docsWithField == null || docsWithField.get(upto)) {
if (docsWithField.get(upto)) {
value = v;
} else {
value = null;
}
} else {
value = docsWithField != null ? null : MISSING;
value = null;
}
upto++;
return value;

View File

@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
@ -56,7 +57,7 @@ final class SegmentCoreReaders implements Accountable {
private final AtomicInteger ref = new AtomicInteger(1);
final FieldsProducer fields;
final DocValuesProducer normsProducer;
final NormsProducer normsProducer;
final StoredFieldsReader fieldsReaderOrig;
final TermVectorsReader termVectorsReaderOrig;
@ -166,7 +167,7 @@ final class SegmentCoreReaders implements Accountable {
return null;
}
assert normsProducer != null;
norms = normsProducer.getNumeric(fi);
norms = normsProducer.getNorms(fi);
normFields.put(field, norms);
return norms;
}

View File

@ -0,0 +1,173 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.Version;
/** Encapsulates multiple producers when there are docvalues updates as one producer */
// TODO: try to clean up close? no-op?
// TODO: add shared base class (also used by per-field-pf?) to allow "punching thru" to low level producer?
class SegmentDocValuesProducer extends DocValuesProducer {
private static final long LONG_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Long.class);
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SegmentDocValuesProducer.class);
final Map<String,DocValuesProducer> dvProducersByField = new HashMap<>();
final Set<DocValuesProducer> dvProducers = Collections.newSetFromMap(new IdentityHashMap<DocValuesProducer,Boolean>());
final List<Long> dvGens = new ArrayList<>();
SegmentDocValuesProducer(SegmentCommitInfo si, Directory dir, FieldInfos fieldInfos, SegmentDocValues segDocValues, DocValuesFormat dvFormat) throws IOException {
Version ver = si.info.getVersion();
if (ver != null && ver.onOrAfter(Version.LUCENE_4_9_0)) {
DocValuesProducer baseProducer = null;
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
long docValuesGen = fi.getDocValuesGen();
if (docValuesGen == -1) {
if (baseProducer == null) {
// the base producer gets all the fields, so the Codec can validate properly
baseProducer = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
dvGens.add(docValuesGen);
dvProducers.add(baseProducer);
}
dvProducersByField.put(fi.name, baseProducer);
} else {
assert !dvGens.contains(docValuesGen);
final DocValuesProducer dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(new FieldInfo[] { fi }));
dvGens.add(docValuesGen);
dvProducers.add(dvp);
dvProducersByField.put(fi.name, dvp);
}
}
} else {
// For pre-4.9 indexes, especially with doc-values updates, multiple
// FieldInfos could belong to the same dvGen. Therefore need to make sure
// we initialize each DocValuesProducer once per gen.
Map<Long,List<FieldInfo>> genInfos = new HashMap<>();
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
List<FieldInfo> genFieldInfos = genInfos.get(fi.getDocValuesGen());
if (genFieldInfos == null) {
genFieldInfos = new ArrayList<>();
genInfos.put(fi.getDocValuesGen(), genFieldInfos);
}
genFieldInfos.add(fi);
}
for (Map.Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
long docValuesGen = e.getKey();
List<FieldInfo> infos = e.getValue();
final DocValuesProducer dvp;
if (docValuesGen == -1) {
// we need to send all FieldInfos to gen=-1, but later we need to
// record the DVP only for the "true" gen=-1 fields (not updated)
dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
} else {
dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])));
}
dvGens.add(docValuesGen);
dvProducers.add(dvp);
for (FieldInfo fi : infos) {
dvProducersByField.put(fi.name, dvp);
}
}
}
}
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
DocValuesProducer dvProducer = dvProducersByField.get(field.name);
assert dvProducer != null;
return dvProducer.getNumeric(field);
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
DocValuesProducer dvProducer = dvProducersByField.get(field.name);
assert dvProducer != null;
return dvProducer.getBinary(field);
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
DocValuesProducer dvProducer = dvProducersByField.get(field.name);
assert dvProducer != null;
return dvProducer.getSorted(field);
}
@Override
public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
DocValuesProducer dvProducer = dvProducersByField.get(field.name);
assert dvProducer != null;
return dvProducer.getSortedNumeric(field);
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
DocValuesProducer dvProducer = dvProducersByField.get(field.name);
assert dvProducer != null;
return dvProducer.getSortedSet(field);
}
@Override
public Bits getDocsWithField(FieldInfo field) throws IOException {
DocValuesProducer dvProducer = dvProducersByField.get(field.name);
assert dvProducer != null;
return dvProducer.getDocsWithField(field);
}
@Override
public void checkIntegrity() throws IOException {
for (DocValuesProducer producer : dvProducers) {
producer.checkIntegrity();
}
}
@Override
public void close() throws IOException {
throw new UnsupportedOperationException(); // there is separate ref tracking
}
@Override
public long ramBytesUsed() {
long ramBytesUsed = BASE_RAM_BYTES_USED;
ramBytesUsed += dvGens.size() * LONG_RAM_BYTES_USED;
ramBytesUsed += dvProducers.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
ramBytesUsed += dvProducersByField.size() * 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
for (DocValuesProducer producer : dvProducers) {
ramBytesUsed += producer.ramBytesUsed();
}
return ramBytesUsed;
}
}

View File

@ -18,19 +18,17 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
@ -90,7 +88,6 @@ final class SegmentMerger {
// IndexWriter.close(false) takes to actually stop the
// threads.
mergeFieldInfos();
setMatchingSegmentReaders();
long t0 = 0;
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
@ -158,72 +155,7 @@ final class SegmentMerger {
DocValuesConsumer consumer = codec.docValuesFormat().fieldsConsumer(segmentWriteState);
boolean success = false;
try {
for (FieldInfo field : mergeState.fieldInfos) {
DocValuesType type = field.getDocValuesType();
if (type != null) {
if (type == DocValuesType.NUMERIC) {
List<NumericDocValues> toMerge = new ArrayList<>();
List<Bits> docsWithField = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
NumericDocValues values = reader.getNumericDocValues(field.name);
Bits bits = reader.getDocsWithField(field.name);
if (values == null) {
values = DocValues.emptyNumeric();
bits = new Bits.MatchNoBits(reader.maxDoc());
}
toMerge.add(values);
docsWithField.add(bits);
}
consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.BINARY) {
List<BinaryDocValues> toMerge = new ArrayList<>();
List<Bits> docsWithField = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
BinaryDocValues values = reader.getBinaryDocValues(field.name);
Bits bits = reader.getDocsWithField(field.name);
if (values == null) {
values = DocValues.emptyBinary();
bits = new Bits.MatchNoBits(reader.maxDoc());
}
toMerge.add(values);
docsWithField.add(bits);
}
consumer.mergeBinaryField(field, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.SORTED) {
List<SortedDocValues> toMerge = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
SortedDocValues values = reader.getSortedDocValues(field.name);
if (values == null) {
values = DocValues.emptySorted();
}
toMerge.add(values);
}
consumer.mergeSortedField(field, mergeState, toMerge);
} else if (type == DocValuesType.SORTED_SET) {
List<SortedSetDocValues> toMerge = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
SortedSetDocValues values = reader.getSortedSetDocValues(field.name);
if (values == null) {
values = DocValues.emptySortedSet();
}
toMerge.add(values);
}
consumer.mergeSortedSetField(field, mergeState, toMerge);
} else if (type == DocValuesType.SORTED_NUMERIC) {
List<SortedNumericDocValues> toMerge = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
SortedNumericDocValues values = reader.getSortedNumericDocValues(field.name);
if (values == null) {
values = DocValues.emptySortedNumeric(reader.maxDoc());
}
toMerge.add(values);
}
consumer.mergeSortedNumericField(field, mergeState, toMerge);
} else {
throw new AssertionError("type=" + type);
}
}
}
consumer.merge(mergeState);
success = true;
} finally {
if (success) {
@ -235,24 +167,10 @@ final class SegmentMerger {
}
private void mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
DocValuesConsumer consumer = codec.normsFormat().normsConsumer(segmentWriteState);
NormsConsumer consumer = codec.normsFormat().normsConsumer(segmentWriteState);
boolean success = false;
try {
for (FieldInfo field : mergeState.fieldInfos) {
if (field.hasNorms()) {
List<NumericDocValues> toMerge = new ArrayList<>();
List<Bits> docsWithField = new ArrayList<>();
for (AtomicReader reader : mergeState.readers) {
NumericDocValues norms = reader.getNormValues(field.name);
if (norms == null) {
norms = DocValues.emptyNumeric();
}
toMerge.add(norms);
docsWithField.add(new Bits.MatchAllBits(reader.maxDoc()));
}
consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
}
}
consumer.merge(mergeState);
success = true;
} finally {
if (success) {
@ -263,50 +181,6 @@ final class SegmentMerger {
}
}
private void setMatchingSegmentReaders() {
// If the i'th reader is a SegmentReader and has
// identical fieldName -> number mapping, then this
// array will be non-null at position i:
int numReaders = mergeState.readers.size();
mergeState.matchingSegmentReaders = new SegmentReader[numReaders];
// If this reader is a SegmentReader, and all of its
// field name -> number mappings match the "merged"
// FieldInfos, then we can do a bulk copy of the
// stored fields:
for (int i = 0; i < numReaders; i++) {
AtomicReader reader = mergeState.readers.get(i);
// TODO: we may be able to broaden this to
// non-SegmentReaders, since FieldInfos is now
// required? But... this'd also require exposing
// bulk-copy (TVs and stored fields) API in foreign
// readers..
if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader;
boolean same = true;
FieldInfos segmentFieldInfos = segmentReader.getFieldInfos();
for (FieldInfo fi : segmentFieldInfos) {
FieldInfo other = mergeState.fieldInfos.fieldInfo(fi.number);
if (other == null || !other.name.equals(fi.name)) {
same = false;
break;
}
}
if (same) {
mergeState.matchingSegmentReaders[i] = segmentReader;
mergeState.matchedCount++;
}
}
}
if (mergeState.infoStream.isEnabled("SM")) {
mergeState.infoStream.message("SM", "merge store matchedCount=" + mergeState.matchedCount + " vs " + mergeState.readers.size());
if (mergeState.matchedCount != mergeState.readers.size()) {
mergeState.infoStream.message("SM", "" + (mergeState.readers.size() - mergeState.matchedCount) + " non-bulk merges");
}
}
}
public void mergeFieldInfos() throws IOException {
for (AtomicReader reader : mergeState.readers) {
FieldInfos readerFieldInfos = reader.getFieldInfos();
@ -318,7 +192,7 @@ final class SegmentMerger {
}
/**
*
* Merge stored fields from each of the segments into the new one.
* @return The number of documents in all of the readers
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
@ -326,12 +200,20 @@ final class SegmentMerger {
private int mergeFields() throws IOException {
final StoredFieldsWriter fieldsWriter = codec.storedFieldsFormat().fieldsWriter(directory, mergeState.segmentInfo, context);
boolean success = false;
int numDocs;
try {
return fieldsWriter.merge(mergeState);
numDocs = fieldsWriter.merge(mergeState);
success = true;
} finally {
fieldsWriter.close();
if (success) {
IOUtils.close(fieldsWriter);
} else {
IOUtils.closeWhileHandlingException(fieldsWriter);
}
}
return numDocs;
}
/**
* Merge the TermVectors from each of the segments into the new one.
@ -340,12 +222,20 @@ final class SegmentMerger {
private int mergeVectors() throws IOException {
final TermVectorsWriter termVectorsWriter = codec.termVectorsFormat().vectorsWriter(directory, mergeState.segmentInfo, context);
boolean success = false;
int numDocs;
try {
return termVectorsWriter.merge(mergeState);
numDocs = termVectorsWriter.merge(mergeState);
success = true;
} finally {
termVectorsWriter.close();
if (success) {
IOUtils.close(termVectorsWriter);
} else {
IOUtils.closeWhileHandlingException(termVectorsWriter);
}
}
return numDocs;
}
// NOTE: removes any "all deleted" readers from mergeState.readers
private int setDocMaps() throws IOException {
@ -374,31 +264,10 @@ final class SegmentMerger {
}
private void mergeTerms(SegmentWriteState segmentWriteState) throws IOException {
final List<Fields> fields = new ArrayList<>();
final List<ReaderSlice> slices = new ArrayList<>();
int docBase = 0;
for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
final AtomicReader reader = mergeState.readers.get(readerIndex);
final Fields f = reader.fields();
final int maxDoc = reader.maxDoc();
if (f != null) {
slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
fields.add(f);
}
docBase += maxDoc;
}
Fields mergedFields = new MappedMultiFields(mergeState,
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderSlice.EMPTY_ARRAY)));
FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
boolean success = false;
try {
consumer.write(mergedFields);
consumer.merge(mergeState);
success = true;
} finally {
if (success) {

View File

@ -18,18 +18,15 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.FieldInfo.DocValuesType;
@ -41,8 +38,6 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.Version;
/**
* IndexReader implementation over a single segment.
@ -56,7 +51,6 @@ public final class SegmentReader extends AtomicReader implements Accountable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SegmentReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(SegmentDocValues.class);
private static final long LONG_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Long.class);
private final SegmentCommitInfo si;
private final Bits liveDocs;
@ -83,13 +77,9 @@ public final class SegmentReader extends AtomicReader implements Accountable {
}
};
final Map<String,DocValuesProducer> dvProducersByField = new HashMap<>();
final Set<DocValuesProducer> dvProducers = Collections.newSetFromMap(new IdentityHashMap<DocValuesProducer,Boolean>());
final DocValuesProducer docValuesProducer;
final FieldInfos fieldInfos;
private final List<Long> dvGens = new ArrayList<>();
/**
* Constructs a new SegmentReader with a new core.
* @throws CorruptIndexException if the index is corrupt
@ -121,7 +111,9 @@ public final class SegmentReader extends AtomicReader implements Accountable {
numDocs = si.info.getDocCount() - si.getDelCount();
if (fieldInfos.hasDocValues()) {
initDocValuesProducers(codec);
docValuesProducer = initDocValuesProducer(codec);
} else {
docValuesProducer = null;
}
success = true;
@ -171,7 +163,9 @@ public final class SegmentReader extends AtomicReader implements Accountable {
}
if (fieldInfos.hasDocValues()) {
initDocValuesProducers(codec);
docValuesProducer = initDocValuesProducer(codec);
} else {
docValuesProducer = null;
}
success = true;
} finally {
@ -182,81 +176,15 @@ public final class SegmentReader extends AtomicReader implements Accountable {
}
// initialize the per-field DocValuesProducer
@SuppressWarnings("deprecation")
private void initDocValuesProducers(Codec codec) throws IOException {
private DocValuesProducer initDocValuesProducer(Codec codec) throws IOException {
final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
final DocValuesFormat dvFormat = codec.docValuesFormat();
if (!si.hasFieldUpdates()) {
// simple case, no DocValues updates
final DocValuesProducer dvp = segDocValues.getDocValuesProducer(-1L, si, IOContext.READ, dir, dvFormat, fieldInfos);
dvGens.add(-1L);
dvProducers.add(dvp);
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
assert fi.getDocValuesGen() == -1;
dvProducersByField.put(fi.name, dvp);
}
return;
}
Version ver = si.info.getVersion();
if (ver != null && ver.onOrAfter(Version.LUCENE_4_9_0)) {
DocValuesProducer baseProducer = null;
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
long docValuesGen = fi.getDocValuesGen();
if (docValuesGen == -1) {
if (baseProducer == null) {
// System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
// the base producer gets all the fields, so the Codec can validate properly
baseProducer = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
dvGens.add(docValuesGen);
dvProducers.add(baseProducer);
}
// System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
dvProducersByField.put(fi.name, baseProducer);
return segDocValues.getDocValuesProducer(-1L, si, IOContext.READ, dir, dvFormat, fieldInfos);
} else {
assert !dvGens.contains(docValuesGen);
// System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
final DocValuesProducer dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(new FieldInfo[] { fi }));
dvGens.add(docValuesGen);
dvProducers.add(dvp);
dvProducersByField.put(fi.name, dvp);
}
}
} else {
// For pre-4.9 indexes, especially with doc-values updates, multiple
// FieldInfos could belong to the same dvGen. Therefore need to make sure
// we initialize each DocValuesProducer once per gen.
Map<Long,List<FieldInfo>> genInfos = new HashMap<>();
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
List<FieldInfo> genFieldInfos = genInfos.get(fi.getDocValuesGen());
if (genFieldInfos == null) {
genFieldInfos = new ArrayList<>();
genInfos.put(fi.getDocValuesGen(), genFieldInfos);
}
genFieldInfos.add(fi);
}
for (Map.Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
long docValuesGen = e.getKey();
List<FieldInfo> infos = e.getValue();
final DocValuesProducer dvp;
if (docValuesGen == -1) {
// we need to send all FieldInfos to gen=-1, but later we need to
// record the DVP only for the "true" gen=-1 fields (not updated)
dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
} else {
dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])));
}
dvGens.add(docValuesGen);
dvProducers.add(dvp);
for (FieldInfo fi : infos) {
dvProducersByField.put(fi.name, dvp);
}
}
return new SegmentDocValuesProducer(si, dir, fieldInfos, segDocValues, dvFormat);
}
}
@ -305,11 +233,14 @@ public final class SegmentReader extends AtomicReader implements Accountable {
try {
core.decRef();
} finally {
dvProducersByField.clear();
try {
IOUtils.close(docValuesLocal, docsWithFieldLocal);
} finally {
segDocValues.decRef(dvGens);
if (docValuesProducer instanceof SegmentDocValuesProducer) {
segDocValues.decRef(((SegmentDocValuesProducer)docValuesProducer).dvGens);
} else if (docValuesProducer != null) {
segDocValues.decRef(Collections.singletonList(-1L));
}
}
}
}
@ -320,14 +251,6 @@ public final class SegmentReader extends AtomicReader implements Accountable {
return fieldInfos;
}
/** Expert: retrieve thread-private {@link
* StoredFieldsReader}
* @lucene.internal */
public StoredFieldsReader getFieldsReader() {
ensureOpen();
return core.fieldsReaderLocal.get();
}
@Override
public void document(int docID, StoredFieldVisitor visitor) throws IOException {
checkBounds(docID);
@ -360,6 +283,28 @@ public final class SegmentReader extends AtomicReader implements Accountable {
return core.termVectorsLocal.get();
}
/** Expert: retrieve thread-private {@link
* StoredFieldsReader}
* @lucene.internal */
public StoredFieldsReader getFieldsReader() {
ensureOpen();
return core.fieldsReaderLocal.get();
}
/** Expert: retrieve underlying NormsProducer
* @lucene.internal */
public NormsProducer getNormsReader() {
ensureOpen();
return core.normsProducer;
}
/** Expert: retrieve underlying DocValuesProducer
* @lucene.internal */
public DocValuesProducer getDocValuesReader() {
ensureOpen();
return docValuesProducer;
}
@Override
public Fields getTermVectors(int docID) throws IOException {
TermVectorsReader termVectorsReader = getTermVectorsReader();
@ -455,9 +400,7 @@ public final class SegmentReader extends AtomicReader implements Accountable {
if (fi == null) {
return null;
}
DocValuesProducer dvProducer = dvProducersByField.get(field);
assert dvProducer != null;
NumericDocValues dv = dvProducer.getNumeric(fi);
NumericDocValues dv = docValuesProducer.getNumeric(fi);
dvFields.put(field, dv);
return dv;
}
@ -481,9 +424,7 @@ public final class SegmentReader extends AtomicReader implements Accountable {
// Field was not indexed with doc values
return null;
}
DocValuesProducer dvProducer = dvProducersByField.get(field);
assert dvProducer != null;
Bits dv = dvProducer.getDocsWithField(fi);
Bits dv = docValuesProducer.getDocsWithField(fi);
dvFields.put(field, dv);
return dv;
}
@ -501,9 +442,7 @@ public final class SegmentReader extends AtomicReader implements Accountable {
BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
if (dvs == null) {
DocValuesProducer dvProducer = dvProducersByField.get(field);
assert dvProducer != null;
dvs = dvProducer.getBinary(fi);
dvs = docValuesProducer.getBinary(fi);
dvFields.put(field, dvs);
}
@ -523,9 +462,7 @@ public final class SegmentReader extends AtomicReader implements Accountable {
if (fi == null) {
return null;
}
DocValuesProducer dvProducer = dvProducersByField.get(field);
assert dvProducer != null;
SortedDocValues dv = dvProducer.getSorted(fi);
SortedDocValues dv = docValuesProducer.getSorted(fi);
dvFields.put(field, dv);
return dv;
}
@ -544,9 +481,7 @@ public final class SegmentReader extends AtomicReader implements Accountable {
if (fi == null) {
return null;
}
DocValuesProducer dvProducer = dvProducersByField.get(field);
assert dvProducer != null;
SortedNumericDocValues dv = dvProducer.getSortedNumeric(fi);
SortedNumericDocValues dv = docValuesProducer.getSortedNumeric(fi);
dvFields.put(field, dv);
return dv;
}
@ -565,9 +500,7 @@ public final class SegmentReader extends AtomicReader implements Accountable {
if (fi == null) {
return null;
}
DocValuesProducer dvProducer = dvProducersByField.get(field);
assert dvProducer != null;
SortedSetDocValues dv = dvProducer.getSortedSet(fi);
SortedSetDocValues dv = docValuesProducer.getSortedSet(fi);
dvFields.put(field, dv);
return dv;
}
@ -595,13 +528,8 @@ public final class SegmentReader extends AtomicReader implements Accountable {
public long ramBytesUsed() {
ensureOpen();
long ramBytesUsed = BASE_RAM_BYTES_USED;
ramBytesUsed += dvGens.size() * LONG_RAM_BYTES_USED;
ramBytesUsed += dvProducers.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
ramBytesUsed += dvProducersByField.size() * 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
if (dvProducers != null) {
for (DocValuesProducer producer : dvProducers) {
ramBytesUsed += producer.ramBytesUsed();
}
if (docValuesProducer != null) {
ramBytesUsed += docValuesProducer.ramBytesUsed();
}
if (core != null) {
ramBytesUsed += core.ramBytesUsed();
@ -633,10 +561,8 @@ public final class SegmentReader extends AtomicReader implements Accountable {
}
// docvalues
if (dvProducers != null) {
for (DocValuesProducer producer : dvProducers) {
producer.checkIntegrity();
}
if (docValuesProducer != null) {
docValuesProducer.checkIntegrity();
}
}
}

View File

@ -25,7 +25,7 @@ import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
@ -40,8 +40,8 @@ public class TestFileSwitchDirectory extends BaseDirectoryTestCase {
*/
public void testBasic() throws IOException {
Set<String> fileExtensions = new HashSet<>();
fileExtensions.add(Lucene40StoredFieldsWriter.FIELDS_EXTENSION);
fileExtensions.add(Lucene40StoredFieldsWriter.FIELDS_INDEX_EXTENSION);
fileExtensions.add(CompressingStoredFieldsWriter.FIELDS_EXTENSION);
fileExtensions.add(CompressingStoredFieldsWriter.FIELDS_INDEX_EXTENSION);
MockDirectoryWrapper primaryDir = new MockDirectoryWrapper(random(), new RAMDirectory());
primaryDir.setCheckIndexOnClose(false); // only part of an index

View File

@ -207,54 +207,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
}
}
static class AssertingNormsConsumer extends DocValuesConsumer {
private final DocValuesConsumer in;
private final int maxDoc;
AssertingNormsConsumer(DocValuesConsumer in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
int count = 0;
for (Number v : values) {
assert v != null;
count++;
}
assert count == maxDoc;
checkIterator(values.iterator(), maxDoc, false);
in.addNumericField(field, values);
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
throw new IllegalStateException();
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
throw new IllegalStateException();
}
@Override
public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
throw new IllegalStateException();
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
throw new IllegalStateException();
}
}
private static <T> void checkIterator(Iterator<T> iterator, long expectedSize, boolean allowNull) {
static <T> void checkIterator(Iterator<T> iterator, long expectedSize, boolean allowNull) {
for (long i = 0; i < expectedSize; i++) {
boolean hasNext = iterator.hasNext();
assert hasNext;
@ -287,8 +240,7 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
assert field.getDocValuesType() == FieldInfo.DocValuesType.NUMERIC ||
field.getNormType() == FieldInfo.DocValuesType.NUMERIC;
assert field.getDocValuesType() == FieldInfo.DocValuesType.NUMERIC;
NumericDocValues values = in.getNumeric(field);
assert values != null;
return new AssertingAtomicReader.AssertingNumericDocValues(values, maxDoc);

View File

@ -19,12 +19,13 @@ package org.apache.lucene.codecs.asserting;
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat.AssertingNormsConsumer;
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat.AssertingDocValuesProducer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
import org.apache.lucene.index.AssertingAtomicReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -35,17 +36,77 @@ public class AssertingNormsFormat extends NormsFormat {
private final NormsFormat in = new Lucene49NormsFormat();
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
DocValuesConsumer consumer = in.normsConsumer(state);
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
NormsConsumer consumer = in.normsConsumer(state);
assert consumer != null;
return new AssertingNormsConsumer(consumer, state.segmentInfo.getDocCount());
}
@Override
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
assert state.fieldInfos.hasNorms();
DocValuesProducer producer = in.normsProducer(state);
NormsProducer producer = in.normsProducer(state);
assert producer != null;
return new AssertingDocValuesProducer(producer, state.segmentInfo.getDocCount());
return new AssertingNormsProducer(producer, state.segmentInfo.getDocCount());
}
static class AssertingNormsConsumer extends NormsConsumer {
private final NormsConsumer in;
private final int maxDoc;
AssertingNormsConsumer(NormsConsumer in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
}
@Override
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
int count = 0;
for (Number v : values) {
assert v != null;
count++;
}
assert count == maxDoc;
AssertingDocValuesFormat.checkIterator(values.iterator(), maxDoc, false);
in.addNormsField(field, values);
}
@Override
public void close() throws IOException {
in.close();
}
}
static class AssertingNormsProducer extends NormsProducer {
private final NormsProducer in;
private final int maxDoc;
AssertingNormsProducer(NormsProducer in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
}
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
assert field.getNormType() == FieldInfo.DocValuesType.NUMERIC;
NumericDocValues values = in.getNorms(field);
assert values != null;
return new AssertingAtomicReader.AssertingNumericDocValues(values, maxDoc);
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public long ramBytesUsed() {
return in.ramBytesUsed();
}
@Override
public void checkIntegrity() throws IOException {
in.checkIntegrity();
}
}
}

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.index.AssertingAtomicReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
@ -33,10 +33,10 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
/**
* Just like {@link Lucene40TermVectorsFormat} but with additional asserts.
* Just like {@link Lucene42TermVectorsFormat} but with additional asserts.
*/
public class AssertingTermVectorsFormat extends TermVectorsFormat {
private final TermVectorsFormat in = new Lucene40TermVectorsFormat();
private final TermVectorsFormat in = new Lucene42TermVectorsFormat();
@Override
public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {

View File

@ -23,11 +23,11 @@ import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
/** Codec that tries to use as little ram as possible because he spent all his money on beer */
@ -37,9 +37,8 @@ public class CheapBastardCodec extends FilterCodec {
// TODO: would be better to have no terms index at all and bsearch a terms dict
private final PostingsFormat postings = new Lucene41PostingsFormat(100, 200);
// uncompressing versions, waste lots of disk but no ram
private final StoredFieldsFormat storedFields = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat termVectors = new Lucene40TermVectorsFormat();
private final StoredFieldsFormat storedFields = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat termVectors = new Lucene42TermVectorsFormat();
private final DocValuesFormat docValues = new Lucene410DocValuesFormat();
private final NormsFormat norms = new Lucene49NormsFormat();

View File

@ -20,9 +20,10 @@ package org.apache.lucene.codecs.cranky;
import java.io.IOException;
import java.util.Random;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -36,15 +37,41 @@ class CrankyNormsFormat extends NormsFormat {
}
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
if (random.nextInt(100) == 0) {
throw new IOException("Fake IOException from NormsFormat.fieldsConsumer()");
throw new IOException("Fake IOException from NormsFormat.normsConsumer()");
}
return new CrankyDocValuesFormat.CrankyDocValuesConsumer(delegate.normsConsumer(state), random);
return new CrankyNormsConsumer(delegate.normsConsumer(state), random);
}
@Override
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
public NormsProducer normsProducer(SegmentReadState state) throws IOException {
return delegate.normsProducer(state);
}
static class CrankyNormsConsumer extends NormsConsumer {
final NormsConsumer delegate;
final Random random;
CrankyNormsConsumer(NormsConsumer delegate, Random random) {
this.delegate = delegate;
this.random = random;
}
@Override
public void close() throws IOException {
delegate.close();
if (random.nextInt(100) == 0) {
throw new IOException("Fake IOException from NormsConsumer.close()");
}
}
@Override
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
if (random.nextInt(100) == 0) {
throw new IOException("Fake IOException from NormsConsumer.addNormsField()");
}
delegate.addNormsField(field, values);
}
}
}

View File

@ -6,7 +6,8 @@ import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.util.LuceneTestCase;
/*
@ -43,6 +44,8 @@ public final class Lucene40RWCodec extends Lucene40Codec {
private final DocValuesFormat docValues = new Lucene40RWDocValuesFormat();
private final NormsFormat norms = new Lucene40RWNormsFormat();
private final StoredFieldsFormat stored = new Lucene40RWStoredFieldsFormat();
private final TermVectorsFormat vectors = new Lucene40RWTermVectorsFormat();
@Override
public FieldInfosFormat fieldInfosFormat() {
@ -59,4 +62,13 @@ public final class Lucene40RWCodec extends Lucene40Codec {
return norms;
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return stored;
}
@Override
public TermVectorsFormat termVectorsFormat() {
return vectors;
}
}

View File

@ -19,7 +19,8 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
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.util.LuceneTestCase;
@ -29,14 +30,25 @@ import org.apache.lucene.util.LuceneTestCase;
public class Lucene40RWNormsFormat extends Lucene40NormsFormat {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
if (!LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
return super.normsConsumer(state);
} else {
String filename = IndexFileNames.segmentFileName(state.segmentInfo.name,
"nrm",
IndexFileNames.COMPOUND_FILE_EXTENSION);
return new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
final Lucene40DocValuesWriter impl = new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
return new NormsConsumer() {
@Override
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
impl.addNumericField(field, values);
}
@Override
public void close() throws IOException {
impl.close();
}
};
}
}
}

View File

@ -0,0 +1,41 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.StoredFieldsWriter;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.LuceneTestCase;
/**
* Simulates writing Lucene 4.0 Stored Fields Format.
*/
public class Lucene40RWStoredFieldsFormat extends Lucene40StoredFieldsFormat {
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
if (!LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
throw new UnsupportedOperationException("this codec can only be used for reading");
} else {
return new Lucene40StoredFieldsWriter(directory, si.name, context);
}
}
}

View File

@ -0,0 +1,41 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.TermVectorsWriter;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.LuceneTestCase;
/**
* Simulates writing Lucene 4.0 Stored Fields Format.
*/
public class Lucene40RWTermVectorsFormat extends Lucene40TermVectorsFormat {
@Override
public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
if (!LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
throw new UnsupportedOperationException("this codec can only be used for reading");
} else {
return new Lucene40TermVectorsWriter(directory, segmentInfo.name, context);
}
}
}

View File

@ -0,0 +1,188 @@
package org.apache.lucene.codecs.lucene40;
/**
* Copyright 2004 The Apache Software Foundation
*
* Licensed 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.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsReader.*;
/**
* Class responsible for writing stored document fields.
* <p/>
* It uses &lt;segment&gt;.fdt and &lt;segment&gt;.fdx; files.
*
* @see Lucene40StoredFieldsFormat
* @lucene.experimental
*/
public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
private final Directory directory;
private final String segment;
private IndexOutput fieldsStream;
private IndexOutput indexStream;
private final RAMOutputStream fieldsBuffer = new RAMOutputStream();
/** Sole constructor. */
public Lucene40StoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
assert directory != null;
this.directory = directory;
this.segment = segment;
boolean success = false;
try {
fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION), context);
CodecUtil.writeHeader(fieldsStream, CODEC_NAME_DAT, VERSION_CURRENT);
CodecUtil.writeHeader(indexStream, CODEC_NAME_IDX, VERSION_CURRENT);
assert HEADER_LENGTH_DAT == fieldsStream.getFilePointer();
assert HEADER_LENGTH_IDX == indexStream.getFilePointer();
success = true;
} finally {
if (!success) {
abort();
}
}
}
int numStoredFields;
// Writes the contents of buffer into the fields stream
// and adds a new entry for this document into the index
// stream. This assumes the buffer was already written
// in the correct fields format.
@Override
public void startDocument() throws IOException {
indexStream.writeLong(fieldsStream.getFilePointer());
}
@Override
public void finishDocument() throws IOException {
fieldsStream.writeVInt(numStoredFields);
fieldsBuffer.writeTo(fieldsStream);
fieldsBuffer.reset();
numStoredFields = 0;
}
@Override
public void close() throws IOException {
try {
IOUtils.close(fieldsStream, indexStream);
} finally {
fieldsStream = indexStream = null;
}
}
@Override
public void abort() {
try {
close();
} catch (Throwable ignored) {}
IOUtils.deleteFilesIgnoringExceptions(directory,
IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION),
IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
}
@Override
public void writeField(FieldInfo info, StorableField field) throws IOException {
numStoredFields++;
fieldsBuffer.writeVInt(info.number);
int bits = 0;
final BytesRef bytes;
final String string;
// TODO: maybe a field should serialize itself?
// this way we don't bake into indexer all these
// specific encodings for different fields? and apps
// can customize...
Number number = field.numericValue();
if (number != null) {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
bits |= FIELD_IS_NUMERIC_INT;
} else if (number instanceof Long) {
bits |= FIELD_IS_NUMERIC_LONG;
} else if (number instanceof Float) {
bits |= FIELD_IS_NUMERIC_FLOAT;
} else if (number instanceof Double) {
bits |= FIELD_IS_NUMERIC_DOUBLE;
} else {
throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
}
string = null;
bytes = null;
} else {
bytes = field.binaryValue();
if (bytes != null) {
bits |= FIELD_IS_BINARY;
string = null;
} else {
string = field.stringValue();
if (string == null) {
throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
}
}
}
fieldsBuffer.writeByte((byte) bits);
if (bytes != null) {
fieldsBuffer.writeVInt(bytes.length);
fieldsBuffer.writeBytes(bytes.bytes, bytes.offset, bytes.length);
} else if (string != null) {
fieldsBuffer.writeString(field.stringValue());
} else {
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
fieldsBuffer.writeInt(number.intValue());
} else if (number instanceof Long) {
fieldsBuffer.writeLong(number.longValue());
} else if (number instanceof Float) {
fieldsBuffer.writeInt(Float.floatToIntBits(number.floatValue()));
} else if (number instanceof Double) {
fieldsBuffer.writeLong(Double.doubleToLongBits(number.doubleValue()));
} else {
throw new AssertionError("Cannot get here");
}
}
}
@Override
public void finish(FieldInfos fis, int numDocs) {
long indexFP = indexStream.getFilePointer();
if (HEADER_LENGTH_IDX+((long) numDocs)*8 != indexFP)
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
// we detect that the bug has struck, here, and
// throw an exception to prevent the corruption from
// entering the index. See LUCENE-1282 for
// details.
throw new RuntimeException("fdx size mismatch: docCount is " + numDocs + " but fdx file size is " + indexFP + " (wrote numDocs=" + ((indexFP-HEADER_LENGTH_IDX)/8.0) + " file=" + indexStream.toString() + "; now aborting this merge to prevent index corruption");
}
}

View File

@ -20,21 +20,15 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
@ -286,144 +280,6 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_FIELDS_EXTENSION));
}
/**
* Do a bulk copy of numDocs documents from reader to our
* streams. This is used to expedite merging, if the
* field numbers are congruent.
*/
private void addRawDocuments(Lucene40TermVectorsReader reader, int[] tvdLengths, int[] tvfLengths, int numDocs) throws IOException {
long tvdPosition = tvd.getFilePointer();
long tvfPosition = tvf.getFilePointer();
long tvdStart = tvdPosition;
long tvfStart = tvfPosition;
for(int i=0;i<numDocs;i++) {
tvx.writeLong(tvdPosition);
tvdPosition += tvdLengths[i];
tvx.writeLong(tvfPosition);
tvfPosition += tvfLengths[i];
}
tvd.copyBytes(reader.getTvdStream(), tvdPosition-tvdStart);
tvf.copyBytes(reader.getTvfStream(), tvfPosition-tvfStart);
assert tvd.getFilePointer() == tvdPosition;
assert tvf.getFilePointer() == tvfPosition;
}
@Override
public final int merge(MergeState mergeState) throws IOException {
// Used for bulk-reading raw bytes for term vectors
int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
int rawDocLengths2[] = new int[MAX_RAW_MERGE_DOCS];
int idx = 0;
int numDocs = 0;
for (int i = 0; i < mergeState.readers.size(); i++) {
final AtomicReader reader = mergeState.readers.get(i);
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
Lucene40TermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) {
TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();
if (vectorsReader != null && vectorsReader instanceof Lucene40TermVectorsReader) {
matchingVectorsReader = (Lucene40TermVectorsReader) vectorsReader;
}
}
if (reader.getLiveDocs() != null) {
numDocs += copyVectorsWithDeletions(mergeState, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
} else {
numDocs += copyVectorsNoDeletions(mergeState, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
}
}
finish(mergeState.fieldInfos, numDocs);
return numDocs;
}
/** Maximum number of contiguous documents to bulk-copy
when merging term vectors */
private final static int MAX_RAW_MERGE_DOCS = 4192;
private int copyVectorsWithDeletions(MergeState mergeState,
final Lucene40TermVectorsReader matchingVectorsReader,
final AtomicReader reader,
int rawDocLengths[],
int rawDocLengths2[])
throws IOException {
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
int totalNumDocs = 0;
if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
for (int docNum = 0; docNum < maxDoc;) {
if (!liveDocs.get(docNum)) {
// skip deleted docs
++docNum;
continue;
}
// We can optimize this case (doing a bulk byte copy) since the field
// numbers are identical
int start = docNum, numDocs = 0;
do {
docNum++;
numDocs++;
if (docNum >= maxDoc) break;
if (!liveDocs.get(docNum)) {
docNum++;
break;
}
} while(numDocs < MAX_RAW_MERGE_DOCS);
matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
totalNumDocs += numDocs;
mergeState.checkAbort.work(300 * numDocs);
}
} else {
for (int docNum = 0; docNum < maxDoc; docNum++) {
if (!liveDocs.get(docNum)) {
// skip deleted docs
continue;
}
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.getTermVectors(docNum);
addAllDocVectors(vectors, mergeState);
totalNumDocs++;
mergeState.checkAbort.work(300);
}
}
return totalNumDocs;
}
private int copyVectorsNoDeletions(MergeState mergeState,
final Lucene40TermVectorsReader matchingVectorsReader,
final AtomicReader reader,
int rawDocLengths[],
int rawDocLengths2[])
throws IOException {
final int maxDoc = reader.maxDoc();
if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
int docCount = 0;
while (docCount < maxDoc) {
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, docCount, len);
addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, len);
docCount += len;
mergeState.checkAbort.work(300 * len);
}
} else {
for (int docNum = 0; docNum < maxDoc; docNum++) {
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.getTermVectors(docNum);
addAllDocVectors(vectors, mergeState);
mergeState.checkAbort.work(300);
}
}
return maxDoc;
}
@Override
public void finish(FieldInfos fis, int numDocs) {
long indexFP = tvx.getFilePointer();

View File

@ -7,10 +7,12 @@ import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosWriter;
import org.apache.lucene.codecs.lucene40.Lucene40RWDocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40RWNormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40RWTermVectorsFormat;
import org.apache.lucene.util.LuceneTestCase;
/*
@ -49,6 +51,7 @@ public class Lucene41RWCodec extends Lucene41Codec {
private final DocValuesFormat docValues = new Lucene40RWDocValuesFormat();
private final NormsFormat norms = new Lucene40RWNormsFormat();
private final TermVectorsFormat vectors = new Lucene40RWTermVectorsFormat();
@Override
public FieldInfosFormat fieldInfosFormat() {
@ -70,4 +73,8 @@ public class Lucene41RWCodec extends Lucene41Codec {
return norms;
}
@Override
public TermVectorsFormat termVectorsFormat() {
return vectors;
}
}

View File

@ -22,12 +22,11 @@ import java.util.HashMap;
import java.util.HashSet;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
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.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MathUtil;
import org.apache.lucene.util.packed.BlockPackedWriter;
@ -39,7 +38,7 @@ import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.VERSIO
/**
* Writer for {@link Lucene42NormsFormat}
*/
class Lucene42NormsConsumer extends DocValuesConsumer {
class Lucene42NormsConsumer extends NormsConsumer {
static final byte NUMBER = 0;
static final int BLOCK_SIZE = 4096;
@ -73,7 +72,7 @@ class Lucene42NormsConsumer extends DocValuesConsumer {
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(NUMBER);
meta.writeLong(data.getFilePointer());
@ -194,24 +193,4 @@ class Lucene42NormsConsumer extends DocValuesConsumer {
meta = data = null;
}
}
@Override
public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
throw new UnsupportedOperationException();
}
}

View File

@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene42;
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.LuceneTestCase;
@ -29,7 +29,7 @@ import org.apache.lucene.util.LuceneTestCase;
public class Lucene42RWNormsFormat extends Lucene42NormsFormat {
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
if (LuceneTestCase.OLD_FORMAT_IMPERSONATION_IS_ACTIVE) {
return new Lucene42NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
} else {