LUCENE-5611: put current patch on branch

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5611@1590721 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-04-28 17:31:13 +00:00
parent 0acad0e334
commit 67ebc26fa9
55 changed files with 1329 additions and 2199 deletions

View File

@ -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();

View File

@ -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++;
}

View File

@ -31,7 +31,7 @@ import org.apache.lucene.index.AtomicReader;
* Codec API for writing stored fields:
* <p>
* <ol>
* <li>For every document, {@link #startDocument(int)} is called,
* <li>For every document, {@link #startDocument()} is called,
* informing the Codec how many fields will be written.
* <li>{@link #writeField(FieldInfo, StorableField)} is called for
* each field in the document.
@ -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);

View File

@ -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);

View File

@ -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");
}

View File

@ -244,9 +244,23 @@ 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");
if (!type.indexed()) {
if (type.storeTermVectors()) {
throw new IllegalArgumentException("cannot store term vectors "
+ "for a field that is not indexed (field=\"" + name + "\")");
}
if (type.storeTermVectorPositions()) {
throw new IllegalArgumentException("cannot store term vector positions "
+ "for a field that is not indexed (field=\"" + name + "\")");
}
if (type.storeTermVectorOffsets()) {
throw new IllegalArgumentException("cannot store term vector offsets "
+ "for a field that is not indexed (field=\"" + name + "\")");
}
if (type.storeTermVectorPayloads()) {
throw new IllegalArgumentException("cannot store term vector payloads "
+ "for a field that is not indexed (field=\"" + name + "\")");
}
}
this.type = type;

View File

@ -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");

View File

@ -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();

View File

@ -0,0 +1,669 @@
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();
if (invertState.attributeSource != stream) {
// EmptyTokenStream gets angry otherwise:
if (stream.hasAttribute(TermToBytesRefAttribute.class)) {
invertState.termAttribute = stream.getAttribute(TermToBytesRefAttribute.class);
} else {
invertState.termAttribute = null;
}
invertState.posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
invertState.offsetAttribute = stream.addAttribute(OffsetAttribute.class);
if (stream.hasAttribute(PayloadAttribute.class)) {
invertState.payloadAttribute = stream.getAttribute(PayloadAttribute.class);
} else {
invertState.payloadAttribute = null;
}
invertState.attributeSource = 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();
}
}
}

View File

@ -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();
}

View File

@ -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;
}

View File

@ -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();
}

View File

@ -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();
}
}
}

View File

@ -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();
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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();
}
}

View File

@ -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;
}

View File

@ -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);

View File

@ -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;

View File

@ -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;

View File

@ -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,6 @@ public final class FieldInvertState {
maxTermFrequency = 0;
uniqueTermCount = 0;
boost = 1.0f;
attributeSource = null;
}
/**

View File

@ -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;

View File

@ -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));
}
}

View File

@ -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();
}
}

View File

@ -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();
/**

View File

@ -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;
}

View File

@ -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();
}

View File

@ -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;
}

View File

@ -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();
}

View File

@ -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);
}
}

View File

@ -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() {
//
}
}

View File

@ -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();

View File

@ -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[];

View File

@ -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[];

View File

@ -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;
}

View File

@ -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");
}
}
}

View File

@ -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;
}
}
}

View File

@ -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) {

View File

@ -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();
}
}
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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();
}
}

View File

@ -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;

View File

@ -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;

View File

@ -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();
}
/**

View File

@ -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();
}
}

View File

@ -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();

View File

@ -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) {

View File

@ -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,59 @@ 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);
try {
new Field("field", "value", ft);
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);
try {
new Field("field", "value", ft);
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);
try {
new Field("field", "value", ft);
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);
try {
new Field("field", "value", ft);
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();

View File

@ -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();

View File

@ -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 {

View File

@ -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;
}

View File

@ -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,9 @@ public abstract class LuceneTestCase extends Assert {
.around(new TestRuleFieldCacheSanity())
.around(parentChainCallRule);
private static final Map<String,FieldType> fieldTermVectorOptions = new HashMap<String,FieldType>();
private static final Set<String> fieldNamesSeen = Collections.newSetFromMap(new ConcurrentHashMap<String,Boolean>());
// -----------------------------------------------------------------
// Suite and test case setup/ cleanup.
// -----------------------------------------------------------------
@ -619,6 +624,8 @@ public abstract class LuceneTestCase extends Assert {
@After
public void tearDown() throws Exception {
parentChainCallRule.teardownCalled = true;
fieldTermVectorOptions.clear();
fieldNamesSeen.clear();
}
@ -1168,11 +1175,12 @@ 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) {
name = new String(name);
if (usually(random) || !type.indexed()) {
// most of the time, don't modify the params
fieldNamesSeen.add(name);
return new Field(name, value, type);
}
@ -1185,19 +1193,39 @@ public abstract class LuceneTestCase extends Assert {
newType.setStored(true); // randomly store it
}
if (!newType.storeTermVectors() && random.nextBoolean()) {
newType.setStoreTermVectors(true);
if (!newType.storeTermVectorOffsets()) {
newType.setStoreTermVectorOffsets(random.nextBoolean());
}
if (!newType.storeTermVectorPositions()) {
newType.setStoreTermVectorPositions(random.nextBoolean());
// Randomly turn on term vector options, but always do
// so consistently for the same field name:
if (!newType.storeTermVectors() && fieldNamesSeen.contains(name) == false && random.nextBoolean()) {
FieldType prev;
synchronized(fieldTermVectorOptions) {
prev = fieldTermVectorOptions.get(name);
if (prev == null) {
newType.setStoreTermVectors(true);
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());
}
}
}
fieldTermVectorOptions.put(name, prev);
}
}
if (prev != null) {
newType.setStoreTermVectors(prev.storeTermVectors());
newType.setStoreTermVectorOffsets(prev.storeTermVectorOffsets());
newType.setStoreTermVectorPositions(prev.storeTermVectorPositions());
newType.setStoreTermVectorPayloads(prev.storeTermVectorPayloads());
}
System.out.println(" LTC: upgrade name=" + name + " type=" + newType);
}
newType.freeze();
// TODO: we need to do this, but smarter, ie, most of
// the time we set the same value for a given field but