mirror of https://github.com/apache/lucene.git
LUCENE-5611: simplify the default indexing chain
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1591025 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
f5de5d01e8
|
@ -77,6 +77,13 @@ Other
|
|||
|
||||
======================= Lucene 4.9.0 =======================
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-5611: Changing the term vector options for multiple field
|
||||
instances by the same name in one document is not longer accepted;
|
||||
IndexWriter will now throw IllegalArgumentException. (Robert Muir,
|
||||
Mike McCandless)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-5610: Add Terms.getMin and Terms.getMax to get the lowest and
|
||||
|
@ -93,6 +100,13 @@ API Changes
|
|||
* LUCENE-5621: Deprecate IndexOutput.flush: this is not used by Lucene.
|
||||
(Robert Muir)
|
||||
|
||||
* LUCENE-5611: Simplified Lucene's default indexing chain / APIs.
|
||||
AttributeSource/TokenStream.getAttribute now returns null if the
|
||||
attribute is not present (previously it threw
|
||||
IllegalArgumentException). StoredFieldsWriter.startDocument no
|
||||
longer receives the number of fields that will be added (Robert
|
||||
Muir, Mike McCandless)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-5603: hunspell stemmer more efficiently strips prefixes
|
||||
|
|
|
@ -98,13 +98,12 @@ public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
|
|||
@Override
|
||||
public void visitDocument(int n, StoredFieldVisitor visitor) throws IOException {
|
||||
in.seek(offsets[n]);
|
||||
readLine();
|
||||
assert StringHelper.startsWith(scratch, NUM);
|
||||
int numFields = parseIntAt(NUM.length);
|
||||
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
while (true) {
|
||||
readLine();
|
||||
assert StringHelper.startsWith(scratch, FIELD);
|
||||
if (StringHelper.startsWith(scratch, FIELD) == false) {
|
||||
break;
|
||||
}
|
||||
int fieldNumber = parseIntAt(FIELD.length);
|
||||
FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
|
||||
readLine();
|
||||
|
|
|
@ -53,7 +53,6 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
|
|||
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
final static BytesRef DOC = new BytesRef("doc ");
|
||||
final static BytesRef NUM = new BytesRef(" numfields ");
|
||||
final static BytesRef FIELD = new BytesRef(" field ");
|
||||
final static BytesRef NAME = new BytesRef(" name ");
|
||||
final static BytesRef TYPE = new BytesRef(" type ");
|
||||
|
@ -76,15 +75,11 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void startDocument(int numStoredFields) throws IOException {
|
||||
public void startDocument() throws IOException {
|
||||
write(DOC);
|
||||
write(Integer.toString(numDocsWritten));
|
||||
newLine();
|
||||
|
||||
write(NUM);
|
||||
write(Integer.toString(numStoredFields));
|
||||
newLine();
|
||||
|
||||
numDocsWritten++;
|
||||
}
|
||||
|
||||
|
|
|
@ -171,8 +171,9 @@ public abstract class TokenStream extends AttributeSource implements Closeable {
|
|||
*/
|
||||
public void end() throws IOException {
|
||||
clearAttributes(); // LUCENE-3849: don't consume dirty atts
|
||||
if (hasAttribute(PositionIncrementAttribute.class)) {
|
||||
getAttribute(PositionIncrementAttribute.class).setPositionIncrement(0);
|
||||
PositionIncrementAttribute posIncAtt = getAttribute(PositionIncrementAttribute.class);
|
||||
if (posIncAtt != null) {
|
||||
posIncAtt.setPositionIncrement(0);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -31,8 +31,8 @@ import org.apache.lucene.index.AtomicReader;
|
|||
* Codec API for writing stored fields:
|
||||
* <p>
|
||||
* <ol>
|
||||
* <li>For every document, {@link #startDocument(int)} is called,
|
||||
* informing the Codec how many fields will be written.
|
||||
* <li>For every document, {@link #startDocument()} is called,
|
||||
* informing the Codec that a new document has started.
|
||||
* <li>{@link #writeField(FieldInfo, StorableField)} is called for
|
||||
* each field in the document.
|
||||
* <li>After all documents have been written, {@link #finish(FieldInfos, int)}
|
||||
|
@ -51,10 +51,9 @@ public abstract class StoredFieldsWriter implements Closeable {
|
|||
|
||||
/** Called before writing the stored fields of the document.
|
||||
* {@link #writeField(FieldInfo, StorableField)} will be called
|
||||
* <code>numStoredFields</code> times. Note that this is
|
||||
* called even if the document has no stored fields, in
|
||||
* this case <code>numStoredFields</code> will be zero. */
|
||||
public abstract void startDocument(int numStoredFields) throws IOException;
|
||||
* for each stored field. Note that this is
|
||||
* called even if the document has no stored fields. */
|
||||
public abstract void startDocument() throws IOException;
|
||||
|
||||
/** Called when a document and all its fields have been added. */
|
||||
public void finishDocument() throws IOException {}
|
||||
|
@ -69,14 +68,14 @@ public abstract class StoredFieldsWriter implements Closeable {
|
|||
/** Called before {@link #close()}, passing in the number
|
||||
* of documents that were written. Note that this is
|
||||
* intentionally redundant (equivalent to the number of
|
||||
* calls to {@link #startDocument(int)}, but a Codec should
|
||||
* calls to {@link #startDocument()}, but a Codec should
|
||||
* check that this is the case to detect the JRE bug described
|
||||
* in LUCENE-1282. */
|
||||
public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
|
||||
|
||||
/** Merges in the stored fields from the readers in
|
||||
* <code>mergeState</code>. The default implementation skips
|
||||
* over deleted documents, and uses {@link #startDocument(int)},
|
||||
* over deleted documents, and uses {@link #startDocument()},
|
||||
* {@link #writeField(FieldInfo, StorableField)}, and {@link #finish(FieldInfos, int)},
|
||||
* returning the number of documents that were written.
|
||||
* Implementations can override this method for more sophisticated
|
||||
|
@ -109,12 +108,7 @@ public abstract class StoredFieldsWriter implements Closeable {
|
|||
|
||||
/** sugar method for startDocument() + writeField() for every stored field in the document */
|
||||
protected final void addDocument(Iterable<? extends StorableField> doc, FieldInfos fieldInfos) throws IOException {
|
||||
int storedCount = 0;
|
||||
for (StorableField field : doc) {
|
||||
storedCount++;
|
||||
}
|
||||
|
||||
startDocument(storedCount);
|
||||
startDocument();
|
||||
|
||||
for (StorableField field : doc) {
|
||||
writeField(fieldInfos.fieldInfo(field.name()), field);
|
||||
|
|
|
@ -145,20 +145,23 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
private int numStoredFieldsInDoc;
|
||||
|
||||
@Override
|
||||
public void startDocument(int numStoredFields) throws IOException {
|
||||
public void startDocument() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDocument() throws IOException {
|
||||
if (numBufferedDocs == this.numStoredFields.length) {
|
||||
final int newLength = ArrayUtil.oversize(numBufferedDocs + 1, 4);
|
||||
this.numStoredFields = Arrays.copyOf(this.numStoredFields, newLength);
|
||||
endOffsets = Arrays.copyOf(endOffsets, newLength);
|
||||
}
|
||||
this.numStoredFields[numBufferedDocs] = numStoredFields;
|
||||
this.numStoredFields[numBufferedDocs] = numStoredFieldsInDoc;
|
||||
numStoredFieldsInDoc = 0;
|
||||
endOffsets[numBufferedDocs] = bufferedDocs.length;
|
||||
++numBufferedDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDocument() throws IOException {
|
||||
endOffsets[numBufferedDocs - 1] = bufferedDocs.length;
|
||||
if (triggerFlush()) {
|
||||
flush();
|
||||
}
|
||||
|
@ -242,6 +245,9 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
|
|||
@Override
|
||||
public void writeField(FieldInfo info, StorableField field)
|
||||
throws IOException {
|
||||
|
||||
++numStoredFieldsInDoc;
|
||||
|
||||
int bits = 0;
|
||||
final BytesRef bytes;
|
||||
final String string;
|
||||
|
@ -391,8 +397,9 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
|
|||
// copy non-deleted docs
|
||||
for (; docID < it.docBase + it.chunkDocs; docID = nextLiveDoc(docID + 1, liveDocs, maxDoc)) {
|
||||
final int diff = docID - it.docBase;
|
||||
startDocument(it.numStoredFields[diff]);
|
||||
startDocument();
|
||||
bufferedDocs.writeBytes(it.bytes.bytes, it.bytes.offset + startOffsets[diff], it.lengths[diff]);
|
||||
numStoredFieldsInDoc = it.numStoredFields[diff];
|
||||
finishDocument();
|
||||
++docCount;
|
||||
mergeState.checkAbort.work(300);
|
||||
|
|
|
@ -36,6 +36,7 @@ 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;
|
||||
|
@ -85,6 +86,7 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
|||
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 {
|
||||
|
@ -109,14 +111,23 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
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(int numStoredFields) throws IOException {
|
||||
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
|
||||
|
@ -140,7 +151,9 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
|||
|
||||
@Override
|
||||
public void writeField(FieldInfo info, StorableField field) throws IOException {
|
||||
fieldsStream.writeVInt(info.number);
|
||||
numStoredFields++;
|
||||
|
||||
fieldsBuffer.writeVInt(info.number);
|
||||
int bits = 0;
|
||||
final BytesRef bytes;
|
||||
final String string;
|
||||
|
@ -177,22 +190,22 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
fieldsStream.writeByte((byte) bits);
|
||||
fieldsBuffer.writeByte((byte) bits);
|
||||
|
||||
if (bytes != null) {
|
||||
fieldsStream.writeVInt(bytes.length);
|
||||
fieldsStream.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
fieldsBuffer.writeVInt(bytes.length);
|
||||
fieldsBuffer.writeBytes(bytes.bytes, bytes.offset, bytes.length);
|
||||
} else if (string != null) {
|
||||
fieldsStream.writeString(field.stringValue());
|
||||
fieldsBuffer.writeString(field.stringValue());
|
||||
} else {
|
||||
if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
|
||||
fieldsStream.writeInt(number.intValue());
|
||||
fieldsBuffer.writeInt(number.intValue());
|
||||
} else if (number instanceof Long) {
|
||||
fieldsStream.writeLong(number.longValue());
|
||||
fieldsBuffer.writeLong(number.longValue());
|
||||
} else if (number instanceof Float) {
|
||||
fieldsStream.writeInt(Float.floatToIntBits(number.floatValue()));
|
||||
fieldsBuffer.writeInt(Float.floatToIntBits(number.floatValue()));
|
||||
} else if (number instanceof Double) {
|
||||
fieldsStream.writeLong(Double.doubleToLongBits(number.doubleValue()));
|
||||
fieldsBuffer.writeLong(Double.doubleToLongBits(number.doubleValue()));
|
||||
} else {
|
||||
throw new AssertionError("Cannot get here");
|
||||
}
|
||||
|
|
|
@ -88,27 +88,51 @@ final class Lucene41SkipWriter extends MultiLevelSkipListWriter {
|
|||
this.fieldHasOffsets = fieldHasOffsets;
|
||||
this.fieldHasPayloads = fieldHasPayloads;
|
||||
}
|
||||
|
||||
// tricky: we only skip data for blocks (terms with more than 128 docs), but re-init'ing the skipper
|
||||
// is pretty slow for rare terms in large segments as we have to fill O(log #docs in segment) of junk.
|
||||
// this is the vast majority of terms (worst case: ID field or similar). so in resetSkip() we save
|
||||
// away the previous pointers, and lazy-init only if we need to buffer skip data for the term.
|
||||
private boolean initialized;
|
||||
long lastDocFP;
|
||||
long lastPosFP;
|
||||
long lastPayFP;
|
||||
|
||||
@Override
|
||||
public void resetSkip() {
|
||||
super.resetSkip();
|
||||
Arrays.fill(lastSkipDoc, 0);
|
||||
Arrays.fill(lastSkipDocPointer, docOut.getFilePointer());
|
||||
lastDocFP = docOut.getFilePointer();
|
||||
if (fieldHasPositions) {
|
||||
Arrays.fill(lastSkipPosPointer, posOut.getFilePointer());
|
||||
if (fieldHasPayloads) {
|
||||
Arrays.fill(lastPayloadByteUpto, 0);
|
||||
}
|
||||
lastPosFP = posOut.getFilePointer();
|
||||
if (fieldHasOffsets || fieldHasPayloads) {
|
||||
Arrays.fill(lastSkipPayPointer, payOut.getFilePointer());
|
||||
lastPayFP = payOut.getFilePointer();
|
||||
}
|
||||
}
|
||||
initialized = false;
|
||||
}
|
||||
|
||||
public void initSkip() {
|
||||
if (!initialized) {
|
||||
super.resetSkip();
|
||||
Arrays.fill(lastSkipDoc, 0);
|
||||
Arrays.fill(lastSkipDocPointer, lastDocFP);
|
||||
if (fieldHasPositions) {
|
||||
Arrays.fill(lastSkipPosPointer, lastPosFP);
|
||||
if (fieldHasPayloads) {
|
||||
Arrays.fill(lastPayloadByteUpto, 0);
|
||||
}
|
||||
if (fieldHasOffsets || fieldHasPayloads) {
|
||||
Arrays.fill(lastSkipPayPointer, lastPayFP);
|
||||
}
|
||||
}
|
||||
initialized = true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the values for the current skip data.
|
||||
*/
|
||||
public void bufferSkip(int doc, int numDocs, long posFP, long payFP, int posBufferUpto, int payloadByteUpto) throws IOException {
|
||||
initSkip();
|
||||
this.curDoc = doc;
|
||||
this.curDocPointer = docOut.getFilePointer();
|
||||
this.curPosPointer = posFP;
|
||||
|
|
|
@ -244,11 +244,6 @@ public class Field implements IndexableField, StorableField {
|
|||
throw new IllegalArgumentException("it doesn't make sense to have a field that "
|
||||
+ "is neither indexed nor stored");
|
||||
}
|
||||
if (!type.indexed() && (type.storeTermVectors())) {
|
||||
throw new IllegalArgumentException("cannot store term vector information "
|
||||
+ "for a field that is not indexed");
|
||||
}
|
||||
|
||||
this.type = type;
|
||||
this.name = name;
|
||||
this.fieldsData = value;
|
||||
|
|
|
@ -382,9 +382,9 @@ public class FieldType implements IndexableFieldType {
|
|||
}
|
||||
if (storeTermVectorPositions()) {
|
||||
result.append(",termVectorPosition");
|
||||
if (storeTermVectorPayloads()) {
|
||||
result.append(",termVectorPayloads");
|
||||
}
|
||||
}
|
||||
if (storeTermVectorPayloads()) {
|
||||
result.append(",termVectorPayloads");
|
||||
}
|
||||
if (omitNorms()) {
|
||||
result.append(",omitNorms");
|
||||
|
|
|
@ -121,10 +121,6 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
}
|
||||
|
||||
// iterates over the values we have in ram
|
||||
private class BytesIterator implements Iterator<BytesRef> {
|
||||
final BytesRef value = new BytesRef();
|
||||
|
|
|
@ -0,0 +1,653 @@
|
|||
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.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** Default general purpose indexing chain, which handles
|
||||
* indexing all types of fields. */
|
||||
final class DefaultIndexingChain extends DocConsumer {
|
||||
final Counter bytesUsed;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
final FieldInfos.Builder fieldInfos;
|
||||
|
||||
// Writes postings and term vectors:
|
||||
final TermsHash termsHash;
|
||||
|
||||
final StoredFieldsWriter storedFieldsWriter;
|
||||
private int lastStoredDocID;
|
||||
|
||||
// NOTE: I tried using Hash Map<String,PerField>
|
||||
// but it was ~2% slower on Wiki and Geonames with Java
|
||||
// 1.7.0_25:
|
||||
private PerField[] fieldHash = new PerField[2];
|
||||
private int hashMask = 1;
|
||||
|
||||
private int totalFieldCount;
|
||||
private long nextFieldGen;
|
||||
|
||||
// Lazy init:
|
||||
private NumericDocValuesWriter norms;
|
||||
|
||||
// Holds fields seen in each document
|
||||
private PerField[] fields = new PerField[1];
|
||||
|
||||
public DefaultIndexingChain(DocumentsWriterPerThread docWriter) throws IOException {
|
||||
this.docWriter = docWriter;
|
||||
this.fieldInfos = docWriter.getFieldInfosBuilder();
|
||||
this.docState = docWriter.docState;
|
||||
this.bytesUsed = docWriter.bytesUsed;
|
||||
|
||||
// Writes stored fields:
|
||||
storedFieldsWriter = docWriter.codec.storedFieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegmentInfo(), IOContext.DEFAULT);
|
||||
|
||||
TermsHash termVectorsWriter = new TermVectorsConsumer(docWriter);
|
||||
termsHash = new FreqProxTermsWriter(docWriter, termVectorsWriter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state) throws IOException {
|
||||
|
||||
// NOTE: caller (DocumentsWriterPerThread) handles
|
||||
// aborting on any exception from this method
|
||||
|
||||
int numDocs = state.segmentInfo.getDocCount();
|
||||
writeNorms(state);
|
||||
writeDocValues(state);
|
||||
|
||||
fillStoredFields(numDocs);
|
||||
storedFieldsWriter.finish(state.fieldInfos, numDocs);
|
||||
storedFieldsWriter.close();
|
||||
|
||||
Map<String,TermsHashPerField> fieldsToFlush = new HashMap<>();
|
||||
for (int i=0;i<fieldHash.length;i++) {
|
||||
PerField perField = fieldHash[i];
|
||||
while (perField != null) {
|
||||
if (perField.invertState != null) {
|
||||
fieldsToFlush.put(perField.fieldInfo.name, perField.termsHashPerField);
|
||||
}
|
||||
perField = perField.next;
|
||||
}
|
||||
}
|
||||
|
||||
termsHash.flush(fieldsToFlush, state);
|
||||
|
||||
// Important to save after asking consumer to flush so
|
||||
// consumer can alter the FieldInfo* if necessary. EG,
|
||||
// FreqProxTermsWriter does this with
|
||||
// FieldInfo.storePayload.
|
||||
FieldInfosWriter infosWriter = docWriter.codec.fieldInfosFormat().getFieldInfosWriter();
|
||||
infosWriter.write(state.directory, state.segmentInfo.name, "", state.fieldInfos, IOContext.DEFAULT);
|
||||
}
|
||||
|
||||
/** Writes all buffered doc values (called from {@link #flush}). */
|
||||
private void writeDocValues(SegmentWriteState state) throws IOException {
|
||||
int docCount = state.segmentInfo.getDocCount();
|
||||
DocValuesConsumer dvConsumer = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
for (int i=0;i<fieldHash.length;i++) {
|
||||
PerField perField = fieldHash[i];
|
||||
while (perField != null) {
|
||||
if (perField.docValuesWriter != null) {
|
||||
if (dvConsumer == null) {
|
||||
// lazy init
|
||||
DocValuesFormat fmt = state.segmentInfo.getCodec().docValuesFormat();
|
||||
dvConsumer = fmt.fieldsConsumer(state);
|
||||
}
|
||||
|
||||
perField.docValuesWriter.finish(docCount);
|
||||
perField.docValuesWriter.flush(state, dvConsumer);
|
||||
perField.docValuesWriter = null;
|
||||
}
|
||||
perField = perField.next;
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: catch missing DV fields here? else we have
|
||||
// null/"" depending on how docs landed in segments?
|
||||
// but we can't detect all cases, and we should leave
|
||||
// this behavior undefined. dv is not "schemaless": its column-stride.
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(dvConsumer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(dvConsumer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Catch up for all docs before us that had no stored
|
||||
* fields, or hit non-aborting exceptions before writing
|
||||
* stored fields. */
|
||||
private void fillStoredFields(int docID) throws IOException {
|
||||
while (lastStoredDocID < docID) {
|
||||
storedFieldsWriter.startDocument();
|
||||
lastStoredDocID++;
|
||||
storedFieldsWriter.finishDocument();
|
||||
}
|
||||
}
|
||||
|
||||
private void writeNorms(SegmentWriteState state) throws IOException {
|
||||
boolean success = false;
|
||||
DocValuesConsumer normsConsumer = null;
|
||||
try {
|
||||
if (state.fieldInfos.hasNorms()) {
|
||||
NormsFormat normsFormat = state.segmentInfo.getCodec().normsFormat();
|
||||
assert normsFormat != null;
|
||||
normsConsumer = normsFormat.normsConsumer(state);
|
||||
|
||||
for (FieldInfo fi : state.fieldInfos) {
|
||||
PerField perField = getPerField(fi.name);
|
||||
assert perField != null;
|
||||
|
||||
// we must check the final value of omitNorms for the fieldinfo: it could have
|
||||
// changed for this field since the first time we added it.
|
||||
if (fi.omitsNorms() == false) {
|
||||
if (perField.norms != null) {
|
||||
perField.norms.finish(state.segmentInfo.getDocCount());
|
||||
perField.norms.flush(state, normsConsumer);
|
||||
assert fi.getNormType() == DocValuesType.NUMERIC;
|
||||
} else if (fi.isIndexed()) {
|
||||
assert fi.getNormType() == null: "got " + fi.getNormType() + "; field=" + fi.name;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(normsConsumer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(normsConsumer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
// E.g. close any open files in the stored fields writer:
|
||||
storedFieldsWriter.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
|
||||
try {
|
||||
// E.g. close any open files in the term vectors writer:
|
||||
termsHash.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
|
||||
Arrays.fill(fieldHash, null);
|
||||
}
|
||||
|
||||
private void rehash() {
|
||||
int newHashSize = (fieldHash.length*2);
|
||||
assert newHashSize > fieldHash.length;
|
||||
|
||||
PerField newHashArray[] = new PerField[newHashSize];
|
||||
|
||||
// Rehash
|
||||
int newHashMask = newHashSize-1;
|
||||
for(int j=0;j<fieldHash.length;j++) {
|
||||
PerField fp0 = fieldHash[j];
|
||||
while(fp0 != null) {
|
||||
final int hashPos2 = fp0.fieldInfo.name.hashCode() & newHashMask;
|
||||
PerField nextFP0 = fp0.next;
|
||||
fp0.next = newHashArray[hashPos2];
|
||||
newHashArray[hashPos2] = fp0;
|
||||
fp0 = nextFP0;
|
||||
}
|
||||
}
|
||||
|
||||
fieldHash = newHashArray;
|
||||
hashMask = newHashMask;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processDocument() throws IOException {
|
||||
|
||||
// How many indexed field names we've seen (collapses
|
||||
// multiple field instances by the same name):
|
||||
int fieldCount = 0;
|
||||
|
||||
long fieldGen = nextFieldGen++;
|
||||
|
||||
// NOTE: we need two passes here, in case there are
|
||||
// multi-valued fields, because we must process all
|
||||
// instances of a given field at once, since the
|
||||
// analyzer is free to reuse TokenStream across fields
|
||||
// (i.e., we cannot have more than one TokenStream
|
||||
// running "at once"):
|
||||
|
||||
termsHash.startDocument();
|
||||
|
||||
// Invert indexed fields:
|
||||
for (IndexableField field : docState.doc.indexableFields()) {
|
||||
IndexableFieldType fieldType = field.fieldType();
|
||||
PerField fp = getOrAddField(field.name(), fieldType, true);
|
||||
boolean first = fp.fieldGen != fieldGen;
|
||||
fp.invert(field, first);
|
||||
|
||||
if (first) {
|
||||
fields[fieldCount++] = fp;
|
||||
fp.fieldGen = fieldGen;
|
||||
}
|
||||
}
|
||||
|
||||
// Finish each field name seen in the document:
|
||||
for (int i=0;i<fieldCount;i++) {
|
||||
fields[i].finish();
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
termsHash.finishDocument();
|
||||
success = true;
|
||||
} finally {
|
||||
if (success == false) {
|
||||
// Must abort, on the possibility that on-disk term
|
||||
// vectors are now corrupt:
|
||||
docWriter.setAborting();
|
||||
}
|
||||
}
|
||||
|
||||
// Add stored fields:
|
||||
fillStoredFields(docState.docID);
|
||||
storedFieldsWriter.startDocument();
|
||||
lastStoredDocID++;
|
||||
|
||||
success = false;
|
||||
try {
|
||||
for (StorableField field : docState.doc.storableFields()) {
|
||||
final String fieldName = field.name();
|
||||
IndexableFieldType fieldType = field.fieldType();
|
||||
|
||||
verifyFieldType(fieldName, fieldType);
|
||||
|
||||
PerField fp = getOrAddField(fieldName, fieldType, false);
|
||||
if (fieldType.stored()) {
|
||||
storedFieldsWriter.writeField(fp.fieldInfo, field);
|
||||
}
|
||||
|
||||
DocValuesType dvType = fieldType.docValueType();
|
||||
if (dvType != null) {
|
||||
indexDocValue(fp, dvType, field);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success == false) {
|
||||
// We must abort, on the possibility that the
|
||||
// stored fields file is now corrupt:
|
||||
docWriter.setAborting();
|
||||
} else {
|
||||
storedFieldsWriter.finishDocument();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyFieldType(String name, IndexableFieldType ft) {
|
||||
if (ft.indexed() == false) {
|
||||
if (ft.storeTermVectors()) {
|
||||
throw new IllegalArgumentException("cannot store term vectors "
|
||||
+ "for a field that is not indexed (field=\"" + name + "\")");
|
||||
}
|
||||
if (ft.storeTermVectorPositions()) {
|
||||
throw new IllegalArgumentException("cannot store term vector positions "
|
||||
+ "for a field that is not indexed (field=\"" + name + "\")");
|
||||
}
|
||||
if (ft.storeTermVectorOffsets()) {
|
||||
throw new IllegalArgumentException("cannot store term vector offsets "
|
||||
+ "for a field that is not indexed (field=\"" + name + "\")");
|
||||
}
|
||||
if (ft.storeTermVectorPayloads()) {
|
||||
throw new IllegalArgumentException("cannot store term vector payloads "
|
||||
+ "for a field that is not indexed (field=\"" + name + "\")");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Called from processDocument to index one field's doc
|
||||
* value */
|
||||
private void indexDocValue(PerField fp, DocValuesType dvType, StorableField field) throws IOException {
|
||||
|
||||
boolean hasDocValues = fp.fieldInfo.hasDocValues();
|
||||
|
||||
// This will throw an exc if the caller tried to
|
||||
// change the DV type for the field:
|
||||
fp.fieldInfo.setDocValuesType(dvType);
|
||||
if (hasDocValues == false) {
|
||||
fieldInfos.globalFieldNumbers.setDocValuesType(fp.fieldInfo.number, fp.fieldInfo.name, dvType);
|
||||
}
|
||||
|
||||
int docID = docState.docID;
|
||||
|
||||
switch(dvType) {
|
||||
|
||||
case NUMERIC:
|
||||
if (fp.docValuesWriter == null) {
|
||||
fp.docValuesWriter = new NumericDocValuesWriter(fp.fieldInfo, bytesUsed, true);
|
||||
}
|
||||
((NumericDocValuesWriter) fp.docValuesWriter).addValue(docID, field.numericValue().longValue());
|
||||
break;
|
||||
|
||||
case BINARY:
|
||||
if (fp.docValuesWriter == null) {
|
||||
fp.docValuesWriter = new BinaryDocValuesWriter(fp.fieldInfo, bytesUsed);
|
||||
}
|
||||
((BinaryDocValuesWriter) fp.docValuesWriter).addValue(docID, field.binaryValue());
|
||||
break;
|
||||
|
||||
case SORTED:
|
||||
if (fp.docValuesWriter == null) {
|
||||
fp.docValuesWriter = new SortedDocValuesWriter(fp.fieldInfo, bytesUsed);
|
||||
}
|
||||
((SortedDocValuesWriter) fp.docValuesWriter).addValue(docID, field.binaryValue());
|
||||
break;
|
||||
|
||||
case SORTED_SET:
|
||||
if (fp.docValuesWriter == null) {
|
||||
fp.docValuesWriter = new SortedSetDocValuesWriter(fp.fieldInfo, bytesUsed);
|
||||
}
|
||||
((SortedSetDocValuesWriter) fp.docValuesWriter).addValue(docID, field.binaryValue());
|
||||
break;
|
||||
|
||||
default:
|
||||
throw new AssertionError("unrecognized DocValues.Type: " + dvType);
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns a previously created {@link PerField}, or null
|
||||
* if this field name wasn't seen yet. */
|
||||
private PerField getPerField(String name) {
|
||||
final int hashPos = name.hashCode() & hashMask;
|
||||
PerField fp = fieldHash[hashPos];
|
||||
while (fp != null && !fp.fieldInfo.name.equals(name)) {
|
||||
fp = fp.next;
|
||||
}
|
||||
return fp;
|
||||
}
|
||||
|
||||
/** Returns a previously created {@link PerField},
|
||||
* absorbing the type information from {@link FieldType},
|
||||
* and creates a new {@link PerField} if this field name
|
||||
* wasn't seen yet. */
|
||||
private PerField getOrAddField(String name, IndexableFieldType fieldType, boolean invert) {
|
||||
|
||||
// Make sure we have a PerField allocated
|
||||
final int hashPos = name.hashCode() & hashMask;
|
||||
PerField fp = fieldHash[hashPos];
|
||||
while (fp != null && !fp.fieldInfo.name.equals(name)) {
|
||||
fp = fp.next;
|
||||
}
|
||||
|
||||
if (fp == null) {
|
||||
// First time we are seeing this field in this segment
|
||||
|
||||
FieldInfo fi = fieldInfos.addOrUpdate(name, fieldType);
|
||||
|
||||
fp = new PerField(fi, invert);
|
||||
fp.next = fieldHash[hashPos];
|
||||
fieldHash[hashPos] = fp;
|
||||
totalFieldCount++;
|
||||
|
||||
// At most 50% load factor:
|
||||
if (totalFieldCount >= fieldHash.length/2) {
|
||||
rehash();
|
||||
}
|
||||
|
||||
if (totalFieldCount > fields.length) {
|
||||
PerField[] newFields = new PerField[ArrayUtil.oversize(totalFieldCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
|
||||
System.arraycopy(fields, 0, newFields, 0, fields.length);
|
||||
fields = newFields;
|
||||
}
|
||||
|
||||
} else {
|
||||
fp.fieldInfo.update(fieldType);
|
||||
|
||||
if (invert && fp.invertState == null) {
|
||||
fp.setInvertState();
|
||||
}
|
||||
}
|
||||
|
||||
return fp;
|
||||
}
|
||||
|
||||
/** NOTE: not static: accesses at least docState, termsHash. */
|
||||
private final class PerField implements Comparable<PerField> {
|
||||
|
||||
final FieldInfo fieldInfo;
|
||||
final Similarity similarity;
|
||||
|
||||
FieldInvertState invertState;
|
||||
TermsHashPerField termsHashPerField;
|
||||
|
||||
// Non-null if this field ever had doc values in this
|
||||
// segment:
|
||||
DocValuesWriter docValuesWriter;
|
||||
|
||||
/** We use this to know when a PerField is seen for the
|
||||
* first time in the current document. */
|
||||
long fieldGen = -1;
|
||||
|
||||
// Used by the hash table
|
||||
PerField next;
|
||||
|
||||
// Lazy init'd:
|
||||
NumericDocValuesWriter norms;
|
||||
|
||||
public PerField(FieldInfo fieldInfo, boolean invert) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
similarity = docState.similarity;
|
||||
if (invert) {
|
||||
setInvertState();
|
||||
}
|
||||
}
|
||||
|
||||
void setInvertState() {
|
||||
invertState = new FieldInvertState(fieldInfo.name);
|
||||
termsHashPerField = termsHash.addField(invertState, fieldInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(PerField other) {
|
||||
return this.fieldInfo.name.compareTo(other.fieldInfo.name);
|
||||
}
|
||||
|
||||
public void finish() throws IOException {
|
||||
if (fieldInfo.omitsNorms() == false) {
|
||||
if (norms == null) {
|
||||
fieldInfo.setNormValueType(FieldInfo.DocValuesType.NUMERIC);
|
||||
norms = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed, false);
|
||||
}
|
||||
norms.addValue(docState.docID, similarity.computeNorm(invertState));
|
||||
}
|
||||
|
||||
termsHashPerField.finish();
|
||||
}
|
||||
|
||||
/** Inverts one field for one document; first is true
|
||||
* if this is the first time we are seeing this field
|
||||
* name in this document. */
|
||||
public void invert(IndexableField field, boolean first) throws IOException {
|
||||
if (first) {
|
||||
// First time we're seeing this field (indexed) in
|
||||
// this document:
|
||||
invertState.reset();
|
||||
}
|
||||
|
||||
IndexableFieldType fieldType = field.fieldType();
|
||||
|
||||
// if the field omits norms, the boost cannot be indexed.
|
||||
if (fieldType.omitNorms() && field.boost() != 1.0f) {
|
||||
throw new UnsupportedOperationException("You cannot set an index-time boost: norms are omitted for field '" + field.name() + "'");
|
||||
}
|
||||
|
||||
final boolean analyzed = fieldType.tokenized() && docState.analyzer != null;
|
||||
|
||||
// only bother checking offsets if something will consume them.
|
||||
// TODO: after we fix analyzers, also check if termVectorOffsets will be indexed.
|
||||
final boolean checkOffsets = fieldType.indexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
|
||||
|
||||
int lastStartOffset = 0;
|
||||
|
||||
/*
|
||||
* To assist people in tracking down problems in analysis components, we wish to write the field name to the infostream
|
||||
* when we fail. We expect some caller to eventually deal with the real exception, so we don't want any 'catch' clauses,
|
||||
* but rather a finally that takes note of the problem.
|
||||
*/
|
||||
boolean aborting = false;
|
||||
boolean succeededInProcessingField = false;
|
||||
try (TokenStream stream = field.tokenStream(docState.analyzer)) {
|
||||
// reset the TokenStream to the first token
|
||||
stream.reset();
|
||||
invertState.setAttributeSource(stream);
|
||||
|
||||
termsHashPerField.start(field, first);
|
||||
|
||||
while (stream.incrementToken()) {
|
||||
|
||||
// If we hit an exception in stream.next below
|
||||
// (which is fairly common, e.g. if analyzer
|
||||
// chokes on a given document), then it's
|
||||
// non-aborting and (above) this one document
|
||||
// will be marked as deleted, but still
|
||||
// consume a docID
|
||||
|
||||
final int posIncr = invertState.posIncrAttribute.getPositionIncrement();
|
||||
if (posIncr < 0) {
|
||||
throw new IllegalArgumentException("position increment must be >=0 (got " + posIncr + ") for field '" + field.name() + "'");
|
||||
}
|
||||
if (invertState.position == 0 && posIncr == 0) {
|
||||
throw new IllegalArgumentException("first position increment must be > 0 (got 0) for field '" + field.name() + "'");
|
||||
}
|
||||
int position = invertState.position + posIncr;
|
||||
if (position > 0) {
|
||||
// NOTE: confusing: this "mirrors" the
|
||||
// position++ we do below
|
||||
position--;
|
||||
} else if (position < 0) {
|
||||
throw new IllegalArgumentException("position overflow for field '" + field.name() + "'");
|
||||
}
|
||||
|
||||
// position is legal, we can safely place it in invertState now.
|
||||
// not sure if anything will use invertState after non-aborting exc...
|
||||
invertState.position = position;
|
||||
|
||||
if (posIncr == 0) {
|
||||
invertState.numOverlap++;
|
||||
}
|
||||
|
||||
if (checkOffsets) {
|
||||
int startOffset = invertState.offset + invertState.offsetAttribute.startOffset();
|
||||
int endOffset = invertState.offset + invertState.offsetAttribute.endOffset();
|
||||
if (startOffset < 0 || endOffset < startOffset) {
|
||||
throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, "
|
||||
+ "startOffset=" + startOffset + ",endOffset=" + endOffset + " for field '" + field.name() + "'");
|
||||
}
|
||||
if (startOffset < lastStartOffset) {
|
||||
throw new IllegalArgumentException("offsets must not go backwards startOffset="
|
||||
+ startOffset + " is < lastStartOffset=" + lastStartOffset + " for field '" + field.name() + "'");
|
||||
}
|
||||
lastStartOffset = startOffset;
|
||||
}
|
||||
|
||||
//System.out.println(" term=" + invertState.termAttribute);
|
||||
|
||||
// If we hit an exception in here, we abort
|
||||
// all buffered documents since the last
|
||||
// flush, on the likelihood that the
|
||||
// internal state of the terms hash is now
|
||||
// corrupt and should not be flushed to a
|
||||
// new segment:
|
||||
aborting = true;
|
||||
termsHashPerField.add();
|
||||
aborting = false;
|
||||
|
||||
invertState.length++;
|
||||
invertState.position++;
|
||||
}
|
||||
|
||||
// trigger streams to perform end-of-stream operations
|
||||
stream.end();
|
||||
|
||||
// TODO: maybe add some safety? then again, its already checked
|
||||
// when we come back around to the field...
|
||||
invertState.position += invertState.posIncrAttribute.getPositionIncrement();
|
||||
invertState.offset += invertState.offsetAttribute.endOffset();
|
||||
|
||||
/* if there is an exception coming through, we won't set this to true here:*/
|
||||
succeededInProcessingField = true;
|
||||
} catch (MaxBytesLengthExceededException e) {
|
||||
aborting = false;
|
||||
byte[] prefix = new byte[30];
|
||||
BytesRef bigTerm = invertState.termAttribute.getBytesRef();
|
||||
System.arraycopy(bigTerm.bytes, bigTerm.offset, prefix, 0, 30);
|
||||
String msg = "Document contains at least one immense term in field=\"" + fieldInfo.name + "\" (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + Arrays.toString(prefix) + "...'";
|
||||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", "ERROR: " + msg);
|
||||
}
|
||||
// Document will be deleted above:
|
||||
throw new IllegalArgumentException(msg);
|
||||
} finally {
|
||||
if (succeededInProcessingField == false && aborting) {
|
||||
docState.docWriter.setAborting();
|
||||
}
|
||||
|
||||
if (!succeededInProcessingField && docState.infoStream.isEnabled("DW")) {
|
||||
docState.infoStream.message("DW", "An exception was thrown while processing field " + fieldInfo.name);
|
||||
}
|
||||
}
|
||||
|
||||
if (analyzed) {
|
||||
invertState.position += docState.analyzer.getPositionIncrementGap(fieldInfo.name);
|
||||
invertState.offset += docState.analyzer.getOffsetGap(fieldInfo.name);
|
||||
}
|
||||
|
||||
invertState.boost *= field.boost();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,8 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
|
||||
abstract class DocConsumer {
|
||||
abstract void processDocument(FieldInfos.Builder fieldInfos) throws IOException;
|
||||
abstract void finishDocument() throws IOException;
|
||||
abstract void processDocument() throws IOException;
|
||||
abstract void flush(final SegmentWriteState state) throws IOException;
|
||||
abstract void abort();
|
||||
}
|
||||
|
|
|
@ -1,36 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
abstract class DocFieldConsumer {
|
||||
/** Called when DocumentsWriterPerThread decides to create a new
|
||||
* segment */
|
||||
abstract void flush(Map<String, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
|
||||
|
||||
/** Called when an aborting exception is hit */
|
||||
abstract void abort();
|
||||
|
||||
abstract void startDocument() throws IOException;
|
||||
|
||||
abstract DocFieldConsumerPerField addField(FieldInfo fi);
|
||||
|
||||
abstract void finishDocument() throws IOException;
|
||||
}
|
|
@ -1,27 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
abstract class DocFieldConsumerPerField {
|
||||
/** Processes all occurrences of a single field */
|
||||
abstract void processFields(IndexableField[] fields, int count) throws IOException;
|
||||
abstract void abort();
|
||||
abstract FieldInfo getFieldInfo();
|
||||
}
|
|
@ -1,278 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Counter;
|
||||
|
||||
/**
|
||||
* This is a DocConsumer that gathers all fields under the
|
||||
* same name, and calls per-field consumers to process field
|
||||
* by field. This class doesn't doesn't do any "real" work
|
||||
* of its own: it just forwards the fields to a
|
||||
* DocFieldConsumer.
|
||||
*/
|
||||
|
||||
final class DocFieldProcessor extends DocConsumer {
|
||||
|
||||
final DocFieldConsumer consumer;
|
||||
final StoredFieldsConsumer storedConsumer;
|
||||
final Codec codec;
|
||||
|
||||
// Holds all fields seen in current doc
|
||||
DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1];
|
||||
int fieldCount;
|
||||
|
||||
// Hash table for all fields ever seen
|
||||
DocFieldProcessorPerField[] fieldHash = new DocFieldProcessorPerField[2];
|
||||
int hashMask = 1;
|
||||
int totalFieldCount;
|
||||
|
||||
int fieldGen;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
|
||||
final Counter bytesUsed;
|
||||
|
||||
public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer, StoredFieldsConsumer storedConsumer) {
|
||||
this.docState = docWriter.docState;
|
||||
this.codec = docWriter.codec;
|
||||
this.bytesUsed = docWriter.bytesUsed;
|
||||
this.consumer = consumer;
|
||||
this.storedConsumer = storedConsumer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state) throws IOException {
|
||||
|
||||
Map<String,DocFieldConsumerPerField> childFields = new HashMap<>();
|
||||
Collection<DocFieldConsumerPerField> fields = fields();
|
||||
for (DocFieldConsumerPerField f : fields) {
|
||||
childFields.put(f.getFieldInfo().name, f);
|
||||
}
|
||||
|
||||
assert fields.size() == totalFieldCount;
|
||||
|
||||
storedConsumer.flush(state);
|
||||
consumer.flush(childFields, state);
|
||||
|
||||
// Important to save after asking consumer to flush so
|
||||
// consumer can alter the FieldInfo* if necessary. EG,
|
||||
// FreqProxTermsWriter does this with
|
||||
// FieldInfo.storePayload.
|
||||
FieldInfosWriter infosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
|
||||
infosWriter.write(state.directory, state.segmentInfo.name, "", state.fieldInfos, IOContext.DEFAULT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
Throwable th = null;
|
||||
|
||||
for (DocFieldProcessorPerField field : fieldHash) {
|
||||
while (field != null) {
|
||||
final DocFieldProcessorPerField next = field.next;
|
||||
try {
|
||||
field.abort();
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
field = next;
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
storedConsumer.abort();
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
consumer.abort();
|
||||
} catch (Throwable t) {
|
||||
if (th == null) {
|
||||
th = t;
|
||||
}
|
||||
}
|
||||
|
||||
// If any errors occured, throw it.
|
||||
if (th != null) {
|
||||
if (th instanceof RuntimeException) throw (RuntimeException) th;
|
||||
if (th instanceof Error) throw (Error) th;
|
||||
// defensive code - we should not hit unchecked exceptions
|
||||
throw new RuntimeException(th);
|
||||
}
|
||||
}
|
||||
|
||||
public Collection<DocFieldConsumerPerField> fields() {
|
||||
Collection<DocFieldConsumerPerField> fields = new HashSet<>();
|
||||
for(int i=0;i<fieldHash.length;i++) {
|
||||
DocFieldProcessorPerField field = fieldHash[i];
|
||||
while(field != null) {
|
||||
fields.add(field.consumer);
|
||||
field = field.next;
|
||||
}
|
||||
}
|
||||
assert fields.size() == totalFieldCount;
|
||||
return fields;
|
||||
}
|
||||
|
||||
private void rehash() {
|
||||
final int newHashSize = (fieldHash.length*2);
|
||||
assert newHashSize > fieldHash.length;
|
||||
|
||||
final DocFieldProcessorPerField newHashArray[] = new DocFieldProcessorPerField[newHashSize];
|
||||
|
||||
// Rehash
|
||||
int newHashMask = newHashSize-1;
|
||||
for(int j=0;j<fieldHash.length;j++) {
|
||||
DocFieldProcessorPerField fp0 = fieldHash[j];
|
||||
while(fp0 != null) {
|
||||
final int hashPos2 = fp0.fieldInfo.name.hashCode() & newHashMask;
|
||||
DocFieldProcessorPerField nextFP0 = fp0.next;
|
||||
fp0.next = newHashArray[hashPos2];
|
||||
newHashArray[hashPos2] = fp0;
|
||||
fp0 = nextFP0;
|
||||
}
|
||||
}
|
||||
|
||||
fieldHash = newHashArray;
|
||||
hashMask = newHashMask;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processDocument(FieldInfos.Builder fieldInfos) throws IOException {
|
||||
|
||||
consumer.startDocument();
|
||||
storedConsumer.startDocument();
|
||||
|
||||
fieldCount = 0;
|
||||
|
||||
final int thisFieldGen = fieldGen++;
|
||||
|
||||
// Absorb any new fields first seen in this document.
|
||||
// Also absorb any changes to fields we had already
|
||||
// seen before (eg suddenly turning on norms or
|
||||
// vectors, etc.):
|
||||
|
||||
for(IndexableField field : docState.doc.indexableFields()) {
|
||||
final String fieldName = field.name();
|
||||
IndexableFieldType ft = field.fieldType();
|
||||
|
||||
DocFieldProcessorPerField fp = processField(fieldInfos, thisFieldGen, fieldName, ft);
|
||||
|
||||
fp.addField(field);
|
||||
}
|
||||
|
||||
for (StorableField field: docState.doc.storableFields()) {
|
||||
final String fieldName = field.name();
|
||||
IndexableFieldType ft = field.fieldType();
|
||||
FieldInfo fieldInfo = fieldInfos.addOrUpdate(fieldName, ft);
|
||||
storedConsumer.addField(docState.docID, field, fieldInfo);
|
||||
}
|
||||
|
||||
// If we are writing vectors then we must visit
|
||||
// fields in sorted order so they are written in
|
||||
// sorted order. TODO: we actually only need to
|
||||
// sort the subset of fields that have vectors
|
||||
// enabled; we could save [small amount of] CPU
|
||||
// here.
|
||||
ArrayUtil.introSort(fields, 0, fieldCount, fieldsComp);
|
||||
for(int i=0;i<fieldCount;i++) {
|
||||
final DocFieldProcessorPerField perField = fields[i];
|
||||
perField.consumer.processFields(perField.fields, perField.fieldCount);
|
||||
}
|
||||
}
|
||||
|
||||
private DocFieldProcessorPerField processField(FieldInfos.Builder fieldInfos,
|
||||
final int thisFieldGen, final String fieldName, IndexableFieldType ft) {
|
||||
|
||||
// Make sure we have a PerField allocated
|
||||
final int hashPos = fieldName.hashCode() & hashMask;
|
||||
DocFieldProcessorPerField fp = fieldHash[hashPos];
|
||||
while(fp != null && !fp.fieldInfo.name.equals(fieldName)) {
|
||||
fp = fp.next;
|
||||
}
|
||||
|
||||
if (fp == null) {
|
||||
|
||||
// TODO FI: we need to genericize the "flags" that a
|
||||
// field holds, and, how these flags are merged; it
|
||||
// needs to be more "pluggable" such that if I want
|
||||
// to have a new "thing" my Fields can do, I can
|
||||
// easily add it
|
||||
FieldInfo fi = fieldInfos.addOrUpdate(fieldName, ft);
|
||||
|
||||
fp = new DocFieldProcessorPerField(this, fi);
|
||||
fp.next = fieldHash[hashPos];
|
||||
fieldHash[hashPos] = fp;
|
||||
totalFieldCount++;
|
||||
|
||||
if (totalFieldCount >= fieldHash.length/2) {
|
||||
rehash();
|
||||
}
|
||||
} else {
|
||||
fp.fieldInfo.update(ft);
|
||||
}
|
||||
|
||||
if (thisFieldGen != fp.lastGen) {
|
||||
|
||||
// First time we're seeing this field for this doc
|
||||
fp.fieldCount = 0;
|
||||
|
||||
if (fieldCount == fields.length) {
|
||||
final int newSize = fields.length*2;
|
||||
DocFieldProcessorPerField newArray[] = new DocFieldProcessorPerField[newSize];
|
||||
System.arraycopy(fields, 0, newArray, 0, fieldCount);
|
||||
fields = newArray;
|
||||
}
|
||||
|
||||
fields[fieldCount++] = fp;
|
||||
fp.lastGen = thisFieldGen;
|
||||
}
|
||||
return fp;
|
||||
}
|
||||
|
||||
private static final Comparator<DocFieldProcessorPerField> fieldsComp = new Comparator<DocFieldProcessorPerField>() {
|
||||
@Override
|
||||
public int compare(DocFieldProcessorPerField o1, DocFieldProcessorPerField o2) {
|
||||
return o1.fieldInfo.name.compareTo(o2.fieldInfo.name);
|
||||
}
|
||||
};
|
||||
|
||||
@Override
|
||||
void finishDocument() throws IOException {
|
||||
try {
|
||||
storedConsumer.finishDocument();
|
||||
} finally {
|
||||
consumer.finishDocument();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,57 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Holds all per thread, per field state.
|
||||
*/
|
||||
|
||||
final class DocFieldProcessorPerField {
|
||||
|
||||
final DocFieldConsumerPerField consumer;
|
||||
final FieldInfo fieldInfo;
|
||||
|
||||
DocFieldProcessorPerField next;
|
||||
int lastGen = -1;
|
||||
|
||||
int fieldCount;
|
||||
IndexableField[] fields = new IndexableField[1];
|
||||
|
||||
public DocFieldProcessorPerField(final DocFieldProcessor docFieldProcessor, final FieldInfo fieldInfo) {
|
||||
this.consumer = docFieldProcessor.consumer.addField(fieldInfo);
|
||||
this.fieldInfo = fieldInfo;
|
||||
}
|
||||
|
||||
public void addField(IndexableField field) {
|
||||
if (fieldCount == fields.length) {
|
||||
int newSize = ArrayUtil.oversize(fieldCount + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||
IndexableField[] newArray = new IndexableField[newSize];
|
||||
System.arraycopy(fields, 0, newArray, 0, fieldCount);
|
||||
fields = newArray;
|
||||
}
|
||||
|
||||
fields[fieldCount++] = field;
|
||||
}
|
||||
|
||||
public void abort() {
|
||||
consumer.abort();
|
||||
}
|
||||
}
|
|
@ -1,84 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/** This is a DocFieldConsumer that inverts each field,
|
||||
* separately, from a Document, and accepts a
|
||||
* InvertedTermsConsumer to process those terms. */
|
||||
|
||||
final class DocInverter extends DocFieldConsumer {
|
||||
|
||||
final InvertedDocConsumer consumer;
|
||||
final InvertedDocEndConsumer endConsumer;
|
||||
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
|
||||
public DocInverter(DocumentsWriterPerThread.DocState docState, InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
|
||||
this.docState = docState;
|
||||
this.consumer = consumer;
|
||||
this.endConsumer = endConsumer;
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(Map<String, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
|
||||
|
||||
Map<String, InvertedDocConsumerPerField> childFieldsToFlush = new HashMap<>();
|
||||
Map<String, InvertedDocEndConsumerPerField> endChildFieldsToFlush = new HashMap<>();
|
||||
|
||||
for (Map.Entry<String, DocFieldConsumerPerField> fieldToFlush : fieldsToFlush.entrySet()) {
|
||||
DocInverterPerField perField = (DocInverterPerField) fieldToFlush.getValue();
|
||||
childFieldsToFlush.put(fieldToFlush.getKey(), perField.consumer);
|
||||
endChildFieldsToFlush.put(fieldToFlush.getKey(), perField.endConsumer);
|
||||
}
|
||||
|
||||
consumer.flush(childFieldsToFlush, state);
|
||||
endConsumer.flush(endChildFieldsToFlush, state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startDocument() throws IOException {
|
||||
consumer.startDocument();
|
||||
endConsumer.startDocument();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDocument() throws IOException {
|
||||
// TODO: allow endConsumer.finishDocument to also return
|
||||
// a DocWriter
|
||||
endConsumer.finishDocument();
|
||||
consumer.finishDocument();
|
||||
}
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
try {
|
||||
consumer.abort();
|
||||
} finally {
|
||||
endConsumer.abort();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocFieldConsumerPerField addField(FieldInfo fi) {
|
||||
return new DocInverterPerField(this, fi);
|
||||
}
|
||||
}
|
|
@ -1,220 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
|
||||
/**
|
||||
* Holds state for inverting all occurrences of a single
|
||||
* field in the document. This class doesn't do anything
|
||||
* itself; instead, it forwards the tokens produced by
|
||||
* analysis to its own consumer
|
||||
* (InvertedDocConsumerPerField). It also interacts with an
|
||||
* endConsumer (InvertedDocEndConsumerPerField).
|
||||
*/
|
||||
|
||||
final class DocInverterPerField extends DocFieldConsumerPerField {
|
||||
|
||||
final FieldInfo fieldInfo;
|
||||
final InvertedDocConsumerPerField consumer;
|
||||
final InvertedDocEndConsumerPerField endConsumer;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final FieldInvertState fieldState;
|
||||
|
||||
public DocInverterPerField(DocInverter parent, FieldInfo fieldInfo) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
docState = parent.docState;
|
||||
fieldState = new FieldInvertState(fieldInfo.name);
|
||||
this.consumer = parent.consumer.addField(this, fieldInfo);
|
||||
this.endConsumer = parent.endConsumer.addField(this, fieldInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
try {
|
||||
consumer.abort();
|
||||
} finally {
|
||||
endConsumer.abort();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processFields(final IndexableField[] fields,
|
||||
final int count) throws IOException {
|
||||
|
||||
fieldState.reset();
|
||||
|
||||
final boolean doInvert = consumer.start(fields, count);
|
||||
|
||||
for(int i=0;i<count;i++) {
|
||||
|
||||
final IndexableField field = fields[i];
|
||||
final IndexableFieldType fieldType = field.fieldType();
|
||||
|
||||
// TODO FI: this should be "genericized" to querying
|
||||
// consumer if it wants to see this particular field
|
||||
// tokenized.
|
||||
if (doInvert) {
|
||||
final boolean analyzed = fieldType.tokenized() && docState.analyzer != null;
|
||||
|
||||
// if the field omits norms, the boost cannot be indexed.
|
||||
if (fieldType.omitNorms() && field.boost() != 1.0f) {
|
||||
throw new UnsupportedOperationException("You cannot set an index-time boost: norms are omitted for field '" + field.name() + "'");
|
||||
}
|
||||
|
||||
// only bother checking offsets if something will consume them.
|
||||
// TODO: after we fix analyzers, also check if termVectorOffsets will be indexed.
|
||||
final boolean checkOffsets = fieldType.indexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
|
||||
int lastStartOffset = 0;
|
||||
|
||||
if (i > 0) {
|
||||
fieldState.position += analyzed ? docState.analyzer.getPositionIncrementGap(fieldInfo.name) : 0;
|
||||
}
|
||||
|
||||
/*
|
||||
* To assist people in tracking down problems in analysis components, we wish to write the field name to the infostream
|
||||
* when we fail. We expect some caller to eventually deal with the real exception, so we don't want any 'catch' clauses,
|
||||
* but rather a finally that takes note of the problem.
|
||||
*/
|
||||
|
||||
boolean succeededInProcessingField = false;
|
||||
try (TokenStream stream = field.tokenStream(docState.analyzer)) {
|
||||
// reset the TokenStream to the first token
|
||||
stream.reset();
|
||||
boolean hasMoreTokens = stream.incrementToken();
|
||||
|
||||
fieldState.attributeSource = stream;
|
||||
|
||||
OffsetAttribute offsetAttribute = fieldState.attributeSource.addAttribute(OffsetAttribute.class);
|
||||
PositionIncrementAttribute posIncrAttribute = fieldState.attributeSource.addAttribute(PositionIncrementAttribute.class);
|
||||
|
||||
if (hasMoreTokens) {
|
||||
consumer.start(field);
|
||||
|
||||
do {
|
||||
// If we hit an exception in stream.next below
|
||||
// (which is fairly common, eg if analyzer
|
||||
// chokes on a given document), then it's
|
||||
// non-aborting and (above) this one document
|
||||
// will be marked as deleted, but still
|
||||
// consume a docID
|
||||
|
||||
final int posIncr = posIncrAttribute.getPositionIncrement();
|
||||
if (posIncr < 0) {
|
||||
throw new IllegalArgumentException("position increment must be >=0 (got " + posIncr + ") for field '" + field.name() + "'");
|
||||
}
|
||||
if (fieldState.position == 0 && posIncr == 0) {
|
||||
throw new IllegalArgumentException("first position increment must be > 0 (got 0) for field '" + field.name() + "'");
|
||||
}
|
||||
int position = fieldState.position + posIncr;
|
||||
if (position > 0) {
|
||||
// NOTE: confusing: this "mirrors" the
|
||||
// position++ we do below
|
||||
position--;
|
||||
} else if (position < 0) {
|
||||
throw new IllegalArgumentException("position overflow for field '" + field.name() + "'");
|
||||
}
|
||||
|
||||
// position is legal, we can safely place it in fieldState now.
|
||||
// not sure if anything will use fieldState after non-aborting exc...
|
||||
fieldState.position = position;
|
||||
|
||||
if (posIncr == 0)
|
||||
fieldState.numOverlap++;
|
||||
|
||||
if (checkOffsets) {
|
||||
int startOffset = fieldState.offset + offsetAttribute.startOffset();
|
||||
int endOffset = fieldState.offset + offsetAttribute.endOffset();
|
||||
if (startOffset < 0 || endOffset < startOffset) {
|
||||
throw new IllegalArgumentException("startOffset must be non-negative, and endOffset must be >= startOffset, "
|
||||
+ "startOffset=" + startOffset + ",endOffset=" + endOffset + " for field '" + field.name() + "'");
|
||||
}
|
||||
if (startOffset < lastStartOffset) {
|
||||
throw new IllegalArgumentException("offsets must not go backwards startOffset="
|
||||
+ startOffset + " is < lastStartOffset=" + lastStartOffset + " for field '" + field.name() + "'");
|
||||
}
|
||||
lastStartOffset = startOffset;
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
// If we hit an exception in here, we abort
|
||||
// all buffered documents since the last
|
||||
// flush, on the likelihood that the
|
||||
// internal state of the consumer is now
|
||||
// corrupt and should not be flushed to a
|
||||
// new segment:
|
||||
consumer.add();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docState.docWriter.setAborting();
|
||||
}
|
||||
}
|
||||
fieldState.length++;
|
||||
fieldState.position++;
|
||||
} while (stream.incrementToken());
|
||||
}
|
||||
// trigger streams to perform end-of-stream operations
|
||||
stream.end();
|
||||
// TODO: maybe add some safety? then again, its already checked
|
||||
// when we come back around to the field...
|
||||
fieldState.position += posIncrAttribute.getPositionIncrement();
|
||||
fieldState.offset += offsetAttribute.endOffset();
|
||||
|
||||
|
||||
if (docState.maxTermPrefix != null) {
|
||||
final String msg = "Document contains at least one immense term in field=\"" + fieldInfo.name + "\" (whose UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'";
|
||||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", "ERROR: " + msg);
|
||||
}
|
||||
docState.maxTermPrefix = null;
|
||||
throw new IllegalArgumentException(msg);
|
||||
}
|
||||
|
||||
/* if success was false above there is an exception coming through and we won't get here.*/
|
||||
succeededInProcessingField = true;
|
||||
} finally {
|
||||
if (!succeededInProcessingField && docState.infoStream.isEnabled("DW")) {
|
||||
docState.infoStream.message("DW", "An exception was thrown while processing field " + fieldInfo.name);
|
||||
}
|
||||
}
|
||||
|
||||
fieldState.offset += analyzed ? docState.analyzer.getOffsetGap(fieldInfo.name) : 0;
|
||||
fieldState.boost *= field.boost();
|
||||
}
|
||||
|
||||
// LUCENE-2387: don't hang onto the field, so GC can
|
||||
// reclaim
|
||||
fields[i] = null;
|
||||
}
|
||||
|
||||
consumer.finish();
|
||||
endConsumer.finish();
|
||||
}
|
||||
|
||||
@Override
|
||||
FieldInfo getFieldInfo() {
|
||||
return fieldInfo;
|
||||
}
|
||||
}
|
|
@ -1,177 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
final class DocValuesProcessor extends StoredFieldsConsumer {
|
||||
|
||||
// TODO: somewhat wasteful we also keep a map here; would
|
||||
// be more efficient if we could "reuse" the map/hash
|
||||
// lookup DocFieldProcessor already did "above"
|
||||
private final Map<String,DocValuesWriter> writers = new HashMap<>();
|
||||
private final Counter bytesUsed;
|
||||
|
||||
public DocValuesProcessor(Counter bytesUsed) {
|
||||
this.bytesUsed = bytesUsed;
|
||||
}
|
||||
|
||||
@Override
|
||||
void startDocument() {
|
||||
}
|
||||
|
||||
@Override
|
||||
void finishDocument() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addField(int docID, StorableField field, FieldInfo fieldInfo) {
|
||||
final DocValuesType dvType = field.fieldType().docValueType();
|
||||
if (dvType != null) {
|
||||
fieldInfo.setDocValuesType(dvType);
|
||||
if (dvType == DocValuesType.BINARY) {
|
||||
addBinaryField(fieldInfo, docID, field.binaryValue());
|
||||
} else if (dvType == DocValuesType.SORTED) {
|
||||
addSortedField(fieldInfo, docID, field.binaryValue());
|
||||
} else if (dvType == DocValuesType.SORTED_SET) {
|
||||
addSortedSetField(fieldInfo, docID, field.binaryValue());
|
||||
} else if (dvType == DocValuesType.NUMERIC) {
|
||||
if (!(field.numericValue() instanceof Long)) {
|
||||
throw new IllegalArgumentException("illegal type " + field.numericValue().getClass() + ": DocValues types must be Long");
|
||||
}
|
||||
addNumericField(fieldInfo, docID, field.numericValue().longValue());
|
||||
} else {
|
||||
assert false: "unrecognized DocValues.Type: " + dvType;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(SegmentWriteState state) throws IOException {
|
||||
if (!writers.isEmpty()) {
|
||||
DocValuesFormat fmt = state.segmentInfo.getCodec().docValuesFormat();
|
||||
DocValuesConsumer dvConsumer = fmt.fieldsConsumer(state);
|
||||
boolean success = false;
|
||||
try {
|
||||
for(DocValuesWriter writer : writers.values()) {
|
||||
writer.finish(state.segmentInfo.getDocCount());
|
||||
writer.flush(state, dvConsumer);
|
||||
}
|
||||
// TODO: catch missing DV fields here? else we have
|
||||
// null/"" depending on how docs landed in segments?
|
||||
// but we can't detect all cases, and we should leave
|
||||
// this behavior undefined. dv is not "schemaless": its column-stride.
|
||||
writers.clear();
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(dvConsumer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(dvConsumer);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void addBinaryField(FieldInfo fieldInfo, int docID, BytesRef value) {
|
||||
DocValuesWriter writer = writers.get(fieldInfo.name);
|
||||
BinaryDocValuesWriter binaryWriter;
|
||||
if (writer == null) {
|
||||
binaryWriter = new BinaryDocValuesWriter(fieldInfo, bytesUsed);
|
||||
writers.put(fieldInfo.name, binaryWriter);
|
||||
} else if (!(writer instanceof BinaryDocValuesWriter)) {
|
||||
throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to binary");
|
||||
} else {
|
||||
binaryWriter = (BinaryDocValuesWriter) writer;
|
||||
}
|
||||
binaryWriter.addValue(docID, value);
|
||||
}
|
||||
|
||||
void addSortedField(FieldInfo fieldInfo, int docID, BytesRef value) {
|
||||
DocValuesWriter writer = writers.get(fieldInfo.name);
|
||||
SortedDocValuesWriter sortedWriter;
|
||||
if (writer == null) {
|
||||
sortedWriter = new SortedDocValuesWriter(fieldInfo, bytesUsed);
|
||||
writers.put(fieldInfo.name, sortedWriter);
|
||||
} else if (!(writer instanceof SortedDocValuesWriter)) {
|
||||
throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to sorted");
|
||||
} else {
|
||||
sortedWriter = (SortedDocValuesWriter) writer;
|
||||
}
|
||||
sortedWriter.addValue(docID, value);
|
||||
}
|
||||
|
||||
void addSortedSetField(FieldInfo fieldInfo, int docID, BytesRef value) {
|
||||
DocValuesWriter writer = writers.get(fieldInfo.name);
|
||||
SortedSetDocValuesWriter sortedSetWriter;
|
||||
if (writer == null) {
|
||||
sortedSetWriter = new SortedSetDocValuesWriter(fieldInfo, bytesUsed);
|
||||
writers.put(fieldInfo.name, sortedSetWriter);
|
||||
} else if (!(writer instanceof SortedSetDocValuesWriter)) {
|
||||
throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to sorted");
|
||||
} else {
|
||||
sortedSetWriter = (SortedSetDocValuesWriter) writer;
|
||||
}
|
||||
sortedSetWriter.addValue(docID, value);
|
||||
}
|
||||
|
||||
void addNumericField(FieldInfo fieldInfo, int docID, long value) {
|
||||
DocValuesWriter writer = writers.get(fieldInfo.name);
|
||||
NumericDocValuesWriter numericWriter;
|
||||
if (writer == null) {
|
||||
numericWriter = new NumericDocValuesWriter(fieldInfo, bytesUsed, true);
|
||||
writers.put(fieldInfo.name, numericWriter);
|
||||
} else if (!(writer instanceof NumericDocValuesWriter)) {
|
||||
throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to numeric");
|
||||
} else {
|
||||
numericWriter = (NumericDocValuesWriter) writer;
|
||||
}
|
||||
numericWriter.addValue(docID, value);
|
||||
}
|
||||
|
||||
private String getTypeDesc(DocValuesWriter obj) {
|
||||
if (obj instanceof BinaryDocValuesWriter) {
|
||||
return "binary";
|
||||
} else if (obj instanceof NumericDocValuesWriter) {
|
||||
return "numeric";
|
||||
} else {
|
||||
assert obj instanceof SortedDocValuesWriter;
|
||||
return "sorted";
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() throws IOException {
|
||||
for(DocValuesWriter writer : writers.values()) {
|
||||
try {
|
||||
writer.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
}
|
||||
writers.clear();
|
||||
}
|
||||
}
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
|||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
|
||||
abstract class DocValuesWriter {
|
||||
abstract void abort() throws IOException;
|
||||
abstract void finish(int numDoc);
|
||||
abstract void flush(SegmentWriteState state, DocValuesConsumer consumer) throws IOException;
|
||||
}
|
||||
|
|
|
@ -43,19 +43,13 @@ import org.apache.lucene.util.InfoStream;
|
|||
* This class accepts multiple added documents and directly
|
||||
* writes segment files.
|
||||
*
|
||||
* Each added document is passed to the {@link DocConsumer},
|
||||
* which in turn processes the document and interacts with
|
||||
* other consumers in the indexing chain. Certain
|
||||
* consumers, like {@link StoredFieldsConsumer} and {@link
|
||||
* TermVectorsConsumer}, digest a document and
|
||||
* immediately write bytes to the "doc store" files (ie,
|
||||
* they do not consume RAM per document, except while they
|
||||
* are processing the document).
|
||||
* Each added document is passed to the indexing chain,
|
||||
* which in turn processes the document into the different
|
||||
* codec formats. Some formats write bytes to files
|
||||
* immediately, e.g. stored fields and term vectors, while
|
||||
* others are buffered by the indexing chain and written
|
||||
* only on flush.
|
||||
*
|
||||
* Other consumers, eg {@link FreqProxTermsWriter} and
|
||||
* {@link NormsConsumer}, buffer bytes in RAM and flush only
|
||||
* when a new segment is produced.
|
||||
|
||||
* Once we have used our allowed RAM buffer, or the number
|
||||
* of added docs is large enough (in the case we are
|
||||
* flushing by doc count instead of RAM usage), we create a
|
||||
|
@ -398,7 +392,7 @@ final class DocumentsWriter implements Closeable {
|
|||
return hasEvents;
|
||||
}
|
||||
|
||||
private final void ensureInitialized(ThreadState state) {
|
||||
private final void ensureInitialized(ThreadState state) throws IOException {
|
||||
if (state.isActive() && state.dwpt == null) {
|
||||
final FieldInfos.Builder infos = new FieldInfos.Builder(
|
||||
writer.globalFieldNumberMap);
|
||||
|
|
|
@ -51,48 +51,15 @@ class DocumentsWriterPerThread {
|
|||
* documents.
|
||||
*/
|
||||
abstract static class IndexingChain {
|
||||
abstract DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread);
|
||||
abstract DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread) throws IOException;
|
||||
}
|
||||
|
||||
|
||||
static final IndexingChain defaultIndexingChain = new IndexingChain() {
|
||||
|
||||
@Override
|
||||
DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread) {
|
||||
/*
|
||||
This is the current indexing chain:
|
||||
|
||||
DocConsumer / DocConsumerPerThread
|
||||
--> code: DocFieldProcessor
|
||||
--> DocFieldConsumer / DocFieldConsumerPerField
|
||||
--> code: DocFieldConsumers / DocFieldConsumersPerField
|
||||
--> code: DocInverter / DocInverterPerField
|
||||
--> InvertedDocConsumer / InvertedDocConsumerPerField
|
||||
--> code: TermsHash / TermsHashPerField
|
||||
--> TermsHashConsumer / TermsHashConsumerPerField
|
||||
--> code: FreqProxTermsWriter / FreqProxTermsWriterPerField
|
||||
--> code: TermVectorsTermsWriter / TermVectorsTermsWriterPerField
|
||||
--> InvertedDocEndConsumer / InvertedDocConsumerPerField
|
||||
--> code: NormsConsumer / NormsConsumerPerField
|
||||
--> StoredFieldsConsumer
|
||||
--> TwoStoredFieldConsumers
|
||||
-> code: StoredFieldsProcessor
|
||||
-> code: DocValuesProcessor
|
||||
*/
|
||||
|
||||
// Build up indexing chain:
|
||||
|
||||
final TermsHashConsumer termVectorsWriter = new TermVectorsConsumer(documentsWriterPerThread);
|
||||
final TermsHashConsumer freqProxWriter = new FreqProxTermsWriter();
|
||||
|
||||
final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
|
||||
new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
|
||||
final NormsConsumer normsWriter = new NormsConsumer();
|
||||
final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter);
|
||||
final StoredFieldsConsumer storedFields = new TwoStoredFieldsConsumers(
|
||||
new StoredFieldsProcessor(documentsWriterPerThread),
|
||||
new DocValuesProcessor(documentsWriterPerThread.bytesUsed));
|
||||
return new DocFieldProcessor(documentsWriterPerThread, docInverter, storedFields);
|
||||
DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread) throws IOException {
|
||||
return new DefaultIndexingChain(documentsWriterPerThread);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -103,7 +70,6 @@ class DocumentsWriterPerThread {
|
|||
Similarity similarity;
|
||||
int docID;
|
||||
IndexDocument doc;
|
||||
String maxTermPrefix;
|
||||
|
||||
DocState(DocumentsWriterPerThread docWriter, InfoStream infoStream) {
|
||||
this.docWriter = docWriter;
|
||||
|
@ -180,7 +146,7 @@ class DocumentsWriterPerThread {
|
|||
boolean aborting = false; // True if an abort is pending
|
||||
boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting
|
||||
|
||||
private FieldInfos.Builder fieldInfos;
|
||||
private final FieldInfos.Builder fieldInfos;
|
||||
private final InfoStream infoStream;
|
||||
private int numDocsInRAM;
|
||||
final DocumentsWriterDeleteQueue deleteQueue;
|
||||
|
@ -192,7 +158,7 @@ class DocumentsWriterPerThread {
|
|||
|
||||
|
||||
public DocumentsWriterPerThread(String segmentName, Directory directory, LiveIndexWriterConfig indexWriterConfig, InfoStream infoStream, DocumentsWriterDeleteQueue deleteQueue,
|
||||
FieldInfos.Builder fieldInfos) {
|
||||
FieldInfos.Builder fieldInfos) throws IOException {
|
||||
this.directoryOrig = directory;
|
||||
this.directory = new TrackingDirectoryWrapper(directory);
|
||||
this.fieldInfos = fieldInfos;
|
||||
|
@ -218,13 +184,16 @@ class DocumentsWriterPerThread {
|
|||
// this should be the last call in the ctor
|
||||
// it really sucks that we need to pull this within the ctor and pass this ref to the chain!
|
||||
consumer = indexWriterConfig.getIndexingChain().getChain(this);
|
||||
|
||||
}
|
||||
|
||||
void setAborting() {
|
||||
aborting = true;
|
||||
}
|
||||
|
||||
public FieldInfos.Builder getFieldInfosBuilder() {
|
||||
return fieldInfos;
|
||||
}
|
||||
|
||||
boolean checkAndResetHasAborted() {
|
||||
final boolean retval = hasAborted;
|
||||
hasAborted = false;
|
||||
|
@ -250,7 +219,7 @@ class DocumentsWriterPerThread {
|
|||
boolean success = false;
|
||||
try {
|
||||
try {
|
||||
consumer.processDocument(fieldInfos);
|
||||
consumer.processDocument();
|
||||
} finally {
|
||||
docState.clear();
|
||||
}
|
||||
|
@ -266,15 +235,6 @@ class DocumentsWriterPerThread {
|
|||
}
|
||||
}
|
||||
}
|
||||
success = false;
|
||||
try {
|
||||
consumer.finishDocument();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
abort(filesToDelete);
|
||||
}
|
||||
}
|
||||
finishDocument(delTerm);
|
||||
}
|
||||
|
||||
|
@ -296,7 +256,7 @@ class DocumentsWriterPerThread {
|
|||
|
||||
boolean success = false;
|
||||
try {
|
||||
consumer.processDocument(fieldInfos);
|
||||
consumer.processDocument();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -310,16 +270,6 @@ class DocumentsWriterPerThread {
|
|||
}
|
||||
}
|
||||
}
|
||||
success = false;
|
||||
try {
|
||||
consumer.finishDocument();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
abort(filesToDelete);
|
||||
}
|
||||
}
|
||||
|
||||
finishDocument(null);
|
||||
}
|
||||
allDocsIndexed = true;
|
||||
|
|
|
@ -169,16 +169,10 @@ public final class FieldInfo {
|
|||
// should only be called by FieldInfos#addOrUpdate
|
||||
void update(boolean indexed, boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions) {
|
||||
//System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" + omitNorms + " this.omitNorms=" + this.omitNorms);
|
||||
if (this.indexed != indexed) {
|
||||
this.indexed = true; // once indexed, always index
|
||||
}
|
||||
this.indexed |= indexed; // once indexed, always indexed
|
||||
if (indexed) { // if updated field data is not for indexing, leave the updates out
|
||||
if (this.storeTermVector != storeTermVector) {
|
||||
this.storeTermVector = true; // once vector, always vector
|
||||
}
|
||||
if (this.storePayloads != storePayloads) {
|
||||
this.storePayloads = true;
|
||||
}
|
||||
this.storeTermVector |= storeTermVector; // once vector, always vector
|
||||
this.storePayloads |= storePayloads;
|
||||
if (this.omitNorms != omitNorms) {
|
||||
this.omitNorms = true; // if one require omitNorms at least once, it remains off for life
|
||||
this.normType = null;
|
||||
|
|
|
@ -17,6 +17,10 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream; // javadocs
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
|
||||
/**
|
||||
|
@ -37,6 +41,11 @@ public final class FieldInvertState {
|
|||
float boost;
|
||||
AttributeSource attributeSource;
|
||||
|
||||
OffsetAttribute offsetAttribute;
|
||||
PositionIncrementAttribute posIncrAttribute;
|
||||
PayloadAttribute payloadAttribute;
|
||||
TermToBytesRefAttribute termAttribute;
|
||||
|
||||
/** Creates {code FieldInvertState} for the specified
|
||||
* field name. */
|
||||
public FieldInvertState(String name) {
|
||||
|
@ -65,7 +74,20 @@ public final class FieldInvertState {
|
|||
maxTermFrequency = 0;
|
||||
uniqueTermCount = 0;
|
||||
boost = 1.0f;
|
||||
attributeSource = null;
|
||||
}
|
||||
|
||||
// TODO: better name?
|
||||
/**
|
||||
* Sets attributeSource to a new instance.
|
||||
*/
|
||||
void setAttributeSource(AttributeSource attributeSource) {
|
||||
if (this.attributeSource != attributeSource) {
|
||||
this.attributeSource = attributeSource;
|
||||
termAttribute = attributeSource.getAttribute(TermToBytesRefAttribute.class);
|
||||
posIncrAttribute = attributeSource.addAttribute(PositionIncrementAttribute.class);
|
||||
offsetAttribute = attributeSource.addAttribute(OffsetAttribute.class);
|
||||
payloadAttribute = attributeSource.getAttribute(PayloadAttribute.class);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -126,7 +126,7 @@ class FreqProxFields extends Fields {
|
|||
|
||||
@Override
|
||||
public boolean hasPayloads() {
|
||||
return terms.hasPayloads;
|
||||
return terms.sawPayloads;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -140,10 +140,10 @@ class FreqProxFields extends Fields {
|
|||
|
||||
public FreqProxTermsEnum(FreqProxTermsWriterPerField terms) {
|
||||
this.terms = terms;
|
||||
this.numTerms = terms.termsHashPerField.bytesHash.size();
|
||||
this.numTerms = terms.bytesHash.size();
|
||||
sortedTermIDs = terms.sortedTermIDs;
|
||||
assert sortedTermIDs != null;
|
||||
postingsArray = (FreqProxPostingsArray) terms.termsHashPerField.postingsArray;
|
||||
postingsArray = (FreqProxPostingsArray) terms.postingsArray;
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
|
@ -161,7 +161,7 @@ class FreqProxFields extends Fields {
|
|||
while (hi >= lo) {
|
||||
int mid = (lo + hi) >>> 1;
|
||||
int textStart = postingsArray.textStarts[sortedTermIDs[mid]];
|
||||
terms.termsHashPerField.bytePool.setBytesRef(scratch, textStart);
|
||||
terms.bytePool.setBytesRef(scratch, textStart);
|
||||
int cmp = scratch.compareTo(text);
|
||||
if (cmp < 0) {
|
||||
lo = mid + 1;
|
||||
|
@ -186,7 +186,7 @@ class FreqProxFields extends Fields {
|
|||
public void seekExact(long ord) {
|
||||
this.ord = (int) ord;
|
||||
int textStart = postingsArray.textStarts[sortedTermIDs[this.ord]];
|
||||
terms.termsHashPerField.bytePool.setBytesRef(scratch, textStart);
|
||||
terms.bytePool.setBytesRef(scratch, textStart);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -196,7 +196,7 @@ class FreqProxFields extends Fields {
|
|||
return null;
|
||||
} else {
|
||||
int textStart = postingsArray.textStarts[sortedTermIDs[ord]];
|
||||
terms.termsHashPerField.bytePool.setBytesRef(scratch, textStart);
|
||||
terms.bytePool.setBytesRef(scratch, textStart);
|
||||
return scratch;
|
||||
}
|
||||
}
|
||||
|
@ -324,7 +324,7 @@ class FreqProxFields extends Fields {
|
|||
|
||||
public void reset(int termID) {
|
||||
this.termID = termID;
|
||||
terms.termsHashPerField.initReader(reader, termID, 0);
|
||||
terms.initReader(reader, termID, 0);
|
||||
ended = false;
|
||||
docID = 0;
|
||||
}
|
||||
|
@ -415,8 +415,8 @@ class FreqProxFields extends Fields {
|
|||
|
||||
public void reset(int termID) {
|
||||
this.termID = termID;
|
||||
terms.termsHashPerField.initReader(reader, termID, 0);
|
||||
terms.termsHashPerField.initReader(posReader, termID, 1);
|
||||
terms.initReader(reader, termID, 0);
|
||||
terms.initReader(posReader, termID, 1);
|
||||
ended = false;
|
||||
docID = 0;
|
||||
posLeft = 0;
|
||||
|
|
|
@ -23,13 +23,13 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
|
||||
final class FreqProxTermsWriter extends TermsHashConsumer {
|
||||
final class FreqProxTermsWriter extends TermsHash {
|
||||
|
||||
@Override
|
||||
void abort() {}
|
||||
public FreqProxTermsWriter(DocumentsWriterPerThread docWriter, TermsHash termVectors) {
|
||||
super(docWriter, true, termVectors);
|
||||
}
|
||||
|
||||
private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException {
|
||||
// Process any pending Term deletes for this newly
|
||||
|
@ -77,21 +77,16 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
// TODO: would be nice to factor out more of this, eg the
|
||||
// FreqProxFieldMergeState, and code to visit all Fields
|
||||
// under the same FieldInfo together, up into TermsHash*.
|
||||
// Other writers would presumably share alot of this...
|
||||
|
||||
@Override
|
||||
public void flush(Map<String,TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
public void flush(Map<String,TermsHashPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
super.flush(fieldsToFlush, state);
|
||||
|
||||
// Gather all FieldData's that have postings, across all
|
||||
// ThreadStates
|
||||
// Gather all fields that saw any postings:
|
||||
List<FreqProxTermsWriterPerField> allFields = new ArrayList<>();
|
||||
|
||||
for (TermsHashConsumerPerField f : fieldsToFlush.values()) {
|
||||
for (TermsHashPerField f : fieldsToFlush.values()) {
|
||||
final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) f;
|
||||
if (perField.termsHashPerField.bytesHash.size() > 0) {
|
||||
if (perField.bytesHash.size() > 0) {
|
||||
perField.sortPostings();
|
||||
assert perField.fieldInfo.isIndexed();
|
||||
allFields.add(perField);
|
||||
|
@ -108,18 +103,8 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
|
|||
state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state).write(fields);
|
||||
}
|
||||
|
||||
BytesRef payload;
|
||||
|
||||
@Override
|
||||
public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
|
||||
return new FreqProxTermsWriterPerField(termsHashPerField, this, fieldInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
void finishDocument(TermsHash termsHash) {
|
||||
}
|
||||
|
||||
@Override
|
||||
void startDocument() {
|
||||
public TermsHashPerField addField(FieldInvertState invertState, FieldInfo fieldInfo) {
|
||||
return new FreqProxTermsWriterPerField(invertState, this, fieldInfo, nextTermsHash.addField(invertState, fieldInfo));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,8 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
|
@ -26,16 +28,13 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
// TODO: break into separate freq and prox writers as
|
||||
// codecs; make separate container (tii/tis/skip/*) that can
|
||||
// be configured as any number of files 1..N
|
||||
final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable<FreqProxTermsWriterPerField> {
|
||||
final class FreqProxTermsWriterPerField extends TermsHashPerField {
|
||||
|
||||
final FreqProxTermsWriter parent;
|
||||
final TermsHashPerField termsHashPerField;
|
||||
final FieldInfo fieldInfo;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final FieldInvertState fieldState;
|
||||
boolean hasFreq;
|
||||
boolean hasProx;
|
||||
boolean hasOffsets;
|
||||
private FreqProxPostingsArray freqProxPostingsArray;
|
||||
|
||||
final boolean hasFreq;
|
||||
final boolean hasProx;
|
||||
final boolean hasOffsets;
|
||||
PayloadAttribute payloadAttribute;
|
||||
OffsetAttribute offsetAttribute;
|
||||
long sumTotalTermFreq;
|
||||
|
@ -44,111 +43,67 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
// How many docs have this field:
|
||||
int docCount;
|
||||
|
||||
public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriter parent, FieldInfo fieldInfo) {
|
||||
this.termsHashPerField = termsHashPerField;
|
||||
this.parent = parent;
|
||||
this.fieldInfo = fieldInfo;
|
||||
docState = termsHashPerField.docState;
|
||||
fieldState = termsHashPerField.fieldState;
|
||||
setIndexOptions(fieldInfo.getIndexOptions());
|
||||
/** Set to true if any token had a payload in the current
|
||||
* segment. */
|
||||
boolean sawPayloads;
|
||||
|
||||
public FreqProxTermsWriterPerField(FieldInvertState invertState, TermsHash termsHash, FieldInfo fieldInfo, TermsHashPerField nextPerField) {
|
||||
super(fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 ? 2 : 1, invertState, termsHash, nextPerField, fieldInfo);
|
||||
IndexOptions indexOptions = fieldInfo.getIndexOptions();
|
||||
assert indexOptions != null;
|
||||
hasFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
|
||||
hasProx = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
||||
hasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getStreamCount() {
|
||||
if (!hasProx) {
|
||||
return 1;
|
||||
} else {
|
||||
return 2;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void finish() {
|
||||
void finish() throws IOException {
|
||||
super.finish();
|
||||
sumDocFreq += fieldState.uniqueTermCount;
|
||||
sumTotalTermFreq += fieldState.length;
|
||||
if (fieldState.length > 0) {
|
||||
docCount++;
|
||||
}
|
||||
|
||||
if (hasPayloads) {
|
||||
if (sawPayloads) {
|
||||
fieldInfo.setStorePayloads();
|
||||
}
|
||||
}
|
||||
|
||||
boolean hasPayloads;
|
||||
|
||||
@Override
|
||||
void skippingLongTerm() {}
|
||||
|
||||
@Override
|
||||
public int compareTo(FreqProxTermsWriterPerField other) {
|
||||
return fieldInfo.name.compareTo(other.fieldInfo.name);
|
||||
}
|
||||
|
||||
private void setIndexOptions(IndexOptions indexOptions) {
|
||||
if (indexOptions == null) {
|
||||
// field could later be updated with indexed=true, so set everything on
|
||||
hasFreq = hasProx = hasOffsets = true;
|
||||
} else {
|
||||
hasFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
|
||||
hasProx = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
||||
hasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean start(IndexableField[] fields, int count) {
|
||||
boolean start(IndexableField f, boolean first) {
|
||||
super.start(f, first);
|
||||
payloadAttribute = fieldState.payloadAttribute;
|
||||
offsetAttribute = fieldState.offsetAttribute;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
void start(IndexableField f) {
|
||||
if (fieldState.attributeSource.hasAttribute(PayloadAttribute.class)) {
|
||||
payloadAttribute = fieldState.attributeSource.getAttribute(PayloadAttribute.class);
|
||||
} else {
|
||||
payloadAttribute = null;
|
||||
}
|
||||
if (hasOffsets) {
|
||||
offsetAttribute = fieldState.attributeSource.addAttribute(OffsetAttribute.class);
|
||||
} else {
|
||||
offsetAttribute = null;
|
||||
}
|
||||
}
|
||||
|
||||
void writeProx(final int termID, int proxCode) {
|
||||
//System.out.println("writeProx termID=" + termID + " proxCode=" + proxCode);
|
||||
assert hasProx;
|
||||
final BytesRef payload;
|
||||
void writeProx(int termID, int proxCode) {
|
||||
if (payloadAttribute == null) {
|
||||
payload = null;
|
||||
writeVInt(1, proxCode<<1);
|
||||
} else {
|
||||
payload = payloadAttribute.getPayload();
|
||||
BytesRef payload = payloadAttribute.getPayload();
|
||||
if (payload != null && payload.length > 0) {
|
||||
writeVInt(1, (proxCode<<1)|1);
|
||||
writeVInt(1, payload.length);
|
||||
writeBytes(1, payload.bytes, payload.offset, payload.length);
|
||||
sawPayloads = true;
|
||||
} else {
|
||||
writeVInt(1, proxCode<<1);
|
||||
}
|
||||
}
|
||||
|
||||
if (payload != null && payload.length > 0) {
|
||||
termsHashPerField.writeVInt(1, (proxCode<<1)|1);
|
||||
termsHashPerField.writeVInt(1, payload.length);
|
||||
termsHashPerField.writeBytes(1, payload.bytes, payload.offset, payload.length);
|
||||
hasPayloads = true;
|
||||
} else {
|
||||
termsHashPerField.writeVInt(1, proxCode<<1);
|
||||
}
|
||||
|
||||
FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
|
||||
postings.lastPositions[termID] = fieldState.position;
|
||||
assert postingsArray == freqProxPostingsArray;
|
||||
freqProxPostingsArray.lastPositions[termID] = fieldState.position;
|
||||
}
|
||||
|
||||
void writeOffsets(final int termID, int offsetAccum) {
|
||||
assert hasOffsets;
|
||||
void writeOffsets(int termID, int offsetAccum) {
|
||||
final int startOffset = offsetAccum + offsetAttribute.startOffset();
|
||||
final int endOffset = offsetAccum + offsetAttribute.endOffset();
|
||||
//System.out.println("writeOffsets termID=" + termID + " prevOffset=" + prevOffset + " startOff=" + startOffset + " endOff=" + endOffset);
|
||||
FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
|
||||
assert startOffset - postings.lastOffsets[termID] >= 0;
|
||||
termsHashPerField.writeVInt(1, startOffset - postings.lastOffsets[termID]);
|
||||
termsHashPerField.writeVInt(1, endOffset - startOffset);
|
||||
|
||||
postings.lastOffsets[termID] = startOffset;
|
||||
assert startOffset - freqProxPostingsArray.lastOffsets[termID] >= 0;
|
||||
writeVInt(1, startOffset - freqProxPostingsArray.lastOffsets[termID]);
|
||||
writeVInt(1, endOffset - startOffset);
|
||||
freqProxPostingsArray.lastOffsets[termID] = startOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -157,9 +112,11 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
// flush
|
||||
assert docState.testPoint("FreqProxTermsWriterPerField.newTerm start");
|
||||
|
||||
FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
|
||||
final FreqProxPostingsArray postings = freqProxPostingsArray;
|
||||
|
||||
postings.lastDocIDs[termID] = docState.docID;
|
||||
if (!hasFreq) {
|
||||
assert postings.termFreqs == null;
|
||||
postings.lastDocCodes[termID] = docState.docID;
|
||||
} else {
|
||||
postings.lastDocCodes[termID] = docState.docID << 1;
|
||||
|
@ -182,15 +139,16 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
|
||||
assert docState.testPoint("FreqProxTermsWriterPerField.addTerm start");
|
||||
|
||||
FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
|
||||
final FreqProxPostingsArray postings = freqProxPostingsArray;
|
||||
|
||||
assert !hasFreq || postings.termFreqs[termID] > 0;
|
||||
|
||||
if (!hasFreq) {
|
||||
assert postings.termFreqs == null;
|
||||
if (docState.docID != postings.lastDocIDs[termID]) {
|
||||
// New document; now encode docCode for previous doc:
|
||||
assert docState.docID > postings.lastDocIDs[termID];
|
||||
termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]);
|
||||
writeVInt(0, postings.lastDocCodes[termID]);
|
||||
postings.lastDocCodes[termID] = docState.docID - postings.lastDocIDs[termID];
|
||||
postings.lastDocIDs[termID] = docState.docID;
|
||||
fieldState.uniqueTermCount++;
|
||||
|
@ -203,11 +161,13 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
// Now that we know doc freq for previous doc,
|
||||
// write it & lastDocCode
|
||||
if (1 == postings.termFreqs[termID]) {
|
||||
termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]|1);
|
||||
writeVInt(0, postings.lastDocCodes[termID]|1);
|
||||
} else {
|
||||
termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]);
|
||||
termsHashPerField.writeVInt(0, postings.termFreqs[termID]);
|
||||
writeVInt(0, postings.lastDocCodes[termID]);
|
||||
writeVInt(0, postings.termFreqs[termID]);
|
||||
}
|
||||
|
||||
// Init freq for the current document
|
||||
postings.termFreqs[termID] = 1;
|
||||
fieldState.maxTermFrequency = Math.max(1, fieldState.maxTermFrequency);
|
||||
postings.lastDocCodes[termID] = (docState.docID - postings.lastDocIDs[termID]) << 1;
|
||||
|
@ -226,15 +186,25 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
fieldState.maxTermFrequency = Math.max(fieldState.maxTermFrequency, ++postings.termFreqs[termID]);
|
||||
if (hasProx) {
|
||||
writeProx(termID, fieldState.position-postings.lastPositions[termID]);
|
||||
}
|
||||
if (hasOffsets) {
|
||||
writeOffsets(termID, fieldState.offset);
|
||||
if (hasOffsets) {
|
||||
writeOffsets(termID, fieldState.offset);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newPostingsArray() {
|
||||
freqProxPostingsArray = (FreqProxPostingsArray) postingsArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
ParallelPostingsArray createPostingsArray(int size) {
|
||||
IndexOptions indexOptions = fieldInfo.getIndexOptions();
|
||||
assert indexOptions != null;
|
||||
boolean hasFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
|
||||
boolean hasProx = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
||||
boolean hasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
|
||||
return new FreqProxPostingsArray(size, hasFreq, hasProx, hasOffsets);
|
||||
}
|
||||
|
||||
|
@ -307,15 +277,4 @@ final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implem
|
|||
return bytes;
|
||||
}
|
||||
}
|
||||
|
||||
public void abort() {}
|
||||
|
||||
BytesRef payload;
|
||||
|
||||
int[] sortedTermIDs;
|
||||
|
||||
void sortPostings() {
|
||||
assert sortedTermIDs == null;
|
||||
sortedTermIDs = termsHashPerField.sortPostings();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,6 +39,8 @@ public interface IndexableFieldType {
|
|||
* <p>
|
||||
* This has no effect if {@link #indexed()} returns false.
|
||||
*/
|
||||
// TODO: shouldn't we remove this? Whether/how a field is
|
||||
// tokenized is an impl detail under Field?
|
||||
public boolean tokenized();
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,36 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
abstract class InvertedDocConsumer {
|
||||
|
||||
/** Abort (called after hitting AbortException) */
|
||||
abstract void abort();
|
||||
|
||||
/** Flush a new segment */
|
||||
abstract void flush(Map<String, InvertedDocConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
|
||||
|
||||
abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
|
||||
|
||||
abstract void startDocument() throws IOException;
|
||||
|
||||
abstract void finishDocument() throws IOException;
|
||||
}
|
|
@ -1,42 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
abstract class InvertedDocConsumerPerField {
|
||||
|
||||
// Called once per field, and is given all IndexableField
|
||||
// occurrences for this field in the document. Return
|
||||
// true if you wish to see inverted tokens for these
|
||||
// fields:
|
||||
abstract boolean start(IndexableField[] fields, int count) throws IOException;
|
||||
|
||||
// Called before a field instance is being processed
|
||||
abstract void start(IndexableField field);
|
||||
|
||||
// Called once per inverted token
|
||||
abstract void add() throws IOException;
|
||||
|
||||
// Called once per field per document, after all IndexableFields
|
||||
// are inverted
|
||||
abstract void finish() throws IOException;
|
||||
|
||||
// Called on hitting an aborting exception
|
||||
abstract void abort();
|
||||
}
|
|
@ -1,29 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
abstract class InvertedDocEndConsumer {
|
||||
abstract void flush(Map<String, InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
|
||||
abstract void abort();
|
||||
abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
|
||||
abstract void startDocument() throws IOException;
|
||||
abstract void finishDocument() throws IOException;
|
||||
}
|
|
@ -1,25 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/*
|
||||
* 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 class InvertedDocEndConsumerPerField {
|
||||
abstract void finish() throws IOException;
|
||||
abstract void abort();
|
||||
}
|
|
@ -1,84 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
// TODO FI: norms could actually be stored as doc store
|
||||
|
||||
/** Writes norms. Each thread X field accumulates the norms
|
||||
* for the doc/fields it saw, then the flush method below
|
||||
* merges all of these together into a single _X.nrm file.
|
||||
*/
|
||||
|
||||
final class NormsConsumer extends InvertedDocEndConsumer {
|
||||
|
||||
@Override
|
||||
void abort() {}
|
||||
|
||||
@Override
|
||||
public void flush(Map<String,InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
|
||||
boolean success = false;
|
||||
DocValuesConsumer normsConsumer = null;
|
||||
try {
|
||||
if (state.fieldInfos.hasNorms()) {
|
||||
NormsFormat normsFormat = state.segmentInfo.getCodec().normsFormat();
|
||||
assert normsFormat != null;
|
||||
normsConsumer = normsFormat.normsConsumer(state);
|
||||
|
||||
for (FieldInfo fi : state.fieldInfos) {
|
||||
final NormsConsumerPerField toWrite = (NormsConsumerPerField) fieldsToFlush.get(fi.name);
|
||||
// we must check the final value of omitNorms for the fieldinfo, it could have
|
||||
// changed for this field since the first time we added it.
|
||||
if (!fi.omitsNorms()) {
|
||||
if (toWrite != null && !toWrite.isEmpty()) {
|
||||
toWrite.flush(state, normsConsumer);
|
||||
assert fi.getNormType() == DocValuesType.NUMERIC;
|
||||
} else if (fi.isIndexed()) {
|
||||
assert fi.getNormType() == null: "got " + fi.getNormType() + "; field=" + fi.name;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(normsConsumer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(normsConsumer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void finishDocument() {}
|
||||
|
||||
@Override
|
||||
void startDocument() {}
|
||||
|
||||
@Override
|
||||
InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo) {
|
||||
return new NormsConsumerPerField(docInverterPerField, fieldInfo, this);
|
||||
}
|
||||
}
|
|
@ -1,72 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
|
||||
final class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
|
||||
private final FieldInfo fieldInfo;
|
||||
private final DocumentsWriterPerThread.DocState docState;
|
||||
private final Similarity similarity;
|
||||
private final FieldInvertState fieldState;
|
||||
private NumericDocValuesWriter consumer;
|
||||
|
||||
public NormsConsumerPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo, NormsConsumer parent) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
docState = docInverterPerField.docState;
|
||||
fieldState = docInverterPerField.fieldState;
|
||||
similarity = docState.similarity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(NormsConsumerPerField other) {
|
||||
return fieldInfo.name.compareTo(other.fieldInfo.name);
|
||||
}
|
||||
|
||||
@Override
|
||||
void finish() throws IOException {
|
||||
if (fieldInfo.isIndexed() && !fieldInfo.omitsNorms()) {
|
||||
if (consumer == null) {
|
||||
fieldInfo.setNormValueType(FieldInfo.DocValuesType.NUMERIC);
|
||||
consumer = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed, false);
|
||||
}
|
||||
consumer.addValue(docState.docID, similarity.computeNorm(fieldState));
|
||||
}
|
||||
}
|
||||
|
||||
void flush(SegmentWriteState state, DocValuesConsumer normsWriter) throws IOException {
|
||||
int docCount = state.segmentInfo.getDocCount();
|
||||
if (consumer == null) {
|
||||
return; // null type - not omitted but not written -
|
||||
// meaning the only docs that had
|
||||
// norms hit exceptions (but indexed=true is set...)
|
||||
}
|
||||
consumer.finish(docCount);
|
||||
consumer.flush(state, normsWriter);
|
||||
}
|
||||
|
||||
boolean isEmpty() {
|
||||
return consumer == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
//
|
||||
}
|
||||
}
|
|
@ -97,10 +97,6 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
}
|
||||
|
||||
// iterates over the values we have in ram
|
||||
private class NumericIterator implements Iterator<Number> {
|
||||
final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
|
||||
|
|
|
@ -97,7 +97,7 @@ class PrefixCodedTerms implements Iterable<Term> {
|
|||
/** Builds a PrefixCodedTerms: call add repeatedly, then finish. */
|
||||
public static class Builder {
|
||||
private RAMFile buffer = new RAMFile();
|
||||
private RAMOutputStream output = new RAMOutputStream(buffer);
|
||||
private RAMOutputStream output = new RAMOutputStream(buffer, false);
|
||||
private Term lastTerm = new Term("");
|
||||
|
||||
/** add a term */
|
||||
|
|
|
@ -139,10 +139,6 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
}
|
||||
|
||||
// iterates over the unique values we have in ram
|
||||
private class ValuesIterator implements Iterator<BytesRef> {
|
||||
final int sortedValues[];
|
||||
|
|
|
@ -187,10 +187,6 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
|
|||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
}
|
||||
|
||||
// iterates over the unique values we have in ram
|
||||
private class ValuesIterator implements Iterator<BytesRef> {
|
||||
final int sortedValues[];
|
||||
|
|
|
@ -1,28 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
abstract class StoredFieldsConsumer {
|
||||
abstract void addField(int docID, StorableField field, FieldInfo fieldInfo) throws IOException;
|
||||
abstract void flush(SegmentWriteState state) throws IOException;
|
||||
abstract void abort() throws IOException;
|
||||
abstract void startDocument() throws IOException;
|
||||
abstract void finishDocument() throws IOException;
|
||||
}
|
|
@ -1,157 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** This is a StoredFieldsConsumer that writes stored fields. */
|
||||
final class StoredFieldsProcessor extends StoredFieldsConsumer {
|
||||
|
||||
StoredFieldsWriter fieldsWriter;
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
int lastDocID;
|
||||
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final Codec codec;
|
||||
|
||||
public StoredFieldsProcessor(DocumentsWriterPerThread docWriter) {
|
||||
this.docWriter = docWriter;
|
||||
this.docState = docWriter.docState;
|
||||
this.codec = docWriter.codec;
|
||||
}
|
||||
|
||||
private int numStoredFields;
|
||||
private StorableField[] storedFields = new StorableField[1];
|
||||
private FieldInfo[] fieldInfos = new FieldInfo[1];
|
||||
|
||||
public void reset() {
|
||||
numStoredFields = 0;
|
||||
Arrays.fill(storedFields, null);
|
||||
Arrays.fill(fieldInfos, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startDocument() {
|
||||
reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state) throws IOException {
|
||||
int numDocs = state.segmentInfo.getDocCount();
|
||||
if (numDocs > 0) {
|
||||
// It's possible that all documents seen in this segment
|
||||
// hit non-aborting exceptions, in which case we will
|
||||
// not have yet init'd the FieldsWriter:
|
||||
initFieldsWriter(state.context);
|
||||
fill(numDocs);
|
||||
}
|
||||
if (fieldsWriter != null) {
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldsWriter.finish(state.fieldInfos, numDocs);
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(fieldsWriter);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(fieldsWriter);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void initFieldsWriter(IOContext context) throws IOException {
|
||||
if (fieldsWriter == null) {
|
||||
fieldsWriter = codec.storedFieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegmentInfo(), context);
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
reset();
|
||||
|
||||
if (fieldsWriter != null) {
|
||||
fieldsWriter.abort();
|
||||
fieldsWriter = null;
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
||||
/** Fills in any hole in the docIDs */
|
||||
void fill(int docID) throws IOException {
|
||||
// We must "catch up" for all docs before us
|
||||
// that had no stored fields:
|
||||
while(lastDocID < docID) {
|
||||
fieldsWriter.startDocument(0);
|
||||
lastDocID++;
|
||||
fieldsWriter.finishDocument();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void finishDocument() throws IOException {
|
||||
assert docWriter.testPoint("StoredFieldsWriter.finishDocument start");
|
||||
|
||||
initFieldsWriter(IOContext.DEFAULT);
|
||||
fill(docState.docID);
|
||||
|
||||
if (fieldsWriter != null && numStoredFields > 0) {
|
||||
fieldsWriter.startDocument(numStoredFields);
|
||||
for (int i = 0; i < numStoredFields; i++) {
|
||||
fieldsWriter.writeField(fieldInfos[i], storedFields[i]);
|
||||
}
|
||||
fieldsWriter.finishDocument();
|
||||
lastDocID++;
|
||||
}
|
||||
|
||||
reset();
|
||||
assert docWriter.testPoint("StoredFieldsWriter.finishDocument end");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addField(int docID, StorableField field, FieldInfo fieldInfo) {
|
||||
if (field.fieldType().stored()) {
|
||||
if (numStoredFields == storedFields.length) {
|
||||
int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||
StorableField[] newArray = new StorableField[newSize];
|
||||
System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
|
||||
storedFields = newArray;
|
||||
|
||||
FieldInfo[] newInfoArray = new FieldInfo[newSize];
|
||||
System.arraycopy(fieldInfos, 0, newInfoArray, 0, numStoredFields);
|
||||
fieldInfos = newInfoArray;
|
||||
}
|
||||
|
||||
storedFields[numStoredFields] = field;
|
||||
fieldInfos[numStoredFields] = fieldInfo;
|
||||
numStoredFields++;
|
||||
|
||||
assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -29,28 +29,32 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
final class TermVectorsConsumer extends TermsHashConsumer {
|
||||
final class TermVectorsConsumer extends TermsHash {
|
||||
|
||||
TermVectorsWriter writer;
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
|
||||
/** Scratch term used by TermVectorsConsumerPerField.finishDocument. */
|
||||
final BytesRef flushTerm = new BytesRef();
|
||||
|
||||
// Used by perField when serializing the term vectors
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
|
||||
/** Used by TermVectorsConsumerPerField when serializing
|
||||
* the term vectors. */
|
||||
final ByteSliceReader vectorSliceReaderPos = new ByteSliceReader();
|
||||
final ByteSliceReader vectorSliceReaderOff = new ByteSliceReader();
|
||||
|
||||
boolean hasVectors;
|
||||
int numVectorFields;
|
||||
int lastDocID;
|
||||
private TermVectorsConsumerPerField[] perFields = new TermVectorsConsumerPerField[1];
|
||||
|
||||
public TermVectorsConsumer(DocumentsWriterPerThread docWriter) {
|
||||
super(docWriter, false, null);
|
||||
this.docWriter = docWriter;
|
||||
docState = docWriter.docState;
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(Map<String, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
void flush(Map<String, TermsHashPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
if (writer != null) {
|
||||
int numDocs = state.segmentInfo.getDocCount();
|
||||
assert numDocs > 0;
|
||||
|
@ -78,7 +82,7 @@ final class TermVectorsConsumer extends TermsHashConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
private final void initTermVectorsWriter() throws IOException {
|
||||
private void initTermVectorsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
|
||||
writer = docWriter.codec.termVectorsFormat().vectorsWriter(docWriter.directory, docWriter.getSegmentInfo(), context);
|
||||
|
@ -87,7 +91,7 @@ final class TermVectorsConsumer extends TermsHashConsumer {
|
|||
}
|
||||
|
||||
@Override
|
||||
void finishDocument(TermsHash termsHash) throws IOException {
|
||||
void finishDocument() throws IOException {
|
||||
|
||||
assert docWriter.testPoint("TermVectorsTermsWriter.finishDocument start");
|
||||
|
||||
|
@ -95,6 +99,9 @@ final class TermVectorsConsumer extends TermsHashConsumer {
|
|||
return;
|
||||
}
|
||||
|
||||
// Fields in term vectors are UTF16 sorted:
|
||||
ArrayUtil.introSort(perFields, 0, numVectorFields);
|
||||
|
||||
initTermVectorsWriter();
|
||||
|
||||
fill(docState.docID);
|
||||
|
@ -110,32 +117,35 @@ final class TermVectorsConsumer extends TermsHashConsumer {
|
|||
|
||||
lastDocID++;
|
||||
|
||||
termsHash.reset();
|
||||
reset();
|
||||
super.reset();
|
||||
resetFields();
|
||||
assert docWriter.testPoint("TermVectorsTermsWriter.finishDocument end");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
hasVectors = false;
|
||||
try {
|
||||
super.abort();
|
||||
} finally {
|
||||
if (writer != null) {
|
||||
writer.abort();
|
||||
writer = null;
|
||||
}
|
||||
|
||||
if (writer != null) {
|
||||
writer.abort();
|
||||
writer = null;
|
||||
lastDocID = 0;
|
||||
reset();
|
||||
}
|
||||
|
||||
lastDocID = 0;
|
||||
reset();
|
||||
}
|
||||
|
||||
void reset() {
|
||||
Arrays.fill(perFields, null);// don't hang onto stuff from previous doc
|
||||
void resetFields() {
|
||||
Arrays.fill(perFields, null); // don't hang onto stuff from previous doc
|
||||
numVectorFields = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
|
||||
return new TermVectorsConsumerPerField(termsHashPerField, this, fieldInfo);
|
||||
public TermsHashPerField addField(FieldInvertState invertState, FieldInfo fieldInfo) {
|
||||
return new TermVectorsConsumerPerField(invertState, this, fieldInfo);
|
||||
}
|
||||
|
||||
void addFieldToFlush(TermVectorsConsumerPerField fieldToFlush) {
|
||||
|
@ -151,24 +161,7 @@ final class TermVectorsConsumer extends TermsHashConsumer {
|
|||
|
||||
@Override
|
||||
void startDocument() {
|
||||
assert clearLastVectorFieldName();
|
||||
reset();
|
||||
resetFields();
|
||||
numVectorFields = 0;
|
||||
}
|
||||
|
||||
// Called only by assert
|
||||
final boolean clearLastVectorFieldName() {
|
||||
lastVectorFieldName = null;
|
||||
return true;
|
||||
}
|
||||
|
||||
// Called only by assert
|
||||
String lastVectorFieldName;
|
||||
final boolean vectorFieldsInOrder(FieldInfo fi) {
|
||||
try {
|
||||
return lastVectorFieldName != null ? lastVectorFieldName.compareTo(fi.name) < 0 : true;
|
||||
} finally {
|
||||
lastVectorFieldName = fi.name;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,149 +22,69 @@ import java.io.IOException;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
||||
final class TermVectorsConsumerPerField extends TermsHashPerField {
|
||||
|
||||
private TermVectorsPostingsArray termVectorsPostingsArray;
|
||||
|
||||
final TermsHashPerField termsHashPerField;
|
||||
final TermVectorsConsumer termsWriter;
|
||||
final FieldInfo fieldInfo;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final FieldInvertState fieldState;
|
||||
|
||||
boolean doVectors;
|
||||
boolean doVectorPositions;
|
||||
boolean doVectorOffsets;
|
||||
boolean doVectorPayloads;
|
||||
|
||||
int maxNumPostings;
|
||||
OffsetAttribute offsetAttribute;
|
||||
PayloadAttribute payloadAttribute;
|
||||
boolean hasPayloads; // if enabled, and we actually saw any for this field
|
||||
|
||||
public TermVectorsConsumerPerField(TermsHashPerField termsHashPerField, TermVectorsConsumer termsWriter, FieldInfo fieldInfo) {
|
||||
this.termsHashPerField = termsHashPerField;
|
||||
public TermVectorsConsumerPerField(FieldInvertState invertState, TermVectorsConsumer termsWriter, FieldInfo fieldInfo) {
|
||||
super(2, invertState, termsWriter, null, fieldInfo);
|
||||
this.termsWriter = termsWriter;
|
||||
this.fieldInfo = fieldInfo;
|
||||
docState = termsHashPerField.docState;
|
||||
fieldState = termsHashPerField.fieldState;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getStreamCount() {
|
||||
return 2;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean start(IndexableField[] fields, int count) {
|
||||
doVectors = false;
|
||||
doVectorPositions = false;
|
||||
doVectorOffsets = false;
|
||||
doVectorPayloads = false;
|
||||
hasPayloads = false;
|
||||
|
||||
for(int i=0;i<count;i++) {
|
||||
IndexableField field = fields[i];
|
||||
if (field.fieldType().indexed()) {
|
||||
if (field.fieldType().storeTermVectors()) {
|
||||
doVectors = true;
|
||||
doVectorPositions |= field.fieldType().storeTermVectorPositions();
|
||||
doVectorOffsets |= field.fieldType().storeTermVectorOffsets();
|
||||
if (doVectorPositions) {
|
||||
doVectorPayloads |= field.fieldType().storeTermVectorPayloads();
|
||||
} else if (field.fieldType().storeTermVectorPayloads()) {
|
||||
// TODO: move this check somewhere else, and impl the other missing ones
|
||||
throw new IllegalArgumentException("cannot index term vector payloads without term vector positions (field=\"" + field.name() + "\")");
|
||||
}
|
||||
} else {
|
||||
if (field.fieldType().storeTermVectorOffsets()) {
|
||||
throw new IllegalArgumentException("cannot index term vector offsets when term vectors are not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (field.fieldType().storeTermVectorPositions()) {
|
||||
throw new IllegalArgumentException("cannot index term vector positions when term vectors are not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (field.fieldType().storeTermVectorPayloads()) {
|
||||
throw new IllegalArgumentException("cannot index term vector payloads when term vectors are not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (field.fieldType().storeTermVectors()) {
|
||||
throw new IllegalArgumentException("cannot index term vectors when field is not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (field.fieldType().storeTermVectorOffsets()) {
|
||||
throw new IllegalArgumentException("cannot index term vector offsets when field is not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (field.fieldType().storeTermVectorPositions()) {
|
||||
throw new IllegalArgumentException("cannot index term vector positions when field is not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (field.fieldType().storeTermVectorPayloads()) {
|
||||
throw new IllegalArgumentException("cannot index term vector payloads when field is not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (doVectors) {
|
||||
termsWriter.hasVectors = true;
|
||||
if (termsHashPerField.bytesHash.size() != 0) {
|
||||
// Only necessary if previous doc hit a
|
||||
// non-aborting exception while writing vectors in
|
||||
// this field:
|
||||
termsHashPerField.reset();
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: only if needed for performance
|
||||
//perThread.postingsCount = 0;
|
||||
|
||||
return doVectors;
|
||||
}
|
||||
|
||||
public void abort() {}
|
||||
|
||||
/** Called once per field per document if term vectors
|
||||
* are enabled, to write the vectors to
|
||||
* RAMOutputStream, which is then quickly flushed to
|
||||
* the real term vectors files in the Directory. */ @Override
|
||||
void finish() {
|
||||
if (!doVectors || termsHashPerField.bytesHash.size() == 0) {
|
||||
if (!doVectors || bytesHash.size() == 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
termsWriter.addFieldToFlush(this);
|
||||
}
|
||||
|
||||
void finishDocument() throws IOException {
|
||||
if (doVectors == false) {
|
||||
return;
|
||||
}
|
||||
|
||||
doVectors = false;
|
||||
|
||||
assert docState.testPoint("TermVectorsTermsWriterPerField.finish start");
|
||||
|
||||
final int numPostings = termsHashPerField.bytesHash.size();
|
||||
final int numPostings = bytesHash.size();
|
||||
|
||||
final BytesRef flushTerm = termsWriter.flushTerm;
|
||||
|
||||
assert numPostings >= 0;
|
||||
|
||||
if (numPostings > maxNumPostings)
|
||||
maxNumPostings = numPostings;
|
||||
|
||||
// This is called once, after inverting all occurrences
|
||||
// of a given field in the doc. At this point we flush
|
||||
// our hash into the DocWriter.
|
||||
|
||||
assert termsWriter.vectorFieldsInOrder(fieldInfo);
|
||||
|
||||
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
|
||||
TermVectorsPostingsArray postings = termVectorsPostingsArray;
|
||||
final TermVectorsWriter tv = termsWriter.writer;
|
||||
|
||||
final int[] termIDs = termsHashPerField.sortPostings();
|
||||
final int[] termIDs = sortPostings();
|
||||
|
||||
tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets, hasPayloads);
|
||||
|
||||
final ByteSliceReader posReader = doVectorPositions ? termsWriter.vectorSliceReaderPos : null;
|
||||
final ByteSliceReader offReader = doVectorOffsets ? termsWriter.vectorSliceReaderOff : null;
|
||||
|
||||
final ByteBlockPool termBytePool = termsHashPerField.termBytePool;
|
||||
|
||||
for(int j=0;j<numPostings;j++) {
|
||||
final int termID = termIDs[j];
|
||||
final int freq = postings.freqs[termID];
|
||||
|
@ -175,10 +95,10 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
|
||||
if (doVectorPositions || doVectorOffsets) {
|
||||
if (posReader != null) {
|
||||
termsHashPerField.initReader(posReader, termID, 0);
|
||||
initReader(posReader, termID, 0);
|
||||
}
|
||||
if (offReader != null) {
|
||||
termsHashPerField.initReader(offReader, termID, 1);
|
||||
initReader(offReader, termID, 1);
|
||||
}
|
||||
tv.addProx(freq, posReader, offReader);
|
||||
}
|
||||
|
@ -186,23 +106,91 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
}
|
||||
tv.finishField();
|
||||
|
||||
termsHashPerField.reset();
|
||||
reset();
|
||||
|
||||
fieldInfo.setStoreTermVectors();
|
||||
}
|
||||
|
||||
@Override
|
||||
void start(IndexableField f) {
|
||||
if (doVectorOffsets) {
|
||||
offsetAttribute = fieldState.attributeSource.addAttribute(OffsetAttribute.class);
|
||||
boolean start(IndexableField field, boolean first) {
|
||||
assert field.fieldType().indexed();
|
||||
|
||||
if (first) {
|
||||
|
||||
if (bytesHash.size() != 0) {
|
||||
// Only necessary if previous doc hit a
|
||||
// non-aborting exception while writing vectors in
|
||||
// this field:
|
||||
reset();
|
||||
}
|
||||
|
||||
bytesHash.reinit();
|
||||
|
||||
hasPayloads = false;
|
||||
|
||||
doVectors = field.fieldType().storeTermVectors();
|
||||
|
||||
if (doVectors) {
|
||||
|
||||
termsWriter.hasVectors = true;
|
||||
|
||||
doVectorPositions = field.fieldType().storeTermVectorPositions();
|
||||
|
||||
// Somewhat confusingly, unlike postings, you are
|
||||
// allowed to index TV offsets without TV positions:
|
||||
doVectorOffsets = field.fieldType().storeTermVectorOffsets();
|
||||
|
||||
if (doVectorPositions) {
|
||||
doVectorPayloads = field.fieldType().storeTermVectorPayloads();
|
||||
} else {
|
||||
doVectorPayloads = false;
|
||||
if (field.fieldType().storeTermVectorPayloads()) {
|
||||
// TODO: move this check somewhere else, and impl the other missing ones
|
||||
throw new IllegalArgumentException("cannot index term vector payloads without term vector positions (field=\"" + field.name() + "\")");
|
||||
}
|
||||
}
|
||||
|
||||
} else {
|
||||
if (field.fieldType().storeTermVectorOffsets()) {
|
||||
throw new IllegalArgumentException("cannot index term vector offsets when term vectors are not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (field.fieldType().storeTermVectorPositions()) {
|
||||
throw new IllegalArgumentException("cannot index term vector positions when term vectors are not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (field.fieldType().storeTermVectorPayloads()) {
|
||||
throw new IllegalArgumentException("cannot index term vector payloads when term vectors are not indexed (field=\"" + field.name() + "\")");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
offsetAttribute = null;
|
||||
if (doVectors != field.fieldType().storeTermVectors()) {
|
||||
throw new IllegalArgumentException("all instances of a given field name must have the same term vectors settings (storeTermVectors changed for field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (doVectorPositions != field.fieldType().storeTermVectorPositions()) {
|
||||
throw new IllegalArgumentException("all instances of a given field name must have the same term vectors settings (storeTermVectorPositions changed for field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (doVectorOffsets != field.fieldType().storeTermVectorOffsets()) {
|
||||
throw new IllegalArgumentException("all instances of a given field name must have the same term vectors settings (storeTermVectorOffsets changed for field=\"" + field.name() + "\")");
|
||||
}
|
||||
if (doVectorPayloads != field.fieldType().storeTermVectorPayloads()) {
|
||||
throw new IllegalArgumentException("all instances of a given field name must have the same term vectors settings (storeTermVectorPayloads changed for field=\"" + field.name() + "\")");
|
||||
}
|
||||
}
|
||||
if (doVectorPayloads && fieldState.attributeSource.hasAttribute(PayloadAttribute.class)) {
|
||||
payloadAttribute = fieldState.attributeSource.getAttribute(PayloadAttribute.class);
|
||||
} else {
|
||||
payloadAttribute = null;
|
||||
|
||||
if (doVectors) {
|
||||
if (doVectorOffsets) {
|
||||
offsetAttribute = fieldState.offsetAttribute;
|
||||
assert offsetAttribute != null;
|
||||
}
|
||||
|
||||
if (doVectorPayloads) {
|
||||
// Can be null:
|
||||
payloadAttribute = fieldState.payloadAttribute;
|
||||
} else {
|
||||
payloadAttribute = null;
|
||||
}
|
||||
}
|
||||
|
||||
return doVectors;
|
||||
}
|
||||
|
||||
void writeProx(TermVectorsPostingsArray postings, int termID) {
|
||||
|
@ -210,8 +198,8 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
int startOffset = fieldState.offset + offsetAttribute.startOffset();
|
||||
int endOffset = fieldState.offset + offsetAttribute.endOffset();
|
||||
|
||||
termsHashPerField.writeVInt(1, startOffset - postings.lastOffsets[termID]);
|
||||
termsHashPerField.writeVInt(1, endOffset - startOffset);
|
||||
writeVInt(1, startOffset - postings.lastOffsets[termID]);
|
||||
writeVInt(1, endOffset - startOffset);
|
||||
postings.lastOffsets[termID] = endOffset;
|
||||
}
|
||||
|
||||
|
@ -225,12 +213,12 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
|
||||
final int pos = fieldState.position - postings.lastPositions[termID];
|
||||
if (payload != null && payload.length > 0) {
|
||||
termsHashPerField.writeVInt(0, (pos<<1)|1);
|
||||
termsHashPerField.writeVInt(0, payload.length);
|
||||
termsHashPerField.writeBytes(0, payload.bytes, payload.offset, payload.length);
|
||||
writeVInt(0, (pos<<1)|1);
|
||||
writeVInt(0, payload.length);
|
||||
writeBytes(0, payload.bytes, payload.offset, payload.length);
|
||||
hasPayloads = true;
|
||||
} else {
|
||||
termsHashPerField.writeVInt(0, pos<<1);
|
||||
writeVInt(0, pos<<1);
|
||||
}
|
||||
postings.lastPositions[termID] = fieldState.position;
|
||||
}
|
||||
|
@ -239,7 +227,7 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
@Override
|
||||
void newTerm(final int termID) {
|
||||
assert docState.testPoint("TermVectorsTermsWriterPerField.newTerm start");
|
||||
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
|
||||
TermVectorsPostingsArray postings = termVectorsPostingsArray;
|
||||
|
||||
postings.freqs[termID] = 1;
|
||||
postings.lastOffsets[termID] = 0;
|
||||
|
@ -251,7 +239,7 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
@Override
|
||||
void addTerm(final int termID) {
|
||||
assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
|
||||
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
|
||||
TermVectorsPostingsArray postings = termVectorsPostingsArray;
|
||||
|
||||
postings.freqs[termID]++;
|
||||
|
||||
|
@ -259,7 +247,9 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
}
|
||||
|
||||
@Override
|
||||
void skippingLongTerm() {}
|
||||
public void newPostingsArray() {
|
||||
termVectorsPostingsArray = (TermVectorsPostingsArray) postingsArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
ParallelPostingsArray createPostingsArray(int size) {
|
||||
|
|
|
@ -22,21 +22,17 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.IntBlockPool;
|
||||
|
||||
/** This class implements {@link InvertedDocConsumer}, which
|
||||
* is passed each token produced by the analyzer on each
|
||||
* field. It stores these tokens in a hash table, and
|
||||
* allocates separate byte streams per token. Consumers of
|
||||
* this class, eg {@link FreqProxTermsWriter} and {@link
|
||||
* TermVectorsConsumer}, write their own byte streams
|
||||
* under each term.
|
||||
*/
|
||||
final class TermsHash extends InvertedDocConsumer {
|
||||
/** This class is passed each token produced by the analyzer
|
||||
* on each field during indexing, and it stores these
|
||||
* tokens in a hash table, and allocates separate byte
|
||||
* streams per token. Consumers of this class, eg {@link
|
||||
* FreqProxTermsWriter} and {@link TermVectorsConsumer},
|
||||
* write their own byte streams under each term. */
|
||||
abstract class TermsHash {
|
||||
|
||||
final TermsHashConsumer consumer;
|
||||
final TermsHash nextTermsHash;
|
||||
|
||||
final IntBlockPool intPool;
|
||||
|
@ -44,21 +40,12 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
ByteBlockPool termBytePool;
|
||||
final Counter bytesUsed;
|
||||
|
||||
final boolean primary;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
|
||||
// Used when comparing postings via termRefComp, in TermsHashPerField
|
||||
final BytesRef tr1 = new BytesRef();
|
||||
final BytesRef tr2 = new BytesRef();
|
||||
|
||||
// Used by perField to obtain terms from the analysis chain
|
||||
final BytesRef termBytesRef = new BytesRef(10);
|
||||
|
||||
final boolean trackAllocations;
|
||||
|
||||
public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) {
|
||||
TermsHash(final DocumentsWriterPerThread docWriter, boolean trackAllocations, TermsHash nextTermsHash) {
|
||||
this.docState = docWriter.docState;
|
||||
this.consumer = consumer;
|
||||
this.trackAllocations = trackAllocations;
|
||||
this.nextTermsHash = nextTermsHash;
|
||||
this.bytesUsed = trackAllocations ? docWriter.bytesUsed : Counter.newCounter();
|
||||
|
@ -67,19 +54,14 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
|
||||
if (nextTermsHash != null) {
|
||||
// We are primary
|
||||
primary = true;
|
||||
termBytePool = bytePool;
|
||||
nextTermsHash.termBytePool = bytePool;
|
||||
} else {
|
||||
primary = false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
reset();
|
||||
try {
|
||||
consumer.abort();
|
||||
reset();
|
||||
} finally {
|
||||
if (nextTermsHash != null) {
|
||||
nextTermsHash.abort();
|
||||
|
@ -94,50 +76,27 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
bytePool.reset(false, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(Map<String,InvertedDocConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
Map<String,TermsHashConsumerPerField> childFields = new HashMap<>();
|
||||
Map<String,InvertedDocConsumerPerField> nextChildFields;
|
||||
|
||||
void flush(Map<String,TermsHashPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
if (nextTermsHash != null) {
|
||||
nextChildFields = new HashMap<>();
|
||||
} else {
|
||||
nextChildFields = null;
|
||||
}
|
||||
|
||||
for (final Map.Entry<String,InvertedDocConsumerPerField> entry : fieldsToFlush.entrySet()) {
|
||||
TermsHashPerField perField = (TermsHashPerField) entry.getValue();
|
||||
childFields.put(entry.getKey(), perField.consumer);
|
||||
if (nextTermsHash != null) {
|
||||
nextChildFields.put(entry.getKey(), perField.nextPerField);
|
||||
Map<String,TermsHashPerField> nextChildFields = new HashMap<>();
|
||||
for (final Map.Entry<String,TermsHashPerField> entry : fieldsToFlush.entrySet()) {
|
||||
nextChildFields.put(entry.getKey(), entry.getValue().nextPerField);
|
||||
}
|
||||
}
|
||||
|
||||
consumer.flush(childFields, state);
|
||||
|
||||
if (nextTermsHash != null) {
|
||||
nextTermsHash.flush(nextChildFields, state);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
|
||||
return new TermsHashPerField(docInverterPerField, this, nextTermsHash, fieldInfo);
|
||||
}
|
||||
abstract TermsHashPerField addField(FieldInvertState fieldInvertState, final FieldInfo fieldInfo);
|
||||
|
||||
@Override
|
||||
void finishDocument() throws IOException {
|
||||
consumer.finishDocument(this);
|
||||
if (nextTermsHash != null) {
|
||||
nextTermsHash.consumer.finishDocument(nextTermsHash);
|
||||
nextTermsHash.finishDocument();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void startDocument() throws IOException {
|
||||
consumer.startDocument();
|
||||
if (nextTermsHash != null) {
|
||||
nextTermsHash.consumer.startDocument();
|
||||
nextTermsHash.startDocument();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,29 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
abstract class TermsHashConsumer {
|
||||
abstract void flush(Map<String, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException;
|
||||
abstract void abort();
|
||||
abstract void startDocument() throws IOException;
|
||||
abstract void finishDocument(TermsHash termsHash) throws IOException;
|
||||
abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
|
||||
}
|
|
@ -1,37 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/** Implement this class to plug into the TermsHash
|
||||
* processor, which inverts & stores Tokens into a hash
|
||||
* table and provides an API for writing bytes into
|
||||
* multiple streams for each unique Token. */
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
abstract class TermsHashConsumerPerField {
|
||||
abstract boolean start(IndexableField[] fields, int count) throws IOException;
|
||||
abstract void finish() throws IOException;
|
||||
abstract void skippingLongTerm() throws IOException;
|
||||
abstract void start(IndexableField field);
|
||||
abstract void newTerm(int termID) throws IOException;
|
||||
abstract void addTerm(int termID) throws IOException;
|
||||
abstract int getStreamCount();
|
||||
|
||||
abstract ParallelPostingsArray createPostingsArray(int size);
|
||||
}
|
|
@ -26,18 +26,15 @@ import org.apache.lucene.util.BytesRefHash;
|
|||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.IntBlockPool;
|
||||
import org.apache.lucene.util.BytesRefHash.BytesStartArray;
|
||||
import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
|
||||
|
||||
final class TermsHashPerField extends InvertedDocConsumerPerField {
|
||||
abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
|
||||
private static final int HASH_INIT_SIZE = 4;
|
||||
|
||||
final TermsHashConsumerPerField consumer;
|
||||
|
||||
final TermsHash termsHash;
|
||||
|
||||
final TermsHashPerField nextPerField;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final FieldInvertState fieldState;
|
||||
protected final DocumentsWriterPerThread.DocState docState;
|
||||
protected final FieldInvertState fieldState;
|
||||
TermToBytesRefAttribute termAtt;
|
||||
BytesRef termBytesRef;
|
||||
|
||||
|
@ -49,44 +46,37 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
final int streamCount;
|
||||
final int numPostingInt;
|
||||
|
||||
final FieldInfo fieldInfo;
|
||||
protected final FieldInfo fieldInfo;
|
||||
|
||||
final BytesRefHash bytesHash;
|
||||
|
||||
ParallelPostingsArray postingsArray;
|
||||
private final Counter bytesUsed;
|
||||
|
||||
public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) {
|
||||
/** streamCount: how many streams this field stores per term.
|
||||
* E.g. doc(+freq) is 1 stream, prox+offset is a second. */
|
||||
|
||||
public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
|
||||
intPool = termsHash.intPool;
|
||||
bytePool = termsHash.bytePool;
|
||||
termBytePool = termsHash.termBytePool;
|
||||
docState = termsHash.docState;
|
||||
this.termsHash = termsHash;
|
||||
bytesUsed = termsHash.bytesUsed;
|
||||
fieldState = docInverterPerField.fieldState;
|
||||
this.consumer = termsHash.consumer.addField(this, fieldInfo);
|
||||
PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
|
||||
bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
|
||||
streamCount = consumer.getStreamCount();
|
||||
this.fieldState = fieldState;
|
||||
this.streamCount = streamCount;
|
||||
numPostingInt = 2*streamCount;
|
||||
this.fieldInfo = fieldInfo;
|
||||
if (nextTermsHash != null)
|
||||
nextPerField = (TermsHashPerField) nextTermsHash.addField(docInverterPerField, fieldInfo);
|
||||
else
|
||||
nextPerField = null;
|
||||
this.nextPerField = nextPerField;
|
||||
PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
|
||||
bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
|
||||
}
|
||||
|
||||
void reset() {
|
||||
bytesHash.clear(false);
|
||||
if (nextPerField != null)
|
||||
if (nextPerField != null) {
|
||||
nextPerField.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
reset();
|
||||
if (nextPerField != null)
|
||||
nextPerField.abort();
|
||||
}
|
||||
}
|
||||
|
||||
public void initReader(ByteSliceReader reader, int termID, int stream) {
|
||||
|
@ -99,34 +89,17 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
ints[upto+stream]);
|
||||
}
|
||||
|
||||
/** Collapse the hash table & sort in-place. */
|
||||
int[] sortedTermIDs;
|
||||
|
||||
/** Collapse the hash table & sort in-place; also sets
|
||||
* this.sortedTermIDs to the results */
|
||||
public int[] sortPostings() {
|
||||
return bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
sortedTermIDs = bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
return sortedTermIDs;
|
||||
}
|
||||
|
||||
private boolean doCall;
|
||||
private boolean doNextCall;
|
||||
|
||||
@Override
|
||||
void start(IndexableField f) {
|
||||
termAtt = fieldState.attributeSource.getAttribute(TermToBytesRefAttribute.class);
|
||||
termBytesRef = termAtt.getBytesRef();
|
||||
consumer.start(f);
|
||||
if (nextPerField != null) {
|
||||
nextPerField.start(f);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean start(IndexableField[] fields, int count) throws IOException {
|
||||
doCall = consumer.start(fields, count);
|
||||
bytesHash.reinit();
|
||||
if (nextPerField != null) {
|
||||
doNextCall = nextPerField.start(fields, count);
|
||||
}
|
||||
return doCall || doNextCall;
|
||||
}
|
||||
|
||||
// Secondary entry point (for 2nd & subsequent TermsHash),
|
||||
// because token text has already been "interned" into
|
||||
// textStart, so we hash by textStart. term vectors use
|
||||
|
@ -137,8 +110,9 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
// First time we are seeing this token since we last
|
||||
// flushed the hash.
|
||||
// Init stream slices
|
||||
if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE)
|
||||
if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {
|
||||
intPool.nextBuffer();
|
||||
}
|
||||
|
||||
if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
|
||||
bytePool.nextBuffer();
|
||||
|
@ -156,20 +130,20 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
}
|
||||
postingsArray.byteStarts[termID] = intUptos[intUptoStart];
|
||||
|
||||
consumer.newTerm(termID);
|
||||
newTerm(termID);
|
||||
|
||||
} else {
|
||||
termID = (-termID)-1;
|
||||
int intStart = postingsArray.intStarts[termID];
|
||||
intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
|
||||
intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
|
||||
consumer.addTerm(termID);
|
||||
addTerm(termID);
|
||||
}
|
||||
}
|
||||
|
||||
// Primary entry point (for first TermsHash); postings use
|
||||
// this API.
|
||||
@Override
|
||||
/** Called once per inverted token. This is the primary
|
||||
* entry point (for first TermsHash); postings use this
|
||||
* API. */
|
||||
void add() throws IOException {
|
||||
|
||||
termAtt.fillBytesRef();
|
||||
|
@ -177,27 +151,10 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
// We are first in the chain so we must "intern" the
|
||||
// term text into textStart address
|
||||
// Get the text & hash of this term.
|
||||
int termID;
|
||||
try {
|
||||
termID = bytesHash.add(termBytesRef);
|
||||
} catch (MaxBytesLengthExceededException e) {
|
||||
// Term is too large; record this here (can't throw an
|
||||
// exc because DocInverterPerField will then abort the
|
||||
// entire segment) and then throw an exc later in
|
||||
// DocInverterPerField.java. LengthFilter can always be
|
||||
// used to prune the term before indexing:
|
||||
if (docState.maxTermPrefix == null) {
|
||||
final int saved = termBytesRef.length;
|
||||
try {
|
||||
termBytesRef.length = Math.min(30, DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8);
|
||||
docState.maxTermPrefix = termBytesRef.toString();
|
||||
} finally {
|
||||
termBytesRef.length = saved;
|
||||
}
|
||||
}
|
||||
consumer.skippingLongTerm();
|
||||
return;
|
||||
}
|
||||
int termID = bytesHash.add(termBytesRef);
|
||||
|
||||
//System.out.println("add term=" + termBytesRef.utf8ToString() + " doc=" + docState.docID + " termID=" + termID);
|
||||
|
||||
if (termID >= 0) {// New posting
|
||||
bytesHash.byteStart(termID);
|
||||
// Init stream slices
|
||||
|
@ -221,18 +178,19 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
}
|
||||
postingsArray.byteStarts[termID] = intUptos[intUptoStart];
|
||||
|
||||
consumer.newTerm(termID);
|
||||
newTerm(termID);
|
||||
|
||||
} else {
|
||||
termID = (-termID)-1;
|
||||
final int intStart = postingsArray.intStarts[termID];
|
||||
int intStart = postingsArray.intStarts[termID];
|
||||
intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
|
||||
intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
|
||||
consumer.addTerm(termID);
|
||||
addTerm(termID);
|
||||
}
|
||||
|
||||
if (doNextCall)
|
||||
if (doNextCall) {
|
||||
nextPerField.add(postingsArray.textStarts[termID]);
|
||||
}
|
||||
}
|
||||
|
||||
int[] intUptos;
|
||||
|
@ -269,13 +227,6 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
writeByte(stream, (byte) i);
|
||||
}
|
||||
|
||||
@Override
|
||||
void finish() throws IOException {
|
||||
consumer.finish();
|
||||
if (nextPerField != null)
|
||||
nextPerField.finish();
|
||||
}
|
||||
|
||||
private static final class PostingsBytesStartArray extends BytesStartArray {
|
||||
|
||||
private final TermsHashPerField perField;
|
||||
|
@ -290,7 +241,8 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
@Override
|
||||
public int[] init() {
|
||||
if (perField.postingsArray == null) {
|
||||
perField.postingsArray = perField.consumer.createPostingsArray(2);
|
||||
perField.postingsArray = perField.createPostingsArray(2);
|
||||
perField.newPostingsArray();
|
||||
bytesUsed.addAndGet(perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
|
||||
}
|
||||
return perField.postingsArray.textStarts;
|
||||
|
@ -301,15 +253,17 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
ParallelPostingsArray postingsArray = perField.postingsArray;
|
||||
final int oldSize = perField.postingsArray.size;
|
||||
postingsArray = perField.postingsArray = postingsArray.grow();
|
||||
perField.newPostingsArray();
|
||||
bytesUsed.addAndGet((postingsArray.bytesPerPosting() * (postingsArray.size - oldSize)));
|
||||
return postingsArray.textStarts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] clear() {
|
||||
if(perField.postingsArray != null) {
|
||||
if (perField.postingsArray != null) {
|
||||
bytesUsed.addAndGet(-(perField.postingsArray.size * perField.postingsArray.bytesPerPosting()));
|
||||
perField.postingsArray = null;
|
||||
perField.newPostingsArray();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -318,7 +272,47 @@ final class TermsHashPerField extends InvertedDocConsumerPerField {
|
|||
public Counter bytesUsed() {
|
||||
return bytesUsed;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(TermsHashPerField other) {
|
||||
return fieldInfo.name.compareTo(other.fieldInfo.name);
|
||||
}
|
||||
|
||||
/** Finish adding all instances of this field to the
|
||||
* current document. */
|
||||
void finish() throws IOException {
|
||||
if (nextPerField != null) {
|
||||
nextPerField.finish();
|
||||
}
|
||||
}
|
||||
|
||||
/** Start adding a new field instance; first is true if
|
||||
* this is the first time this field name was seen in the
|
||||
* document. */
|
||||
boolean start(IndexableField field, boolean first) {
|
||||
termAtt = fieldState.termAttribute;
|
||||
// EmptyTokenStream can have null term att
|
||||
if (termAtt != null) {
|
||||
termBytesRef = termAtt.getBytesRef();
|
||||
}
|
||||
if (nextPerField != null) {
|
||||
doNextCall = nextPerField.start(field, first);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Called when a term is seen for the first time. */
|
||||
abstract void newTerm(int termID) throws IOException;
|
||||
|
||||
/** Called when a previously seen term is seen again. */
|
||||
abstract void addTerm(int termID) throws IOException;
|
||||
|
||||
/** Called when the postings array is initialized or
|
||||
* resized. */
|
||||
abstract void newPostingsArray();
|
||||
|
||||
/** Creates a new postings array of the specified size. */
|
||||
abstract ParallelPostingsArray createPostingsArray(int size);
|
||||
}
|
||||
|
|
|
@ -1,68 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/** Just switches between two {@link DocFieldConsumer}s. */
|
||||
|
||||
class TwoStoredFieldsConsumers extends StoredFieldsConsumer {
|
||||
private final StoredFieldsConsumer first;
|
||||
private final StoredFieldsConsumer second;
|
||||
|
||||
public TwoStoredFieldsConsumers(StoredFieldsConsumer first, StoredFieldsConsumer second) {
|
||||
this.first = first;
|
||||
this.second = second;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addField(int docID, StorableField field, FieldInfo fieldInfo) throws IOException {
|
||||
first.addField(docID, field, fieldInfo);
|
||||
second.addField(docID, field, fieldInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(SegmentWriteState state) throws IOException {
|
||||
first.flush(state);
|
||||
second.flush(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
try {
|
||||
first.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
try {
|
||||
second.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void startDocument() throws IOException {
|
||||
first.startDocument();
|
||||
second.startDocument();
|
||||
}
|
||||
|
||||
@Override
|
||||
void finishDocument() throws IOException {
|
||||
first.finishDocument();
|
||||
second.finishDocument();
|
||||
}
|
||||
}
|
|
@ -171,7 +171,7 @@ public class RAMDirectory extends BaseDirectory {
|
|||
existing.directory = null;
|
||||
}
|
||||
fileMap.put(name, file);
|
||||
return new RAMOutputStream(file);
|
||||
return new RAMOutputStream(file, true);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -38,20 +38,25 @@ public class RAMOutputStream extends IndexOutput {
|
|||
private long bufferStart;
|
||||
private int bufferLength;
|
||||
|
||||
private Checksum crc = new BufferedChecksum(new CRC32());
|
||||
private final Checksum crc;
|
||||
|
||||
/** Construct an empty output buffer. */
|
||||
public RAMOutputStream() {
|
||||
this(new RAMFile());
|
||||
this(new RAMFile(), false);
|
||||
}
|
||||
|
||||
public RAMOutputStream(RAMFile f) {
|
||||
public RAMOutputStream(RAMFile f, boolean checksum) {
|
||||
file = f;
|
||||
|
||||
// make sure that we switch to the
|
||||
// first needed buffer lazily
|
||||
currentBufferIndex = -1;
|
||||
currentBuffer = null;
|
||||
if (checksum) {
|
||||
crc = new BufferedChecksum(new CRC32());
|
||||
} else {
|
||||
crc = null;
|
||||
}
|
||||
}
|
||||
|
||||
/** Copy the current contents of this buffer to the named output. */
|
||||
|
@ -99,7 +104,9 @@ public class RAMOutputStream extends IndexOutput {
|
|||
bufferStart = 0;
|
||||
bufferLength = 0;
|
||||
file.setLength(0);
|
||||
crc.reset();
|
||||
if (crc != null) {
|
||||
crc.reset();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -113,14 +120,18 @@ public class RAMOutputStream extends IndexOutput {
|
|||
currentBufferIndex++;
|
||||
switchCurrentBuffer();
|
||||
}
|
||||
crc.update(b);
|
||||
if (crc != null) {
|
||||
crc.update(b);
|
||||
}
|
||||
currentBuffer[bufferPosition++] = b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(byte[] b, int offset, int len) throws IOException {
|
||||
assert b != null;
|
||||
crc.update(b, offset, len);
|
||||
if (crc != null) {
|
||||
crc.update(b, offset, len);
|
||||
}
|
||||
while (len > 0) {
|
||||
if (bufferPosition == bufferLength) {
|
||||
currentBufferIndex++;
|
||||
|
@ -171,6 +182,10 @@ public class RAMOutputStream extends IndexOutput {
|
|||
|
||||
@Override
|
||||
public long getChecksum() throws IOException {
|
||||
return crc.getValue();
|
||||
if (crc == null) {
|
||||
throw new IllegalStateException("internal RAMOutputStream created with checksum disabled");
|
||||
} else {
|
||||
return crc.getValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -287,22 +287,20 @@ public class AttributeSource {
|
|||
}
|
||||
|
||||
/**
|
||||
* The caller must pass in a Class<? extends Attribute> value.
|
||||
* Returns the instance of the passed in Attribute contained in this AttributeSource
|
||||
* <p>
|
||||
* The caller must pass in a Class<? extends Attribute> value.
|
||||
*
|
||||
* @throws IllegalArgumentException if this AttributeSource does not contain the
|
||||
* Attribute. It is recommended to always use {@link #addAttribute} even in consumers
|
||||
* of TokenStreams, because you cannot know if a specific TokenStream really uses
|
||||
* a specific Attribute. {@link #addAttribute} will automatically make the attribute
|
||||
* available. If you want to only use the attribute, if it is available (to optimize
|
||||
* @return instance of the passed in Attribute, or {@code null} if this AttributeSource
|
||||
* does not contain the Attribute. It is recommended to always use
|
||||
* {@link #addAttribute} even in consumers of TokenStreams, because you cannot
|
||||
* know if a specific TokenStream really uses a specific Attribute.
|
||||
* {@link #addAttribute} will automatically make the attribute available.
|
||||
* If you want to only use the attribute, if it is available (to optimize
|
||||
* consuming), use {@link #hasAttribute}.
|
||||
*/
|
||||
public final <T extends Attribute> T getAttribute(Class<T> attClass) {
|
||||
AttributeImpl attImpl = attributes.get(attClass);
|
||||
if (attImpl == null) {
|
||||
throw new IllegalArgumentException("This AttributeSource does not have the attribute '" + attClass.getName() + "'.");
|
||||
}
|
||||
return attClass.cast(attImpl);
|
||||
return attClass.cast(attributes.get(attClass));
|
||||
}
|
||||
|
||||
private State getCurrentState() {
|
||||
|
|
|
@ -80,8 +80,7 @@ public final class BytesRefHash {
|
|||
/**
|
||||
* Creates a new {@link BytesRefHash}
|
||||
*/
|
||||
public BytesRefHash(ByteBlockPool pool, int capacity,
|
||||
BytesStartArray bytesStartArray) {
|
||||
public BytesRefHash(ByteBlockPool pool, int capacity, BytesStartArray bytesStartArray) {
|
||||
hashSize = capacity;
|
||||
hashHalfSize = hashSize >> 1;
|
||||
hashMask = hashSize - 1;
|
||||
|
|
|
@ -207,12 +207,8 @@ public class QueryBuilder {
|
|||
buffer = new CachingTokenFilter(source);
|
||||
buffer.reset();
|
||||
|
||||
if (buffer.hasAttribute(TermToBytesRefAttribute.class)) {
|
||||
termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
|
||||
}
|
||||
if (buffer.hasAttribute(PositionIncrementAttribute.class)) {
|
||||
posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
|
||||
}
|
||||
termAtt = buffer.getAttribute(TermToBytesRefAttribute.class);
|
||||
posIncrAtt = buffer.getAttribute(PositionIncrementAttribute.class);
|
||||
|
||||
if (termAtt != null) {
|
||||
try {
|
||||
|
|
|
@ -31,10 +31,12 @@ public class TestNumericTokenStream extends BaseTokenStreamTestCase {
|
|||
|
||||
public void testLongStream() throws Exception {
|
||||
final NumericTokenStream stream=new NumericTokenStream().setLongValue(lvalue);
|
||||
// use getAttribute to test if attributes really exist, if not an IAE will be throwed
|
||||
final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
|
||||
assertNotNull(bytesAtt);
|
||||
final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
|
||||
assertNotNull(typeAtt);
|
||||
final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
|
||||
assertNotNull(numericAtt);
|
||||
final BytesRef bytes = bytesAtt.getBytesRef();
|
||||
stream.reset();
|
||||
assertEquals(64, numericAtt.getValueSize());
|
||||
|
@ -53,10 +55,12 @@ public class TestNumericTokenStream extends BaseTokenStreamTestCase {
|
|||
|
||||
public void testIntStream() throws Exception {
|
||||
final NumericTokenStream stream=new NumericTokenStream().setIntValue(ivalue);
|
||||
// use getAttribute to test if attributes really exist, if not an IAE will be throwed
|
||||
final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
|
||||
assertNotNull(bytesAtt);
|
||||
final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
|
||||
assertNotNull(typeAtt);
|
||||
final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
|
||||
assertNotNull(numericAtt);
|
||||
final BytesRef bytes = bytesAtt.getBytesRef();
|
||||
stream.reset();
|
||||
assertEquals(32, numericAtt.getValueSize());
|
||||
|
|
|
@ -131,7 +131,7 @@ public class TestDocument extends LuceneTestCase {
|
|||
assertEquals(0, doc.getFields().size());
|
||||
}
|
||||
|
||||
public void testConstructorExceptions() {
|
||||
public void testConstructorExceptions() throws Exception {
|
||||
FieldType ft = new FieldType();
|
||||
ft.setStored(true);
|
||||
new Field("name", "value", ft); // okay
|
||||
|
@ -142,16 +142,23 @@ public class TestDocument extends LuceneTestCase {
|
|||
} catch (IllegalArgumentException e) {
|
||||
// expected exception
|
||||
}
|
||||
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
new Field("name", "value", ft); // okay
|
||||
Document doc = new Document();
|
||||
FieldType ft2 = new FieldType();
|
||||
ft2.setStored(true);
|
||||
ft2.setStoreTermVectors(true);
|
||||
doc.add(new Field("name", "value", ft2));
|
||||
try {
|
||||
FieldType ft2 = new FieldType();
|
||||
ft2.setStored(true);
|
||||
ft2.setStoreTermVectors(true);
|
||||
new Field("name", "value", ft2);
|
||||
w.addDocument(doc);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// expected exception
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testClearDocument() {
|
||||
|
|
|
@ -31,11 +31,10 @@ import org.apache.lucene.analysis.CannedBinaryTokenStream; // javadocs
|
|||
*/
|
||||
public final class BinaryTokenStream extends TokenStream {
|
||||
private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
|
||||
private final BytesRef bytes;
|
||||
private boolean available = true;
|
||||
|
||||
public BinaryTokenStream(BytesRef bytes) {
|
||||
this.bytes = bytes;
|
||||
bytesAtt.setBytesRef(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -43,7 +42,6 @@ public final class BinaryTokenStream extends TokenStream {
|
|||
if (available) {
|
||||
clearAttributes();
|
||||
available = false;
|
||||
bytesAtt.setBytesRef(bytes);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
|
|
|
@ -279,21 +279,14 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
writer.addDocument(doc);
|
||||
try {
|
||||
writer.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("all instances of a given field name must have the same term vectors settings (storeTermVectors changed for field=\"f1\")",
|
||||
iae.getMessage());
|
||||
}
|
||||
writer.shutdown();
|
||||
|
||||
TestUtil.checkIndex(dir);
|
||||
|
||||
IndexReader reader = DirectoryReader.open(dir);
|
||||
// f1
|
||||
Terms tfv1 = reader.getTermVectors(0).terms("f1");
|
||||
assertNotNull(tfv1);
|
||||
assertEquals("the 'with_tv' setting should rule!",2,tfv1.size());
|
||||
// f2
|
||||
Terms tfv2 = reader.getTermVectors(0).terms("f2");
|
||||
assertNotNull(tfv2);
|
||||
assertEquals("the 'with_tv' setting should rule!",2,tfv2.size());
|
||||
reader.close();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -20,11 +20,14 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
|
@ -378,4 +381,63 @@ public class TestIndexableField extends LuceneTestCase {
|
|||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private static class CustomField implements StorableField {
|
||||
@Override
|
||||
public BytesRef binaryValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String stringValue() {
|
||||
return "foobar";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Reader readerValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number numericValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return "field";
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexableFieldType fieldType() {
|
||||
FieldType ft = new FieldType(StoredField.TYPE);
|
||||
ft.setStoreTermVectors(true);
|
||||
ft.freeze();
|
||||
return ft;
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-5611
|
||||
public void testNotIndexedTermVectors() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
try {
|
||||
w.addDocument(
|
||||
new IndexDocument() {
|
||||
@Override
|
||||
public Iterable<IndexableField> indexableFields() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
@Override
|
||||
public Iterable<StorableField> storableFields() {
|
||||
return Collections.<StorableField>singletonList(new CustomField());
|
||||
}
|
||||
});
|
||||
fail("didn't hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -98,7 +98,7 @@ public class TestParallelReaderEmptyIndex extends LuceneTestCase {
|
|||
doc.add(newField("test", "", customType));
|
||||
idField.setStringValue("1");
|
||||
iw.addDocument(doc);
|
||||
doc.add(newTextField("test", "", Field.Store.NO));
|
||||
doc.add(newField("test", "", customType));
|
||||
idField.setStringValue("2");
|
||||
iw.addDocument(doc);
|
||||
iw.shutdown();
|
||||
|
|
|
@ -639,8 +639,8 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
int freq1 = dpEnum1.freq();
|
||||
int freq2 = dpEnum2.freq();
|
||||
assertEquals(freq1, freq2);
|
||||
OffsetAttribute offsetAtt1 = dpEnum1.attributes().hasAttribute(OffsetAttribute.class) ? dpEnum1.attributes().getAttribute(OffsetAttribute.class) : null;
|
||||
OffsetAttribute offsetAtt2 = dpEnum2.attributes().hasAttribute(OffsetAttribute.class) ? dpEnum2.attributes().getAttribute(OffsetAttribute.class) : null;
|
||||
OffsetAttribute offsetAtt1 = dpEnum1.attributes().getAttribute(OffsetAttribute.class);
|
||||
OffsetAttribute offsetAtt2 = dpEnum2.attributes().getAttribute(OffsetAttribute.class);
|
||||
|
||||
if (offsetAtt1 != null) {
|
||||
assertNotNull(offsetAtt2);
|
||||
|
@ -773,24 +773,39 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
Field idField = newField("id", idString, customType1);
|
||||
fields.add(idField);
|
||||
|
||||
Map<String,FieldType> tvTypes = new HashMap<>();
|
||||
|
||||
int nFields = nextInt(maxFields);
|
||||
for (int i=0; i<nFields; i++) {
|
||||
|
||||
FieldType customType = new FieldType();
|
||||
switch (nextInt(4)) {
|
||||
case 0:
|
||||
break;
|
||||
case 1:
|
||||
customType.setStoreTermVectors(true);
|
||||
break;
|
||||
case 2:
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
break;
|
||||
case 3:
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
break;
|
||||
String fieldName = "f" + nextInt(100);
|
||||
FieldType customType;
|
||||
|
||||
// Use the same term vector settings if we already
|
||||
// added this field to the doc:
|
||||
FieldType oldTVType = tvTypes.get(fieldName);
|
||||
if (oldTVType != null) {
|
||||
customType = new FieldType(oldTVType);
|
||||
} else {
|
||||
customType = new FieldType();
|
||||
switch (nextInt(4)) {
|
||||
case 0:
|
||||
break;
|
||||
case 1:
|
||||
customType.setStoreTermVectors(true);
|
||||
break;
|
||||
case 2:
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
break;
|
||||
case 3:
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
break;
|
||||
}
|
||||
FieldType newType = new FieldType(customType);
|
||||
newType.freeze();
|
||||
tvTypes.put(fieldName, newType);
|
||||
}
|
||||
|
||||
switch (nextInt(4)) {
|
||||
|
@ -798,26 +813,30 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
customType.setStored(true);
|
||||
customType.setOmitNorms(true);
|
||||
customType.setIndexed(true);
|
||||
fields.add(newField("f" + nextInt(100), getString(1), customType));
|
||||
customType.freeze();
|
||||
fields.add(newField(fieldName, getString(1), customType));
|
||||
break;
|
||||
case 1:
|
||||
customType.setIndexed(true);
|
||||
customType.setTokenized(true);
|
||||
fields.add(newField("f" + nextInt(100), getString(0), customType));
|
||||
customType.freeze();
|
||||
fields.add(newField(fieldName, getString(0), customType));
|
||||
break;
|
||||
case 2:
|
||||
customType.setStored(true);
|
||||
customType.setStoreTermVectors(false);
|
||||
customType.setStoreTermVectorOffsets(false);
|
||||
customType.setStoreTermVectorPositions(false);
|
||||
fields.add(newField("f" + nextInt(100), getString(0), customType));
|
||||
customType.freeze();
|
||||
fields.add(newField(fieldName, getString(0), customType));
|
||||
break;
|
||||
case 3:
|
||||
customType.setStored(true);
|
||||
customType.setIndexed(true);
|
||||
customType.setTokenized(true);
|
||||
fields.add(newField("f" + nextInt(100), getString(bigFieldSize), customType));
|
||||
break;
|
||||
customType.freeze();
|
||||
fields.add(newField(fieldName, getString(bigFieldSize), customType));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -872,8 +891,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
e.printStackTrace();
|
||||
Assert.fail(e.toString());
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.lucene.codecs.TermVectorsReader;
|
|||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -109,6 +110,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
|
|||
}
|
||||
else if (!testFieldsStorePos[i] && testFieldsStoreOff[i]) {
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
}
|
||||
else {
|
||||
|
@ -333,7 +335,9 @@ public class TestTermVectorsReader extends LuceneTestCase {
|
|||
|
||||
public void testIllegalIndexableField() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
MockAnalyzer a = new MockAnalyzer(random());
|
||||
a.setEnableChecks(false);
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir, a);
|
||||
FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
|
||||
ft.setStoreTermVectors(true);
|
||||
ft.setStoreTermVectorPayloads(true);
|
||||
|
@ -386,6 +390,67 @@ public class TestTermVectorsReader extends LuceneTestCase {
|
|||
assertEquals("cannot index term vector payloads when term vectors are not indexed (field=\"field\")", iae.getMessage());
|
||||
}
|
||||
|
||||
ft = new FieldType(TextField.TYPE_NOT_STORED);
|
||||
ft.setStoreTermVectors(true);
|
||||
ft.setStoreTermVectorPayloads(true);
|
||||
doc = new Document();
|
||||
doc.add(new Field("field", "value", ft));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// Expected
|
||||
assertEquals("cannot index term vector payloads without term vector positions (field=\"field\")", iae.getMessage());
|
||||
}
|
||||
|
||||
ft = new FieldType(StoredField.TYPE);
|
||||
ft.setStoreTermVectors(true);
|
||||
doc = new Document();
|
||||
doc.add(new Field("field", "value", ft));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// Expected
|
||||
assertEquals("cannot store term vectors for a field that is not indexed (field=\"field\")", iae.getMessage());
|
||||
}
|
||||
|
||||
ft = new FieldType(StoredField.TYPE);
|
||||
ft.setStoreTermVectorPositions(true);
|
||||
doc = new Document();
|
||||
doc.add(new Field("field", "value", ft));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// Expected
|
||||
assertEquals("cannot store term vector positions for a field that is not indexed (field=\"field\")", iae.getMessage());
|
||||
}
|
||||
|
||||
ft = new FieldType(StoredField.TYPE);
|
||||
ft.setStoreTermVectorOffsets(true);
|
||||
doc = new Document();
|
||||
doc.add(new Field("field", "value", ft));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// Expected
|
||||
assertEquals("cannot store term vector offsets for a field that is not indexed (field=\"field\")", iae.getMessage());
|
||||
}
|
||||
|
||||
ft = new FieldType(StoredField.TYPE);
|
||||
ft.setStoreTermVectorPayloads(true);
|
||||
doc = new Document();
|
||||
doc.add(new Field("field", "value", ft));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
fail("did not hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// Expected
|
||||
assertEquals("cannot store term vector payloads for a field that is not indexed (field=\"field\")", iae.getMessage());
|
||||
}
|
||||
|
||||
w.shutdown();
|
||||
|
||||
dir.close();
|
||||
|
|
|
@ -534,6 +534,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
|
|||
|
||||
FieldType customType = new FieldType(StringField.TYPE_NOT_STORED);
|
||||
customType.setStoreTermVectors(true);
|
||||
document = new Document();
|
||||
document.add(newField("tvtest", "a b c", customType));
|
||||
iw.addDocument(document);
|
||||
// Make 2nd segment
|
||||
|
@ -567,6 +568,7 @@ public class TestTermVectorsWriter extends LuceneTestCase {
|
|||
FieldType customType2 = new FieldType(StringField.TYPE_NOT_STORED);
|
||||
customType2.setStoreTermVectors(true);
|
||||
document.add(newField("tvtest", "a b c", customType2));
|
||||
document = new Document();
|
||||
iw.addDocument(document);
|
||||
// Make 2nd segment
|
||||
iw.commit();
|
||||
|
|
|
@ -26,15 +26,10 @@ import org.apache.lucene.document.Field;
|
|||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.English;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
@ -116,44 +111,14 @@ public class TestTermVectors extends LuceneTestCase {
|
|||
doc.add(newField("field", "one", ft3));
|
||||
doc.add(newField("field", "one", ft4));
|
||||
doc.add(newField("field", "one", ft5));
|
||||
writer.addDocument(doc);
|
||||
IndexReader reader = writer.getReader();
|
||||
try {
|
||||
writer.addDocument(doc);
|
||||
fail("should have hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
assertEquals("all instances of a given field name must have the same term vectors settings (storeTermVectors changed for field=\"field\")",
|
||||
iae.getMessage());
|
||||
}
|
||||
writer.shutdown();
|
||||
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
|
||||
Query query = new TermQuery(new Term("field", "one"));
|
||||
ScoreDoc[] hits = searcher.search(query, null, 1000).scoreDocs;
|
||||
assertEquals(1, hits.length);
|
||||
|
||||
Fields vectors = searcher.reader.getTermVectors(hits[0].doc);
|
||||
assertNotNull(vectors);
|
||||
assertEquals(1, vectors.size());
|
||||
Terms vector = vectors.terms("field");
|
||||
assertNotNull(vector);
|
||||
assertEquals(1, vector.size());
|
||||
TermsEnum termsEnum = vector.iterator(null);
|
||||
assertNotNull(termsEnum.next());
|
||||
assertEquals("one", termsEnum.term().utf8ToString());
|
||||
assertEquals(5, termsEnum.totalTermFreq());
|
||||
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
|
||||
assertNotNull(dpEnum);
|
||||
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
|
||||
assertEquals(5, dpEnum.freq());
|
||||
for(int i=0;i<5;i++) {
|
||||
assertEquals(i, dpEnum.nextPosition());
|
||||
}
|
||||
|
||||
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
|
||||
assertNotNull(dpEnum);
|
||||
assertTrue(dpEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
|
||||
assertEquals(5, dpEnum.freq());
|
||||
for(int i=0;i<5;i++) {
|
||||
dpEnum.nextPosition();
|
||||
assertEquals(4*i, dpEnum.startOffset());
|
||||
assertEquals(4*i+3, dpEnum.endOffset());
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
|
||||
private IndexWriter createWriter(Directory dir) throws IOException {
|
||||
|
|
|
@ -54,7 +54,7 @@ public class TestHugeRamFile extends LuceneTestCase {
|
|||
public void testHugeFile() throws IOException {
|
||||
DenseRAMFile f = new DenseRAMFile();
|
||||
// output part
|
||||
RAMOutputStream out = new RAMOutputStream(f);
|
||||
RAMOutputStream out = new RAMOutputStream(f, true);
|
||||
byte b1[] = new byte[RAMOutputStream.BUFFER_SIZE];
|
||||
byte b2[] = new byte[RAMOutputStream.BUFFER_SIZE / 3];
|
||||
for (int i = 0; i < b1.length; i++) {
|
||||
|
|
|
@ -90,7 +90,9 @@ public class TestAttributeSource extends LuceneTestCase {
|
|||
assertFalse("No more attributes", it.hasNext());
|
||||
|
||||
final FlagsAttribute flagsAtt2 = clone.getAttribute(FlagsAttribute.class);
|
||||
assertNotNull(flagsAtt2);
|
||||
final TypeAttribute typeAtt2 = clone.getAttribute(TypeAttribute.class);
|
||||
assertNotNull(typeAtt2);
|
||||
assertNotSame("FlagsAttribute of original and clone must be different instances", flagsAtt2, flagsAtt);
|
||||
assertNotSame("TypeAttribute of original and clone must be different instances", typeAtt2, typeAtt);
|
||||
assertEquals("FlagsAttribute of original and clone must be equal", flagsAtt2, flagsAtt);
|
||||
|
|
|
@ -582,7 +582,7 @@ public class SortingAtomicReader extends FilterAtomicReader {
|
|||
file = new RAMFile();
|
||||
sorter = new DocOffsetSorter(maxDoc);
|
||||
}
|
||||
final IndexOutput out = new RAMOutputStream(file);
|
||||
final IndexOutput out = new RAMOutputStream(file, false);
|
||||
int doc;
|
||||
int i = 0;
|
||||
while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
|
|
|
@ -685,11 +685,11 @@ public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
|
|||
int remainder = random.nextInt(10);
|
||||
Reader reader = new StringReader(text);
|
||||
TokenStream ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
|
||||
CharTermAttribute termAtt = ts.hasAttribute(CharTermAttribute.class) ? ts.getAttribute(CharTermAttribute.class) : null;
|
||||
OffsetAttribute offsetAtt = ts.hasAttribute(OffsetAttribute.class) ? ts.getAttribute(OffsetAttribute.class) : null;
|
||||
PositionIncrementAttribute posIncAtt = ts.hasAttribute(PositionIncrementAttribute.class) ? ts.getAttribute(PositionIncrementAttribute.class) : null;
|
||||
PositionLengthAttribute posLengthAtt = ts.hasAttribute(PositionLengthAttribute.class) ? ts.getAttribute(PositionLengthAttribute.class) : null;
|
||||
TypeAttribute typeAtt = ts.hasAttribute(TypeAttribute.class) ? ts.getAttribute(TypeAttribute.class) : null;
|
||||
CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
|
||||
OffsetAttribute offsetAtt = ts.getAttribute(OffsetAttribute.class);
|
||||
PositionIncrementAttribute posIncAtt = ts.getAttribute(PositionIncrementAttribute.class);
|
||||
PositionLengthAttribute posLengthAtt = ts.getAttribute(PositionLengthAttribute.class);
|
||||
TypeAttribute typeAtt = ts.getAttribute(TypeAttribute.class);
|
||||
List<String> tokens = new ArrayList<>();
|
||||
List<String> types = new ArrayList<>();
|
||||
List<Integer> positions = new ArrayList<>();
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
|
||||
import org.apache.lucene.util.Attribute;
|
||||
|
||||
// TODO: rename to OffsetsXXXTF? ie we only validate
|
||||
// offsets (now anyway...)
|
||||
|
@ -47,23 +46,14 @@ public final class ValidatingTokenFilter extends TokenFilter {
|
|||
private final Map<Integer,Integer> posToStartOffset = new HashMap<>();
|
||||
private final Map<Integer,Integer> posToEndOffset = new HashMap<>();
|
||||
|
||||
private final PositionIncrementAttribute posIncAtt = getAttrIfExists(PositionIncrementAttribute.class);
|
||||
private final PositionLengthAttribute posLenAtt = getAttrIfExists(PositionLengthAttribute.class);
|
||||
private final OffsetAttribute offsetAtt = getAttrIfExists(OffsetAttribute.class);
|
||||
private final CharTermAttribute termAtt = getAttrIfExists(CharTermAttribute.class);
|
||||
private final PositionIncrementAttribute posIncAtt = getAttribute(PositionIncrementAttribute.class);
|
||||
private final PositionLengthAttribute posLenAtt = getAttribute(PositionLengthAttribute.class);
|
||||
private final OffsetAttribute offsetAtt = getAttribute(OffsetAttribute.class);
|
||||
private final CharTermAttribute termAtt = getAttribute(CharTermAttribute.class);
|
||||
private final boolean offsetsAreCorrect;
|
||||
|
||||
private final String name;
|
||||
|
||||
// Returns null if the attr wasn't already added
|
||||
private <A extends Attribute> A getAttrIfExists(Class<A> att) {
|
||||
if (hasAttribute(att)) {
|
||||
return getAttribute(att);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/** The name arg is used to identify this stage when
|
||||
* throwing exceptions (useful if you have more than one
|
||||
* instance in your chain). */
|
||||
|
|
|
@ -90,7 +90,6 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
static class AssertingStoredFieldsWriter extends StoredFieldsWriter {
|
||||
private final StoredFieldsWriter in;
|
||||
private int numWritten;
|
||||
private int fieldCount;
|
||||
private Status docStatus;
|
||||
|
||||
AssertingStoredFieldsWriter(StoredFieldsWriter in) {
|
||||
|
@ -99,11 +98,9 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void startDocument(int numStoredFields) throws IOException {
|
||||
public void startDocument() throws IOException {
|
||||
assert docStatus != Status.STARTED;
|
||||
in.startDocument(numStoredFields);
|
||||
assert fieldCount == 0;
|
||||
fieldCount = numStoredFields;
|
||||
in.startDocument();
|
||||
numWritten++;
|
||||
docStatus = Status.STARTED;
|
||||
}
|
||||
|
@ -111,7 +108,6 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
@Override
|
||||
public void finishDocument() throws IOException {
|
||||
assert docStatus == Status.STARTED;
|
||||
assert fieldCount == 0;
|
||||
in.finishDocument();
|
||||
docStatus = Status.FINISHED;
|
||||
}
|
||||
|
@ -120,8 +116,6 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
public void writeField(FieldInfo info, StorableField field) throws IOException {
|
||||
assert docStatus == Status.STARTED;
|
||||
in.writeField(info, field);
|
||||
assert fieldCount > 0;
|
||||
fieldCount--;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -133,7 +127,6 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
|
|||
public void finish(FieldInfos fis, int numDocs) throws IOException {
|
||||
assert docStatus == (numDocs > 0 ? Status.FINISHED : Status.UNDEFINED);
|
||||
in.finish(fis, numDocs);
|
||||
assert fieldCount == 0;
|
||||
assert numDocs == numWritten;
|
||||
}
|
||||
|
||||
|
|
|
@ -17,9 +17,6 @@ package org.apache.lucene.util;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsBoolean;
|
||||
import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsInt;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
|
@ -40,14 +37,17 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -56,8 +56,8 @@ import java.util.logging.Logger;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
|
@ -76,8 +76,8 @@ import org.apache.lucene.index.FieldFilterAtomicReader;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LogByteSizeMergePolicy;
|
||||
import org.apache.lucene.index.LogDocMergePolicy;
|
||||
|
@ -98,25 +98,25 @@ import org.apache.lucene.index.SortedSetDocValues;
|
|||
import org.apache.lucene.index.StorableField;
|
||||
import org.apache.lucene.index.StoredDocument;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TieredMergePolicy;
|
||||
import org.apache.lucene.search.AssertingIndexSearcher;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
import org.apache.lucene.search.FieldCache.CacheEntry;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.QueryUtils.FCInvisibleMultiReader;
|
||||
import org.apache.lucene.store.BaseDirectoryWrapper;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IOContext.Context;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.LockFactory;
|
||||
import org.apache.lucene.store.MergeInfo;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.store.NRTCachingDirectory;
|
||||
import org.apache.lucene.store.RateLimitedDirectoryWrapper;
|
||||
import org.apache.lucene.util.FieldCacheSanityChecker.Insanity;
|
||||
|
@ -134,7 +134,6 @@ import org.junit.Test;
|
|||
import org.junit.rules.RuleChain;
|
||||
import org.junit.rules.TestRule;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.JUnit4MethodProvider;
|
||||
import com.carrotsearch.randomizedtesting.LifecycleScope;
|
||||
import com.carrotsearch.randomizedtesting.MixWithSuiteName;
|
||||
|
@ -145,16 +144,16 @@ import com.carrotsearch.randomizedtesting.annotations.Listeners;
|
|||
import com.carrotsearch.randomizedtesting.annotations.SeedDecorators;
|
||||
import com.carrotsearch.randomizedtesting.annotations.TestGroup;
|
||||
import com.carrotsearch.randomizedtesting.annotations.TestMethodProviders;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup.Group;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies;
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
|
||||
|
@ -163,6 +162,9 @@ import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
|
|||
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
|
||||
import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter;
|
||||
|
||||
import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsBoolean;
|
||||
import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsInt;
|
||||
|
||||
/**
|
||||
* Base class for all Lucene unit tests, Junit3 or Junit4 variant.
|
||||
*
|
||||
|
@ -601,6 +603,8 @@ public abstract class LuceneTestCase extends Assert {
|
|||
.around(new TestRuleFieldCacheSanity())
|
||||
.around(parentChainCallRule);
|
||||
|
||||
private static final Map<String,FieldType> fieldToType = new HashMap<String,FieldType>();
|
||||
|
||||
// -----------------------------------------------------------------
|
||||
// Suite and test case setup/ cleanup.
|
||||
// -----------------------------------------------------------------
|
||||
|
@ -619,6 +623,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
@After
|
||||
public void tearDown() throws Exception {
|
||||
parentChainCallRule.teardownCalled = true;
|
||||
fieldToType.clear();
|
||||
}
|
||||
|
||||
|
||||
|
@ -1168,11 +1173,43 @@ public abstract class LuceneTestCase extends Assert {
|
|||
public static Field newField(String name, String value, FieldType type) {
|
||||
return newField(random(), name, value, type);
|
||||
}
|
||||
|
||||
public static Field newField(Random random, String name, String value, FieldType type) {
|
||||
|
||||
/** Returns a FieldType derived from newType but whose
|
||||
* term vector options match the old type */
|
||||
private static FieldType mergeTermVectorOptions(FieldType newType, FieldType oldType) {
|
||||
if (newType.indexed() && oldType.storeTermVectors() == true && newType.storeTermVectors() == false) {
|
||||
newType = new FieldType(newType);
|
||||
newType.setStoreTermVectors(oldType.storeTermVectors());
|
||||
newType.setStoreTermVectorPositions(oldType.storeTermVectorPositions());
|
||||
newType.setStoreTermVectorOffsets(oldType.storeTermVectorOffsets());
|
||||
newType.setStoreTermVectorPayloads(oldType.storeTermVectorPayloads());
|
||||
newType.freeze();
|
||||
}
|
||||
|
||||
return newType;
|
||||
}
|
||||
|
||||
// TODO: if we can pull out the "make term vector options
|
||||
// consistent across all instances of the same field name"
|
||||
// write-once schema sort of helper class then we can
|
||||
// remove the sync here. We can also fold the random
|
||||
// "enable norms" (now commented out, below) into that:
|
||||
public synchronized static Field newField(Random random, String name, String value, FieldType type) {
|
||||
|
||||
// Defeat any consumers that illegally rely on intern'd
|
||||
// strings (we removed this from Lucene a while back):
|
||||
name = new String(name);
|
||||
if (usually(random) || !type.indexed()) {
|
||||
|
||||
FieldType prevType = fieldToType.get(name);
|
||||
|
||||
if (usually(random) || !type.indexed() || prevType != null) {
|
||||
// most of the time, don't modify the params
|
||||
if (prevType == null) {
|
||||
fieldToType.put(name, new FieldType(type));
|
||||
} else {
|
||||
type = mergeTermVectorOptions(type, prevType);
|
||||
}
|
||||
|
||||
return new Field(name, value, type);
|
||||
}
|
||||
|
||||
|
@ -1185,19 +1222,29 @@ public abstract class LuceneTestCase extends Assert {
|
|||
newType.setStored(true); // randomly store it
|
||||
}
|
||||
|
||||
// Randomly turn on term vector options, but always do
|
||||
// so consistently for the same field name:
|
||||
if (!newType.storeTermVectors() && random.nextBoolean()) {
|
||||
newType.setStoreTermVectors(true);
|
||||
if (!newType.storeTermVectorOffsets()) {
|
||||
newType.setStoreTermVectorOffsets(random.nextBoolean());
|
||||
}
|
||||
if (!newType.storeTermVectorPositions()) {
|
||||
newType.setStoreTermVectorPositions(random.nextBoolean());
|
||||
|
||||
if (newType.storeTermVectorPositions() && !newType.storeTermVectorPayloads()) {
|
||||
newType.setStoreTermVectorPayloads(random.nextBoolean());
|
||||
if (newType.storeTermVectorPositions()) {
|
||||
if (!newType.storeTermVectorPayloads()) {
|
||||
newType.setStoreTermVectorPayloads(random.nextBoolean());
|
||||
}
|
||||
if (!newType.storeTermVectorOffsets()) {
|
||||
newType.setStoreTermVectorOffsets(random.nextBoolean());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("NOTE: LuceneTestCase: upgrade name=" + name + " type=" + newType);
|
||||
}
|
||||
}
|
||||
newType.freeze();
|
||||
fieldToType.put(name, newType);
|
||||
|
||||
// TODO: we need to do this, but smarter, ie, most of
|
||||
// the time we set the same value for a given field but
|
||||
|
|
|
@ -230,10 +230,10 @@ public class JsonPreAnalyzedParser implements PreAnalyzedParser {
|
|||
Map<String,Object> tok = new TreeMap<>();
|
||||
while (it.hasNext()) {
|
||||
Class<? extends Attribute> cl = it.next();
|
||||
if (!ts.hasAttribute(cl)) {
|
||||
Attribute att = ts.getAttribute(cl);
|
||||
if (att == null) {
|
||||
continue;
|
||||
}
|
||||
Attribute att = ts.getAttribute(cl);
|
||||
if (cl.isAssignableFrom(CharTermAttribute.class)) {
|
||||
CharTermAttribute catt = (CharTermAttribute)att;
|
||||
cTerm = new String(catt.buffer(), 0, catt.length());
|
||||
|
|
|
@ -482,10 +482,10 @@ public final class SimplePreAnalyzedParser implements PreAnalyzedParser {
|
|||
String tTerm = null;
|
||||
while (it.hasNext()) {
|
||||
Class<? extends Attribute> cl = it.next();
|
||||
if (!ts.hasAttribute(cl)) {
|
||||
Attribute att = ts.getAttribute(cl);
|
||||
if (att == null) {
|
||||
continue;
|
||||
}
|
||||
Attribute att = ts.getAttribute(cl);
|
||||
if (cl.isAssignableFrom(CharTermAttribute.class)) {
|
||||
CharTermAttribute catt = (CharTermAttribute)att;
|
||||
cTerm = escape(catt.buffer(), catt.length());
|
||||
|
|
Loading…
Reference in New Issue