LUCENE-1301: refactor DocumentsWriter into a package-private indexing chain, as a first step towards flexible indexing

git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@677865 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2008-07-18 09:20:12 +00:00
parent 12e4e67403
commit 7198ead131
65 changed files with 5493 additions and 2464 deletions

View File

@ -229,3 +229,12 @@ from those sources:
* for internal or external distribution as long as this notice
* remains attached.
*/
Some code in src/java/org/apache/lucene/util/ArrayUtil.java was
derived from Python 2.4.2 sources available at
http://www.python.org. Full license is here:
http://www.python.org/download/releases/2.4.2/license/

View File

@ -1,60 +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.store.RAMOutputStream;
import org.apache.lucene.search.Similarity;
/* Stores norms, buffered in RAM, until they are flushed
* to a partial segment. */
final class BufferedNorms {
RAMOutputStream out;
int upto;
private static final byte defaultNorm = Similarity.encodeNorm(1.0f);
BufferedNorms() {
out = new RAMOutputStream();
}
void add(float norm) throws IOException {
byte b = Similarity.encodeNorm(norm);
out.writeByte(b);
upto++;
}
void reset() {
out.reset();
upto = 0;
}
void fill(int docID) throws IOException {
// Must now fill in docs that didn't have this
// field. Note that this is how norms can consume
// tremendous storage when the docs have widely
// varying different fields, because we are not
// storing the norms sparsely (see LUCENE-830)
if (upto < docID) {
DocumentsWriter.fillBytes(out, defaultNorm, docID-upto);
upto = docID;
}
}
}

View File

@ -104,10 +104,10 @@ final class ByteBlockPool {
}
// Size of each slice. These arrays should be at most 16
// elements. First array is just a compact way to encode
// X+1 with a max. Second array is the length of each
// slice, ie first slice is 5 bytes, next slice is 14
// bytes, etc.
// elements (index is encoded with 4 bits). First array
// is just a compact way to encode X+1 with a max. Second
// array is the length of each slice, ie first slice is 5
// bytes, next slice is 14 bytes, etc.
final static int[] nextLevelArray = {1, 2, 3, 4, 5, 6, 7, 8, 9, 9};
final static int[] levelSizeArray = {5, 14, 20, 30, 40, 40, 80, 80, 120, 200};
final static int FIRST_LEVEL_SIZE = levelSizeArray[0];

View File

@ -39,7 +39,9 @@ final class ByteSliceReader extends IndexInput {
public void init(ByteBlockPool pool, int startIndex, int endIndex) {
assert endIndex-startIndex > 0;
assert endIndex-startIndex >= 0;
assert startIndex >= 0;
assert endIndex >= 0;
this.pool = pool;
this.endIndex = endIndex;
@ -59,9 +61,14 @@ final class ByteSliceReader extends IndexInput {
limit = upto+firstSize-4;
}
public boolean eof() {
assert upto + bufferOffset <= endIndex;
return upto + bufferOffset == endIndex;
}
public byte readByte() {
// Assert that we are not @ EOF
assert upto + bufferOffset < endIndex;
assert !eof();
assert upto <= limit;
if (upto == limit)
nextSlice();
return buffer[upto++];

View File

@ -23,11 +23,11 @@ final class CharBlockPool {
int numBuffer;
int bufferUpto = -1; // Which buffer we are upto
public int byteUpto = DocumentsWriter.CHAR_BLOCK_SIZE; // Where we are in head buffer
public int charUpto = DocumentsWriter.CHAR_BLOCK_SIZE; // Where we are in head buffer
public char[] buffer; // Current head buffer
public int byteOffset = -DocumentsWriter.CHAR_BLOCK_SIZE; // Current head offset
private DocumentsWriter docWriter;
public int charOffset = -DocumentsWriter.CHAR_BLOCK_SIZE; // Current head offset
final private DocumentsWriter docWriter;
public CharBlockPool(DocumentsWriter docWriter) {
this.docWriter = docWriter;
@ -36,8 +36,8 @@ final class CharBlockPool {
public void reset() {
docWriter.recycleCharBlocks(buffers, 1+bufferUpto);
bufferUpto = -1;
byteUpto = DocumentsWriter.CHAR_BLOCK_SIZE;
byteOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;
charUpto = DocumentsWriter.CHAR_BLOCK_SIZE;
charOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;
}
public void nextBuffer() {
@ -49,8 +49,8 @@ final class CharBlockPool {
buffer = buffers[1+bufferUpto] = docWriter.getCharBlock();
bufferUpto++;
byteUpto = 0;
byteOffset += DocumentsWriter.CHAR_BLOCK_SIZE;
charUpto = 0;
charOffset += DocumentsWriter.CHAR_BLOCK_SIZE;
}
}

View File

@ -17,14 +17,13 @@ package org.apache.lucene.index;
* limitations under the License.
*/
/* Used by DocumentsWriter to track data for term vectors.
* One of these exists per unique term seen in each field in
* the document. */
class PostingVector {
Posting p; // Corresponding Posting instance for this term
int lastOffset; // Last offset we saw
int offsetStart; // Address of first slice for offsets
int offsetUpto; // Next write address for offsets
int posStart; // Address of first slice for positions
int posUpto; // Next write address for positions
import java.io.IOException;
import java.util.Collection;
abstract class DocConsumer {
abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException;
abstract void flush(final Collection threads, final DocumentsWriter.FlushState state) throws IOException;
abstract void closeDocStore(final DocumentsWriter.FlushState state) throws IOException;
abstract void abort();
abstract boolean freeRAM();
}

View File

@ -0,0 +1,33 @@
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 DocConsumerPerThread {
/** Process the document. If there is
* something for this document to be done in docID order,
* you should encapsulate that as a
* DocumentsWriter.DocWriter and return it.
* DocumentsWriter then calls finish() on this object
* when it's its turn. */
abstract DocumentsWriter.DocWriter processDocument() throws IOException;
abstract void abort();
}

View File

@ -0,0 +1,49 @@
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 {
FieldInfos fieldInfos;
/** Called when DocumentsWriter decides to create a new
* segment */
abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
/** Called when DocumentsWriter decides to close the doc
* stores */
abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
/** Called when an aborting exception is hit */
abstract void abort();
/** Add a new thread */
abstract DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException;
/** Called when DocumentsWriter is using too much RAM.
* The consumer should free RAM, if possible, returning
* true if any RAM was in fact freed. */
abstract boolean freeRAM();
void setFieldInfos(FieldInfos fieldInfos) {
this.fieldInfos = fieldInfos;
}
}

View File

@ -18,12 +18,10 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import org.apache.lucene.document.Fieldable;
// Used only internally to DW to call abort "up the stack"
class AbortException extends IOException {
public AbortException(Throwable cause, DocumentsWriter docWriter) {
super();
initCause(cause);
docWriter.setAborting();
}
abstract class DocFieldConsumerPerField {
/** Processes all occurrences of a single field */
abstract void processFields(Fieldable[] fields, int count) throws IOException;
abstract void abort();
}

View File

@ -0,0 +1,27 @@
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 DocFieldConsumerPerThread {
abstract void startDocument() throws IOException;
abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
abstract DocFieldConsumerPerField addField(FieldInfo fi);
abstract void abort();
}

View File

@ -0,0 +1,162 @@
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.util.HashMap;
import java.util.Collection;
import java.util.Iterator;
import java.util.Map;
import java.util.HashSet;
import java.io.IOException;
import org.apache.lucene.util.ArrayUtil;
/** This is just a "splitter" class: it lets you wrap two
* DocFieldConsumer instances as a single consumer. */
final class DocFieldConsumers extends DocFieldConsumer {
final DocFieldConsumer one;
final DocFieldConsumer two;
public DocFieldConsumers(DocFieldConsumer one, DocFieldConsumer two) {
this.one = one;
this.two = two;
}
void setFieldInfos(FieldInfos fieldInfos) {
super.setFieldInfos(fieldInfos);
one.setFieldInfos(fieldInfos);
two.setFieldInfos(fieldInfos);
}
public void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
Map oneThreadsAndFields = new HashMap();
Map twoThreadsAndFields = new HashMap();
Iterator it = threadsAndFields.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
Collection fields = (Collection) entry.getValue();
Iterator fieldsIt = fields.iterator();
Collection oneFields = new HashSet();
Collection twoFields = new HashSet();
while(fieldsIt.hasNext()) {
DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldsIt.next();
oneFields.add(perField.one);
twoFields.add(perField.two);
}
oneThreadsAndFields.put(perThread.one, oneFields);
twoThreadsAndFields.put(perThread.two, twoFields);
}
one.flush(oneThreadsAndFields, state);
two.flush(twoThreadsAndFields, state);
}
public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
try {
one.closeDocStore(state);
} finally {
two.closeDocStore(state);
}
}
public void abort() {
try {
one.abort();
} finally {
two.abort();
}
}
public boolean freeRAM() {
boolean any = one.freeRAM();
any |= two.freeRAM();
return any;
}
public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException {
return new DocFieldConsumersPerThread(docFieldProcessorPerThread, this, one.addThread(docFieldProcessorPerThread), two.addThread(docFieldProcessorPerThread));
}
PerDoc[] docFreeList = new PerDoc[1];
int freeCount;
int allocCount;
synchronized PerDoc getPerDoc() {
if (freeCount == 0) {
allocCount++;
if (allocCount > docFreeList.length) {
// Grow our free list up front to make sure we have
// enough space to recycle all outstanding PerDoc
// instances
assert allocCount == 1+docFreeList.length;
docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
}
return new PerDoc();
} else
return docFreeList[--freeCount];
}
synchronized void freePerDoc(PerDoc perDoc) {
assert freeCount < docFreeList.length;
docFreeList[freeCount++] = perDoc;
}
class PerDoc extends DocumentsWriter.DocWriter {
DocumentsWriter.DocWriter one;
DocumentsWriter.DocWriter two;
public long sizeInBytes() {
return one.sizeInBytes() + two.sizeInBytes();
}
public void finish() throws IOException {
try {
try {
one.finish();
} finally {
two.finish();
}
} finally {
freePerDoc(this);
}
}
public void abort() {
try {
try {
one.abort();
} finally {
two.abort();
}
} finally {
freePerDoc(this);
}
}
}
}

View File

@ -0,0 +1,47 @@
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.document.Fieldable;
final class DocFieldConsumersPerField extends DocFieldConsumerPerField {
final DocFieldConsumerPerField one;
final DocFieldConsumerPerField two;
final DocFieldConsumersPerThread perThread;
public DocFieldConsumersPerField(DocFieldConsumersPerThread perThread, DocFieldConsumerPerField one, DocFieldConsumerPerField two) {
this.perThread = perThread;
this.one = one;
this.two = two;
}
public void processFields(Fieldable[] fields, int count) throws IOException {
one.processFields(fields, count);
two.processFields(fields, count);
}
public void abort() {
try {
one.abort();
} finally {
two.abort();
}
}
}

View File

@ -0,0 +1,71 @@
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;
final class DocFieldConsumersPerThread extends DocFieldConsumerPerThread {
final DocFieldConsumerPerThread one;
final DocFieldConsumerPerThread two;
final DocFieldConsumers parent;
final DocumentsWriter.DocState docState;
public DocFieldConsumersPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread,
DocFieldConsumers parent, DocFieldConsumerPerThread one, DocFieldConsumerPerThread two) {
this.parent = parent;
this.one = one;
this.two = two;
docState = docFieldProcessorPerThread.docState;
}
public void startDocument() throws IOException {
one.startDocument();
two.startDocument();
}
public void abort() {
try {
one.abort();
} finally {
two.abort();
}
}
public DocumentsWriter.DocWriter finishDocument() throws IOException {
final DocumentsWriter.DocWriter oneDoc = one.finishDocument();
final DocumentsWriter.DocWriter twoDoc = two.finishDocument();
if (oneDoc == null)
return twoDoc;
else if (twoDoc == null)
return oneDoc;
else {
DocFieldConsumers.PerDoc both = parent.getPerDoc();
both.docID = docState.docID;
assert oneDoc.docID == docState.docID;
assert twoDoc.docID == docState.docID;
both.one = oneDoc;
both.two = twoDoc;
return both;
}
}
public DocFieldConsumerPerField addField(FieldInfo fi) {
return new DocFieldConsumersPerField(this, one.addField(fi), two.addField(fi));
}
}

View File

@ -0,0 +1,80 @@
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.Map;
import java.util.HashMap;
import java.util.Iterator;
/**
* 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 DocumentsWriter docWriter;
final FieldInfos fieldInfos = new FieldInfos();
final DocFieldConsumer consumer;
public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) {
this.docWriter = docWriter;
this.consumer = consumer;
consumer.setFieldInfos(fieldInfos);
}
public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
consumer.closeDocStore(state);
}
public void flush(Collection threads, DocumentsWriter.FlushState state) throws IOException {
Map childThreadsAndFields = new HashMap();
Iterator it = threads.iterator();
while(it.hasNext()) {
DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) it.next();
childThreadsAndFields.put(perThread.consumer, perThread.fields());
perThread.trimFields(state);
}
consumer.flush(childThreadsAndFields, state);
// Important to save after asking consumer to flush so
// consumer can alter the FieldInfo* if necessary. EG,
// FreqProxTermsWriter does this with
// FieldInfo.storePayload.
fieldInfos.write(state.directory, state.segmentName + ".fnm");
}
public void abort() {
consumer.abort();
}
public boolean freeRAM() {
return consumer.freeRAM();
}
public DocConsumerPerThread addThread(DocumentsWriterThreadState threadState) throws IOException {
return new DocFieldProcessorPerThread(threadState, this);
}
}

View File

@ -0,0 +1,45 @@
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.document.Fieldable;
/**
* Holds all per thread, per field state.
*/
final class DocFieldProcessorPerField {
final DocFieldConsumerPerField consumer;
final FieldInfo fieldInfo;
DocFieldProcessorPerField next;
int lastGen = -1;
int fieldCount;
Fieldable[] fields = new Fieldable[1];
public DocFieldProcessorPerField(final DocFieldProcessorPerThread perThread, final FieldInfo fieldInfo) {
this.consumer = perThread.consumer.addField(fieldInfo);
this.fieldInfo = fieldInfo;
}
public void abort() {
consumer.abort();
}
}

View File

@ -0,0 +1,302 @@
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.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable;
/**
* Gathers all Fieldables for a document under the same
* name, updates FieldInfos, and calls per-field consumers
* to process field by field.
*
* Currently, only a single thread visits the fields,
* sequentially, for processing.
*/
final class DocFieldProcessorPerThread extends DocConsumerPerThread {
float docBoost;
int fieldGen;
final DocFieldProcessor docFieldProcessor;
final FieldInfos fieldInfos;
final DocFieldConsumerPerThread consumer;
// 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;
final DocumentsWriter.DocState docState;
public DocFieldProcessorPerThread(DocumentsWriterThreadState threadState, DocFieldProcessor docFieldProcessor) throws IOException {
this.docState = threadState.docState;
this.docFieldProcessor = docFieldProcessor;
this.fieldInfos = docFieldProcessor.fieldInfos;
this.consumer = docFieldProcessor.consumer.addThread(this);
}
public void abort() {
for(int i=0;i<fieldHash.length;i++) {
DocFieldProcessorPerField field = fieldHash[i];
while(field != null) {
final DocFieldProcessorPerField next = field.next;
field.abort();
field = next;
}
}
consumer.abort();
}
public Collection fields() {
Collection 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;
}
/** If there are fields we've seen but did not see again
* in the last run, then free them up. */
void trimFields(DocumentsWriter.FlushState state) {
for(int i=0;i<fieldHash.length;i++) {
DocFieldProcessorPerField perField = fieldHash[i];
DocFieldProcessorPerField lastPerField = null;
while (perField != null) {
if (perField.lastGen == -1) {
// This field was not seen since the previous
// flush, so, free up its resources now
// Unhash
if (lastPerField == null)
fieldHash[i] = perField.next;
else
lastPerField.next = perField.next;
if (state.docWriter.infoStream != null)
state.docWriter.infoStream.println(" purge field=" + perField.fieldInfo.name);
totalFieldCount--;
} else {
// Reset
perField.lastGen = -1;
lastPerField = perField;
}
perField = perField.next;
}
}
}
private void rehash() {
final int newHashSize = (int) (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;
}
public DocumentsWriter.DocWriter processDocument() throws IOException {
consumer.startDocument();
final Document doc = docState.doc;
assert docFieldProcessor.docWriter.writer.testPoint("DocumentsWriter.ThreadState.init start");
fieldCount = 0;
final int thisFieldGen = fieldGen++;
final List docFields = doc.getFields();
final int numDocFields = docFields.size();
// 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(int i=0;i<numDocFields;i++) {
Fieldable field = (Fieldable) docFields.get(i);
final String fieldName = field.name();
// 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.add(fieldName, field.isIndexed(), field.isTermVectorStored(),
field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
field.getOmitNorms(), false);
fp = new DocFieldProcessorPerField(this, fi);
fp.next = fieldHash[hashPos];
fieldHash[hashPos] = fp;
totalFieldCount++;
if (totalFieldCount >= fieldHash.length/2)
rehash();
} else
fp.fieldInfo.update(field.isIndexed(), field.isTermVectorStored(),
field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
field.getOmitNorms(), false);
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;
}
if (fp.fieldCount == fp.fields.length) {
Fieldable[] newArray = new Fieldable[fp.fields.length*2];
System.arraycopy(fp.fields, 0, newArray, 0, fp.fieldCount);
fp.fields = newArray;
}
fp.fields[fp.fieldCount++] = field;
}
// 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.
quickSort(fields, 0, fieldCount-1);
for(int i=0;i<fieldCount;i++)
fields[i].consumer.processFields(fields[i].fields, fields[i].fieldCount);
if (docState.maxTermPrefix != null && docState.infoStream != null)
docState.infoStream.println("WARNING: document contains at least one immense term (longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");
return consumer.finishDocument();
}
void quickSort(DocFieldProcessorPerField[] array, int lo, int hi) {
if (lo >= hi)
return;
else if (hi == 1+lo) {
if (array[lo].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
final DocFieldProcessorPerField tmp = array[lo];
array[lo] = array[hi];
array[hi] = tmp;
}
return;
}
int mid = (lo + hi) >>> 1;
if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
DocFieldProcessorPerField tmp = array[lo];
array[lo] = array[mid];
array[mid] = tmp;
}
if (array[mid].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
DocFieldProcessorPerField tmp = array[mid];
array[mid] = array[hi];
array[hi] = tmp;
if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
DocFieldProcessorPerField tmp2 = array[lo];
array[lo] = array[mid];
array[mid] = tmp2;
}
}
int left = lo + 1;
int right = hi - 1;
if (left >= right)
return;
DocFieldProcessorPerField partition = array[mid];
for (; ;) {
while (array[right].fieldInfo.name.compareTo(partition.fieldInfo.name) > 0)
--right;
while (left < right && array[left].fieldInfo.name.compareTo(partition.fieldInfo.name) <= 0)
++left;
if (left < right) {
DocFieldProcessorPerField tmp = array[left];
array[left] = array[right];
array[right] = tmp;
--right;
} else {
break;
}
}
quickSort(array, lo, left);
quickSort(array, left + 1, hi);
}
}

View File

@ -0,0 +1,109 @@
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.util.Map;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Collection;
import java.util.Iterator;
import java.io.IOException;
/** 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;
public DocInverter(InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
this.consumer = consumer;
this.endConsumer = endConsumer;
}
void setFieldInfos(FieldInfos fieldInfos) {
super.setFieldInfos(fieldInfos);
consumer.setFieldInfos(fieldInfos);
endConsumer.setFieldInfos(fieldInfos);
}
void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
Map childThreadsAndFields = new HashMap();
Map endChildThreadsAndFields = new HashMap();
Iterator it = threadsAndFields.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey();
Collection fields = (Collection) entry.getValue();
Iterator fieldsIt = fields.iterator();
Collection childFields = new HashSet();
Collection endChildFields = new HashSet();
while(fieldsIt.hasNext()) {
DocInverterPerField perField = (DocInverterPerField) fieldsIt.next();
childFields.add(perField.consumer);
endChildFields.add(perField.endConsumer);
}
childThreadsAndFields.put(perThread.consumer, childFields);
endChildThreadsAndFields.put(perThread.endConsumer, endChildFields);
}
consumer.flush(childThreadsAndFields, state);
endConsumer.flush(endChildThreadsAndFields, state);
}
public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
consumer.closeDocStore(state);
endConsumer.closeDocStore(state);
}
void abort() {
consumer.abort();
endConsumer.abort();
}
public boolean freeRAM() {
return consumer.freeRAM();
}
public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) {
return new DocInverterPerThread(docFieldProcessorPerThread, this);
}
final static class FieldInvertState {
int position;
int length;
int offset;
float boost;
void reset(float docBoost) {
position = 0;
length = 0;
offset = 0;
boost = docBoost;
}
}
}

View File

@ -0,0 +1,181 @@
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.io.Reader;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
/**
* 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 private DocInverterPerThread perThread;
final private FieldInfo fieldInfo;
final InvertedDocConsumerPerField consumer;
final InvertedDocEndConsumerPerField endConsumer;
final DocumentsWriter.DocState docState;
final DocInverter.FieldInvertState fieldState;
public DocInverterPerField(DocInverterPerThread perThread, FieldInfo fieldInfo) {
this.perThread = perThread;
this.fieldInfo = fieldInfo;
docState = perThread.docState;
fieldState = perThread.fieldState;
this.consumer = perThread.consumer.addField(this, fieldInfo);
this.endConsumer = perThread.endConsumer.addField(this, fieldInfo);
}
void abort() {
consumer.abort();
endConsumer.abort();
}
public void processFields(final Fieldable[] fields,
final int count) throws IOException {
fieldState.reset(docState.doc.getBoost());
final int maxFieldLength = docState.maxFieldLength;
final boolean doInvert = consumer.start(fields, count);
for(int i=0;i<count;i++) {
final Fieldable field = fields[i];
// TODO FI: this should be "genericized" to querying
// consumer if it wants to see this particular field
// tokenized.
if (field.isIndexed() && doInvert) {
if (fieldState.length > 0)
fieldState.position += docState.analyzer.getPositionIncrementGap(fieldInfo.name);
if (!field.isTokenized()) { // un-tokenized field
String stringValue = field.stringValue();
final int valueLength = stringValue.length();
Token token = perThread.localToken;
token.clear();
char[] termBuffer = token.termBuffer();
if (termBuffer.length < valueLength)
termBuffer = token.resizeTermBuffer(valueLength);
stringValue.getChars(0, valueLength, termBuffer, 0);
token.setTermLength(valueLength);
token.setStartOffset(fieldState.offset);
token.setEndOffset(fieldState.offset + stringValue.length());
boolean success = false;
try {
consumer.add(token);
success = true;
} finally {
if (!success)
docState.docWriter.setAborting();
}
fieldState.offset += stringValue.length();
fieldState.length++;
fieldState.position++;
} else { // tokenized field
final TokenStream stream;
final TokenStream streamValue = field.tokenStreamValue();
if (streamValue != null)
stream = streamValue;
else {
// the field does not have a TokenStream,
// so we have to obtain one from the analyzer
final Reader reader; // find or make Reader
final Reader readerValue = field.readerValue();
if (readerValue != null)
reader = readerValue;
else {
String stringValue = field.stringValue();
if (stringValue == null)
throw new IllegalArgumentException("field must have either TokenStream, String or Reader value");
perThread.stringReader.init(stringValue);
reader = perThread.stringReader;
}
// Tokenize field and add to postingTable
stream = docState.analyzer.reusableTokenStream(fieldInfo.name, reader);
}
// reset the TokenStream to the first token
stream.reset();
try {
int offsetEnd = fieldState.offset-1;
final Token localToken = perThread.localToken;
for(;;) {
// 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
Token token = stream.next(localToken);
if (token == null) break;
fieldState.position += (token.getPositionIncrement() - 1);
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(token);
success = true;
} finally {
if (!success)
docState.docWriter.setAborting();
}
fieldState.position++;
offsetEnd = fieldState.offset + token.endOffset();
if (++fieldState.length >= maxFieldLength) {
if (docState.infoStream != null)
docState.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens");
break;
}
}
fieldState.offset = offsetEnd+1;
} finally {
stream.close();
}
}
fieldState.boost *= field.getBoost();
}
}
consumer.finish();
endConsumer.finish();
}
}

View File

@ -0,0 +1,70 @@
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.Token;
/** This is a DocFieldConsumer that inverts each field,
* separately, from a Document, and accepts a
* InvertedTermsConsumer to process those terms. */
final class DocInverterPerThread extends DocFieldConsumerPerThread {
final DocInverter docInverter;
final InvertedDocConsumerPerThread consumer;
final InvertedDocEndConsumerPerThread endConsumer;
final Token localToken = new Token();
final DocumentsWriter.DocState docState;
final DocInverter.FieldInvertState fieldState = new DocInverter.FieldInvertState();
// Used to read a string value for a field
final ReusableStringReader stringReader = new ReusableStringReader();
public DocInverterPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, DocInverter docInverter) {
this.docInverter = docInverter;
docState = docFieldProcessorPerThread.docState;
consumer = docInverter.consumer.addThread(this);
endConsumer = docInverter.endConsumer.addThread(this);
}
public void startDocument() throws IOException {
consumer.startDocument();
endConsumer.startDocument();
}
public DocumentsWriter.DocWriter finishDocument() throws IOException {
// TODO: allow endConsumer.finishDocument to also return
// a DocWriter
endConsumer.finishDocument();
return consumer.finishDocument();
}
void abort() {
try {
consumer.abort();
} finally {
endConsumer.abort();
}
}
public DocFieldConsumerPerField addField(FieldInfo fi) {
return new DocInverterPerField(this, fi);
}
}

File diff suppressed because it is too large Load Diff

View File

@ -1,682 +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.document.Fieldable;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.UnicodeUtil;
import java.io.IOException;
import java.io.Reader;
import java.util.Arrays;
/** Used by DocumentsWriter to hold data associated with a
* single field in a single ThreadState, including the
* Postings hash. A document may have many occurrences for
* a given field name; we gather all such occurrences here
* (in docFields) so that we can process the entire field
* at once. */
final class DocumentsWriterFieldData implements Comparable {
final DocumentsWriterThreadState threadState;
FieldInfo fieldInfo;
int fieldCount;
Fieldable[] docFields = new Fieldable[1];
int lastGen = -1;
DocumentsWriterFieldData next;
boolean doNorms;
boolean doVectors;
boolean doVectorPositions;
boolean doVectorOffsets;
boolean postingsCompacted;
int numPostings;
Posting[] postingsHash;
int postingsHashSize;
int postingsHashHalfSize;
int postingsHashMask;
int position;
int length;
int offset;
float boost;
int postingsVectorsUpto;
final ByteSliceWriter sliceWriter;
final ByteSliceWriter vectorsSliceWriter;
public DocumentsWriterFieldData(DocumentsWriterThreadState threadState, FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo;
this.threadState = threadState;
sliceWriter = new ByteSliceWriter(threadState.postingsPool);
vectorsSliceWriter = new ByteSliceWriter(threadState.vectorsPool);
}
void resetPostingArrays() {
if (!postingsCompacted)
compactPostings();
threadState.docWriter.recyclePostings(this.postingsHash, numPostings);
Arrays.fill(postingsHash, 0, postingsHash.length, null);
postingsCompacted = false;
numPostings = 0;
}
void initPostingArrays() {
// Target hash fill factor of <= 50%
// NOTE: must be a power of two for hash collision
// strategy to work correctly
postingsHashSize = 4;
postingsHashHalfSize = 2;
postingsHashMask = postingsHashSize-1;
postingsHash = new Posting[postingsHashSize];
}
public int compareTo(Object o) {
return fieldInfo.name.compareTo(((DocumentsWriterFieldData) o).fieldInfo.name);
}
private void compactPostings() {
int upto = 0;
for(int i=0;i<postingsHashSize;i++)
if (postingsHash[i] != null)
postingsHash[upto++] = postingsHash[i];
assert upto == numPostings;
postingsCompacted = true;
}
/** Collapse the hash table & sort in-place. */
public Posting[] sortPostings() {
compactPostings();
threadState.doPostingSort(postingsHash, numPostings);
return postingsHash;
}
/** Process all occurrences of one field in the document. */
public void processField(Analyzer analyzer) throws IOException, AbortException {
length = 0;
position = 0;
offset = 0;
boost = threadState.docBoost;
final int maxFieldLength = threadState.docWriter.writer.getMaxFieldLength();
final int limit = fieldCount;
final Fieldable[] docFieldsFinal = docFields;
boolean doWriteVectors = true;
// Walk through all occurrences in this doc for this
// field:
try {
for(int j=0;j<limit;j++) {
Fieldable field = docFieldsFinal[j];
if (field.isIndexed())
invertField(field, analyzer, maxFieldLength);
if (field.isStored()) {
threadState.numStoredFields++;
boolean success = false;
try {
threadState.localFieldsWriter.writeField(fieldInfo, field);
success = true;
} finally {
// If we hit an exception inside
// localFieldsWriter.writeField, the
// contents of fdtLocal can be corrupt, so
// we must discard all stored fields for
// this document:
if (!success)
threadState.fdtLocal.reset();
}
}
docFieldsFinal[j] = null;
}
} catch (AbortException ae) {
doWriteVectors = false;
throw ae;
} finally {
if (postingsVectorsUpto > 0) {
try {
if (doWriteVectors) {
// Add term vectors for this field
boolean success = false;
try {
writeVectors(fieldInfo);
success = true;
} finally {
if (!success) {
// If we hit an exception inside
// writeVectors, the contents of tvfLocal
// can be corrupt, so we must discard all
// term vectors for this document:
threadState.numVectorFields = 0;
threadState.tvfLocal.reset();
}
}
}
} finally {
if (postingsVectorsUpto > threadState.maxPostingsVectors)
threadState.maxPostingsVectors = postingsVectorsUpto;
postingsVectorsUpto = 0;
threadState.vectorsPool.reset();
}
}
}
}
int offsetEnd;
Token localToken = new Token();
/* Invert one occurrence of one field in the document */
public void invertField(Fieldable field, Analyzer analyzer, final int maxFieldLength) throws IOException, AbortException {
if (length>0)
position += analyzer.getPositionIncrementGap(fieldInfo.name);
if (!field.isTokenized()) { // un-tokenized field
String stringValue = field.stringValue();
final int valueLength = stringValue.length();
Token token = localToken;
token.clear();
char[] termBuffer = token.termBuffer();
if (termBuffer.length < valueLength)
termBuffer = token.resizeTermBuffer(valueLength);
stringValue.getChars(0, valueLength, termBuffer, 0);
token.setTermLength(valueLength);
token.setStartOffset(offset);
token.setEndOffset(offset + stringValue.length());
addPosition(token);
offset += stringValue.length();
length++;
} else { // tokenized field
final TokenStream stream;
final TokenStream streamValue = field.tokenStreamValue();
if (streamValue != null)
stream = streamValue;
else {
// the field does not have a TokenStream,
// so we have to obtain one from the analyzer
final Reader reader; // find or make Reader
final Reader readerValue = field.readerValue();
if (readerValue != null)
reader = readerValue;
else {
String stringValue = field.stringValue();
if (stringValue == null)
throw new IllegalArgumentException("field must have either TokenStream, String or Reader value");
threadState.stringReader.init(stringValue);
reader = threadState.stringReader;
}
// Tokenize field and add to postingTable
stream = analyzer.reusableTokenStream(fieldInfo.name, reader);
}
// reset the TokenStream to the first token
stream.reset();
try {
offsetEnd = offset-1;
for(;;) {
Token token = stream.next(localToken);
if (token == null) break;
position += (token.getPositionIncrement() - 1);
addPosition(token);
if (++length >= maxFieldLength) {
if (threadState.docWriter.infoStream != null)
threadState.docWriter.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens");
break;
}
}
offset = offsetEnd+1;
} finally {
stream.close();
}
}
boost *= field.getBoost();
}
/** Only called when term vectors are enabled. This
* is called the first time we see a given term for
* each document, to allocate a PostingVector
* instance that is used to record data needed to
* write the posting vectors. */
private PostingVector addNewVector() {
if (postingsVectorsUpto == threadState.postingsVectors.length) {
final int newSize;
if (threadState.postingsVectors.length < 2)
newSize = 2;
else
newSize = (int) (1.5*threadState.postingsVectors.length);
PostingVector[] newArray = new PostingVector[newSize];
System.arraycopy(threadState.postingsVectors, 0, newArray, 0, threadState.postingsVectors.length);
threadState.postingsVectors = newArray;
}
p.vector = threadState.postingsVectors[postingsVectorsUpto];
if (p.vector == null)
p.vector = threadState.postingsVectors[postingsVectorsUpto] = new PostingVector();
postingsVectorsUpto++;
final PostingVector v = p.vector;
v.p = p;
if (doVectorPositions) {
final int upto = threadState.vectorsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
v.posStart = v.posUpto = threadState.vectorsPool.byteOffset + upto;
}
if (doVectorOffsets) {
final int upto = threadState.vectorsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
v.offsetStart = v.offsetUpto = threadState.vectorsPool.byteOffset + upto;
}
return v;
}
int offsetStartCode;
int offsetStart;
// Current posting we are working on
Posting p;
PostingVector vector;
/** Test whether the text for current Posting p equals
* current tokenText. */
boolean postingEquals(final char[] tokenText, final int tokenTextLen) {
final char[] text = threadState.charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
assert text != null;
int pos = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
int tokenPos = 0;
for(;tokenPos<tokenTextLen;pos++,tokenPos++)
if (tokenText[tokenPos] != text[pos])
return false;
return 0xffff == text[pos];
}
/** This is the hotspot of indexing: it's called once
* for every term of every document. Its job is to *
* update the postings byte stream (Postings hash) *
* based on the occurence of a single term. */
private void addPosition(Token token) throws AbortException {
final Payload payload = token.getPayload();
// Get the text of this term. Term can either
// provide a String token or offset into a char[]
// array
final char[] tokenText = token.termBuffer();
final int tokenTextLen = token.termLength();
int code = 0;
// Compute hashcode & replace any invalid UTF16 sequences
int downto = tokenTextLen;
while (downto > 0) {
char ch = tokenText[--downto];
if (ch >= UnicodeUtil.UNI_SUR_LOW_START && ch <= UnicodeUtil.UNI_SUR_LOW_END) {
if (0 == downto) {
// Unpaired
ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
} else {
final char ch2 = tokenText[downto-1];
if (ch2 >= UnicodeUtil.UNI_SUR_HIGH_START && ch2 <= UnicodeUtil.UNI_SUR_HIGH_END) {
// OK: high followed by low. This is a valid
// surrogate pair.
code = ((code*31) + ch)*31+ch2;
downto--;
continue;
} else {
// Unpaired
ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
}
}
} else if (ch >= UnicodeUtil.UNI_SUR_HIGH_START && ch <= UnicodeUtil.UNI_SUR_HIGH_END)
// Unpaired
ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
code = (code*31) + ch;
}
// System.out.println(" addPosition: field=" + fieldInfo.name + " buffer=" + new String(tokenText, 0, tokenTextLen) + " pos=" + position + " offsetStart=" + (offset+token.startOffset()) + " offsetEnd=" + (offset + token.endOffset()) + " docID=" + docID + " doPos=" + doVectorPositions + " doOffset=" + doVectorOffsets);
int hashPos = code & postingsHashMask;
assert !postingsCompacted;
// Locate Posting in hash
p = postingsHash[hashPos];
if (p != null && !postingEquals(tokenText, tokenTextLen)) {
// Conflict: keep searching different locations in
// the hash table.
final int inc = ((code>>8)+code)|1;
do {
code += inc;
hashPos = code & postingsHashMask;
p = postingsHash[hashPos];
} while (p != null && !postingEquals(tokenText, tokenTextLen));
}
final int proxCode;
// If we hit an exception below, it's possible the
// posting list or term vectors data will be
// partially written and thus inconsistent if
// flushed, so we have to abort all documents
// since the last flush:
try {
if (p != null) { // term seen since last flush
if (threadState.docID != p.lastDocID) { // term not yet seen in this doc
// System.out.println(" seen before (new docID=" + docID + ") freqUpto=" + p.freqUpto +" proxUpto=" + p.proxUpto);
assert p.docFreq > 0;
// Now that we know doc freq for previous doc,
// write it & lastDocCode
sliceWriter.init(p.freqUpto);
if (1 == p.docFreq)
sliceWriter.writeVInt(p.lastDocCode|1);
else {
sliceWriter.writeVInt(p.lastDocCode);
sliceWriter.writeVInt(p.docFreq);
}
p.freqUpto = sliceWriter.getAddress();
if (doVectors) {
vector = addNewVector();
if (doVectorOffsets) {
offsetStartCode = offsetStart = offset + token.startOffset();
offsetEnd = offset + token.endOffset();
}
}
proxCode = position;
p.docFreq = 1;
// Store code so we can write this after we're
// done with this new doc
p.lastDocCode = (threadState.docID-p.lastDocID) << 1;
p.lastDocID = threadState.docID;
} else { // term already seen in this doc
// System.out.println(" seen before (same docID=" + docID + ") proxUpto=" + p.proxUpto);
p.docFreq++;
proxCode = position-p.lastPosition;
if (doVectors) {
vector = p.vector;
if (vector == null)
vector = addNewVector();
if (doVectorOffsets) {
offsetStart = offset + token.startOffset();
offsetEnd = offset + token.endOffset();
offsetStartCode = offsetStart-vector.lastOffset;
}
}
}
} else { // term not seen before
// System.out.println(" never seen docID=" + docID);
// Refill?
if (0 == threadState.postingsFreeCount) {
threadState.docWriter.getPostings(threadState.postingsFreeList);
threadState.postingsFreeCount = threadState.postingsFreeList.length;
}
final int textLen1 = 1+tokenTextLen;
if (textLen1 + threadState.charPool.byteUpto > DocumentsWriter.CHAR_BLOCK_SIZE) {
if (textLen1 > DocumentsWriter.CHAR_BLOCK_SIZE) {
// Just skip this term, to remain as robust as
// possible during indexing. A TokenFilter
// can be inserted into the analyzer chain if
// other behavior is wanted (pruning the term
// to a prefix, throwing an exception, etc).
if (threadState.maxTermPrefix == null)
threadState.maxTermPrefix = new String(tokenText, 0, 30);
// Still increment position:
position++;
return;
}
threadState.charPool.nextBuffer();
}
final char[] text = threadState.charPool.buffer;
final int textUpto = threadState.charPool.byteUpto;
// Pull next free Posting from free list
p = threadState.postingsFreeList[--threadState.postingsFreeCount];
p.textStart = textUpto + threadState.charPool.byteOffset;
threadState.charPool.byteUpto += textLen1;
System.arraycopy(tokenText, 0, text, textUpto, tokenTextLen);
text[textUpto+tokenTextLen] = 0xffff;
assert postingsHash[hashPos] == null;
postingsHash[hashPos] = p;
numPostings++;
if (numPostings == postingsHashHalfSize)
rehashPostings(2*postingsHashSize);
// Init first slice for freq & prox streams
final int upto1 = threadState.postingsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
p.freqStart = p.freqUpto = threadState.postingsPool.byteOffset + upto1;
final int upto2 = threadState.postingsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
p.proxStart = p.proxUpto = threadState.postingsPool.byteOffset + upto2;
p.lastDocCode = threadState.docID << 1;
p.lastDocID = threadState.docID;
p.docFreq = 1;
if (doVectors) {
vector = addNewVector();
if (doVectorOffsets) {
offsetStart = offsetStartCode = offset + token.startOffset();
offsetEnd = offset + token.endOffset();
}
}
proxCode = position;
}
sliceWriter.init(p.proxUpto);
if (payload != null && payload.length > 0) {
sliceWriter.writeVInt((proxCode<<1)|1);
sliceWriter.writeVInt(payload.length);
sliceWriter.writeBytes(payload.data, payload.offset, payload.length);
fieldInfo.storePayloads = true;
} else
sliceWriter.writeVInt(proxCode<<1);
p.proxUpto = sliceWriter.getAddress();
p.lastPosition = position++;
if (doVectorPositions) {
vectorsSliceWriter.init(vector.posUpto);
vectorsSliceWriter.writeVInt(proxCode);
vector.posUpto = vectorsSliceWriter.getAddress();
}
if (doVectorOffsets) {
vectorsSliceWriter.init(vector.offsetUpto);
vectorsSliceWriter.writeVInt(offsetStartCode);
vectorsSliceWriter.writeVInt(offsetEnd-offsetStart);
vector.lastOffset = offsetEnd;
vector.offsetUpto = vectorsSliceWriter.getAddress();
}
} catch (Throwable t) {
throw new AbortException(t, threadState.docWriter);
}
}
/** Called when postings hash is too small (> 50%
* occupied) or too large (< 20% occupied). */
void rehashPostings(final int newSize) {
final int newMask = newSize-1;
Posting[] newHash = new Posting[newSize];
for(int i=0;i<postingsHashSize;i++) {
Posting p0 = postingsHash[i];
if (p0 != null) {
final int start = p0.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
final char[] text = threadState.charPool.buffers[p0.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
int pos = start;
while(text[pos] != 0xffff)
pos++;
int code = 0;
while (pos > start)
code = (code*31) + text[--pos];
int hashPos = code & newMask;
assert hashPos >= 0;
if (newHash[hashPos] != null) {
final int inc = ((code>>8)+code)|1;
do {
code += inc;
hashPos = code & newMask;
} while (newHash[hashPos] != null);
}
newHash[hashPos] = p0;
}
}
postingsHashMask = newMask;
postingsHash = newHash;
postingsHashSize = newSize;
postingsHashHalfSize = newSize >> 1;
}
final ByteSliceReader vectorSliceReader = new ByteSliceReader();
/** 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. */
void writeVectors(FieldInfo fieldInfo) throws IOException {
assert fieldInfo.storeTermVector;
assert threadState.vectorFieldsInOrder(fieldInfo);
threadState.vectorFieldNumbers[threadState.numVectorFields] = fieldInfo.number;
threadState.vectorFieldPointers[threadState.numVectorFields] = threadState.tvfLocal.getFilePointer();
threadState.numVectorFields++;
final int numPostingsVectors = postingsVectorsUpto;
final PostingVector[] postingsVectors = threadState.postingsVectors;
final IndexOutput tvfLocal = threadState.tvfLocal;
threadState.tvfLocal.writeVInt(numPostingsVectors);
byte bits = 0x0;
if (doVectorPositions)
bits |= TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
if (doVectorOffsets)
bits |= TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
threadState.tvfLocal.writeByte(bits);
threadState.doVectorSort(postingsVectors, numPostingsVectors);
int encoderUpto = 0;
int lastTermBytesCount = 0;
final ByteSliceReader reader = vectorSliceReader;
final char[][] charBuffers = threadState.charPool.buffers;
for(int j=0;j<numPostingsVectors;j++) {
final PostingVector vector = postingsVectors[j];
Posting posting = vector.p;
final int freq = posting.docFreq;
final char[] text2 = charBuffers[posting.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
final int start2 = posting.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
// We swap between two encoders to save copying
// last Term's byte array
final UnicodeUtil.UTF8Result utf8Result = threadState.utf8Results[encoderUpto];
// TODO: we could do this incrementally
UnicodeUtil.UTF16toUTF8(text2, start2, utf8Result);
final int termBytesCount = utf8Result.length;
// TODO: UTF16toUTF8 could tell us this prefix
// Compute common prefix between last term and
// this term
int prefix = 0;
if (j > 0) {
final byte[] lastTermBytes = threadState.utf8Results[1-encoderUpto].result;
final byte[] termBytes = threadState.utf8Results[encoderUpto].result;
while(prefix < lastTermBytesCount && prefix < termBytesCount) {
if (lastTermBytes[prefix] != termBytes[prefix])
break;
prefix++;
}
}
encoderUpto = 1-encoderUpto;
lastTermBytesCount = termBytesCount;
final int suffix = termBytesCount - prefix;
tvfLocal.writeVInt(prefix);
tvfLocal.writeVInt(suffix);
tvfLocal.writeBytes(utf8Result.result, prefix, suffix);
tvfLocal.writeVInt(freq);
if (doVectorPositions) {
reader.init(threadState.vectorsPool, vector.posStart, vector.posUpto);
reader.writeTo(tvfLocal);
}
if (doVectorOffsets) {
reader.init(threadState.vectorsPool, vector.offsetStart, vector.offsetUpto);
reader.writeTo(tvfLocal);
}
}
}
}

View File

@ -18,13 +18,6 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.List;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.util.UnicodeUtil;
/** Used by DocumentsWriter to maintain per-thread state.
* We keep a separate Posting hash and other state for each
@ -32,693 +25,26 @@ import org.apache.lucene.util.UnicodeUtil;
* when writing the segment. */
final class DocumentsWriterThreadState {
Posting[] postingsFreeList; // Free Posting instances
int postingsFreeCount;
RAMOutputStream tvfLocal = new RAMOutputStream(); // Term vectors for one doc
RAMOutputStream fdtLocal = new RAMOutputStream(); // Stored fields for one doc
FieldsWriter localFieldsWriter; // Fields for one doc
long[] vectorFieldPointers;
int[] vectorFieldNumbers;
boolean isIdle = true; // Whether we are in use
int numThreads = 1; // Number of threads that use this instance
int docID; // docID we are now working on
int numStoredFields; // How many stored fields in current doc
float docBoost; // Boost for current doc
DocumentsWriterFieldData[] fieldDataArray; // Fields touched by current doc
int numFieldData; // How many fields in current doc
int numVectorFields; // How many vector fields in current doc
DocumentsWriterFieldData[] allFieldDataArray = new DocumentsWriterFieldData[10]; // All FieldData instances
int numAllFieldData;
DocumentsWriterFieldData[] fieldDataHash; // Hash FieldData instances by field name
int fieldDataHashMask;
String maxTermPrefix; // Non-null prefix of a too-large term if this
// doc has one
boolean doFlushAfter;
boolean isIdle = true; // false if this is currently in use by a thread
int numThreads = 1; // Number of threads that share this instance
boolean doFlushAfter; // true if we should flush after processing current doc
final DocConsumerPerThread consumer;
final DocumentsWriter.DocState docState;
final DocumentsWriter docWriter;
final ByteBlockPool postingsPool;
final ByteBlockPool vectorsPool;
final CharBlockPool charPool;
public DocumentsWriterThreadState(DocumentsWriter docWriter) {
public DocumentsWriterThreadState(DocumentsWriter docWriter) throws IOException {
this.docWriter = docWriter;
fieldDataArray = new DocumentsWriterFieldData[8];
fieldDataHash = new DocumentsWriterFieldData[16];
fieldDataHashMask = 15;
vectorFieldPointers = new long[10];
vectorFieldNumbers = new int[10];
postingsFreeList = new Posting[256];
postingsFreeCount = 0;
postingsPool = new ByteBlockPool(docWriter.byteBlockAllocator, true);
vectorsPool = new ByteBlockPool(docWriter.byteBlockAllocator, false);
charPool = new CharBlockPool(docWriter);
docState = new DocumentsWriter.DocState();
docState.maxFieldLength = docWriter.maxFieldLength;
docState.infoStream = docWriter.infoStream;
docState.similarity = docWriter.similarity;
docState.docWriter = docWriter;
consumer = docWriter.consumer.addThread(this);
}
/** Clear the postings hash and return objects back to
* shared pool */
public void resetPostings() throws IOException {
fieldGen = 0;
maxPostingsVectors = 0;
void doAfterFlush() {
numThreads = 0;
doFlushAfter = false;
if (localFieldsWriter != null) {
localFieldsWriter.close();
localFieldsWriter = null;
}
postingsPool.reset();
charPool.reset();
docWriter.recyclePostings(postingsFreeList, postingsFreeCount);
postingsFreeCount = 0;
for(int i=0;i<numAllFieldData;i++) {
DocumentsWriterFieldData fp = allFieldDataArray[i];
fp.lastGen = -1;
if (fp.numPostings > 0)
fp.resetPostingArrays();
}
}
/** Move all per-document state that was accumulated in
* the ThreadState into the "real" stores. */
public void writeDocument() throws IOException, AbortException {
// If we hit an exception while appending to the
// stored fields or term vectors files, we have to
// abort all documents since we last flushed because
// it means those files are possibly inconsistent.
try {
docWriter.numDocsInStore++;
// Append stored fields to the real FieldsWriter:
docWriter.fieldsWriter.flushDocument(numStoredFields, fdtLocal);
fdtLocal.reset();
// Append term vectors to the real outputs:
final IndexOutput tvx = docWriter.tvx;
final IndexOutput tvd = docWriter.tvd;
final IndexOutput tvf = docWriter.tvf;
if (tvx != null) {
tvx.writeLong(tvd.getFilePointer());
tvx.writeLong(tvf.getFilePointer());
tvd.writeVInt(numVectorFields);
if (numVectorFields > 0) {
for(int i=0;i<numVectorFields;i++)
tvd.writeVInt(vectorFieldNumbers[i]);
assert 0 == vectorFieldPointers[0];
long lastPos = vectorFieldPointers[0];
for(int i=1;i<numVectorFields;i++) {
long pos = vectorFieldPointers[i];
tvd.writeVLong(pos-lastPos);
lastPos = pos;
}
tvfLocal.writeTo(tvf);
tvfLocal.reset();
}
}
// Append norms for the fields we saw:
for(int i=0;i<numFieldData;i++) {
DocumentsWriterFieldData fp = fieldDataArray[i];
if (fp.doNorms) {
BufferedNorms bn = docWriter.norms[fp.fieldInfo.number];
assert bn != null;
assert bn.upto <= docID;
bn.fill(docID);
float norm = fp.boost * docWriter.writer.getSimilarity().lengthNorm(fp.fieldInfo.name, fp.length);
bn.add(norm);
}
}
} catch (Throwable t) {
// Forcefully idle this threadstate -- its state will
// be reset by abort()
isIdle = true;
throw new AbortException(t, docWriter);
}
if (docWriter.bufferIsFull && !docWriter.flushPending) {
docWriter.flushPending = true;
doFlushAfter = true;
}
}
int fieldGen;
/** Initializes shared state for this new document */
void init(Document doc, int docID) throws IOException, AbortException {
assert !isIdle;
assert docWriter.writer.testPoint("DocumentsWriter.ThreadState.init start");
this.docID = docID;
docBoost = doc.getBoost();
numStoredFields = 0;
numFieldData = 0;
numVectorFields = 0;
maxTermPrefix = null;
assert 0 == fdtLocal.length();
assert 0 == fdtLocal.getFilePointer();
assert 0 == tvfLocal.length();
assert 0 == tvfLocal.getFilePointer();
final int thisFieldGen = fieldGen++;
List docFields = doc.getFields();
final int numDocFields = docFields.size();
boolean docHasVectors = false;
// 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(int i=0;i<numDocFields;i++) {
Fieldable field = (Fieldable) docFields.get(i);
FieldInfo fi = docWriter.fieldInfos.add(field.name(), field.isIndexed(), field.isTermVectorStored(),
field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
field.getOmitNorms(), false);
if (fi.isIndexed && !fi.omitNorms) {
// Maybe grow our buffered norms
if (docWriter.norms.length <= fi.number) {
int newSize = (int) ((1+fi.number)*1.25);
BufferedNorms[] newNorms = new BufferedNorms[newSize];
System.arraycopy(docWriter.norms, 0, newNorms, 0, docWriter.norms.length);
docWriter.norms = newNorms;
}
if (docWriter.norms[fi.number] == null)
docWriter.norms[fi.number] = new BufferedNorms();
docWriter.hasNorms = true;
}
// Make sure we have a FieldData allocated
int hashPos = fi.name.hashCode() & fieldDataHashMask;
DocumentsWriterFieldData fp = fieldDataHash[hashPos];
while(fp != null && !fp.fieldInfo.name.equals(fi.name))
fp = fp.next;
if (fp == null) {
fp = new DocumentsWriterFieldData(this, fi);
fp.next = fieldDataHash[hashPos];
fieldDataHash[hashPos] = fp;
if (numAllFieldData == allFieldDataArray.length) {
int newSize = (int) (allFieldDataArray.length*1.5);
int newHashSize = fieldDataHash.length*2;
DocumentsWriterFieldData newArray[] = new DocumentsWriterFieldData[newSize];
DocumentsWriterFieldData newHashArray[] = new DocumentsWriterFieldData[newHashSize];
System.arraycopy(allFieldDataArray, 0, newArray, 0, numAllFieldData);
// Rehash
fieldDataHashMask = newSize-1;
for(int j=0;j<fieldDataHash.length;j++) {
DocumentsWriterFieldData fp0 = fieldDataHash[j];
while(fp0 != null) {
hashPos = fp0.fieldInfo.name.hashCode() & fieldDataHashMask;
DocumentsWriterFieldData nextFP0 = fp0.next;
fp0.next = newHashArray[hashPos];
newHashArray[hashPos] = fp0;
fp0 = nextFP0;
}
}
allFieldDataArray = newArray;
fieldDataHash = newHashArray;
}
allFieldDataArray[numAllFieldData++] = fp;
} else {
assert fp.fieldInfo == fi;
}
if (thisFieldGen != fp.lastGen) {
// First time we're seeing this field for this doc
fp.lastGen = thisFieldGen;
fp.fieldCount = 0;
fp.doVectors = fp.doVectorPositions = fp.doVectorOffsets = false;
fp.doNorms = fi.isIndexed && !fi.omitNorms;
if (numFieldData == fieldDataArray.length) {
int newSize = fieldDataArray.length*2;
DocumentsWriterFieldData newArray[] = new DocumentsWriterFieldData[newSize];
System.arraycopy(fieldDataArray, 0, newArray, 0, numFieldData);
fieldDataArray = newArray;
}
fieldDataArray[numFieldData++] = fp;
}
if (field.isTermVectorStored()) {
if (!fp.doVectors && numVectorFields++ == vectorFieldPointers.length) {
final int newSize = (int) (numVectorFields*1.5);
vectorFieldPointers = new long[newSize];
vectorFieldNumbers = new int[newSize];
}
fp.doVectors = true;
docHasVectors = true;
fp.doVectorPositions |= field.isStorePositionWithTermVector();
fp.doVectorOffsets |= field.isStoreOffsetWithTermVector();
}
if (fp.fieldCount == fp.docFields.length) {
Fieldable[] newArray = new Fieldable[fp.docFields.length*2];
System.arraycopy(fp.docFields, 0, newArray, 0, fp.docFields.length);
fp.docFields = newArray;
}
// Lazily allocate arrays for postings:
if (field.isIndexed() && fp.postingsHash == null)
fp.initPostingArrays();
fp.docFields[fp.fieldCount++] = field;
}
// Maybe init the local & global fieldsWriter
if (localFieldsWriter == null) {
if (docWriter.fieldsWriter == null) {
assert docWriter.docStoreSegment == null;
assert docWriter.segment != null;
docWriter.files = null;
docWriter.docStoreSegment = docWriter.segment;
// If we hit an exception while init'ing the
// fieldsWriter, we must abort this segment
// because those files will be in an unknown
// state:
try {
docWriter.fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.docStoreSegment, docWriter.fieldInfos);
} catch (Throwable t) {
throw new AbortException(t, docWriter);
}
}
localFieldsWriter = new FieldsWriter(null, fdtLocal, docWriter.fieldInfos);
}
// First time we see a doc that has field(s) with
// stored vectors, we init our tvx writer
if (docHasVectors) {
if (docWriter.tvx == null) {
assert docWriter.docStoreSegment != null;
docWriter.files = null;
// If we hit an exception while init'ing the term
// vector output files, we must abort this segment
// because those files will be in an unknown
// state:
try {
docWriter.tvx = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
docWriter.tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
docWriter.tvd = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
docWriter.tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
docWriter.tvf = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
docWriter.tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
// We must "catch up" for all docs before us
// that had no vectors:
for(int i=0;i<docWriter.numDocsInStore;i++) {
docWriter.tvx.writeLong(docWriter.tvd.getFilePointer());
docWriter.tvd.writeVInt(0);
docWriter.tvx.writeLong(0);
}
} catch (Throwable t) {
throw new AbortException(t, docWriter);
}
}
numVectorFields = 0;
}
}
/** Do in-place sort of Posting array */
void doPostingSort(Posting[] postings, int numPosting) {
quickSort(postings, 0, numPosting-1);
}
void quickSort(Posting[] postings, int lo, int hi) {
if (lo >= hi)
return;
else if (hi == 1+lo) {
if (comparePostings(postings[lo], postings[hi]) > 0) {
final Posting tmp = postings[lo];
postings[lo] = postings[hi];
postings[hi] = tmp;
}
return;
}
int mid = (lo + hi) >>> 1;
if (comparePostings(postings[lo], postings[mid]) > 0) {
Posting tmp = postings[lo];
postings[lo] = postings[mid];
postings[mid] = tmp;
}
if (comparePostings(postings[mid], postings[hi]) > 0) {
Posting tmp = postings[mid];
postings[mid] = postings[hi];
postings[hi] = tmp;
if (comparePostings(postings[lo], postings[mid]) > 0) {
Posting tmp2 = postings[lo];
postings[lo] = postings[mid];
postings[mid] = tmp2;
}
}
int left = lo + 1;
int right = hi - 1;
if (left >= right)
return;
Posting partition = postings[mid];
for (; ;) {
while (comparePostings(postings[right], partition) > 0)
--right;
while (left < right && comparePostings(postings[left], partition) <= 0)
++left;
if (left < right) {
Posting tmp = postings[left];
postings[left] = postings[right];
postings[right] = tmp;
--right;
} else {
break;
}
}
quickSort(postings, lo, left);
quickSort(postings, left + 1, hi);
}
/** Do in-place sort of PostingVector array */
void doVectorSort(PostingVector[] postings, int numPosting) {
quickSort(postings, 0, numPosting-1);
}
void quickSort(PostingVector[] postings, int lo, int hi) {
if (lo >= hi)
return;
else if (hi == 1+lo) {
if (comparePostings(postings[lo].p, postings[hi].p) > 0) {
final PostingVector tmp = postings[lo];
postings[lo] = postings[hi];
postings[hi] = tmp;
}
return;
}
int mid = (lo + hi) >>> 1;
if (comparePostings(postings[lo].p, postings[mid].p) > 0) {
PostingVector tmp = postings[lo];
postings[lo] = postings[mid];
postings[mid] = tmp;
}
if (comparePostings(postings[mid].p, postings[hi].p) > 0) {
PostingVector tmp = postings[mid];
postings[mid] = postings[hi];
postings[hi] = tmp;
if (comparePostings(postings[lo].p, postings[mid].p) > 0) {
PostingVector tmp2 = postings[lo];
postings[lo] = postings[mid];
postings[mid] = tmp2;
}
}
int left = lo + 1;
int right = hi - 1;
if (left >= right)
return;
PostingVector partition = postings[mid];
for (; ;) {
while (comparePostings(postings[right].p, partition.p) > 0)
--right;
while (left < right && comparePostings(postings[left].p, partition.p) <= 0)
++left;
if (left < right) {
PostingVector tmp = postings[left];
postings[left] = postings[right];
postings[right] = tmp;
--right;
} else {
break;
}
}
quickSort(postings, lo, left);
quickSort(postings, left + 1, hi);
}
void quickSort(DocumentsWriterFieldData[] array, int lo, int hi) {
if (lo >= hi)
return;
else if (hi == 1+lo) {
if (array[lo].compareTo(array[hi]) > 0) {
final DocumentsWriterFieldData tmp = array[lo];
array[lo] = array[hi];
array[hi] = tmp;
}
return;
}
int mid = (lo + hi) >>> 1;
if (array[lo].compareTo(array[mid]) > 0) {
DocumentsWriterFieldData tmp = array[lo];
array[lo] = array[mid];
array[mid] = tmp;
}
if (array[mid].compareTo(array[hi]) > 0) {
DocumentsWriterFieldData tmp = array[mid];
array[mid] = array[hi];
array[hi] = tmp;
if (array[lo].compareTo(array[mid]) > 0) {
DocumentsWriterFieldData tmp2 = array[lo];
array[lo] = array[mid];
array[mid] = tmp2;
}
}
int left = lo + 1;
int right = hi - 1;
if (left >= right)
return;
DocumentsWriterFieldData partition = array[mid];
for (; ;) {
while (array[right].compareTo(partition) > 0)
--right;
while (left < right && array[left].compareTo(partition) <= 0)
++left;
if (left < right) {
DocumentsWriterFieldData tmp = array[left];
array[left] = array[right];
array[right] = tmp;
--right;
} else {
break;
}
}
quickSort(array, lo, left);
quickSort(array, left + 1, hi);
}
/** If there are fields we've seen but did not see again
* in the last run, then free them up. Also reduce
* postings hash size. */
void trimFields() {
int upto = 0;
for(int i=0;i<numAllFieldData;i++) {
DocumentsWriterFieldData fp = allFieldDataArray[i];
if (fp.lastGen == -1) {
// This field was not seen since the previous
// flush, so, free up its resources now
// Unhash
final int hashPos = fp.fieldInfo.name.hashCode() & fieldDataHashMask;
DocumentsWriterFieldData last = null;
DocumentsWriterFieldData fp0 = fieldDataHash[hashPos];
while(fp0 != fp) {
last = fp0;
fp0 = fp0.next;
}
if (last == null)
fieldDataHash[hashPos] = fp.next;
else
last.next = fp.next;
if (docWriter.infoStream != null)
docWriter.infoStream.println(" remove field=" + fp.fieldInfo.name);
} else {
// Reset
fp.lastGen = -1;
allFieldDataArray[upto++] = fp;
if (fp.numPostings > 0 && ((float) fp.numPostings) / fp.postingsHashSize < 0.2) {
int hashSize = fp.postingsHashSize;
// Reduce hash so it's between 25-50% full
while (fp.numPostings < (hashSize>>1) && hashSize >= 2)
hashSize >>= 1;
hashSize <<= 1;
if (hashSize != fp.postingsHash.length)
fp.rehashPostings(hashSize);
}
}
}
// If we didn't see any norms for this field since
// last flush, free it
for(int i=0;i<docWriter.norms.length;i++) {
BufferedNorms n = docWriter.norms[i];
if (n != null && n.upto == 0)
docWriter.norms[i] = null;
}
numAllFieldData = upto;
// Also pare back PostingsVectors if it's excessively
// large
if (maxPostingsVectors * 1.5 < postingsVectors.length) {
final int newSize;
if (0 == maxPostingsVectors)
newSize = 1;
else
newSize = (int) (1.5*maxPostingsVectors);
PostingVector[] newArray = new PostingVector[newSize];
System.arraycopy(postingsVectors, 0, newArray, 0, newSize);
postingsVectors = newArray;
}
}
/** Tokenizes the fields of a document into Postings */
void processDocument(Analyzer analyzer)
throws IOException, AbortException {
final int numFields = numFieldData;
assert clearLastVectorFieldName();
assert 0 == fdtLocal.length();
if (docWriter.tvx != null)
// 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.
quickSort(fieldDataArray, 0, numFields-1);
// We process the document one field at a time
for(int i=0;i<numFields;i++)
fieldDataArray[i].processField(analyzer);
if (docWriter.infoStream != null && maxTermPrefix != null)
docWriter.infoStream.println("WARNING: document contains at least one immense term (longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH + "), all of which were skipped. Please correct the analyzer to not produce such terms. The prefix of the first immense term is: '" + maxTermPrefix + "...'");
}
// USE ONLY FOR DEBUGGING!
/*
public String getPostingText() {
char[] text = charPool.buffers[p.textStart >> CHAR_BLOCK_SHIFT];
int upto = p.textStart & CHAR_BLOCK_MASK;
while(text[upto] != 0xffff)
upto++;
return new String(text, p.textStart, upto-(p.textStart & BYTE_BLOCK_MASK));
}
*/
/** Compares term text for two Posting instance and
* returns -1 if p1 < p2; 1 if p1 > p2; else 0.
*/
int comparePostings(Posting p1, Posting p2) {
if (p1 == p2)
return 0;
final char[] text1 = charPool.buffers[p1.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
int pos1 = p1.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
final char[] text2 = charPool.buffers[p2.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
int pos2 = p2.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
assert text1 != text2 || pos1 != pos2;
while(true) {
final char c1 = text1[pos1++];
final char c2 = text2[pos2++];
if (c1 != c2) {
if (0xffff == c2)
return 1;
else if (0xffff == c1)
return -1;
else
return c1-c2;
} else
// This method should never compare equal postings
// unless p1==p2
assert c1 != 0xffff;
}
}
String lastVectorFieldName;
// Called only by assert
final boolean clearLastVectorFieldName() {
lastVectorFieldName = null;
return true;
}
// Called only by assert
final boolean vectorFieldsInOrder(FieldInfo fi) {
try {
if (lastVectorFieldName != null)
return lastVectorFieldName.compareTo(fi.name) < 0;
else
return true;
} finally {
lastVectorFieldName = fi.name;
}
}
PostingVector[] postingsVectors = new PostingVector[1];
int maxPostingsVectors;
// Used to read a string value for a field
ReusableStringReader stringReader = new ReusableStringReader();
final UnicodeUtil.UTF8Result utf8Results[] = {new UnicodeUtil.UTF8Result(),
new UnicodeUtil.UTF8Result()};
}

View File

@ -48,4 +48,47 @@ final class FieldInfo {
return new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
storeOffsetWithTermVector, omitNorms, storePayloads);
}
void update(boolean isIndexed, boolean storeTermVector, boolean storePositionWithTermVector,
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads) {
if (this.isIndexed != isIndexed) {
this.isIndexed = true; // once indexed, always index
}
if (this.storeTermVector != storeTermVector) {
this.storeTermVector = true; // once vector, always vector
}
if (this.storePositionWithTermVector != storePositionWithTermVector) {
this.storePositionWithTermVector = true; // once vector, always vector
}
if (this.storeOffsetWithTermVector != storeOffsetWithTermVector) {
this.storeOffsetWithTermVector = true; // once vector, always vector
}
if (this.omitNorms != omitNorms) {
this.omitNorms = false; // once norms are stored, always store
}
if (this.storePayloads != storePayloads) {
this.storePayloads = true;
}
}
void update(FieldInfo other) {
if (isIndexed != other.isIndexed) {
isIndexed = true; // once indexed, always index
}
if (storeTermVector != other.storeTermVector) {
storeTermVector = true; // once vector, always vector
}
if (storePositionWithTermVector != other.storePositionWithTermVector) {
storePositionWithTermVector = true; // once vector, always vector
}
if (storeOffsetWithTermVector != other.storeOffsetWithTermVector) {
storeOffsetWithTermVector = true; // once vector, always vector
}
if (omitNorms != other.omitNorms) {
omitNorms = false; // once norms are stored, always store
}
if (storePayloads != other.storePayloads) {
storePayloads = true;
}
}
}

View File

@ -65,7 +65,7 @@ final class FieldInfos {
/**
* Returns a deep clone of this FieldInfos instance.
*/
public Object clone() {
synchronized public Object clone() {
FieldInfos fis = new FieldInfos();
final int numField = byNumber.size();
for(int i=0;i<numField;i++) {
@ -77,7 +77,7 @@ final class FieldInfos {
}
/** Adds field info for a Document. */
public void add(Document doc) {
synchronized public void add(Document doc) {
List fields = doc.getFields();
Iterator fieldIterator = fields.iterator();
while (fieldIterator.hasNext()) {
@ -95,7 +95,7 @@ final class FieldInfos {
* @param storePositionWithTermVector treu if positions should be stored.
* @param storeOffsetWithTermVector true if offsets should be stored
*/
public void addIndexed(Collection names, boolean storeTermVectors, boolean storePositionWithTermVector,
synchronized public void addIndexed(Collection names, boolean storeTermVectors, boolean storePositionWithTermVector,
boolean storeOffsetWithTermVector) {
Iterator i = names.iterator();
while (i.hasNext()) {
@ -111,7 +111,7 @@ final class FieldInfos {
*
* @see #add(String, boolean)
*/
public void add(Collection names, boolean isIndexed) {
synchronized public void add(Collection names, boolean isIndexed) {
Iterator i = names.iterator();
while (i.hasNext()) {
add((String)i.next(), isIndexed);
@ -125,7 +125,7 @@ final class FieldInfos {
* @param isIndexed true if the field is indexed
* @see #add(String, boolean, boolean, boolean, boolean)
*/
public void add(String name, boolean isIndexed) {
synchronized public void add(String name, boolean isIndexed) {
add(name, isIndexed, false, false, false, false);
}
@ -136,7 +136,7 @@ final class FieldInfos {
* @param isIndexed true if the field is indexed
* @param storeTermVector true if the term vector should be stored
*/
public void add(String name, boolean isIndexed, boolean storeTermVector){
synchronized public void add(String name, boolean isIndexed, boolean storeTermVector){
add(name, isIndexed, storeTermVector, false, false, false);
}
@ -151,7 +151,7 @@ final class FieldInfos {
* @param storePositionWithTermVector true if the term vector with positions should be stored
* @param storeOffsetWithTermVector true if the term vector with offsets should be stored
*/
public void add(String name, boolean isIndexed, boolean storeTermVector,
synchronized public void add(String name, boolean isIndexed, boolean storeTermVector,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector) {
add(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, false);
@ -169,7 +169,7 @@ final class FieldInfos {
* @param storeOffsetWithTermVector true if the term vector with offsets should be stored
* @param omitNorms true if the norms for the indexed field should be omitted
*/
public void add(String name, boolean isIndexed, boolean storeTermVector,
synchronized public void add(String name, boolean isIndexed, boolean storeTermVector,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector, boolean omitNorms) {
add(name, isIndexed, storeTermVector, storePositionWithTermVector,
storeOffsetWithTermVector, omitNorms, false);
@ -188,32 +188,26 @@ final class FieldInfos {
* @param omitNorms true if the norms for the indexed field should be omitted
* @param storePayloads true if payloads should be stored for this field
*/
public FieldInfo add(String name, boolean isIndexed, boolean storeTermVector,
synchronized public FieldInfo add(String name, boolean isIndexed, boolean storeTermVector,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
boolean omitNorms, boolean storePayloads) {
FieldInfo fi = fieldInfo(name);
if (fi == null) {
return addInternal(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
} else {
if (fi.isIndexed != isIndexed) {
fi.isIndexed = true; // once indexed, always index
}
if (fi.storeTermVector != storeTermVector) {
fi.storeTermVector = true; // once vector, always vector
}
if (fi.storePositionWithTermVector != storePositionWithTermVector) {
fi.storePositionWithTermVector = true; // once vector, always vector
}
if (fi.storeOffsetWithTermVector != storeOffsetWithTermVector) {
fi.storeOffsetWithTermVector = true; // once vector, always vector
}
if (fi.omitNorms != omitNorms) {
fi.omitNorms = false; // once norms are stored, always store
}
if (fi.storePayloads != storePayloads) {
fi.storePayloads = true;
}
fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
}
return fi;
}
synchronized public FieldInfo add(FieldInfo fieldInfo) {
FieldInfo fi = fieldInfo(fieldInfo.name);
if (fi == null) {
return addInternal(fieldInfo.name, fieldInfo.isIndexed, fieldInfo.storeTermVector,
fieldInfo.storePositionWithTermVector, fieldInfo.storeOffsetWithTermVector,
fieldInfo.omitNorms, fieldInfo.storePayloads);
} else {
fi.update(fieldInfo);
}
return fi;
}

View File

@ -109,7 +109,7 @@ final class FieldsReader {
// Verify the file is long enough to hold all of our
// docs
assert ((int) (indexSize / 8)) >= size + this.docStoreOffset;
assert ((int) (indexSize / 8)) >= size + this.docStoreOffset: "indexSize=" + indexSize + " size=" + size + " docStoreOffset=" + docStoreOffset;
} else {
this.docStoreOffset = 0;
this.size = (int) (indexSize >> 3);

View File

@ -55,7 +55,7 @@ final class FieldsWriter
FieldsWriter(Directory d, String segment, FieldInfos fn) throws IOException {
fieldInfos = fn;
boolean success = false;
final String fieldsName = segment + "." + IndexFileNames.FIELDS_EXTENSION;
try {
@ -112,6 +112,10 @@ final class FieldsWriter
doClose = false;
}
void setFieldsStream(IndexOutput stream) {
this.fieldsStream = stream;
}
// 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
@ -122,6 +126,11 @@ final class FieldsWriter
buffer.writeTo(fieldsStream);
}
void skipDocument() throws IOException {
indexStream.writeLong(fieldsStream.getFilePointer());
fieldsStream.writeVInt(0);
}
void flush() throws IOException {
indexStream.flush();
fieldsStream.flush();
@ -129,10 +138,37 @@ final class FieldsWriter
final void close() throws IOException {
if (doClose) {
if (fieldsStream != null)
fieldsStream.close();
if (indexStream != null)
indexStream.close();
try {
if (fieldsStream != null) {
try {
fieldsStream.close();
} finally {
fieldsStream = null;
}
}
} catch (IOException ioe) {
try {
if (indexStream != null) {
try {
indexStream.close();
} finally {
indexStream = null;
}
}
} catch (IOException ioe2) {
// Ignore so we throw only first IOException hit
}
throw ioe;
} finally {
if (indexStream != null) {
try {
indexStream.close();
} finally {
indexStream = null;
}
}
}
}
}

View File

@ -19,43 +19,51 @@ package org.apache.lucene.index;
import java.io.IOException;
// TODO FI: some of this is "generic" to TermsHash* so we
// should factor it out so other consumers don't have to
// duplicate this code
/** Used by DocumentsWriter to merge the postings from
* multiple ThreadStates when creating a segment */
final class DocumentsWriterFieldMergeState {
final class FreqProxFieldMergeState {
DocumentsWriterFieldData field;
final FreqProxTermsWriterPerField field;
final int numPostings;
final CharBlockPool charPool;
final RawPostingList[] postings;
Posting[] postings;
private Posting p;
private FreqProxTermsWriter.PostingList p;
char[] text;
int textOffset;
private int postingUpto = -1;
ByteSliceReader freq = new ByteSliceReader();
ByteSliceReader prox = new ByteSliceReader();
final ByteSliceReader freq = new ByteSliceReader();
final ByteSliceReader prox = new ByteSliceReader();
int docID;
int termFreq;
public FreqProxFieldMergeState(FreqProxTermsWriterPerField field) {
this.field = field;
this.charPool = field.perThread.termsHashPerThread.charPool;
this.numPostings = field.termsHashPerField.numPostings;
this.postings = field.termsHashPerField.sortPostings();
}
boolean nextTerm() throws IOException {
postingUpto++;
if (postingUpto == field.numPostings)
if (postingUpto == numPostings)
return false;
p = postings[postingUpto];
p = (FreqProxTermsWriter.PostingList) postings[postingUpto];
docID = 0;
text = field.threadState.charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
text = charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
textOffset = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
if (p.freqUpto > p.freqStart)
freq.init(field.threadState.postingsPool, p.freqStart, p.freqUpto);
else
freq.bufferOffset = freq.upto = freq.endIndex = 0;
prox.init(field.threadState.postingsPool, p.proxStart, p.proxUpto);
field.termsHashPerField.initReader(freq, p, 0);
field.termsHashPerField.initReader(prox, p, 1);
// Should always be true
boolean result = nextDoc();
@ -65,7 +73,7 @@ final class DocumentsWriterFieldMergeState {
}
public boolean nextDoc() throws IOException {
if (freq.bufferOffset + freq.upto == freq.endIndex) {
if (freq.eof()) {
if (p.lastDocCode != -1) {
// Return last doc
docID = p.lastDocID;

View File

@ -0,0 +1,366 @@
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.store.IndexOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.UnicodeUtil;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.ArrayList;
import java.util.List;
import java.util.Iterator;
final class FreqProxTermsWriter extends TermsHashConsumer {
FreqProxTermsWriter() {
streamCount = 2;
}
public TermsHashConsumerPerThread addThread(TermsHashPerThread perThread) {
return new FreqProxTermsWriterPerThread(perThread);
}
void createPostings(RawPostingList[] postings, int start, int count) {
final int end = start + count;
for(int i=start;i<end;i++)
postings[i] = new PostingList();
}
private static int compareText(final char[] text1, int pos1, final char[] text2, int pos2) {
while(true) {
final char c1 = text1[pos1++];
final char c2 = text2[pos2++];
if (c1 != c2) {
if (0xffff == c2)
return 1;
else if (0xffff == c1)
return -1;
else
return c1-c2;
} else if (0xffff == c1)
return 0;
}
}
void closeDocStore(DocumentsWriter.FlushState state) {}
void abort() {}
// TODO: would be nice to factor out morme 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...
public void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException {
// Gather all FieldData's that have postings, across all
// ThreadStates
List allFields = new ArrayList();
Iterator it = threadsAndFields.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
Collection fields = (Collection) entry.getValue();
Iterator fieldsIt = fields.iterator();
while(fieldsIt.hasNext()) {
FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) fieldsIt.next();
if (perField.termsHashPerField.numPostings > 0)
allFields.add(perField);
}
}
// Sort by field name
Collections.sort(allFields);
final int numAllFields = allFields.size();
final TermInfosWriter termsOut = new TermInfosWriter(state.directory,
state.segmentName,
fieldInfos,
state.docWriter.writer.getTermIndexInterval());
final IndexOutput freqOut = state.directory.createOutput(state.segmentFileName(IndexFileNames.FREQ_EXTENSION));
final IndexOutput proxOut = state.directory.createOutput(state.segmentFileName(IndexFileNames.PROX_EXTENSION));
final DefaultSkipListWriter skipListWriter = new DefaultSkipListWriter(termsOut.skipInterval,
termsOut.maxSkipLevels,
state.numDocsInRAM, freqOut, proxOut);
int start = 0;
while(start < numAllFields) {
final FieldInfo fieldInfo = ((FreqProxTermsWriterPerField) allFields.get(start)).fieldInfo;
final String fieldName = fieldInfo.name;
int end = start+1;
while(end < numAllFields && ((FreqProxTermsWriterPerField) allFields.get(end)).fieldInfo.name.equals(fieldName))
end++;
FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start];
for(int i=start;i<end;i++) {
fields[i-start] = (FreqProxTermsWriterPerField) allFields.get(i);
// Aggregate the storePayload as seen by the same
// field across multiple threads
fieldInfo.storePayloads |= fields[i-start].hasPayloads;
}
// If this field has postings then add them to the
// segment
appendPostings(state, fields, termsOut, freqOut, proxOut, skipListWriter);
for(int i=0;i<fields.length;i++) {
TermsHashPerField perField = fields[i].termsHashPerField;
int numPostings = perField.numPostings;
perField.reset();
perField.shrinkHash(numPostings);
}
start = end;
}
it = threadsAndFields.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
perThread.termsHashPerThread.reset(true);
}
freqOut.close();
proxOut.close();
termsOut.close();
// Record all files we have flushed
state.flushedFiles.add(state.segmentFileName(IndexFileNames.FIELD_INFOS_EXTENSION));
state.flushedFiles.add(state.segmentFileName(IndexFileNames.FREQ_EXTENSION));
state.flushedFiles.add(state.segmentFileName(IndexFileNames.PROX_EXTENSION));
state.flushedFiles.add(state.segmentFileName(IndexFileNames.TERMS_EXTENSION));
state.flushedFiles.add(state.segmentFileName(IndexFileNames.TERMS_INDEX_EXTENSION));
}
final byte[] copyByteBuffer = new byte[4096];
/** Copy numBytes from srcIn to destIn */
void copyBytes(IndexInput srcIn, IndexOutput destIn, long numBytes) throws IOException {
// TODO: we could do this more efficiently (save a copy)
// because it's always from a ByteSliceReader ->
// IndexOutput
while(numBytes > 0) {
final int chunk;
if (numBytes > 4096)
chunk = 4096;
else
chunk = (int) numBytes;
srcIn.readBytes(copyByteBuffer, 0, chunk);
destIn.writeBytes(copyByteBuffer, 0, chunk);
numBytes -= chunk;
}
}
/* Walk through all unique text tokens (Posting
* instances) found in this field and serialize them
* into a single RAM segment. */
void appendPostings(final DocumentsWriter.FlushState flushState,
FreqProxTermsWriterPerField[] fields,
TermInfosWriter termsOut,
IndexOutput freqOut,
IndexOutput proxOut,
DefaultSkipListWriter skipListWriter)
throws CorruptIndexException, IOException {
final int fieldNumber = fields[0].fieldInfo.number;
int numFields = fields.length;
final FreqProxFieldMergeState[] mergeStates = new FreqProxFieldMergeState[numFields];
for(int i=0;i<numFields;i++) {
FreqProxFieldMergeState fms = mergeStates[i] = new FreqProxFieldMergeState(fields[i]);
assert fms.field.fieldInfo == fields[0].fieldInfo;
// Should always be true
boolean result = fms.nextTerm();
assert result;
}
final int skipInterval = termsOut.skipInterval;
final boolean currentFieldStorePayloads = fields[0].fieldInfo.storePayloads;
FreqProxFieldMergeState[] termStates = new FreqProxFieldMergeState[numFields];
while(numFields > 0) {
// Get the next term to merge
termStates[0] = mergeStates[0];
int numToMerge = 1;
for(int i=1;i<numFields;i++) {
final char[] text = mergeStates[i].text;
final int textOffset = mergeStates[i].textOffset;
final int cmp = compareText(text, textOffset, termStates[0].text, termStates[0].textOffset);
if (cmp < 0) {
termStates[0] = mergeStates[i];
numToMerge = 1;
} else if (cmp == 0)
termStates[numToMerge++] = mergeStates[i];
}
int df = 0;
int lastPayloadLength = -1;
int lastDoc = 0;
final char[] text = termStates[0].text;
final int start = termStates[0].textOffset;
long freqPointer = freqOut.getFilePointer();
long proxPointer = proxOut.getFilePointer();
skipListWriter.resetSkip();
// Now termStates has numToMerge FieldMergeStates
// which all share the same term. Now we must
// interleave the docID streams.
while(numToMerge > 0) {
if ((++df % skipInterval) == 0) {
skipListWriter.setSkipData(lastDoc, currentFieldStorePayloads, lastPayloadLength);
skipListWriter.bufferSkip(df);
}
FreqProxFieldMergeState minState = termStates[0];
for(int i=1;i<numToMerge;i++)
if (termStates[i].docID < minState.docID)
minState = termStates[i];
final int doc = minState.docID;
final int termDocFreq = minState.termFreq;
assert doc < flushState.numDocsInRAM;
assert doc > lastDoc || df == 1;
final int newDocCode = (doc-lastDoc)<<1;
lastDoc = doc;
final ByteSliceReader prox = minState.prox;
// Carefully copy over the prox + payload info,
// changing the format to match Lucene's segment
// format.
for(int j=0;j<termDocFreq;j++) {
final int code = prox.readVInt();
if (currentFieldStorePayloads) {
final int payloadLength;
if ((code & 1) != 0) {
// This position has a payload
payloadLength = prox.readVInt();
} else
payloadLength = 0;
if (payloadLength != lastPayloadLength) {
proxOut.writeVInt(code|1);
proxOut.writeVInt(payloadLength);
lastPayloadLength = payloadLength;
} else
proxOut.writeVInt(code & (~1));
if (payloadLength > 0)
copyBytes(prox, proxOut, payloadLength);
} else {
assert 0 == (code & 1);
proxOut.writeVInt(code>>1);
}
}
if (1 == termDocFreq) {
freqOut.writeVInt(newDocCode|1);
} else {
freqOut.writeVInt(newDocCode);
freqOut.writeVInt(termDocFreq);
}
if (!minState.nextDoc()) {
// Remove from termStates
int upto = 0;
for(int i=0;i<numToMerge;i++)
if (termStates[i] != minState)
termStates[upto++] = termStates[i];
numToMerge--;
assert upto == numToMerge;
// Advance this state to the next term
if (!minState.nextTerm()) {
// OK, no more terms, so remove from mergeStates
// as well
upto = 0;
for(int i=0;i<numFields;i++)
if (mergeStates[i] != minState)
mergeStates[upto++] = mergeStates[i];
numFields--;
assert upto == numFields;
}
}
}
assert df > 0;
// Done merging this term
long skipPointer = skipListWriter.writeSkip(freqOut);
// Write term
termInfo.set(df, freqPointer, proxPointer, (int) (skipPointer - freqPointer));
// TODO: we could do this incrementally
UnicodeUtil.UTF16toUTF8(text, start, termsUTF8);
// TODO: we could save O(n) re-scan of the term by
// computing the shared prefix with the last term
// while during the UTF8 encoding
termsOut.add(fieldNumber,
termsUTF8.result,
termsUTF8.length,
termInfo);
}
}
private final TermInfo termInfo = new TermInfo(); // minimize consing
final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
void files(Collection files) {}
static final class PostingList extends RawPostingList {
int docFreq; // # times this term occurs in the current doc
int lastDocID; // Last docID where this term occurred
int lastDocCode; // Code for prior doc
int lastPosition; // Last position where this term occurred
}
int bytesPerPosting() {
return RawPostingList.BYTES_SIZE + 4 * DocumentsWriter.INT_NUM_BYTE;
}
}

View File

@ -0,0 +1,117 @@
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.document.Fieldable;
import org.apache.lucene.analysis.Token;
// 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 {
final FreqProxTermsWriterPerThread perThread;
final TermsHashPerField termsHashPerField;
final FieldInfo fieldInfo;
final DocumentsWriter.DocState docState;
final DocInverter.FieldInvertState fieldState;
public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriterPerThread perThread, FieldInfo fieldInfo) {
this.termsHashPerField = termsHashPerField;
this.perThread = perThread;
this.fieldInfo = fieldInfo;
docState = termsHashPerField.docState;
fieldState = termsHashPerField.fieldState;
}
void finish() {}
//boolean doNext;
boolean hasPayloads;
void skippingLongTerm(Token t) throws IOException {}
public int compareTo(Object other0) {
FreqProxTermsWriterPerField other = (FreqProxTermsWriterPerField) other0;
return fieldInfo.name.compareTo(other.fieldInfo.name);
}
boolean start(Fieldable[] fields, int count) {
for(int i=0;i<count;i++)
if (fields[i].isIndexed())
return true;
return false;
}
final void writeProx(Token t, FreqProxTermsWriter.PostingList p, int proxCode) {
final Payload payload = t.getPayload();
if (payload != null && payload.length > 0) {
termsHashPerField.writeVInt(1, (proxCode<<1)|1);
termsHashPerField.writeVInt(1, payload.length);
termsHashPerField.writeBytes(1, payload.data, payload.offset, payload.length);
hasPayloads = true;
} else
termsHashPerField.writeVInt(1, proxCode<<1);
p.lastPosition = fieldState.position;
}
final void newTerm(Token t, RawPostingList p0) {
// First time we're seeing this term since the last
// flush
assert docState.testPoint("FreqProxTermsWriterPerField.newTerm start");
FreqProxTermsWriter.PostingList p = (FreqProxTermsWriter.PostingList) p0;
p.lastDocCode = docState.docID << 1;
p.lastDocID = docState.docID;
p.docFreq = 1;
writeProx(t, p, fieldState.position);
}
final void addTerm(Token t, RawPostingList p0) {
assert docState.testPoint("FreqProxTermsWriterPerField.addTerm start");
FreqProxTermsWriter.PostingList p = (FreqProxTermsWriter.PostingList) p0;
assert p.docFreq > 0;
if (docState.docID != p.lastDocID) {
// Term not yet seen in the current doc but previously
// seen in other doc(s) since the last flush
// Now that we know doc freq for previous doc,
// write it & lastDocCode
if (1 == p.docFreq)
termsHashPerField.writeVInt(0, p.lastDocCode|1);
else {
termsHashPerField.writeVInt(0, p.lastDocCode);
termsHashPerField.writeVInt(0, p.docFreq);
}
p.docFreq = 1;
p.lastDocCode = (docState.docID - p.lastDocID) << 1;
p.lastDocID = docState.docID;
writeProx(t, p, fieldState.position);
} else {
p.docFreq++;
writeProx(t, p, fieldState.position-p.lastPosition);
}
}
public void abort() {}
}

View File

@ -0,0 +1,41 @@
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.
*/
final class FreqProxTermsWriterPerThread extends TermsHashConsumerPerThread {
final TermsHashPerThread termsHashPerThread;
final DocumentsWriter.DocState docState;
public FreqProxTermsWriterPerThread(TermsHashPerThread perThread) {
docState = perThread.docState;
termsHashPerThread = perThread;
}
public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
return new FreqProxTermsWriterPerField(termsHashPerField, this, fieldInfo);
}
void startDocument() {
}
DocumentsWriter.DocWriter finishDocument() {
return null;
}
public void abort() {}
}

View File

@ -403,9 +403,9 @@ final class IndexFileDeleter {
final List docWriterFiles;
if (docWriter != null) {
docWriterFiles = docWriter.files();
docWriterFiles = docWriter.openFiles();
if (docWriterFiles != null)
// We must incRef thes files before decRef'ing
// We must incRef these files before decRef'ing
// last files to make sure we don't accidentally
// delete them:
incRef(docWriterFiles);
@ -510,11 +510,13 @@ final class IndexFileDeleter {
/** Delets the specified files, but only if they are new
* (have not yet been incref'd). */
void deleteNewFiles(List files) throws IOException {
final int size = files.size();
for(int i=0;i<size;i++)
if (!refCounts.containsKey(files.get(i)))
deleteFile((String) files.get(i));
void deleteNewFiles(Collection files) throws IOException {
final Iterator it = files.iterator();
while(it.hasNext()) {
final String fileName = (String) it.next();
if (!refCounts.containsKey(fileName))
deleteFile(fileName);
}
}
void deleteFile(String fileName)

View File

@ -426,6 +426,7 @@ public class IndexWriter {
public void setSimilarity(Similarity similarity) {
ensureOpen();
this.similarity = similarity;
docWriter.setSimilarity(similarity);
}
/** Expert: Return the Similarity implementation used by this IndexWriter.
@ -1130,6 +1131,7 @@ public class IndexWriter {
docWriter = new DocumentsWriter(directory, this);
docWriter.setInfoStream(infoStream);
docWriter.setMaxFieldLength(maxFieldLength);
// Default deleter (for backwards compatibility) is
// KeepOnlyLastCommitDeleter:
@ -1267,6 +1269,7 @@ public class IndexWriter {
public void setMaxFieldLength(int maxFieldLength) {
ensureOpen();
this.maxFieldLength = maxFieldLength;
docWriter.setMaxFieldLength(maxFieldLength);
if (infoStream != null)
message("setMaxFieldLength " + maxFieldLength);
}
@ -1715,62 +1718,61 @@ public class IndexWriter {
*/
private synchronized boolean flushDocStores() throws IOException {
List files = docWriter.files();
boolean useCompoundDocStore = false;
if (files.size() > 0) {
String docStoreSegment;
String docStoreSegment;
boolean success = false;
try {
docStoreSegment = docWriter.closeDocStore();
success = true;
} finally {
if (!success) {
if (infoStream != null)
message("hit exception closing doc store segment");
}
}
useCompoundDocStore = mergePolicy.useCompoundDocStore(segmentInfos);
if (useCompoundDocStore && docStoreSegment != null && docWriter.closedFiles().size() != 0) {
// Now build compound doc store file
success = false;
final int numSegments = segmentInfos.size();
final String compoundFileName = docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
boolean success = false;
try {
docStoreSegment = docWriter.closeDocStore();
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
final Iterator it = docWriter.closedFiles().iterator();
while(it.hasNext())
cfsWriter.addFile((String) it.next());
// Perform the merge
cfsWriter.close();
success = true;
} finally {
if (!success) {
if (infoStream != null)
message("hit exception closing doc store segment");
docWriter.abort(null);
message("hit exception building compound file doc store for segment " + docStoreSegment);
deleter.deleteFile(compoundFileName);
}
}
useCompoundDocStore = mergePolicy.useCompoundDocStore(segmentInfos);
if (useCompoundDocStore && docStoreSegment != null) {
// Now build compound doc store file
success = false;
final int numSegments = segmentInfos.size();
final String compoundFileName = docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
try {
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
final int size = files.size();
for(int i=0;i<size;i++)
cfsWriter.addFile((String) files.get(i));
// Perform the merge
cfsWriter.close();
success = true;
} finally {
if (!success) {
if (infoStream != null)
message("hit exception building compound file doc store for segment " + docStoreSegment);
deleter.deleteFile(compoundFileName);
}
}
for(int i=0;i<numSegments;i++) {
SegmentInfo si = segmentInfos.info(i);
if (si.getDocStoreOffset() != -1 &&
si.getDocStoreSegment().equals(docStoreSegment))
si.setDocStoreIsCompoundFile(true);
}
checkpoint();
for(int i=0;i<numSegments;i++) {
SegmentInfo si = segmentInfos.info(i);
if (si.getDocStoreOffset() != -1 &&
si.getDocStoreSegment().equals(docStoreSegment))
si.setDocStoreIsCompoundFile(true);
}
checkpoint();
// In case the files we just merged into a CFS were
// not previously checkpointed:
deleter.deleteNewFiles(docWriter.closedFiles());
}
return useCompoundDocStore;
@ -1947,7 +1949,7 @@ public class IndexWriter {
// If docWriter has some aborted files that were
// never incref'd, then we clean them up here
if (docWriter != null) {
final List files = docWriter.abortedFiles();
final Collection files = docWriter.abortedFiles();
if (files != null)
deleter.deleteNewFiles(files);
}
@ -2076,7 +2078,7 @@ public class IndexWriter {
synchronized (this) {
// If docWriter has some aborted files that were
// never incref'd, then we clean them up here
final List files = docWriter.abortedFiles();
final Collection files = docWriter.abortedFiles();
if (files != null)
deleter.deleteNewFiles(files);
}
@ -2650,8 +2652,8 @@ public class IndexWriter {
// once").
segmentInfos.clear();
segmentInfos.addAll(rollbackSegmentInfos);
docWriter.abort(null);
docWriter.abort();
// Ask deleter to locate unreferenced files & remove
// them:
@ -3338,7 +3340,6 @@ public class IndexWriter {
if (!success) {
if (infoStream != null)
message("hit exception flushing segment " + segment);
docWriter.abort(null);
deleter.refresh(segment);
}
}
@ -3830,8 +3831,9 @@ public class IndexWriter {
// If the segment is referencing the current "live"
// doc store outputs then we must merge
if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment))
if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment)) {
doFlushDocStore = true;
}
}
final int docStoreOffset;
@ -3859,8 +3861,9 @@ public class IndexWriter {
// newly flushed doc store files then we should not
// make compound file out of them...
if (infoStream != null)
message("flush at merge");
flush(false, true, false);
message("now flush at merge");
doFlush(true, false);
//flush(false, true, false);
}
// We must take a full copy at this point so that we can

View File

@ -0,0 +1,65 @@
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.
*/
final class IntBlockPool {
public int[][] buffers = new int[10][];
int bufferUpto = -1; // Which buffer we are upto
public int intUpto = DocumentsWriter.INT_BLOCK_SIZE; // Where we are in head buffer
public int[] buffer; // Current head buffer
public int intOffset = -DocumentsWriter.INT_BLOCK_SIZE; // Current head offset
final private DocumentsWriter docWriter;
final boolean trackAllocations;
public IntBlockPool(DocumentsWriter docWriter, boolean trackAllocations) {
this.docWriter = docWriter;
this.trackAllocations = trackAllocations;
}
public void reset() {
if (bufferUpto != -1) {
if (bufferUpto > 0)
// Recycle all but the first buffer
docWriter.recycleIntBlocks(buffers, 1, 1+bufferUpto);
// Reuse first buffer
bufferUpto = 0;
intUpto = 0;
intOffset = 0;
buffer = buffers[0];
}
}
public void nextBuffer() {
if (1+bufferUpto == buffers.length) {
int[][] newBuffers = new int[(int) (buffers.length*1.5)][];
System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
buffers = newBuffers;
}
buffer = buffers[1+bufferUpto] = docWriter.getIntBlock(trackAllocations);
bufferUpto++;
intUpto = 0;
intOffset += DocumentsWriter.INT_BLOCK_SIZE;
}
}

View File

@ -0,0 +1,46 @@
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.util.Map;
import java.io.IOException;
abstract class InvertedDocConsumer {
/** Add a new thread */
abstract InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
/** Abort (called after hitting AbortException) */
abstract void abort();
/** Flush a new segment */
abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
/** Close doc stores */
abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
/** Attempt to free RAM, returning true if any RAM was
* freed */
abstract boolean freeRAM();
FieldInfos fieldInfos;
void setFieldInfos(FieldInfos fieldInfos) {
this.fieldInfos = fieldInfos;
}
}

View File

@ -0,0 +1,41 @@
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.document.Fieldable;
import org.apache.lucene.analysis.Token;
import java.io.IOException;
abstract class InvertedDocConsumerPerField {
// Called once per field, and is given all Fieldable
// occurrences for this field in the document. Return
// true if you wish to see inverted tokens for these
// fields:
abstract boolean start(Fieldable[] fields, int count) throws IOException;
// Called once per inverted token
abstract void add(Token token) throws IOException;
// Called once per field per document, after all Fieldable
// occurrences are inverted
abstract void finish() throws IOException;
// Called on hitting an aborting exception
abstract void abort();
}

View File

@ -0,0 +1,27 @@
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 InvertedDocConsumerPerThread {
abstract void startDocument() throws IOException;
abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
abstract void abort();
}

View File

@ -0,0 +1,29 @@
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.util.Map;
import java.io.IOException;
abstract class InvertedDocEndConsumer {
abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
abstract void abort();
abstract void setFieldInfos(FieldInfos fieldInfos);
}

View File

@ -0,0 +1,23 @@
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.
*/
abstract class InvertedDocEndConsumerPerField {
abstract void finish();
abstract void abort();
}

View File

@ -0,0 +1,25 @@
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.
*/
abstract class InvertedDocEndConsumerPerThread {
abstract void startDocument();
abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
abstract void finishDocument();
abstract void abort();
}

View File

@ -0,0 +1,175 @@
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.Iterator;
import java.util.HashMap;
import java.util.Map;
import java.util.List;
import java.util.ArrayList;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.search.Similarity;
// 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 NormsWriter extends InvertedDocEndConsumer {
private static final byte defaultNorm = Similarity.encodeNorm(1.0f);
private FieldInfos fieldInfos;
public InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
return new NormsWriterPerThread(docInverterPerThread, this);
}
public void abort() {}
// We only write the _X.nrm file at flush
void files(Collection files) {}
void setFieldInfos(FieldInfos fieldInfos) {
this.fieldInfos = fieldInfos;
}
/** Produce _X.nrm if any document had a field with norms
* not disabled */
public void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
final Map byField = new HashMap();
// Typically, each thread will have encountered the same
// field. So first we collate by field, ie, all
// per-thread field instances that correspond to the
// same FieldInfo
final Iterator it = threadsAndFields.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
Collection fields = (Collection) entry.getValue();
Iterator fieldsIt = fields.iterator();
while(fieldsIt.hasNext()) {
NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next();
if (perField.upto > 0) {
// It has some norms
List l = (List) byField.get(perField.fieldInfo);
if (l == null) {
l = new ArrayList();
byField.put(perField.fieldInfo, l);
}
l.add(perField);
} else
// Remove this field since we haven't seen it
// since the previous flush
fieldsIt.remove();
}
}
final String normsFileName = state.segmentName + "." + IndexFileNames.NORMS_EXTENSION;
state.flushedFiles.add(normsFileName);
IndexOutput normsOut = state.directory.createOutput(normsFileName);
try {
normsOut.writeBytes(SegmentMerger.NORMS_HEADER, 0, SegmentMerger.NORMS_HEADER.length);
final int numField = fieldInfos.size();
int normCount = 0;
for(int fieldNumber=0;fieldNumber<numField;fieldNumber++) {
final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
List toMerge = (List) byField.get(fieldInfo);
int upto = 0;
if (toMerge != null) {
final int numFields = toMerge.size();
normCount++;
final NormsWriterPerField[] fields = new NormsWriterPerField[numFields];
int[] uptos = new int[numFields];
for(int j=0;j<numFields;j++)
fields[j] = (NormsWriterPerField) toMerge.get(j);
int numLeft = numFields;
while(numLeft > 0) {
assert uptos[0] < fields[0].docIDs.length : " uptos[0]=" + uptos[0] + " len=" + (fields[0].docIDs.length);
int minLoc = 0;
int minDocID = fields[0].docIDs[uptos[0]];
for(int j=1;j<numLeft;j++) {
final int docID = fields[j].docIDs[uptos[j]];
if (docID < minDocID) {
minDocID = docID;
minLoc = j;
}
}
assert minDocID < state.numDocsInRAM;
// Fill hole
for(;upto<minDocID;upto++)
normsOut.writeByte(defaultNorm);
normsOut.writeByte(fields[minLoc].norms[uptos[minLoc]]);
(uptos[minLoc])++;
upto++;
if (uptos[minLoc] == fields[minLoc].upto) {
fields[minLoc].reset();
if (minLoc != numLeft-1) {
fields[minLoc] = fields[numLeft-1];
uptos[minLoc] = uptos[numLeft-1];
}
numLeft--;
}
}
// Fill final hole with defaultNorm
for(;upto<state.numDocsInRAM;upto++)
normsOut.writeByte(defaultNorm);
} else if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
normCount++;
// Fill entire field with default norm:
for(;upto<state.numDocsInRAM;upto++)
normsOut.writeByte(defaultNorm);
}
assert 4+normCount*state.numDocsInRAM == normsOut.getFilePointer() : ".nrm file size mismatch: expected=" + (4+normCount*state.numDocsInRAM) + " actual=" + normsOut.getFilePointer();
}
} finally {
normsOut.close();
}
}
void closeDocStore(DocumentsWriter.FlushState state) {}
}

View File

@ -0,0 +1,77 @@
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.search.Similarity;
/** Taps into DocInverter, as an InvertedDocEndConsumer,
* which is called at the end of inverting each field. We
* just look at the length for the field (docState.length)
* and record the norm. */
final class NormsWriterPerField extends InvertedDocEndConsumerPerField implements Comparable {
final NormsWriterPerThread perThread;
final FieldInfo fieldInfo;
final DocumentsWriter.DocState docState;
// Holds all docID/norm pairs we've seen
int[] docIDs = new int[1];
byte[] norms = new byte[1];
int upto;
final DocInverter.FieldInvertState fieldState;
public void reset() {
// Shrink back if we are overallocated now:
docIDs = ArrayUtil.shrink(docIDs, upto);
norms = ArrayUtil.shrink(norms, upto);
upto = 0;
}
public NormsWriterPerField(final DocInverterPerField docInverterPerField, final NormsWriterPerThread perThread, final FieldInfo fieldInfo) {
this.perThread = perThread;
this.fieldInfo = fieldInfo;
docState = perThread.docState;
fieldState = docInverterPerField.fieldState;
}
void abort() {
upto = 0;
}
public int compareTo(Object other) {
return fieldInfo.name.compareTo(((NormsWriterPerField) other).fieldInfo.name);
}
void finish() {
assert docIDs.length == norms.length;
if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
if (docIDs.length <= upto) {
assert docIDs.length == upto;
docIDs = ArrayUtil.grow(docIDs, 1+upto);
norms = ArrayUtil.grow(norms, 1+upto);
}
final float norm = fieldState.boost * docState.similarity.lengthNorm(fieldInfo.name, fieldState.length);
norms[upto] = Similarity.encodeNorm(norm);
docIDs[upto] = docState.docID;
upto++;
}
}
}

View File

@ -0,0 +1,41 @@
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.
*/
final class NormsWriterPerThread extends InvertedDocEndConsumerPerThread {
final NormsWriter normsWriter;
final DocumentsWriter.DocState docState;
public NormsWriterPerThread(DocInverterPerThread docInverterPerThread, NormsWriter normsWriter) {
this.normsWriter = normsWriter;
docState = docInverterPerThread.docState;
}
InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
return new NormsWriterPerField(docInverterPerField, this, fieldInfo);
}
void abort() {}
void startDocument() {}
void finishDocument() {}
boolean freeRAM() {
return false;
}
}

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.
*/
/* Used by DocumentsWriter to track postings for a single
* term. One of these exists per unique term seen since the
* last flush. If you alter this class you must also fix
* DocumentWriter.POSTING_NUM_BYTE to reflect the change as
* this is how RAM usage is measured. */
final class Posting {
int textStart; // Address into char[] blocks where our text is stored
int docFreq; // # times this term occurs in the current doc
int freqStart; // Address of first byte[] slice for freq
int freqUpto; // Next write address for freq
int proxStart; // Address of first byte[] slice
int proxUpto; // Next write address for prox
int lastDocID; // Last docID where this term occurred
int lastDocCode; // Code for prior doc
int lastPosition; // Last position where this term occurred
PostingVector vector; // Corresponding PostingVector instance
}

View File

@ -0,0 +1,36 @@
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.
*/
/** This is the base class for an in-memory posting list,
* keyed by a Token. {@link TermsHash} maintains a hash
* table holding one instance of this per unique Token.
* Consumers of TermsHash (@link TermsHashConsumer} must
* subclass this class with its own concrete class.
* {@link FreqProxTermsWriter.RawPostingList} is the
* subclass used for the freq/prox postings, and {@link
* TermVectorsTermsWriter.PostingList} is the subclass
* used to hold TermVectors postings. */
abstract class RawPostingList {
final static int BYTES_SIZE = DocumentsWriter.OBJECT_HEADER_BYTES + 3*DocumentsWriter.INT_NUM_BYTE;
int textStart;
int intStart;
int byteStart;
}

View File

@ -0,0 +1,190 @@
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.util.Map;
import java.io.IOException;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
/** This is a DocFieldConsumer that writes stored fields. */
final class StoredFieldsWriter extends DocFieldConsumer {
FieldsWriter fieldsWriter;
final DocumentsWriter docWriter;
int lastDocID;
PerDoc[] docFreeList = new PerDoc[1];
int freeCount;
public StoredFieldsWriter(DocumentsWriter docWriter) {
this.docWriter = docWriter;
}
public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException {
return new StoredFieldsWriterPerThread(docFieldProcessorPerThread, this);
}
synchronized public void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
if (state.numDocsInStore > 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();
// Fill fdx file to include any final docs that we
// skipped because they hit non-aborting exceptions
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
}
if (fieldsWriter != null)
fieldsWriter.flush();
}
private void initFieldsWriter() throws IOException {
if (fieldsWriter == null) {
final String docStoreSegment = docWriter.getDocStoreSegment();
if (docStoreSegment != null) {
assert docStoreSegment != null;
fieldsWriter = new FieldsWriter(docWriter.directory,
docStoreSegment,
fieldInfos);
docWriter.addOpenFile(docStoreSegment + "." + IndexFileNames.FIELDS_EXTENSION);
docWriter.addOpenFile(docStoreSegment + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
lastDocID = 0;
}
}
}
synchronized public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
final int inc = state.numDocsInStore - lastDocID;
if (inc > 0) {
initFieldsWriter();
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
}
if (fieldsWriter != null) {
fieldsWriter.close();
fieldsWriter = null;
lastDocID = 0;
assert state.docStoreSegmentName != null;
state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_EXTENSION);
state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
state.docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_EXTENSION);
state.docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
if (4+state.numDocsInStore*8 != state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION))
throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION) + " length in bytes of " + state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
}
}
int allocCount;
synchronized PerDoc getPerDoc() {
if (freeCount == 0) {
allocCount++;
if (allocCount > docFreeList.length) {
// Grow our free list up front to make sure we have
// enough space to recycle all outstanding PerDoc
// instances
assert allocCount == 1+docFreeList.length;
docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
}
return new PerDoc();
} else
return docFreeList[--freeCount];
}
synchronized void abort() {
if (fieldsWriter != null) {
try {
fieldsWriter.close();
} catch (Throwable t) {
}
fieldsWriter = null;
lastDocID = 0;
}
}
/** Fills in any hole in the docIDs */
void fill(int docID) throws IOException {
final int docStoreOffset = docWriter.getDocStoreOffset();
// We must "catch up" for all docs before us
// that had no stored fields:
final int end = docID+docStoreOffset;
while(lastDocID < end) {
fieldsWriter.skipDocument();
lastDocID++;
}
}
synchronized void finishDocument(PerDoc perDoc) throws IOException {
assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
initFieldsWriter();
fill(perDoc.docID);
// Append stored fields to the real FieldsWriter:
fieldsWriter.flushDocument(perDoc.numStoredFields, perDoc.fdt);
lastDocID++;
perDoc.reset();
free(perDoc);
assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
}
public boolean freeRAM() {
return false;
}
synchronized void free(PerDoc perDoc) {
assert freeCount < docFreeList.length;
assert 0 == perDoc.numStoredFields;
assert 0 == perDoc.fdt.length();
assert 0 == perDoc.fdt.getFilePointer();
docFreeList[freeCount++] = perDoc;
}
class PerDoc extends DocumentsWriter.DocWriter {
// TODO: use something more memory efficient; for small
// docs the 1024 buffer size of RAMOutputStream wastes alot
RAMOutputStream fdt = new RAMOutputStream();
int numStoredFields;
void reset() {
fdt.reset();
numStoredFields = 0;
}
void abort() {
reset();
free(this);
}
public long sizeInBytes() {
return fdt.sizeInBytes();
}
public void finish() throws IOException {
finishDocument(this);
}
}
}

View File

@ -0,0 +1,66 @@
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.document.Fieldable;
final class StoredFieldsWriterPerField extends DocFieldConsumerPerField {
final StoredFieldsWriterPerThread perThread;
final FieldInfo fieldInfo;
final DocumentsWriter.DocState docState;
public StoredFieldsWriterPerField(StoredFieldsWriterPerThread perThread, FieldInfo fieldInfo) {
this.perThread = perThread;
this.fieldInfo = fieldInfo;
docState = perThread.docState;
}
// Process all occurrences of a single field in one doc;
// count is 1 if a given field occurs only once in the
// Document, which is the "typical" case
public void processFields(Fieldable[] fields, int count) throws IOException {
final StoredFieldsWriter.PerDoc doc;
if (perThread.doc == null) {
doc = perThread.doc = perThread.storedFieldsWriter.getPerDoc();
doc.docID = docState.docID;
perThread.localFieldsWriter.setFieldsStream(doc.fdt);
assert doc.numStoredFields == 0: "doc.numStoredFields=" + doc.numStoredFields;
assert 0 == doc.fdt.length();
assert 0 == doc.fdt.getFilePointer();
} else {
doc = perThread.doc;
assert doc.docID == docState.docID: "doc.docID=" + doc.docID + " docState.docID=" + docState.docID;
}
for(int i=0;i<count;i++) {
final Fieldable field = fields[i];
if (field.isStored()) {
perThread.localFieldsWriter.writeField(fieldInfo, field);
assert docState.testPoint("StoredFieldsWriterPerField.processFields.writeField");
doc.numStoredFields++;
}
}
}
void abort() {
}
}

View File

@ -0,0 +1,67 @@
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.store.IndexOutput;
final class StoredFieldsWriterPerThread extends DocFieldConsumerPerThread {
final FieldsWriter localFieldsWriter;
final StoredFieldsWriter storedFieldsWriter;
final DocumentsWriter.DocState docState;
StoredFieldsWriter.PerDoc doc;
public StoredFieldsWriterPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, StoredFieldsWriter storedFieldsWriter) throws IOException {
this.storedFieldsWriter = storedFieldsWriter;
this.docState = docFieldProcessorPerThread.docState;
localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, storedFieldsWriter.fieldInfos);
}
public void startDocument() {
if (doc != null) {
// Only happens if previous document hit non-aborting
// exception while writing stored fields into
// localFieldsWriter:
doc.reset();
doc.docID = docState.docID;
}
}
public DocumentsWriter.DocWriter finishDocument() {
// If there were any stored fields in this doc, doc will
// be non-null; else it's null.
try {
return doc;
} finally {
doc = null;
}
}
public void abort() {
if (doc != null) {
doc.abort();
doc = null;
}
}
public DocFieldConsumerPerField addField(FieldInfo fieldInfo) {
return new StoredFieldsWriterPerField(this, fieldInfo);
}
}

View File

@ -0,0 +1,291 @@
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.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import java.io.IOException;
import java.util.Collection;
import java.util.Iterator;
import java.util.Map;
final class TermVectorsTermsWriter extends TermsHashConsumer {
final DocumentsWriter docWriter;
TermVectorsWriter termVectorsWriter;
PerDoc[] docFreeList = new PerDoc[1];
int freeCount;
IndexOutput tvx;
IndexOutput tvd;
IndexOutput tvf;
int lastDocID;
public TermVectorsTermsWriter(DocumentsWriter docWriter) {
this.docWriter = docWriter;
streamCount = 2;
}
public TermsHashConsumerPerThread addThread(TermsHashPerThread termsHashPerThread) {
return new TermVectorsTermsWriterPerThread(termsHashPerThread, this);
}
void createPostings(RawPostingList[] postings, int start, int count) {
final int end = start + count;
for(int i=start;i<end;i++)
postings[i] = new PostingList();
}
synchronized void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException {
if (tvx != null) {
if (state.numDocsInStore > 0)
// In case there are some final documents that we
// didn't see (because they hit a non-aborting exception):
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
tvx.flush();
tvd.flush();
tvf.flush();
}
Iterator it = threadsAndFields.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
Iterator it2 = ((Collection) entry.getValue()).iterator();
while(it2.hasNext()) {
TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) it2.next();
perField.termsHashPerField.reset();
perField.shrinkHash();
}
TermVectorsTermsWriterPerThread perThread = (TermVectorsTermsWriterPerThread) entry.getKey();
perThread.termsHashPerThread.reset(true);
}
}
synchronized void closeDocStore(final DocumentsWriter.FlushState state) throws IOException {
if (tvx != null) {
// At least one doc in this run had term vectors
// enabled
fill(state.numDocsInStore - docWriter.getDocStoreOffset());
tvx.close();
tvf.close();
tvd.close();
tvx = null;
assert state.docStoreSegmentName != null;
if (4+state.numDocsInStore*16 != state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION))
throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION) + " length in bytes of " + state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
state.flushedFiles.add(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
docWriter.removeOpenFile(state.docStoreSegmentName + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
lastDocID = 0;
}
}
int allocCount;
synchronized PerDoc getPerDoc() {
if (freeCount == 0) {
allocCount++;
if (allocCount > docFreeList.length) {
// Grow our free list up front to make sure we have
// enough space to recycle all outstanding PerDoc
// instances
assert allocCount == 1+docFreeList.length;
docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
}
return new PerDoc();
} else
return docFreeList[--freeCount];
}
/** Fills in no-term-vectors for all docs we haven't seen
* since the last doc that had term vectors. */
void fill(int docID) throws IOException {
final int docStoreOffset = docWriter.getDocStoreOffset();
final int end = docID+docStoreOffset;
if (lastDocID < end) {
final long tvfPosition = tvf.getFilePointer();
while(lastDocID < end) {
tvx.writeLong(tvd.getFilePointer());
tvd.writeVInt(0);
tvx.writeLong(tvfPosition);
lastDocID++;
}
}
}
synchronized void initTermVectorsWriter() throws IOException {
if (tvx == null) {
final String docStoreSegment = docWriter.getDocStoreSegment();
if (docStoreSegment == null)
return;
assert docStoreSegment != null;
// If we hit an exception while init'ing the term
// vector output files, we must abort this segment
// because those files will be in an unknown
// state:
tvx = docWriter.directory.createOutput(docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
tvd = docWriter.directory.createOutput(docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
tvf = docWriter.directory.createOutput(docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
docWriter.addOpenFile(docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
docWriter.addOpenFile(docStoreSegment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
docWriter.addOpenFile(docStoreSegment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
lastDocID = 0;
}
}
synchronized void finishDocument(PerDoc perDoc) throws IOException {
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument start");
initTermVectorsWriter();
fill(perDoc.docID);
// Append term vectors to the real outputs:
tvx.writeLong(tvd.getFilePointer());
tvx.writeLong(tvf.getFilePointer());
tvd.writeVInt(perDoc.numVectorFields);
if (perDoc.numVectorFields > 0) {
for(int i=0;i<perDoc.numVectorFields;i++)
tvd.writeVInt(perDoc.fieldNumbers[i]);
assert 0 == perDoc.fieldPointers[0];
long lastPos = perDoc.fieldPointers[0];
for(int i=1;i<perDoc.numVectorFields;i++) {
long pos = perDoc.fieldPointers[i];
tvd.writeVLong(pos-lastPos);
lastPos = pos;
}
perDoc.tvf.writeTo(tvf);
perDoc.tvf.reset();
perDoc.numVectorFields = 0;
}
assert lastDocID == perDoc.docID + docWriter.getDocStoreOffset();
lastDocID++;
free(perDoc);
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
}
public boolean freeRAM() {
// We don't hold any state beyond one doc, so we don't
// free persistent RAM here
return false;
}
public void abort() {
if (tvx != null) {
try {
tvx.close();
} catch (Throwable t) {
}
tvx = null;
}
if (tvd != null) {
try {
tvd.close();
} catch (Throwable t) {
}
tvd = null;
}
if (tvf != null) {
try {
tvf.close();
} catch (Throwable t) {
}
tvf = null;
}
lastDocID = 0;
}
synchronized void free(PerDoc doc) {
assert freeCount < docFreeList.length;
docFreeList[freeCount++] = doc;
}
class PerDoc extends DocumentsWriter.DocWriter {
// TODO: use something more memory efficient; for small
// docs the 1024 buffer size of RAMOutputStream wastes alot
RAMOutputStream tvf = new RAMOutputStream();
int numVectorFields;
int[] fieldNumbers = new int[1];
long[] fieldPointers = new long[1];
void reset() {
tvf.reset();
numVectorFields = 0;
}
void abort() {
reset();
free(this);
}
void addField(final int fieldNumber) {
if (numVectorFields == fieldNumbers.length) {
fieldNumbers = ArrayUtil.grow(fieldNumbers);
fieldPointers = ArrayUtil.grow(fieldPointers);
}
fieldNumbers[numVectorFields] = fieldNumber;
fieldPointers[numVectorFields] = tvf.getFilePointer();
numVectorFields++;
}
public long sizeInBytes() {
return tvf.sizeInBytes();
}
public void finish() throws IOException {
finishDocument(this);
}
}
static final class PostingList extends RawPostingList {
int freq; // How many times this term occurred in the current doc
int lastOffset; // Last offset we saw
int lastPosition; // Last position where this term occurred
}
int bytesPerPosting() {
return RawPostingList.BYTES_SIZE + 3 * DocumentsWriter.INT_NUM_BYTE;
}
}

View File

@ -0,0 +1,235 @@
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.util.UnicodeUtil;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.store.IndexOutput;
final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
final TermVectorsTermsWriterPerThread perThread;
final TermsHashPerField termsHashPerField;
final TermVectorsTermsWriter termsWriter;
final FieldInfo fieldInfo;
final DocumentsWriter.DocState docState;
final DocInverter.FieldInvertState fieldState;
boolean doVectors;
boolean doVectorPositions;
boolean doVectorOffsets;
int maxNumPostings;
public TermVectorsTermsWriterPerField(TermsHashPerField termsHashPerField, TermVectorsTermsWriterPerThread perThread, FieldInfo fieldInfo) {
this.termsHashPerField = termsHashPerField;
this.perThread = perThread;
this.termsWriter = perThread.termsWriter;
this.fieldInfo = fieldInfo;
docState = termsHashPerField.docState;
fieldState = termsHashPerField.fieldState;
}
boolean start(Fieldable[] fields, int count) {
doVectors = false;
doVectorPositions = false;
doVectorOffsets = false;
for(int i=0;i<count;i++) {
Fieldable field = fields[i];
if (field.isIndexed() && field.isTermVectorStored()) {
doVectors = true;
doVectorPositions |= field.isStorePositionWithTermVector();
doVectorOffsets |= field.isStoreOffsetWithTermVector();
}
}
if (doVectors) {
if (perThread.doc == null) {
perThread.doc = termsWriter.getPerDoc();
perThread.doc.docID = docState.docID;
assert perThread.doc.numVectorFields == 0;
assert 0 == perThread.doc.tvf.length();
assert 0 == perThread.doc.tvf.getFilePointer();
} else {
assert perThread.doc.docID == docState.docID;
if (termsHashPerField.numPostings != 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. */
void finish() throws IOException {
assert docState.testPoint("TermVectorsTermsWriterPerField.finish start");
final int numPostings = termsHashPerField.numPostings;
assert numPostings >= 0;
if (!doVectors || numPostings == 0)
return;
if (numPostings > maxNumPostings)
maxNumPostings = numPostings;
final IndexOutput tvf = perThread.doc.tvf;
// This is called once, after inverting all occurences
// of a given field in the doc. At this point we flush
// our hash into the DocWriter.
assert fieldInfo.storeTermVector;
assert perThread.vectorFieldsInOrder(fieldInfo);
perThread.doc.addField(termsHashPerField.fieldInfo.number);
final RawPostingList[] postings = termsHashPerField.sortPostings();
tvf.writeVInt(numPostings);
byte bits = 0x0;
if (doVectorPositions)
bits |= TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
if (doVectorOffsets)
bits |= TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
tvf.writeByte(bits);
int encoderUpto = 0;
int lastTermBytesCount = 0;
final ByteSliceReader reader = perThread.vectorSliceReader;
final char[][] charBuffers = perThread.termsHashPerThread.charPool.buffers;
for(int j=0;j<numPostings;j++) {
final TermVectorsTermsWriter.PostingList posting = (TermVectorsTermsWriter.PostingList) postings[j];
final int freq = posting.freq;
final char[] text2 = charBuffers[posting.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
final int start2 = posting.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
// We swap between two encoders to save copying
// last Term's byte array
final UnicodeUtil.UTF8Result utf8Result = perThread.utf8Results[encoderUpto];
// TODO: we could do this incrementally
UnicodeUtil.UTF16toUTF8(text2, start2, utf8Result);
final int termBytesCount = utf8Result.length;
// TODO: UTF16toUTF8 could tell us this prefix
// Compute common prefix between last term and
// this term
int prefix = 0;
if (j > 0) {
final byte[] lastTermBytes = perThread.utf8Results[1-encoderUpto].result;
final byte[] termBytes = perThread.utf8Results[encoderUpto].result;
while(prefix < lastTermBytesCount && prefix < termBytesCount) {
if (lastTermBytes[prefix] != termBytes[prefix])
break;
prefix++;
}
}
encoderUpto = 1-encoderUpto;
lastTermBytesCount = termBytesCount;
final int suffix = termBytesCount - prefix;
tvf.writeVInt(prefix);
tvf.writeVInt(suffix);
tvf.writeBytes(utf8Result.result, prefix, suffix);
tvf.writeVInt(freq);
if (doVectorPositions) {
termsHashPerField.initReader(reader, posting, 0);
reader.writeTo(tvf);
}
if (doVectorOffsets) {
termsHashPerField.initReader(reader, posting, 1);
reader.writeTo(tvf);
}
}
termsHashPerField.reset();
perThread.termsHashPerThread.reset(false);
}
void shrinkHash() {
termsHashPerField.shrinkHash(maxNumPostings);
maxNumPostings = 0;
}
void newTerm(Token t, RawPostingList p0) {
assert docState.testPoint("TermVectorsTermsWriterPerField.newTerm start");
TermVectorsTermsWriter.PostingList p = (TermVectorsTermsWriter.PostingList) p0;
p.freq = 1;
if (doVectorOffsets) {
final int startOffset = fieldState.offset + t.startOffset();
final int endOffset = fieldState.offset + t.endOffset();
termsHashPerField.writeVInt(1, startOffset);
termsHashPerField.writeVInt(1, endOffset - startOffset);
p.lastOffset = endOffset;
}
if (doVectorPositions) {
termsHashPerField.writeVInt(0, fieldState.position);
p.lastPosition = fieldState.position;
}
}
void addTerm(Token t, RawPostingList p0) {
assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
TermVectorsTermsWriter.PostingList p = (TermVectorsTermsWriter.PostingList) p0;
p.freq++;
if (doVectorOffsets) {
final int startOffset = fieldState.offset + t.startOffset();
final int endOffset = fieldState.offset + t.endOffset();
termsHashPerField.writeVInt(1, startOffset - p.lastOffset);
termsHashPerField.writeVInt(1, endOffset - startOffset);
p.lastOffset = endOffset;
}
if (doVectorPositions) {
termsHashPerField.writeVInt(0, fieldState.position - p.lastPosition);
p.lastPosition = fieldState.position;
}
}
void skippingLongTerm(Token t) {}
}

View File

@ -0,0 +1,87 @@
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.UnicodeUtil;
final class TermVectorsTermsWriterPerThread extends TermsHashConsumerPerThread {
final TermVectorsTermsWriter termsWriter;
final TermsHashPerThread termsHashPerThread;
final DocumentsWriter.DocState docState;
TermVectorsTermsWriter.PerDoc doc;
public TermVectorsTermsWriterPerThread(TermsHashPerThread termsHashPerThread, TermVectorsTermsWriter termsWriter) {
this.termsWriter = termsWriter;
this.termsHashPerThread = termsHashPerThread;
docState = termsHashPerThread.docState;
}
// Used by perField when serializing the term vectors
final ByteSliceReader vectorSliceReader = new ByteSliceReader();
final UnicodeUtil.UTF8Result utf8Results[] = {new UnicodeUtil.UTF8Result(),
new UnicodeUtil.UTF8Result()};
public void startDocument() {
assert clearLastVectorFieldName();
if (doc != null) {
doc.reset();
doc.docID = docState.docID;
}
}
public DocumentsWriter.DocWriter finishDocument() {
try {
return doc;
} finally {
doc = null;
}
}
public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
}
public void abort() {
if (doc != null) {
doc.abort();
doc = null;
}
}
// Called only by assert
final boolean clearLastVectorFieldName() {
lastVectorFieldName = null;
return true;
}
// Called only by assert
String lastVectorFieldName;
final boolean vectorFieldsInOrder(FieldInfo fi) {
try {
if (lastVectorFieldName != null)
return lastVectorFieldName.compareTo(fi.name) < 0;
else
return true;
} finally {
lastVectorFieldName = fi.name;
}
}
}

View File

@ -0,0 +1,244 @@
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.util.Collection;
import java.util.Map;
import java.util.HashMap;
import java.util.Iterator;
import java.util.HashSet;
import java.util.Arrays;
import java.io.IOException;
import org.apache.lucene.util.ArrayUtil;
/** 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
* TermVectorsTermsWriter}, write their own byte streams
* under each term.
*/
final class TermsHash extends InvertedDocConsumer {
final TermsHashConsumer consumer;
final TermsHash nextTermsHash;
final int bytesPerPosting;
final int postingsFreeChunk;
final int streamCount;
final DocumentsWriter docWriter;
TermsHash primaryTermsHash;
RawPostingList[] postingsFreeList = new RawPostingList[1];
int postingsFreeCount;
int postingsAllocCount;
boolean trackAllocations;
public TermsHash(final DocumentsWriter docWriter, boolean trackAllocations, final TermsHashConsumer consumer, final TermsHash nextTermsHash) {
this.docWriter = docWriter;
this.consumer = consumer;
this.streamCount = consumer.streamCount;
this.nextTermsHash = nextTermsHash;
this.trackAllocations = trackAllocations;
// Why + 4*POINTER_NUM_BYTE below?
// +1: Posting is referenced by postingsFreeList array
// +3: Posting is referenced by hash, which
// targets 25-50% fill factor; approximate this
// as 3X # pointers
bytesPerPosting = consumer.bytesPerPosting() + 4*DocumentsWriter.POINTER_NUM_BYTE;
postingsFreeChunk = (int) (DocumentsWriter.BYTE_BLOCK_SIZE / bytesPerPosting);
}
InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, null);
}
TermsHashPerThread addThread(DocInverterPerThread docInverterPerThread, TermsHashPerThread primaryPerThread) {
return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread);
}
void setFieldInfos(FieldInfos fieldInfos) {
this.fieldInfos = fieldInfos;
consumer.setFieldInfos(fieldInfos);
}
synchronized public void abort() {
consumer.abort();
if (nextTermsHash != null)
nextTermsHash.abort();
}
void shrinkFreePostings(Map threadsAndFields, DocumentsWriter.FlushState state) {
assert postingsFreeCount == postingsAllocCount: Thread.currentThread().getName() + ": postingsFreeCount=" + postingsFreeCount + " postingsAllocCount=" + postingsAllocCount + " consumer=" + consumer;
final int newSize = ArrayUtil.getShrinkSize(postingsFreeList.length, postingsAllocCount);
if (newSize != postingsFreeList.length) {
RawPostingList[] newArray = new RawPostingList[newSize];
System.arraycopy(postingsFreeList, 0, newArray, 0, postingsFreeCount);
postingsFreeList = newArray;
}
}
synchronized void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
consumer.closeDocStore(state);
if (nextTermsHash != null)
nextTermsHash.closeDocStore(state);
}
synchronized void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException {
Map childThreadsAndFields = new HashMap();
Map nextThreadsAndFields;
if (nextTermsHash != null)
nextThreadsAndFields = new HashMap();
else
nextThreadsAndFields = null;
Iterator it = threadsAndFields.entrySet().iterator();
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
Collection fields = (Collection) entry.getValue();
Iterator fieldsIt = fields.iterator();
Collection childFields = new HashSet();
Collection nextChildFields;
if (nextTermsHash != null)
nextChildFields = new HashSet();
else
nextChildFields = null;
while(fieldsIt.hasNext()) {
TermsHashPerField perField = (TermsHashPerField) fieldsIt.next();
childFields.add(perField.consumer);
if (nextTermsHash != null)
nextChildFields.add(perField.nextPerField);
}
childThreadsAndFields.put(perThread.consumer, childFields);
if (nextTermsHash != null)
nextThreadsAndFields.put(perThread.nextPerThread, nextChildFields);
}
consumer.flush(childThreadsAndFields, state);
shrinkFreePostings(threadsAndFields, state);
if (nextTermsHash != null)
nextTermsHash.flush(nextThreadsAndFields, state);
}
synchronized public boolean freeRAM() {
if (!trackAllocations)
return false;
boolean any;
final int numToFree;
if (postingsFreeCount >= postingsFreeChunk)
numToFree = postingsFreeChunk;
else
numToFree = postingsFreeCount;
any = numToFree > 0;
if (any) {
Arrays.fill(postingsFreeList, postingsFreeCount-numToFree, postingsFreeCount, null);
postingsFreeCount -= numToFree;
postingsAllocCount -= numToFree;
docWriter.bytesAllocated(-numToFree * bytesPerPosting);
any = true;
}
if (nextTermsHash != null)
any |= nextTermsHash.freeRAM();
return any;
}
// USE ONLY FOR DEBUGGING!
/*
public String getPostingText() {
char[] text = charPool.buffers[p.textStart >> CHAR_BLOCK_SHIFT];
int upto = p.textStart & CHAR_BLOCK_MASK;
while(text[upto] != 0xffff)
upto++;
return new String(text, p.textStart, upto-(p.textStart & BYTE_BLOCK_MASK));
}
*/
synchronized public void recyclePostings(final RawPostingList[] postings, final int numPostings) {
assert postings.length >= numPostings;
// Move all Postings from this ThreadState back to our
// free list. We pre-allocated this array while we were
// creating Postings to make sure it's large enough
assert postingsFreeCount + numPostings <= postingsFreeList.length;
System.arraycopy(postings, 0, postingsFreeList, postingsFreeCount, numPostings);
postingsFreeCount += numPostings;
}
synchronized public void getPostings(final RawPostingList[] postings) {
assert docWriter.writer.testPoint("TermsHash.getPostings start");
assert postingsFreeCount <= postingsFreeList.length;
assert postingsFreeCount <= postingsAllocCount: "postingsFreeCount=" + postingsFreeCount + " postingsAllocCount=" + postingsAllocCount;
final int numToCopy;
if (postingsFreeCount < postings.length)
numToCopy = postingsFreeCount;
else
numToCopy = postings.length;
final int start = postingsFreeCount-numToCopy;
assert start >= 0;
assert start + numToCopy <= postingsFreeList.length;
assert numToCopy <= postings.length;
System.arraycopy(postingsFreeList, start,
postings, 0, numToCopy);
// Directly allocate the remainder if any
if (numToCopy < postings.length) {
final int extra = postings.length - numToCopy;
final int newPostingsAllocCount = postingsAllocCount + extra;
if (newPostingsAllocCount > postingsFreeList.length)
postingsFreeList = new RawPostingList[ArrayUtil.getNextSize(newPostingsAllocCount)];
consumer.createPostings(postings, numToCopy, extra);
assert docWriter.writer.testPoint("TermsHash.getPostings after create");
postingsAllocCount += extra;
if (trackAllocations)
docWriter.bytesAllocated(extra * bytesPerPosting);
}
postingsFreeCount -= numToCopy;
if (trackAllocations)
docWriter.bytesUsed(postings.length * bytesPerPosting);
}
}

View File

@ -0,0 +1,38 @@
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 int bytesPerPosting();
abstract void createPostings(RawPostingList[] postings, int start, int count);
abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
abstract void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException;
abstract void abort();
abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
int streamCount;
FieldInfos fieldInfos;
void setFieldInfos(FieldInfos fieldInfos) {
this.fieldInfos = fieldInfos;
}
}

View File

@ -0,0 +1,35 @@
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;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.analysis.Token;
abstract class TermsHashConsumerPerField {
abstract boolean start(Fieldable[] fields, int count) throws IOException;
abstract void finish() throws IOException;
abstract void skippingLongTerm(Token t) throws IOException;
abstract void newTerm(Token t, RawPostingList p) throws IOException;
abstract void addTerm(Token t, RawPostingList p) throws IOException;
}

View File

@ -0,0 +1,27 @@
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 TermsHashConsumerPerThread {
abstract void startDocument() throws IOException;
abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
abstract public void abort();
}

View File

@ -0,0 +1,545 @@
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.document.Fieldable;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.util.UnicodeUtil;
final class TermsHashPerField extends InvertedDocConsumerPerField {
final TermsHashConsumerPerField consumer;
final TermsHashPerField nextPerField;
final TermsHashPerThread perThread;
final DocumentsWriter.DocState docState;
final DocInverter.FieldInvertState fieldState;
// Copied from our perThread
final CharBlockPool charPool;
final IntBlockPool intPool;
final ByteBlockPool bytePool;
final int streamCount;
final int numPostingInt;
final FieldInfo fieldInfo;
boolean postingsCompacted;
int numPostings;
private int postingsHashSize = 4;
private int postingsHashHalfSize = postingsHashSize/2;
private int postingsHashMask = postingsHashSize-1;
private RawPostingList[] postingsHash = new RawPostingList[postingsHashSize];
private RawPostingList p;
public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHashPerThread perThread, final TermsHashPerThread nextPerThread, final FieldInfo fieldInfo) {
this.perThread = perThread;
intPool = perThread.intPool;
charPool = perThread.charPool;
bytePool = perThread.bytePool;
docState = perThread.docState;
fieldState = docInverterPerField.fieldState;
streamCount = perThread.termsHash.streamCount;
numPostingInt = 2*streamCount;
this.consumer = perThread.consumer.addField(this, fieldInfo);
this.fieldInfo = fieldInfo;
if (nextPerThread != null)
nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo);
else
nextPerField = null;
}
void shrinkHash(int targetSize) {
assert postingsCompacted || numPostings == 0;
// Cannot use ArrayUtil.shrink because we require power
// of 2:
int newSize = postingsHash.length;
while(newSize >= 8 && newSize/4 > targetSize) {
newSize /= 2;
}
if (newSize != postingsHash.length) {
postingsHash = new RawPostingList[newSize];
postingsHashSize = newSize;
postingsHashHalfSize = newSize/2;
postingsHashMask = newSize-1;
}
}
public void reset() {
if (!postingsCompacted)
compactPostings();
assert numPostings <= postingsHash.length;
if (numPostings > 0) {
perThread.termsHash.recyclePostings(postingsHash, numPostings);
Arrays.fill(postingsHash, 0, numPostings, null);
numPostings = 0;
}
postingsCompacted = false;
if (nextPerField != null)
nextPerField.reset();
}
synchronized public void abort() {
reset();
if (nextPerField != null)
nextPerField.abort();
}
public void initReader(ByteSliceReader reader, RawPostingList p, int stream) {
assert stream < streamCount;
final int[] ints = intPool.buffers[p.intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
final int upto = p.intStart & DocumentsWriter.INT_BLOCK_MASK;
reader.init(bytePool,
p.byteStart+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
ints[upto+stream]);
}
private synchronized void compactPostings() {
int upto = 0;
for(int i=0;i<postingsHashSize;i++) {
if (postingsHash[i] != null) {
if (upto < i) {
postingsHash[upto] = postingsHash[i];
postingsHash[i] = null;
}
upto++;
}
}
assert upto == numPostings;
postingsCompacted = true;
}
/** Collapse the hash table & sort in-place. */
public RawPostingList[] sortPostings() {
compactPostings();
quickSort(postingsHash, 0, numPostings-1);
return postingsHash;
}
void quickSort(RawPostingList[] postings, int lo, int hi) {
if (lo >= hi)
return;
else if (hi == 1+lo) {
if (comparePostings(postings[lo], postings[hi]) > 0) {
final RawPostingList tmp = postings[lo];
postings[lo] = postings[hi];
postings[hi] = tmp;
}
return;
}
int mid = (lo + hi) >>> 1;
if (comparePostings(postings[lo], postings[mid]) > 0) {
RawPostingList tmp = postings[lo];
postings[lo] = postings[mid];
postings[mid] = tmp;
}
if (comparePostings(postings[mid], postings[hi]) > 0) {
RawPostingList tmp = postings[mid];
postings[mid] = postings[hi];
postings[hi] = tmp;
if (comparePostings(postings[lo], postings[mid]) > 0) {
RawPostingList tmp2 = postings[lo];
postings[lo] = postings[mid];
postings[mid] = tmp2;
}
}
int left = lo + 1;
int right = hi - 1;
if (left >= right)
return;
RawPostingList partition = postings[mid];
for (; ;) {
while (comparePostings(postings[right], partition) > 0)
--right;
while (left < right && comparePostings(postings[left], partition) <= 0)
++left;
if (left < right) {
RawPostingList tmp = postings[left];
postings[left] = postings[right];
postings[right] = tmp;
--right;
} else {
break;
}
}
quickSort(postings, lo, left);
quickSort(postings, left + 1, hi);
}
/** Compares term text for two Posting instance and
* returns -1 if p1 < p2; 1 if p1 > p2; else 0. */
int comparePostings(RawPostingList p1, RawPostingList p2) {
if (p1 == p2)
return 0;
final char[] text1 = charPool.buffers[p1.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
int pos1 = p1.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
final char[] text2 = charPool.buffers[p2.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
int pos2 = p2.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
assert text1 != text2 || pos1 != pos2;
while(true) {
final char c1 = text1[pos1++];
final char c2 = text2[pos2++];
if (c1 != c2) {
if (0xffff == c2)
return 1;
else if (0xffff == c1)
return -1;
else
return c1-c2;
} else
// This method should never compare equal postings
// unless p1==p2
assert c1 != 0xffff;
}
}
/** Test whether the text for current RawPostingList p equals
* current tokenText. */
private boolean postingEquals(final char[] tokenText, final int tokenTextLen) {
final char[] text = perThread.charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
assert text != null;
int pos = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
int tokenPos = 0;
for(;tokenPos<tokenTextLen;pos++,tokenPos++)
if (tokenText[tokenPos] != text[pos])
return false;
return 0xffff == text[pos];
}
private boolean doCall;
private boolean doNextCall;
boolean start(Fieldable[] fields, int count) throws IOException {
doCall = consumer.start(fields, count);
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
public void add(Token token, int textStart) throws IOException {
int code = textStart;
int hashPos = code & postingsHashMask;
assert !postingsCompacted;
// Locate RawPostingList in hash
p = postingsHash[hashPos];
if (p != null && p.textStart != textStart) {
// Conflict: keep searching different locations in
// the hash table.
final int inc = ((code>>8)+code)|1;
do {
code += inc;
hashPos = code & postingsHashMask;
p = postingsHash[hashPos];
} while (p != null && p.textStart != textStart);
}
if (p == null) {
// First time we are seeing this token since we last
// flushed the hash.
// Refill?
if (0 == perThread.freePostingsCount)
perThread.morePostings();
// Pull next free RawPostingList from free list
p = perThread.freePostings[--perThread.freePostingsCount];
assert p != null;
p.textStart = textStart;
assert postingsHash[hashPos] == null;
postingsHash[hashPos] = p;
numPostings++;
if (numPostings == postingsHashHalfSize)
rehashPostings(2*postingsHashSize);
// Init stream slices
if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE)
intPool.nextBuffer();
if (DocumentsWriter.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE)
bytePool.nextBuffer();
intUptos = intPool.buffer;
intUptoStart = intPool.intUpto;
intPool.intUpto += streamCount;
p.intStart = intUptoStart + intPool.intOffset;
for(int i=0;i<streamCount;i++) {
final int upto = bytePool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
intUptos[intUptoStart+i] = upto + bytePool.byteOffset;
}
p.byteStart = intUptos[intUptoStart];
consumer.newTerm(token, p);
} else {
intUptos = intPool.buffers[p.intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
intUptoStart = p.intStart & DocumentsWriter.INT_BLOCK_MASK;
consumer.addTerm(token, p);
}
}
// Primary entry point (for first TermsHash)
void add(Token token) throws IOException {
assert !postingsCompacted;
// We are first in the chain so we must "intern" the
// term text into textStart address
// Get the text of this term.
final char[] tokenText = token.termBuffer();
final int tokenTextLen = token.termLength();
// Compute hashcode & replace any invalid UTF16 sequences
int downto = tokenTextLen;
int code = 0;
while (downto > 0) {
char ch = tokenText[--downto];
if (ch >= UnicodeUtil.UNI_SUR_LOW_START && ch <= UnicodeUtil.UNI_SUR_LOW_END) {
if (0 == downto) {
// Unpaired
ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
} else {
final char ch2 = tokenText[downto-1];
if (ch2 >= UnicodeUtil.UNI_SUR_HIGH_START && ch2 <= UnicodeUtil.UNI_SUR_HIGH_END) {
// OK: high followed by low. This is a valid
// surrogate pair.
code = ((code*31) + ch)*31+ch2;
downto--;
continue;
} else {
// Unpaired
ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
}
}
} else if (ch >= UnicodeUtil.UNI_SUR_HIGH_START && ch <= UnicodeUtil.UNI_SUR_HIGH_END)
// Unpaired
ch = tokenText[downto] = UnicodeUtil.UNI_REPLACEMENT_CHAR;
code = (code*31) + ch;
}
int hashPos = code & postingsHashMask;
// Locate RawPostingList in hash
p = postingsHash[hashPos];
if (p != null && !postingEquals(tokenText, tokenTextLen)) {
// Conflict: keep searching different locations in
// the hash table.
final int inc = ((code>>8)+code)|1;
do {
code += inc;
hashPos = code & postingsHashMask;
p = postingsHash[hashPos];
} while (p != null && !postingEquals(tokenText, tokenTextLen));
}
if (p == null) {
// First time we are seeing this token since we last
// flushed the hash.
final int textLen1 = 1+tokenTextLen;
if (textLen1 + charPool.charUpto > DocumentsWriter.CHAR_BLOCK_SIZE) {
if (textLen1 > DocumentsWriter.CHAR_BLOCK_SIZE) {
// Just skip this term, to remain as robust as
// possible during indexing. A TokenFilter
// can be inserted into the analyzer chain if
// other behavior is wanted (pruning the term
// to a prefix, throwing an exception, etc).
if (docState.maxTermPrefix == null)
docState.maxTermPrefix = new String(tokenText, 0, 30);
consumer.skippingLongTerm(token);
return;
}
charPool.nextBuffer();
}
// Refill?
if (0 == perThread.freePostingsCount)
perThread.morePostings();
// Pull next free RawPostingList from free list
p = perThread.freePostings[--perThread.freePostingsCount];
assert p != null;
final char[] text = charPool.buffer;
final int textUpto = charPool.charUpto;
p.textStart = textUpto + charPool.charOffset;
charPool.charUpto += textLen1;
System.arraycopy(tokenText, 0, text, textUpto, tokenTextLen);
text[textUpto+tokenTextLen] = 0xffff;
assert postingsHash[hashPos] == null;
postingsHash[hashPos] = p;
numPostings++;
if (numPostings == postingsHashHalfSize)
rehashPostings(2*postingsHashSize);
// Init stream slices
if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE)
intPool.nextBuffer();
if (DocumentsWriter.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE)
bytePool.nextBuffer();
intUptos = intPool.buffer;
intUptoStart = intPool.intUpto;
intPool.intUpto += streamCount;
p.intStart = intUptoStart + intPool.intOffset;
for(int i=0;i<streamCount;i++) {
final int upto = bytePool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
intUptos[intUptoStart+i] = upto + bytePool.byteOffset;
}
p.byteStart = intUptos[intUptoStart];
consumer.newTerm(token, p);
} else {
intUptos = intPool.buffers[p.intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
intUptoStart = p.intStart & DocumentsWriter.INT_BLOCK_MASK;
consumer.addTerm(token, p);
}
if (doNextCall)
nextPerField.add(token, p.textStart);
}
int[] intUptos;
int intUptoStart;
void writeByte(int stream, byte b) {
int upto = intUptos[intUptoStart+stream];
byte[] bytes = bytePool.buffers[upto >> DocumentsWriter.BYTE_BLOCK_SHIFT];
assert bytes != null;
int offset = upto & DocumentsWriter.BYTE_BLOCK_MASK;
if (bytes[offset] != 0) {
// End of slice; allocate a new one
offset = bytePool.allocSlice(bytes, offset);
bytes = bytePool.buffer;
intUptos[intUptoStart+stream] = offset + bytePool.byteOffset;
}
bytes[offset] = b;
(intUptos[intUptoStart+stream])++;
}
public void writeBytes(int stream, byte[] b, int offset, int len) {
// TODO: optimize
final int end = offset + len;
for(int i=offset;i<end;i++)
writeByte(stream, b[i]);
}
void writeVInt(int stream, int i) {
while ((i & ~0x7F) != 0) {
writeByte(stream, (byte)((i & 0x7f) | 0x80));
i >>>= 7;
}
writeByte(stream, (byte) i);
}
void finish() throws IOException {
consumer.finish();
if (nextPerField != null)
nextPerField.finish();
}
/** Called when postings hash is too small (> 50%
* occupied) or too large (< 20% occupied). */
void rehashPostings(final int newSize) {
final int newMask = newSize-1;
RawPostingList[] newHash = new RawPostingList[newSize];
for(int i=0;i<postingsHashSize;i++) {
RawPostingList p0 = postingsHash[i];
if (p0 != null) {
int code;
if (perThread.primary) {
final int start = p0.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
final char[] text = charPool.buffers[p0.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
int pos = start;
while(text[pos] != 0xffff)
pos++;
code = 0;
while (pos > start)
code = (code*31) + text[--pos];
} else
code = p0.textStart;
int hashPos = code & newMask;
assert hashPos >= 0;
if (newHash[hashPos] != null) {
final int inc = ((code>>8)+code)|1;
do {
code += inc;
hashPos = code & newMask;
} while (newHash[hashPos] != null);
}
newHash[hashPos] = p0;
}
}
postingsHashMask = newMask;
postingsHash = newHash;
postingsHashSize = newSize;
postingsHashHalfSize = newSize >> 1;
}
}

View File

@ -0,0 +1,116 @@
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;
final class TermsHashPerThread extends InvertedDocConsumerPerThread {
final TermsHash termsHash;
final TermsHashConsumerPerThread consumer;
final TermsHashPerThread nextPerThread;
final CharBlockPool charPool;
final IntBlockPool intPool;
final ByteBlockPool bytePool;
final boolean primary;
final DocumentsWriter.DocState docState;
final RawPostingList freePostings[] = new RawPostingList[256];
int freePostingsCount;
public TermsHashPerThread(DocInverterPerThread docInverterPerThread, final TermsHash termsHash, final TermsHash nextTermsHash, final TermsHashPerThread primaryPerThread) {
docState = docInverterPerThread.docState;
this.termsHash = termsHash;
this.consumer = termsHash.consumer.addThread(this);
if (nextTermsHash != null) {
// We are primary
charPool = new CharBlockPool(termsHash.docWriter);
primary = true;
} else {
charPool = primaryPerThread.charPool;
primary = false;
}
intPool = new IntBlockPool(termsHash.docWriter, termsHash.trackAllocations);
bytePool = new ByteBlockPool(termsHash.docWriter.byteBlockAllocator, termsHash.trackAllocations);
if (nextTermsHash != null)
nextPerThread = nextTermsHash.addThread(docInverterPerThread, this);
else
nextPerThread = null;
}
InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
return new TermsHashPerField(docInverterPerField, this, nextPerThread, fieldInfo);
}
synchronized public void abort() {
reset(true);
consumer.abort();
if (nextPerThread != null)
nextPerThread.abort();
}
// perField calls this when it needs more postings:
void morePostings() throws IOException {
assert freePostingsCount == 0;
termsHash.getPostings(freePostings);
freePostingsCount = freePostings.length;
for(int i=0;i<freePostingsCount;i++)
assert freePostings[i] != null;
}
public void startDocument() throws IOException {
consumer.startDocument();
if (nextPerThread != null)
nextPerThread.consumer.startDocument();
}
public DocumentsWriter.DocWriter finishDocument() throws IOException {
final DocumentsWriter.DocWriter doc = consumer.finishDocument();
final DocumentsWriter.DocWriter doc2;
if (nextPerThread != null)
doc2 = nextPerThread.consumer.finishDocument();
else
doc2 = null;
if (doc == null)
return doc2;
else {
doc.setNext(doc2);
return doc;
}
}
// Clear all state
void reset(boolean recyclePostings) {
intPool.reset();
bytePool.reset();
if (primary)
charPool.reset();
if (recyclePostings) {
termsHash.recyclePostings(freePostings, freePostingsCount);
freePostingsCount = 0;
}
}
}

View File

@ -149,4 +149,9 @@ public class RAMOutputStream extends IndexOutput {
public long getFilePointer() {
return currentBufferIndex < 0 ? 0 : bufferStart + bufferPosition;
}
/** Returns byte usage of all buffers. */
public long sizeInBytes() {
return file.numBuffers() * BUFFER_SIZE;
}
}

View File

@ -0,0 +1,109 @@
package org.apache.lucene.util;
/**
* 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.
*/
public final class ArrayUtil {
public static int getNextSize(int targetSize) {
/* This over-allocates proportional to the list size, making room
* for additional growth. The over-allocation is mild, but is
* enough to give linear-time amortized behavior over a long
* sequence of appends() in the presence of a poorly-performing
* system realloc().
* The growth pattern is: 0, 4, 8, 16, 25, 35, 46, 58, 72, 88, ...
*/
return (targetSize >> 3) + (targetSize < 9 ? 3 : 6) + targetSize;
}
public static int getShrinkSize(int currentSize, int targetSize) {
final int newSize = getNextSize(targetSize);
if (newSize < currentSize && currentSize > newSize*2)
return newSize;
else
return currentSize;
}
public static int[] grow(int[] array, int minSize) {
if (array.length < minSize) {
int[] newArray = new int[getNextSize(minSize)];
System.arraycopy(array, 0, newArray, 0, array.length);
return newArray;
} else
return array;
}
public static int[] grow(int[] array) {
return grow(array, 1+array.length);
}
public static int[] shrink(int[] array, int targetSize) {
final int newSize = getShrinkSize(array.length, targetSize);
if (newSize != array.length) {
int[] newArray = new int[newSize];
System.arraycopy(array, 0, newArray, 0, newSize);
return newArray;
} else
return array;
}
public static long[] grow(long[] array, int minSize) {
if (array.length < minSize) {
long[] newArray = new long[getNextSize(minSize)];
System.arraycopy(array, 0, newArray, 0, array.length);
return newArray;
} else
return array;
}
public static long[] grow(long[] array) {
return grow(array, 1+array.length);
}
public static long[] shrink(long[] array, int targetSize) {
final int newSize = getShrinkSize(array.length, targetSize);
if (newSize != array.length) {
long[] newArray = new long[newSize];
System.arraycopy(array, 0, newArray, 0, newSize);
return newArray;
} else
return array;
}
public static byte[] grow(byte[] array, int minSize) {
if (array.length < minSize) {
byte[] newArray = new byte[getNextSize(minSize)];
System.arraycopy(array, 0, newArray, 0, array.length);
return newArray;
} else
return array;
}
public static byte[] grow(byte[] array) {
return grow(array, 1+array.length);
}
public static byte[] shrink(byte[] array, int targetSize) {
final int newSize = getShrinkSize(array.length, targetSize);
if (newSize != array.length) {
byte[] newArray = new byte[newSize];
System.arraycopy(array, 0, newArray, 0, newSize);
return newArray;
} else
return array;
}
}

View File

@ -32,6 +32,7 @@ import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
/**
* A very simple demo used in the API documentation (src/java/overview.html).
@ -58,6 +59,8 @@ public class TestDemo extends LuceneTestCase {
Field.Index.TOKENIZED));
iwriter.addDocument(doc);
iwriter.close();
_TestUtil.checkIndex(directory);
// Now search the index:
IndexSearcher isearcher = new IndexSearcher(directory);

View File

@ -121,9 +121,8 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase
try {
writer.addDocument(doc);
} catch (Throwable t) {
RuntimeException re = new RuntimeException("addDocument failed");
re.initCause(t);
throw re;
t.printStackTrace(System.out);
fail("addDocument failed");
}
}
try {

View File

@ -27,6 +27,7 @@ import org.apache.lucene.document.Field.TermVector;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import java.io.IOException;
import java.io.Reader;
@ -250,6 +251,8 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.addDocument(doc);
writer.close();
_TestUtil.checkIndex(ram);
IndexReader reader = IndexReader.open(ram);
// f1
TermFreqVector tfv1 = reader.getTermFreqVector(0, "f1");

View File

@ -553,7 +553,7 @@ public class TestIndexWriter extends LuceneTestCase
RAMDirectory dir = new RAMDirectory();
IndexWriter writer = new IndexWriter(dir, new StandardAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
char[] chars = new char[16383];
char[] chars = new char[DocumentsWriter.CHAR_BLOCK_SIZE-1];
Arrays.fill(chars, 'x');
Document doc = new Document();
final String bigTerm = new String(chars);
@ -1136,7 +1136,9 @@ public class TestIndexWriter extends LuceneTestCase
dir.resetMaxUsedSizeInBytes();
long startDiskUsage = dir.getMaxUsedSizeInBytes();
writer = new IndexWriter(dir, false, new WhitespaceAnalyzer(), false, IndexWriter.MaxFieldLength.LIMITED);
writer = new IndexWriter(dir, false, new WhitespaceAnalyzer(), false, IndexWriter.MaxFieldLength.LIMITED);
writer.setMaxBufferedDocs(10);
writer.setMergeScheduler(new SerialMergeScheduler());
for(int j=0;j<1470;j++) {
addDocWithIndex(writer, j);
}
@ -1144,6 +1146,9 @@ public class TestIndexWriter extends LuceneTestCase
dir.resetMaxUsedSizeInBytes();
writer.optimize();
writer.close();
IndexReader.open(dir).close();
long endDiskUsage = dir.getMaxUsedSizeInBytes();
// Ending index is 50X as large as starting index; due
@ -1154,7 +1159,7 @@ public class TestIndexWriter extends LuceneTestCase
// System.out.println("start " + startDiskUsage + "; mid " + midDiskUsage + ";end " + endDiskUsage);
assertTrue("writer used to much space while adding documents when autoCommit=false",
midDiskUsage < 100*startDiskUsage);
assertTrue("writer used to much space after close when autoCommit=false",
assertTrue("writer used to much space after close when autoCommit=false endDiskUsage=" + endDiskUsage + " startDiskUsage=" + startDiskUsage,
endDiskUsage < 100*startDiskUsage);
}
@ -1584,6 +1589,7 @@ public class TestIndexWriter extends LuceneTestCase
writer.flush();
writer.addDocument(new Document());
writer.close();
_TestUtil.checkIndex(dir);
IndexReader reader = IndexReader.open(dir);
assertEquals(2, reader.numDocs());
}
@ -1849,7 +1855,7 @@ public class TestIndexWriter extends LuceneTestCase
boolean sawAppend = false;
boolean sawFlush = false;
for (int i = 0; i < trace.length; i++) {
if ("org.apache.lucene.index.DocumentsWriter".equals(trace[i].getClassName()) && "appendPostings".equals(trace[i].getMethodName()))
if ("org.apache.lucene.index.FreqProxTermsWriter".equals(trace[i].getClassName()) && "appendPostings".equals(trace[i].getMethodName()))
sawAppend = true;
if ("doFlush".equals(trace[i].getMethodName()))
sawFlush = true;
@ -2287,6 +2293,7 @@ public class TestIndexWriter extends LuceneTestCase
writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
addCount++;
} catch (IOException ioe) {
//System.out.println(Thread.currentThread().getName() + ": hit exc");
//ioe.printStackTrace(System.out);
if (ioe.getMessage().startsWith("fake disk full at") ||
ioe.getMessage().equals("now failing on purpose")) {
@ -2484,6 +2491,8 @@ public class TestIndexWriter extends LuceneTestCase
"flushDocument".equals(trace[i].getMethodName())) {
if (onlyOnce)
doFail = false;
//System.out.println(Thread.currentThread().getName() + ": now fail");
//new Throwable().printStackTrace(System.out);
throw new IOException("now failing on purpose");
}
}
@ -2663,10 +2672,9 @@ public class TestIndexWriter extends LuceneTestCase
if (doFail) {
StackTraceElement[] trace = new Exception().getStackTrace();
for (int i = 0; i < trace.length; i++) {
if ("writeSegment".equals(trace[i].getMethodName())) {
if ("flush".equals(trace[i].getMethodName()) && "org.apache.lucene.index.DocFieldProcessor".equals(trace[i].getClassName())) {
if (onlyOnce)
doFail = false;
// new RuntimeException().printStackTrace(System.out);
throw new IOException("now failing on purpose");
}
}

View File

@ -0,0 +1,215 @@
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.util.Random;
import java.io.IOException;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.store.MockRAMDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.WhitespaceAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
public class TestIndexWriterExceptions extends LuceneTestCase {
final private static boolean DEBUG = false;
private class IndexerThread extends Thread {
IndexWriter writer;
final Random r = new java.util.Random(47);
Throwable failure;
public IndexerThread(int i, IndexWriter writer) {
setName("Indexer " + i);
this.writer = writer;
}
public void run() {
final Document doc = new Document();
doc.add(new Field("content1", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.TOKENIZED));
doc.add(new Field("content6", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
doc.add(new Field("content2", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.UN_TOKENIZED));
doc.add(new Field("content3", "aaa bbb ccc ddd", Field.Store.YES, Field.Index.NO));
doc.add(new Field("content4", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.TOKENIZED));
doc.add(new Field("content5", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.UN_TOKENIZED));
doc.add(new Field("content7", "aaa bbb ccc ddd", Field.Store.NO, Field.Index.UN_TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
final Field idField = new Field("id", "", Field.Store.YES, Field.Index.UN_TOKENIZED);
doc.add(idField);
final long stopTime = System.currentTimeMillis() + 3000;
while(System.currentTimeMillis() < stopTime) {
doFail.set(this);
final String id = ""+r.nextInt(50);
idField.setValue(id);
Term idTerm = new Term("id", id);
try {
writer.updateDocument(idTerm, doc);
} catch (RuntimeException re) {
if (DEBUG) {
System.out.println("EXC: ");
re.printStackTrace(System.out);
}
try {
_TestUtil.checkIndex(writer.getDirectory());
} catch (IOException ioe) {
System.out.println(Thread.currentThread().getName() + ": unexpected exception1");
ioe.printStackTrace(System.out);
failure = ioe;
break;
}
} catch (Throwable t) {
System.out.println(Thread.currentThread().getName() + ": unexpected exception2");
t.printStackTrace(System.out);
failure = t;
break;
}
doFail.set(null);
// After a possible exception (above) I should be able
// to add a new document without hitting an
// exception:
try {
writer.updateDocument(idTerm, doc);
} catch (Throwable t) {
System.out.println(Thread.currentThread().getName() + ": unexpected exception3");
t.printStackTrace(System.out);
failure = t;
break;
}
}
}
}
ThreadLocal doFail = new ThreadLocal();
public class MockIndexWriter extends IndexWriter {
Random r = new java.util.Random(17);
public MockIndexWriter(Directory dir, Analyzer a, boolean create, MaxFieldLength mfl) throws IOException {
super(dir, false, a, create, mfl);
}
boolean testPoint(String name) {
if (doFail.get() != null && !name.equals("startDoFlush") && r.nextInt(20) == 17) {
if (DEBUG) {
System.out.println(Thread.currentThread().getName() + ": NOW FAIL: " + name);
//new Throwable().printStackTrace(System.out);
}
throw new RuntimeException(Thread.currentThread().getName() + ": intentionally failing at " + name);
}
return true;
}
}
public void testRandomExceptions() throws Throwable {
MockRAMDirectory dir = new MockRAMDirectory();
MockIndexWriter writer = new MockIndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
((ConcurrentMergeScheduler) writer.getMergeScheduler()).setSuppressExceptions();
//writer.setMaxBufferedDocs(10);
writer.setRAMBufferSizeMB(0.1);
if (DEBUG)
writer.setInfoStream(System.out);
IndexerThread thread = new IndexerThread(0, writer);
thread.run();
if (thread.failure != null) {
thread.failure.printStackTrace(System.out);
fail("thread " + thread.getName() + ": hit unexpected failure");
}
writer.commit();
try {
writer.close();
} catch (Throwable t) {
System.out.println("exception during close:");
t.printStackTrace(System.out);
writer.rollback();
}
// Confirm that when doc hits exception partway through tokenization, it's deleted:
IndexReader r2 = IndexReader.open(dir);
final int count = r2.docFreq(new Term("content4", "aaa"));
final int count2 = r2.docFreq(new Term("content4", "ddd"));
assertEquals(count, count2);
r2.close();
_TestUtil.checkIndex(dir);
}
public void testRandomExceptionsThreads() throws Throwable {
MockRAMDirectory dir = new MockRAMDirectory();
MockIndexWriter writer = new MockIndexWriter(dir, new WhitespaceAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
((ConcurrentMergeScheduler) writer.getMergeScheduler()).setSuppressExceptions();
//writer.setMaxBufferedDocs(10);
writer.setRAMBufferSizeMB(0.2);
if (DEBUG)
writer.setInfoStream(System.out);
final int NUM_THREADS = 4;
final IndexerThread[] threads = new IndexerThread[NUM_THREADS];
for(int i=0;i<NUM_THREADS;i++) {
threads[i] = new IndexerThread(i, writer);
threads[i].start();
}
for(int i=0;i<NUM_THREADS;i++)
threads[i].join();
for(int i=0;i<NUM_THREADS;i++)
if (threads[i].failure != null)
fail("thread " + threads[i].getName() + ": hit unexpected failure");
writer.commit();
try {
writer.close();
} catch (Throwable t) {
System.out.println("exception during close:");
t.printStackTrace(System.out);
writer.rollback();
}
// Confirm that when doc hits exception partway through tokenization, it's deleted:
IndexReader r2 = IndexReader.open(dir);
final int count = r2.docFreq(new Term("content4", "aaa"));
final int count2 = r2.docFreq(new Term("content4", "ddd"));
assertEquals(count, count2);
r2.close();
_TestUtil.checkIndex(dir);
}
}

View File

@ -19,11 +19,10 @@ import org.apache.lucene.document.*;
import org.apache.lucene.analysis.*;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.search.TermQuery;
import java.util.*;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.io.IOException;
import junit.framework.TestCase;
@ -53,7 +52,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
}
public void testRandom() throws Exception {
public void testRandom() throws Throwable {
Directory dir1 = new MockRAMDirectory();
// dir1 = FSDirectory.getDirectory("foofoofoo");
Directory dir2 = new MockRAMDirectory();
@ -68,17 +67,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
verifyEquals(dir1, dir2, "id");
}
private void checkIndex(Directory dir) throws IOException {
ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
CheckIndex.out = new PrintStream(bos);
if (!CheckIndex.check(dir, false, null)) {
System.out.println("CheckIndex failed");
System.out.println(bos.toString());
fail("CheckIndex failed");
}
}
public void testMultiConfig() throws Exception {
public void testMultiConfig() throws Throwable {
// test lots of smaller different params together
for (int i=0; i<100; i++) { // increase iterations for better testing
sameFieldOrder=r.nextBoolean();
@ -157,7 +146,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
}
checkIndex(dir);
_TestUtil.checkIndex(dir);
return docs;
}
@ -187,7 +176,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
w.close();
}
public static void verifyEquals(Directory dir1, Directory dir2, String idField) throws IOException {
public static void verifyEquals(Directory dir1, Directory dir2, String idField) throws Throwable {
IndexReader r1 = IndexReader.open(dir1);
IndexReader r2 = IndexReader.open(dir2);
verifyEquals(r1, r2, idField);
@ -196,7 +185,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
public static void verifyEquals(IndexReader r1, IndexReader r2, String idField) throws IOException {
public static void verifyEquals(IndexReader r1, IndexReader r2, String idField) throws Throwable {
assertEquals(r1.numDocs(), r2.numDocs());
boolean hasDeletes = !(r1.maxDoc()==r2.maxDoc() && r1.numDocs()==r1.maxDoc());
@ -236,7 +225,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
try {
// verify term vectors are equivalent
verifyEquals(r1.getTermFreqVectors(id1), r2.getTermFreqVectors(id2));
} catch (java.lang.Error e) {
} catch (Throwable e) {
System.out.println("FAILED id=" + term + " id1=" + id1 + " id2=" + id2);
TermFreqVector[] tv1 = r1.getTermFreqVectors(id1);
System.out.println(" d1=" + tv1);
@ -367,6 +356,8 @@ public class TestStressIndexing2 extends LuceneTestCase {
for(int i=0;i<d1.length;i++) {
TermFreqVector v1 = d1[i];
TermFreqVector v2 = d2[i];
if (v1 == null || v2 == null)
System.out.println("v1=" + v1 + " v2=" + v2 + " i=" + i + " of " + d1.length);
assertEquals(v1.size(), v2.size());
int numTerms = v1.size();
String[] terms1 = v1.getTerms();
@ -572,7 +563,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
indexDoc();
}
}
} catch (Exception e) {
} catch (Throwable e) {
e.printStackTrace();
TestCase.fail(e.toString());
}

View File

@ -93,7 +93,7 @@ public class TestTermVectors extends LuceneTestCase {
public void testTermVectorsFieldOrder() throws IOException {
Directory dir = new MockRAMDirectory();
IndexWriter writer = new IndexWriter(dir, new SimpleAnalyzer(), true);
IndexWriter writer = new IndexWriter(dir, new SimpleAnalyzer(), true, IndexWriter.MaxFieldLength.LIMITED);
Document doc = new Document();
doc.add(new Field("c", "some content here", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
doc.add(new Field("a", "some content here", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));