mirror of
https://github.com/apache/lucene.git
synced 2025-02-20 17:07:09 +00:00
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:
parent
12e4e67403
commit
7198ead131
@ -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/
|
||||
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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];
|
||||
|
@ -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++];
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
}
|
33
src/java/org/apache/lucene/index/DocConsumerPerThread.java
Normal file
33
src/java/org/apache/lucene/index/DocConsumerPerThread.java
Normal 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();
|
||||
}
|
49
src/java/org/apache/lucene/index/DocFieldConsumer.java
Normal file
49
src/java/org/apache/lucene/index/DocFieldConsumer.java
Normal 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;
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
@ -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();
|
||||
}
|
162
src/java/org/apache/lucene/index/DocFieldConsumers.java
Normal file
162
src/java/org/apache/lucene/index/DocFieldConsumers.java
Normal 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
@ -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));
|
||||
}
|
||||
}
|
80
src/java/org/apache/lucene/index/DocFieldProcessor.java
Normal file
80
src/java/org/apache/lucene/index/DocFieldProcessor.java
Normal 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);
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
302
src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
Normal file
302
src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
Normal 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);
|
||||
}
|
||||
}
|
109
src/java/org/apache/lucene/index/DocInverter.java
Normal file
109
src/java/org/apache/lucene/index/DocInverter.java
Normal 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;
|
||||
}
|
||||
}
|
||||
}
|
181
src/java/org/apache/lucene/index/DocInverterPerField.java
Normal file
181
src/java/org/apache/lucene/index/DocInverterPerField.java
Normal 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();
|
||||
}
|
||||
}
|
70
src/java/org/apache/lucene/index/DocInverterPerThread.java
Normal file
70
src/java/org/apache/lucene/index/DocInverterPerThread.java
Normal 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
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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()};
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
366
src/java/org/apache/lucene/index/FreqProxTermsWriter.java
Normal file
366
src/java/org/apache/lucene/index/FreqProxTermsWriter.java
Normal 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;
|
||||
}
|
||||
}
|
@ -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() {}
|
||||
}
|
||||
|
@ -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() {}
|
||||
}
|
@ -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)
|
||||
|
@ -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
|
||||
|
65
src/java/org/apache/lucene/index/IntBlockPool.java
Normal file
65
src/java/org/apache/lucene/index/IntBlockPool.java
Normal 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;
|
||||
}
|
||||
}
|
||||
|
46
src/java/org/apache/lucene/index/InvertedDocConsumer.java
Normal file
46
src/java/org/apache/lucene/index/InvertedDocConsumer.java
Normal 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;
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
@ -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();
|
||||
}
|
29
src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
Normal file
29
src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
Normal 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);
|
||||
}
|
@ -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();
|
||||
}
|
@ -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();
|
||||
}
|
175
src/java/org/apache/lucene/index/NormsWriter.java
Normal file
175
src/java/org/apache/lucene/index/NormsWriter.java
Normal 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) {}
|
||||
}
|
77
src/java/org/apache/lucene/index/NormsWriterPerField.java
Normal file
77
src/java/org/apache/lucene/index/NormsWriterPerField.java
Normal 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++;
|
||||
}
|
||||
}
|
||||
}
|
41
src/java/org/apache/lucene/index/NormsWriterPerThread.java
Normal file
41
src/java/org/apache/lucene/index/NormsWriterPerThread.java
Normal 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;
|
||||
}
|
||||
}
|
@ -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
|
||||
}
|
36
src/java/org/apache/lucene/index/RawPostingList.java
Normal file
36
src/java/org/apache/lucene/index/RawPostingList.java
Normal 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;
|
||||
}
|
190
src/java/org/apache/lucene/index/StoredFieldsWriter.java
Normal file
190
src/java/org/apache/lucene/index/StoredFieldsWriter.java
Normal 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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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() {
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
291
src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
Normal file
291
src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
Normal 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;
|
||||
}
|
||||
}
|
@ -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) {}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
244
src/java/org/apache/lucene/index/TermsHash.java
Normal file
244
src/java/org/apache/lucene/index/TermsHash.java
Normal 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);
|
||||
}
|
||||
}
|
38
src/java/org/apache/lucene/index/TermsHashConsumer.java
Normal file
38
src/java/org/apache/lucene/index/TermsHashConsumer.java
Normal 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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
@ -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();
|
||||
}
|
545
src/java/org/apache/lucene/index/TermsHashPerField.java
Normal file
545
src/java/org/apache/lucene/index/TermsHashPerField.java
Normal 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;
|
||||
}
|
||||
}
|
116
src/java/org/apache/lucene/index/TermsHashPerThread.java
Normal file
116
src/java/org/apache/lucene/index/TermsHashPerThread.java
Normal 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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
109
src/java/org/apache/lucene/util/ArrayUtil.java
Normal file
109
src/java/org/apache/lucene/util/ArrayUtil.java
Normal 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;
|
||||
}
|
||||
}
|
@ -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);
|
||||
|
@ -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 {
|
||||
|
@ -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");
|
||||
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
215
src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
Normal file
215
src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
Normal 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);
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
|
@ -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));
|
||||
|
Loading…
x
Reference in New Issue
Block a user