mirror of https://github.com/apache/lucene.git
LUCENE-2621: fix codec to own the filenames and merging of stored fields
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1199779 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b6bdf5bc34
commit
7877d1ff07
|
@ -83,6 +83,7 @@
|
|||
</or>
|
||||
</condition>
|
||||
<property name="tests.multiplier" value="1" />
|
||||
<property name="tests.codec" value="random" />
|
||||
<property name="tests.postingsformat" value="random" />
|
||||
<property name="tests.locale" value="random" />
|
||||
<property name="tests.timezone" value="random" />
|
||||
|
@ -95,6 +96,8 @@
|
|||
<property name="tests.nightly" value="false" />
|
||||
<property name="tests.cleanthreads.sysprop" value="perMethod"/>
|
||||
<property name="tests.asserts.gracious" value="false"/>
|
||||
<property name="tests.verbose" value="false"/>
|
||||
<property name="tests.infostream" value="${tests.verbose}"/>
|
||||
|
||||
<property name="javac.deprecation" value="off"/>
|
||||
<property name="javac.debug" value="on"/>
|
||||
|
@ -491,8 +494,6 @@
|
|||
<compile-test-macro srcdir="${tests.src.dir}" destdir="${build.dir}/classes/test"
|
||||
test.classpath="test.classpath"/>
|
||||
</target>
|
||||
|
||||
<property name="tests.verbose" value="false"/>
|
||||
|
||||
<macrodef name="compile-test-macro" description="Compiles junit tests.">
|
||||
<attribute name="srcdir"/>
|
||||
|
@ -550,9 +551,13 @@
|
|||
|
||||
<!-- allow tests to control debug prints -->
|
||||
<sysproperty key="tests.verbose" value="${tests.verbose}"/>
|
||||
<!-- even more debugging -->
|
||||
<sysproperty key="tests.infostream" value="${tests.infostream}"/>
|
||||
<!-- directory for formatter lock -->
|
||||
<sysproperty key="tests.lockdir" value="${tests.lockdir}"/>
|
||||
<!-- set the codec tests should run with -->
|
||||
<sysproperty key="tests.codec" value="${tests.codec}"/>
|
||||
<!-- set the postingsformat tests should run with -->
|
||||
<sysproperty key="tests.postingsformat" value="${tests.postingsformat}"/>
|
||||
<!-- set the locale tests should run with -->
|
||||
<sysproperty key="tests.locale" value="${tests.locale}"/>
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.lucene.search.highlight.Encoder;
|
|||
import org.apache.lucene.search.vectorhighlight.FieldFragList.WeightedFragInfo.SubInfo;
|
||||
import org.apache.lucene.search.vectorhighlight.FieldFragList.WeightedFragInfo;
|
||||
import org.apache.lucene.search.vectorhighlight.FieldPhraseList.WeightedPhraseInfo.Toffs;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
||||
public abstract class BaseFragmentsBuilder implements FragmentsBuilder {
|
||||
|
||||
|
@ -124,20 +123,19 @@ public abstract class BaseFragmentsBuilder implements FragmentsBuilder {
|
|||
// according to javadoc, doc.getFields(fieldName) cannot be used with lazy loaded field???
|
||||
final List<Field> fields = new ArrayList<Field>();
|
||||
reader.document(docId, new StoredFieldVisitor() {
|
||||
|
||||
@Override
|
||||
public boolean stringField(FieldInfo fieldInfo, IndexInput in, int numUTF8Bytes) throws IOException {
|
||||
if (fieldInfo.name.equals(fieldName)) {
|
||||
final byte[] b = new byte[numUTF8Bytes];
|
||||
in.readBytes(b, 0, b.length);
|
||||
FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
fields.add(new Field(fieldInfo.name, new String(b, "UTF-8"), ft));
|
||||
} else {
|
||||
in.seek(in.getFilePointer() + numUTF8Bytes);
|
||||
}
|
||||
return false;
|
||||
public void stringField(FieldInfo fieldInfo, String value) throws IOException {
|
||||
FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
fields.add(new Field(fieldInfo.name, value, ft));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status needsField(FieldInfo fieldInfo) throws IOException {
|
||||
return fieldInfo.name.equals(fieldName) ? Status.YES : Status.NO;
|
||||
}
|
||||
});
|
||||
return fields.toArray(new Field[fields.size()]);
|
||||
|
|
|
@ -98,7 +98,6 @@ public class TestIndicesEquals extends LuceneTestCase {
|
|||
// create dir data
|
||||
IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(new Random(seed))).setMergePolicy(newLogMergePolicy()));
|
||||
indexWriter.setInfoStream(VERBOSE ? System.out : null);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: make test index");
|
||||
}
|
||||
|
|
|
@ -1,34 +0,0 @@
|
|||
package org.apache.lucene.document;
|
||||
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Similar to a {@link java.io.FileFilter}, the FieldSelector allows one to make decisions about
|
||||
* what Fields get loaded on a {@link Document} by {@link FieldSelectorVisitor}
|
||||
*
|
||||
**/
|
||||
public interface FieldSelector {
|
||||
|
||||
/**
|
||||
*
|
||||
* @param fieldName the field to accept or reject
|
||||
* @return an instance of {@link FieldSelectorResult}
|
||||
* if the {@link Field} named <code>fieldName</code> should be loaded.
|
||||
*/
|
||||
FieldSelectorResult accept(String fieldName);
|
||||
}
|
|
@ -1,78 +0,0 @@
|
|||
package org.apache.lucene.document;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Provides information about what should be done with this Field
|
||||
*
|
||||
**/
|
||||
import org.apache.lucene.index.IndexableField; // for javadocs
|
||||
|
||||
public enum FieldSelectorResult {
|
||||
|
||||
/**
|
||||
* Load this {@link Field} every time the {@link Document} is loaded, reading in the data as it is encountered.
|
||||
* {@link Document#getField(String)} should not return null.
|
||||
*<p/>
|
||||
* {@link Document#add(IndexableField)} should be called by the Reader.
|
||||
*/
|
||||
LOAD,
|
||||
|
||||
/**
|
||||
* Lazily load this {@link Field}. This means the {@link Field} is valid, but it may not actually contain its data until
|
||||
* invoked. {@link Document#getField(String)} is safe to use and should
|
||||
* return a valid instance of a {@link IndexableField}.
|
||||
*<p/>
|
||||
* {@link Document#add(IndexableField)} should be called by the Reader.
|
||||
*/
|
||||
LAZY_LOAD,
|
||||
|
||||
/**
|
||||
* Do not load the {@link Field}. {@link Document#getField(String)} should return null.
|
||||
* {@link Document#add(IndexableField)} is not called.
|
||||
* <p/>
|
||||
* {@link Document#add(IndexableField)} should not be called by the Reader.
|
||||
*/
|
||||
NO_LOAD,
|
||||
|
||||
/**
|
||||
* Load this field as in the {@link #LOAD} case, but immediately return from {@link Field} loading for the {@link Document}. Thus, the
|
||||
* Document may not have its complete set of Fields. {@link Document#getField(String)} should
|
||||
* both be valid for this {@link Field}
|
||||
* <p/>
|
||||
* {@link Document#add(IndexableField)} should be called by the Reader.
|
||||
*/
|
||||
LOAD_AND_BREAK,
|
||||
|
||||
/** Expert: Load the size of this {@link Field} rather than its value.
|
||||
* Size is measured as number of bytes required to store the field == bytes for a binary or any compressed value, and 2*chars for a String value.
|
||||
* The size is stored as a binary value, represented as an int in a byte[], with the higher order byte first in [0]
|
||||
*/
|
||||
SIZE,
|
||||
|
||||
/** Expert: Like {@link #SIZE} but immediately break from the field loading loop, i.e., stop loading further fields, after the size is loaded */
|
||||
SIZE_AND_BREAK,
|
||||
|
||||
/**
|
||||
* Lazily load this {@link Field}, but do not cache the result. This means the {@link Field} is valid, but it may not actually contain its data until
|
||||
* invoked. {@link Document#getField(String)} is safe to use and should
|
||||
* return a valid instance of a {@link IndexableField}.
|
||||
*<p/>
|
||||
* {@link Document#add(IndexableField)} should be called by the Reader.
|
||||
*/
|
||||
LATENT
|
||||
}
|
|
@ -1,309 +0,0 @@
|
|||
package org.apache.lucene.document;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
|
||||
import org.apache.lucene.document.NumericField.DataType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldReaderException;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** Create this, passing a legacy {@link FieldSelector} to it, then
|
||||
* pass this class to {@link IndexReader#document(int,
|
||||
* StoredFieldVisitor)}, then call {@link #getDocument} to
|
||||
* retrieve the loaded document.
|
||||
|
||||
* <p><b>NOTE</b>: If you use Lazy fields, you should not
|
||||
* access the returned document after the reader has been
|
||||
* closed!
|
||||
*/
|
||||
|
||||
public class FieldSelectorVisitor extends StoredFieldVisitor {
|
||||
|
||||
private final FieldSelector selector;
|
||||
private final Document doc;
|
||||
|
||||
public FieldSelectorVisitor(FieldSelector selector) {
|
||||
this.selector = selector;
|
||||
doc = new Document();
|
||||
}
|
||||
|
||||
public Document getDocument() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean binaryField(FieldInfo fieldInfo, IndexInput in, int numBytes) throws IOException {
|
||||
final FieldSelectorResult accept = selector.accept(fieldInfo.name);
|
||||
switch (accept) {
|
||||
case LOAD:
|
||||
case LOAD_AND_BREAK:
|
||||
final byte[] b = new byte[numBytes];
|
||||
in.readBytes(b, 0, b.length);
|
||||
doc.add(new BinaryField(fieldInfo.name, b));
|
||||
return accept != FieldSelectorResult.LOAD;
|
||||
case LAZY_LOAD:
|
||||
case LATENT:
|
||||
addFieldLazy(in, fieldInfo, true, accept == FieldSelectorResult.LAZY_LOAD, numBytes);
|
||||
return false;
|
||||
case SIZE:
|
||||
case SIZE_AND_BREAK:
|
||||
in.seek(in.getFilePointer() + numBytes);
|
||||
addFieldSize(fieldInfo, numBytes);
|
||||
return accept != FieldSelectorResult.SIZE;
|
||||
default:
|
||||
// skip
|
||||
in.seek(in.getFilePointer() + numBytes);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean stringField(FieldInfo fieldInfo, IndexInput in, int numUTF8Bytes) throws IOException {
|
||||
final FieldSelectorResult accept = selector.accept(fieldInfo.name);
|
||||
switch (accept) {
|
||||
case LOAD:
|
||||
case LOAD_AND_BREAK:
|
||||
final byte[] b = new byte[numUTF8Bytes];
|
||||
in.readBytes(b, 0, b.length);
|
||||
FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
doc.add(new Field(fieldInfo.name, new String(b, "UTF-8"), ft));
|
||||
return accept != FieldSelectorResult.LOAD;
|
||||
case LAZY_LOAD:
|
||||
case LATENT:
|
||||
addFieldLazy(in, fieldInfo, false, accept == FieldSelectorResult.LAZY_LOAD, numUTF8Bytes);
|
||||
return false;
|
||||
case SIZE:
|
||||
case SIZE_AND_BREAK:
|
||||
in.seek(in.getFilePointer() + numUTF8Bytes);
|
||||
addFieldSize(fieldInfo, 2*numUTF8Bytes);
|
||||
return accept != FieldSelectorResult.SIZE;
|
||||
default:
|
||||
// skip
|
||||
in.seek(in.getFilePointer() + numUTF8Bytes);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean intField(FieldInfo fieldInfo, int value) throws IOException {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
ft.setIndexOptions(fieldInfo.indexOptions);
|
||||
return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setIntValue(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean longField(FieldInfo fieldInfo, long value) throws IOException {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
ft.setIndexOptions(fieldInfo.indexOptions);
|
||||
return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setLongValue(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean floatField(FieldInfo fieldInfo, float value) throws IOException {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
ft.setIndexOptions(fieldInfo.indexOptions);
|
||||
return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setFloatValue(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doubleField(FieldInfo fieldInfo, double value) throws IOException {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
ft.setIndexOptions(fieldInfo.indexOptions);
|
||||
return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setDoubleValue(value));
|
||||
}
|
||||
|
||||
private boolean addNumericField(FieldInfo fieldInfo, NumericField f) {
|
||||
doc.add(f);
|
||||
final FieldSelectorResult accept = selector.accept(fieldInfo.name);
|
||||
switch (accept) {
|
||||
case LOAD:
|
||||
return false;
|
||||
case LOAD_AND_BREAK:
|
||||
return true;
|
||||
case LAZY_LOAD:
|
||||
case LATENT:
|
||||
return false;
|
||||
case SIZE:
|
||||
return false;
|
||||
case SIZE_AND_BREAK:
|
||||
return true;
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private void addFieldLazy(IndexInput in, FieldInfo fi, boolean binary, boolean cacheResult, int numBytes) throws IOException {
|
||||
final IndexableField f;
|
||||
final long pointer = in.getFilePointer();
|
||||
// Need to move the pointer ahead by toRead positions
|
||||
in.seek(pointer+numBytes);
|
||||
FieldType ft = new FieldType();
|
||||
ft.setStored(true);
|
||||
ft.setOmitNorms(fi.omitNorms);
|
||||
ft.setIndexOptions(fi.indexOptions);
|
||||
|
||||
if (binary) {
|
||||
f = new LazyField(in, fi.name, ft, numBytes, pointer, binary, cacheResult);
|
||||
} else {
|
||||
ft.setStoreTermVectors(fi.storeTermVector);
|
||||
ft.setStoreTermVectorOffsets(fi.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectorPositions(fi.storePositionWithTermVector);
|
||||
f = new LazyField(in, fi.name, ft, numBytes, pointer, binary, cacheResult);
|
||||
}
|
||||
|
||||
doc.add(f);
|
||||
}
|
||||
|
||||
// Add the size of field as a byte[] containing the 4 bytes of the integer byte size (high order byte first; char = 2 bytes)
|
||||
// Read just the size -- caller must skip the field content to continue reading fields
|
||||
// Return the size in bytes or chars, depending on field type
|
||||
private void addFieldSize(FieldInfo fi, int numBytes) throws IOException {
|
||||
byte[] sizebytes = new byte[4];
|
||||
sizebytes[0] = (byte) (numBytes>>>24);
|
||||
sizebytes[1] = (byte) (numBytes>>>16);
|
||||
sizebytes[2] = (byte) (numBytes>>> 8);
|
||||
sizebytes[3] = (byte) numBytes ;
|
||||
doc.add(new BinaryField(fi.name, sizebytes));
|
||||
}
|
||||
|
||||
/**
|
||||
* A Lazy field implementation that defers loading of fields until asked for, instead of when the Document is
|
||||
* loaded.
|
||||
*/
|
||||
private static class LazyField extends Field {
|
||||
private int toRead;
|
||||
private long pointer;
|
||||
private final boolean cacheResult;
|
||||
private final IndexInput in;
|
||||
private boolean isBinary;
|
||||
|
||||
public LazyField(IndexInput in, String name, FieldType ft, int toRead, long pointer, boolean isBinary, boolean cacheResult) {
|
||||
super(name, ft);
|
||||
this.in = in;
|
||||
this.toRead = toRead;
|
||||
this.pointer = pointer;
|
||||
this.isBinary = isBinary;
|
||||
this.cacheResult = cacheResult;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number numericValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataType numericDataType() {
|
||||
return null;
|
||||
}
|
||||
|
||||
private IndexInput localFieldsStream;
|
||||
|
||||
private IndexInput getFieldStream() {
|
||||
if (localFieldsStream == null) {
|
||||
localFieldsStream = (IndexInput) in.clone();
|
||||
}
|
||||
return localFieldsStream;
|
||||
}
|
||||
|
||||
/** The value of the field as a Reader, or null. If null, the String value,
|
||||
* binary value, or TokenStream value is used. Exactly one of stringValue(),
|
||||
* readerValue(), getBinaryValue(), and tokenStreamValue() must be set. */
|
||||
@Override
|
||||
public Reader readerValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/** The value of the field as a String, or null. If null, the Reader value,
|
||||
* binary value, or TokenStream value is used. Exactly one of stringValue(),
|
||||
* readerValue(), getBinaryValue(), and tokenStreamValue() must be set. */
|
||||
@Override
|
||||
synchronized public String stringValue() {
|
||||
if (isBinary) {
|
||||
return null;
|
||||
} else {
|
||||
if (fieldsData == null) {
|
||||
String result = null;
|
||||
IndexInput localFieldsStream = getFieldStream();
|
||||
try {
|
||||
localFieldsStream.seek(pointer);
|
||||
byte[] bytes = new byte[toRead];
|
||||
localFieldsStream.readBytes(bytes, 0, toRead);
|
||||
result = new String(bytes, "UTF-8");
|
||||
} catch (IOException e) {
|
||||
throw new FieldReaderException(e);
|
||||
}
|
||||
if (cacheResult == true){
|
||||
fieldsData = result;
|
||||
}
|
||||
return result;
|
||||
} else {
|
||||
return (String) fieldsData;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
synchronized public BytesRef binaryValue() {
|
||||
if (isBinary) {
|
||||
if (fieldsData == null) {
|
||||
// Allocate new buffer if result is null or too small
|
||||
final byte[] b = new byte[toRead];
|
||||
|
||||
IndexInput localFieldsStream = getFieldStream();
|
||||
|
||||
// Throw this IOException since IndexReader.document does so anyway, so probably not that big of a change for people
|
||||
// since they are already handling this exception when getting the document
|
||||
try {
|
||||
localFieldsStream.seek(pointer);
|
||||
localFieldsStream.readBytes(b, 0, toRead);
|
||||
} catch (IOException e) {
|
||||
throw new FieldReaderException(e);
|
||||
}
|
||||
|
||||
final BytesRef result = new BytesRef(b);
|
||||
result.length = toRead;
|
||||
if (cacheResult == true){
|
||||
fieldsData = result;
|
||||
}
|
||||
return result;
|
||||
} else {
|
||||
return (BytesRef) fieldsData;
|
||||
}
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,186 @@
|
|||
package org.apache.lucene.document;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.document.NumericField.DataType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.IndexableFieldType;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** Defers actually loading a field's value until you ask
|
||||
* for it. You must not use the returned Field instances
|
||||
* after the provided reader has been closed. */
|
||||
|
||||
public class LazyDocument {
|
||||
private IndexReader reader;
|
||||
private final int docID;
|
||||
|
||||
// null until first field is loaded
|
||||
private Document doc;
|
||||
|
||||
private Map<Integer,Integer> fields = new HashMap<Integer,Integer>();
|
||||
|
||||
public LazyDocument(IndexReader reader, int docID) {
|
||||
this.reader = reader;
|
||||
this.docID = docID;
|
||||
}
|
||||
|
||||
public IndexableField getField(FieldInfo fieldInfo) {
|
||||
Integer num = fields.get(fieldInfo.number);
|
||||
if (num == null) {
|
||||
num = 0;
|
||||
} else {
|
||||
num++;
|
||||
}
|
||||
fields.put(fieldInfo.number, num);
|
||||
|
||||
return new LazyField(fieldInfo.name, num);
|
||||
}
|
||||
|
||||
private synchronized Document getDocument() {
|
||||
if (doc == null) {
|
||||
try {
|
||||
doc = reader.document(docID);
|
||||
} catch (IOException ioe) {
|
||||
throw new IllegalStateException("unable to load document", ioe);
|
||||
}
|
||||
reader = null;
|
||||
}
|
||||
return doc;
|
||||
}
|
||||
|
||||
private class LazyField implements IndexableField {
|
||||
private String name;
|
||||
private int num;
|
||||
|
||||
public LazyField(String name, int num) {
|
||||
this.name = name;
|
||||
this.num = num;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float boost() {
|
||||
return 1.0f;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef binaryValue() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).binaryValue();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].binaryValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String stringValue() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).stringValue();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].stringValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Reader readerValue() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).readerValue();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].readerValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean numeric() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).numeric();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].numeric();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataType numericDataType() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).numericDataType();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].numericDataType();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number numericValue() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).numericValue();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].numericValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexableFieldType fieldType() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).fieldType();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].fieldType();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocFieldValues docValues() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).docValues();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].docValues();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType docValuesType() {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).docValuesType();
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].docValuesType();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TokenStream tokenStream(Analyzer analyzer) throws IOException {
|
||||
if (num == 0) {
|
||||
return getDocument().getField(name).tokenStream(analyzer);
|
||||
} else {
|
||||
return getDocument().getFields(name)[num].tokenStream(analyzer);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,29 +0,0 @@
|
|||
package org.apache.lucene.document;
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
/**
|
||||
* Load the First field and break.
|
||||
* <p/>
|
||||
* See {@link FieldSelectorResult#LOAD_AND_BREAK}
|
||||
*/
|
||||
public class LoadFirstFieldSelector implements FieldSelector {
|
||||
|
||||
public FieldSelectorResult accept(String fieldName) {
|
||||
return FieldSelectorResult.LOAD_AND_BREAK;
|
||||
}
|
||||
}
|
|
@ -1,67 +0,0 @@
|
|||
package org.apache.lucene.document;
|
||||
|
||||
/**
|
||||
* 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.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A {@link FieldSelector} based on a Map of field names to {@link FieldSelectorResult}s
|
||||
*
|
||||
*/
|
||||
public class MapFieldSelector implements FieldSelector {
|
||||
|
||||
Map<String,FieldSelectorResult> fieldSelections;
|
||||
|
||||
/** Create a a MapFieldSelector
|
||||
* @param fieldSelections maps from field names (String) to {@link FieldSelectorResult}s
|
||||
*/
|
||||
public MapFieldSelector(Map<String,FieldSelectorResult> fieldSelections) {
|
||||
this.fieldSelections = fieldSelections;
|
||||
}
|
||||
|
||||
/** Create a a MapFieldSelector
|
||||
* @param fields fields to LOAD. List of Strings. All other fields are NO_LOAD.
|
||||
*/
|
||||
public MapFieldSelector(List<String> fields) {
|
||||
fieldSelections = new HashMap<String,FieldSelectorResult>(fields.size()*5/3);
|
||||
for (final String field : fields)
|
||||
fieldSelections.put(field, FieldSelectorResult.LOAD);
|
||||
}
|
||||
|
||||
/** Create a a MapFieldSelector
|
||||
* @param fields fields to LOAD. All other fields are NO_LOAD.
|
||||
*/
|
||||
public MapFieldSelector(String... fields) {
|
||||
this(Arrays.asList(fields));
|
||||
}
|
||||
|
||||
|
||||
|
||||
/** Load field according to its associated value in fieldSelections
|
||||
* @param field a field name
|
||||
* @return the fieldSelections value that field maps to or NO_LOAD if none.
|
||||
*/
|
||||
public FieldSelectorResult accept(String field) {
|
||||
FieldSelectorResult selection = fieldSelections.get(field);
|
||||
return selection!=null ? selection : FieldSelectorResult.NO_LOAD;
|
||||
}
|
||||
|
||||
}
|
|
@ -1,61 +0,0 @@
|
|||
package org.apache.lucene.document;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
|
||||
/**
|
||||
* Declare what fields to load normally and what fields to load lazily
|
||||
*
|
||||
**/
|
||||
|
||||
public class SetBasedFieldSelector implements FieldSelector {
|
||||
|
||||
private Set<String> fieldsToLoad;
|
||||
private Set<String> lazyFieldsToLoad;
|
||||
|
||||
/**
|
||||
* Pass in the Set of {@link Field} names to load and the Set of {@link Field} names to load lazily. If both are null, the
|
||||
* Document will not have any {@link Field} on it.
|
||||
* @param fieldsToLoad A Set of {@link String} field names to load. May be empty, but not null
|
||||
* @param lazyFieldsToLoad A Set of {@link String} field names to load lazily. May be empty, but not null
|
||||
*/
|
||||
public SetBasedFieldSelector(Set<String> fieldsToLoad, Set<String> lazyFieldsToLoad) {
|
||||
this.fieldsToLoad = fieldsToLoad;
|
||||
this.lazyFieldsToLoad = lazyFieldsToLoad;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate whether to load the field with the given name or not. If the {@link Field#name()} is not in either of the
|
||||
* initializing Sets, then {@link org.apache.lucene.document.FieldSelectorResult#NO_LOAD} is returned. If a Field name
|
||||
* is in both <code>fieldsToLoad</code> and <code>lazyFieldsToLoad</code>, lazy has precedence.
|
||||
*
|
||||
* @param fieldName The {@link Field} name to check
|
||||
* @return The {@link FieldSelectorResult}
|
||||
*/
|
||||
public FieldSelectorResult accept(String fieldName) {
|
||||
FieldSelectorResult result = FieldSelectorResult.NO_LOAD;
|
||||
if (fieldsToLoad.contains(fieldName) == true){
|
||||
result = FieldSelectorResult.LOAD;
|
||||
}
|
||||
if (lazyFieldsToLoad.contains(fieldName) == true){
|
||||
result = FieldSelectorResult.LAZY_LOAD;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -19,9 +19,9 @@ package org.apache.lucene.index.codecs.appending;
|
|||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsFormat;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
|
||||
|
@ -44,7 +44,7 @@ public class AppendingCodec extends Codec {
|
|||
|
||||
private final PostingsFormat postings = new AppendingPostingsFormat();
|
||||
private final SegmentInfosFormat infos = new AppendingSegmentInfosFormat();
|
||||
private final FieldsFormat fields = new DefaultFieldsFormat();
|
||||
private final StoredFieldsFormat fields = new DefaultStoredFieldsFormat();
|
||||
private final DocValuesFormat docValues = new DefaultDocValuesFormat();
|
||||
|
||||
@Override
|
||||
|
@ -53,7 +53,7 @@ public class AppendingCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldsFormat fieldsFormat() {
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fields;
|
||||
}
|
||||
|
||||
|
|
|
@ -1,318 +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.File;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.document.FieldSelectorResult;
|
||||
import org.apache.lucene.document.FieldSelectorVisitor;
|
||||
import org.apache.lucene.document.LoadFirstFieldSelector;
|
||||
import org.apache.lucene.document.SetBasedFieldSelector;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
|
||||
public class TestContribFieldsReader extends LuceneTestCase {
|
||||
private static Directory dir;
|
||||
private static org.apache.lucene.document.Document testDoc = new org.apache.lucene.document.Document();
|
||||
private static FieldInfos fieldInfos = null;
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
fieldInfos = new FieldInfos();
|
||||
DocHelper.setupDoc(testDoc);
|
||||
_TestUtil.add(testDoc, fieldInfos);
|
||||
dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy());
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(false);
|
||||
IndexWriter writer = new IndexWriter(dir, conf);
|
||||
writer.addDocument(testDoc);
|
||||
writer.close();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void afterClass() throws Exception {
|
||||
dir.close();
|
||||
dir = null;
|
||||
fieldInfos = null;
|
||||
testDoc = null;
|
||||
}
|
||||
|
||||
private Document getDocument(IndexReader ir, int docID, FieldSelector selector) throws IOException {
|
||||
final FieldSelectorVisitor visitor = new FieldSelectorVisitor(selector);
|
||||
ir.document(docID, visitor);
|
||||
return visitor.getDocument();
|
||||
}
|
||||
|
||||
public void testLazyFields() throws Exception {
|
||||
assertTrue(dir != null);
|
||||
assertTrue(fieldInfos != null);
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
Set<String> loadFieldNames = new HashSet<String>();
|
||||
loadFieldNames.add(DocHelper.TEXT_FIELD_1_KEY);
|
||||
loadFieldNames.add(DocHelper.TEXT_FIELD_UTF1_KEY);
|
||||
Set<String> lazyFieldNames = new HashSet<String>();
|
||||
//new String[]{DocHelper.LARGE_LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_BINARY_KEY};
|
||||
lazyFieldNames.add(DocHelper.LARGE_LAZY_FIELD_KEY);
|
||||
lazyFieldNames.add(DocHelper.LAZY_FIELD_KEY);
|
||||
lazyFieldNames.add(DocHelper.LAZY_FIELD_BINARY_KEY);
|
||||
lazyFieldNames.add(DocHelper.TEXT_FIELD_UTF2_KEY);
|
||||
SetBasedFieldSelector fieldSelector = new SetBasedFieldSelector(loadFieldNames, lazyFieldNames);
|
||||
Document doc = getDocument(reader, 0, fieldSelector);
|
||||
assertTrue("doc is null and it shouldn't be", doc != null);
|
||||
IndexableField field = doc.getField(DocHelper.LAZY_FIELD_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertTrue("field is not lazy and it should be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
String value = field.stringValue();
|
||||
assertTrue("value is null and it shouldn't be", value != null);
|
||||
assertTrue(value + " is not equal to " + DocHelper.LAZY_FIELD_TEXT, value.equals(DocHelper.LAZY_FIELD_TEXT) == true);
|
||||
assertTrue("calling stringValue() twice should give same reference", field.stringValue() == field.stringValue());
|
||||
|
||||
field = doc.getField(DocHelper.TEXT_FIELD_1_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
field = doc.getField(DocHelper.TEXT_FIELD_UTF1_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF1_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF1_TEXT) == true);
|
||||
|
||||
field = doc.getField(DocHelper.TEXT_FIELD_UTF2_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertTrue("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF2_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF2_TEXT) == true);
|
||||
|
||||
field = doc.getField(DocHelper.LAZY_FIELD_BINARY_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertTrue("stringValue isn't null for lazy binary field", field.stringValue() == null);
|
||||
|
||||
byte [] bytes = field.binaryValue().bytes;
|
||||
assertTrue("bytes is null and it shouldn't be", bytes != null);
|
||||
assertTrue("", DocHelper.LAZY_FIELD_BINARY_BYTES.length == bytes.length);
|
||||
assertTrue("calling binaryValue() twice should give same reference", field.binaryValue().bytes == field.binaryValue().bytes);
|
||||
for (int i = 0; i < bytes.length; i++) {
|
||||
assertTrue("byte[" + i + "] is mismatched", bytes[i] == DocHelper.LAZY_FIELD_BINARY_BYTES[i]);
|
||||
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
|
||||
public void testLatentFields() throws Exception {
|
||||
assertTrue(dir != null);
|
||||
assertTrue(fieldInfos != null);
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
Set<String> loadFieldNames = new HashSet<String>();
|
||||
loadFieldNames.add(DocHelper.TEXT_FIELD_1_KEY);
|
||||
loadFieldNames.add(DocHelper.TEXT_FIELD_UTF1_KEY);
|
||||
Set<String> lazyFieldNames = new HashSet<String>();
|
||||
//new String[]{DocHelper.LARGE_LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_BINARY_KEY};
|
||||
lazyFieldNames.add(DocHelper.LARGE_LAZY_FIELD_KEY);
|
||||
lazyFieldNames.add(DocHelper.LAZY_FIELD_KEY);
|
||||
lazyFieldNames.add(DocHelper.LAZY_FIELD_BINARY_KEY);
|
||||
lazyFieldNames.add(DocHelper.TEXT_FIELD_UTF2_KEY);
|
||||
|
||||
// Use LATENT instead of LAZY
|
||||
SetBasedFieldSelector fieldSelector = new SetBasedFieldSelector(loadFieldNames, lazyFieldNames) {
|
||||
@Override
|
||||
public FieldSelectorResult accept(String fieldName) {
|
||||
final FieldSelectorResult result = super.accept(fieldName);
|
||||
if (result == FieldSelectorResult.LAZY_LOAD) {
|
||||
return FieldSelectorResult.LATENT;
|
||||
} else {
|
||||
return result;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
Document doc = getDocument(reader, 0, fieldSelector);
|
||||
assertTrue("doc is null and it shouldn't be", doc != null);
|
||||
IndexableField field = doc.getField(DocHelper.LAZY_FIELD_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertTrue("field is not lazy and it should be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
String value = field.stringValue();
|
||||
assertTrue("value is null and it shouldn't be", value != null);
|
||||
assertTrue(value + " is not equal to " + DocHelper.LAZY_FIELD_TEXT, value.equals(DocHelper.LAZY_FIELD_TEXT) == true);
|
||||
assertTrue("calling stringValue() twice should give different references", field.stringValue() != field.stringValue());
|
||||
|
||||
field = doc.getField(DocHelper.TEXT_FIELD_1_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
assertTrue("calling stringValue() twice should give same reference", field.stringValue() == field.stringValue());
|
||||
|
||||
field = doc.getField(DocHelper.TEXT_FIELD_UTF1_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF1_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF1_TEXT) == true);
|
||||
assertTrue("calling stringValue() twice should give same reference", field.stringValue() == field.stringValue());
|
||||
|
||||
field = doc.getField(DocHelper.TEXT_FIELD_UTF2_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertTrue("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
|
||||
assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF2_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF2_TEXT) == true);
|
||||
assertTrue("calling stringValue() twice should give different references", field.stringValue() != field.stringValue());
|
||||
|
||||
field = doc.getField(DocHelper.LAZY_FIELD_BINARY_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertTrue("stringValue isn't null for lazy binary field", field.stringValue() == null);
|
||||
assertTrue("calling binaryValue() twice should give different references", field.binaryValue().bytes != field.binaryValue().bytes);
|
||||
|
||||
byte [] bytes = field.binaryValue().bytes;
|
||||
assertTrue("bytes is null and it shouldn't be", bytes != null);
|
||||
assertTrue("", DocHelper.LAZY_FIELD_BINARY_BYTES.length == bytes.length);
|
||||
for (int i = 0; i < bytes.length; i++) {
|
||||
assertTrue("byte[" + i + "] is mismatched", bytes[i] == DocHelper.LAZY_FIELD_BINARY_BYTES[i]);
|
||||
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
|
||||
public void testLoadFirst() throws Exception {
|
||||
assertTrue(dir != null);
|
||||
assertTrue(fieldInfos != null);
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
LoadFirstFieldSelector fieldSelector = new LoadFirstFieldSelector();
|
||||
Document doc = getDocument(reader, 0, fieldSelector);
|
||||
assertTrue("doc is null and it shouldn't be", doc != null);
|
||||
int count = 0;
|
||||
List<IndexableField> l = doc.getFields();
|
||||
for (final IndexableField IndexableField : l ) {
|
||||
Field field = (Field) IndexableField;
|
||||
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
String sv = field.stringValue();
|
||||
assertTrue("sv is null and it shouldn't be", sv != null);
|
||||
count++;
|
||||
}
|
||||
assertTrue(count + " does not equal: " + 1, count == 1);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Not really a test per se, but we should have some way of assessing whether this is worthwhile.
|
||||
* <p/>
|
||||
* Must test using a File based directory
|
||||
*
|
||||
* @throws Exception
|
||||
*/
|
||||
public void testLazyPerformance() throws Exception {
|
||||
String userName = System.getProperty("user.name");
|
||||
File file = _TestUtil.getTempDir("lazyDir" + userName);
|
||||
Directory tmpDir = newFSDirectory(file);
|
||||
assertTrue(tmpDir != null);
|
||||
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE).setMergePolicy(newLogMergePolicy());
|
||||
((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(false);
|
||||
IndexWriter writer = new IndexWriter(tmpDir, conf);
|
||||
writer.addDocument(testDoc);
|
||||
writer.close();
|
||||
|
||||
assertTrue(fieldInfos != null);
|
||||
long lazyTime = 0;
|
||||
long regularTime = 0;
|
||||
int length = 10;
|
||||
Set<String> lazyFieldNames = new HashSet<String>();
|
||||
lazyFieldNames.add(DocHelper.LARGE_LAZY_FIELD_KEY);
|
||||
SetBasedFieldSelector fieldSelector = new SetBasedFieldSelector(Collections. <String> emptySet(), lazyFieldNames);
|
||||
|
||||
for (int i = 0; i < length; i++) {
|
||||
IndexReader reader = IndexReader.open(tmpDir);
|
||||
|
||||
Document doc;
|
||||
doc = reader.document(0);//Load all of them
|
||||
assertTrue("doc is null and it shouldn't be", doc != null);
|
||||
IndexableField field = doc.getField(DocHelper.LARGE_LAZY_FIELD_KEY);
|
||||
assertTrue("field is null and it shouldn't be", field != null);
|
||||
assertFalse("field is lazy", field.getClass().getSimpleName().equals("LazyField"));
|
||||
String value;
|
||||
long start;
|
||||
long finish;
|
||||
start = System.currentTimeMillis();
|
||||
//On my machine this was always 0ms.
|
||||
value = field.stringValue();
|
||||
finish = System.currentTimeMillis();
|
||||
assertTrue("value is null and it shouldn't be", value != null);
|
||||
regularTime += (finish - start);
|
||||
reader.close();
|
||||
reader = null;
|
||||
doc = null;
|
||||
//Hmmm, are we still in cache???
|
||||
System.gc();
|
||||
reader = IndexReader.open(tmpDir);
|
||||
doc = getDocument(reader, 0, fieldSelector);
|
||||
field = doc.getField(DocHelper.LARGE_LAZY_FIELD_KEY);
|
||||
assertTrue("field is not lazy", field.getClass().getSimpleName().equals("LazyField"));
|
||||
start = System.currentTimeMillis();
|
||||
//On my machine this took around 50 - 70ms
|
||||
value = field.stringValue();
|
||||
finish = System.currentTimeMillis();
|
||||
assertTrue("value is null and it shouldn't be", value != null);
|
||||
lazyTime += (finish - start);
|
||||
reader.close();
|
||||
|
||||
}
|
||||
tmpDir.close();
|
||||
if (VERBOSE) {
|
||||
System.out.println("Average Non-lazy time (should be very close to zero): " + regularTime / length + " ms for " + length + " reads");
|
||||
System.out.println("Average Lazy Time (should be greater than zero): " + lazyTime / length + " ms for " + length + " reads");
|
||||
}
|
||||
}
|
||||
|
||||
public void testLoadSize() throws IOException {
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
Document doc;
|
||||
|
||||
doc = getDocument(reader, 0, new FieldSelector(){
|
||||
public FieldSelectorResult accept(String fieldName) {
|
||||
if (fieldName.equals(DocHelper.TEXT_FIELD_1_KEY) ||
|
||||
fieldName.equals(DocHelper.LAZY_FIELD_BINARY_KEY))
|
||||
return FieldSelectorResult.SIZE;
|
||||
else if (fieldName.equals(DocHelper.TEXT_FIELD_3_KEY))
|
||||
return FieldSelectorResult.LOAD;
|
||||
else
|
||||
return FieldSelectorResult.NO_LOAD;
|
||||
}
|
||||
});
|
||||
IndexableField f1 = doc.getField(DocHelper.TEXT_FIELD_1_KEY);
|
||||
IndexableField f3 = doc.getField(DocHelper.TEXT_FIELD_3_KEY);
|
||||
IndexableField fb = doc.getField(DocHelper.LAZY_FIELD_BINARY_KEY);
|
||||
assertTrue(f1.binaryValue()!=null);
|
||||
assertTrue(f3.binaryValue()==null);
|
||||
assertTrue(fb.binaryValue()!=null);
|
||||
assertSizeEquals(2*DocHelper.FIELD_1_TEXT.length(), f1.binaryValue().bytes);
|
||||
assertEquals(DocHelper.FIELD_3_TEXT, f3.stringValue());
|
||||
assertSizeEquals(DocHelper.LAZY_FIELD_BINARY_BYTES.length, fb.binaryValue().bytes);
|
||||
|
||||
reader.close();
|
||||
}
|
||||
|
||||
private void assertSizeEquals(int size, byte[] sizebytes) {
|
||||
assertEquals((byte) (size>>>24), sizebytes[0]);
|
||||
assertEquals((byte) (size>>>16), sizebytes[1]);
|
||||
assertEquals((byte) (size>>> 8), sizebytes[2]);
|
||||
assertEquals((byte) size , sizebytes[3]);
|
||||
}
|
||||
}
|
|
@ -1,169 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.BinaryField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.document.FieldSelectorVisitor;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.SetBasedFieldSelector;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestContribIndexReader extends LuceneTestCase {
|
||||
private Document getDocument(IndexReader ir, int docID, FieldSelector selector) throws IOException {
|
||||
final FieldSelectorVisitor visitor = new FieldSelectorVisitor(selector);
|
||||
ir.document(docID, visitor);
|
||||
return visitor.getDocument();
|
||||
}
|
||||
|
||||
static void addDoc(IndexWriter writer, String value) throws IOException {
|
||||
Document doc = new Document();
|
||||
doc.add(newField("content", value, TextField.TYPE_UNSTORED));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
static void addDocumentWithFields(IndexWriter writer) throws IOException {
|
||||
Document doc = new Document();
|
||||
|
||||
FieldType customType3 = new FieldType();
|
||||
customType3.setStored(true);
|
||||
doc.add(newField("keyword", "test1", StringField.TYPE_STORED));
|
||||
doc.add(newField("text", "test1", TextField.TYPE_STORED));
|
||||
doc.add(newField("unindexed", "test1", customType3));
|
||||
doc.add(new TextField("unstored","test1"));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
|
||||
static void addDocumentWithDifferentFields(IndexWriter writer) throws IOException {
|
||||
Document doc = new Document();
|
||||
|
||||
FieldType customType3 = new FieldType();
|
||||
customType3.setStored(true);
|
||||
doc.add(newField("keyword2", "test1", StringField.TYPE_STORED));
|
||||
doc.add(newField("text2", "test1", TextField.TYPE_STORED));
|
||||
doc.add(newField("unindexed2", "test1", customType3));
|
||||
doc.add(new TextField("unstored2","test1"));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
static void addDocumentWithTermVectorFields(IndexWriter writer) throws IOException {
|
||||
Document doc = new Document();
|
||||
FieldType customType5 = new FieldType(TextField.TYPE_STORED);
|
||||
customType5.setStoreTermVectors(true);
|
||||
FieldType customType6 = new FieldType(TextField.TYPE_STORED);
|
||||
customType6.setStoreTermVectors(true);
|
||||
customType6.setStoreTermVectorOffsets(true);
|
||||
FieldType customType7 = new FieldType(TextField.TYPE_STORED);
|
||||
customType7.setStoreTermVectors(true);
|
||||
customType7.setStoreTermVectorPositions(true);
|
||||
FieldType customType8 = new FieldType(TextField.TYPE_STORED);
|
||||
customType8.setStoreTermVectors(true);
|
||||
customType8.setStoreTermVectorOffsets(true);
|
||||
customType8.setStoreTermVectorPositions(true);
|
||||
doc.add(newField("tvnot","tvnot",TextField.TYPE_STORED));
|
||||
doc.add(newField("termvector","termvector",customType5));
|
||||
doc.add(newField("tvoffset","tvoffset", customType6));
|
||||
doc.add(newField("tvposition","tvposition", customType7));
|
||||
doc.add(newField("tvpositionoffset","tvpositionoffset", customType8));
|
||||
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
public void testBinaryFields() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
byte[] bin = new byte[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9};
|
||||
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
addDoc(writer, "document number " + (i + 1));
|
||||
addDocumentWithFields(writer);
|
||||
addDocumentWithDifferentFields(writer);
|
||||
addDocumentWithTermVectorFields(writer);
|
||||
}
|
||||
writer.close();
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND).setMergePolicy(newLogMergePolicy()));
|
||||
Document doc = new Document();
|
||||
doc.add(new BinaryField("bin1", bin));
|
||||
doc.add(new TextField("junk", "junk text"));
|
||||
writer.addDocument(doc);
|
||||
writer.close();
|
||||
IndexReader reader = IndexReader.open(dir, false);
|
||||
Document doc2 = reader.document(reader.maxDoc() - 1);
|
||||
IndexableField[] fields = doc2.getFields("bin1");
|
||||
assertNotNull(fields);
|
||||
assertEquals(1, fields.length);
|
||||
Field b1 = (Field) fields[0];
|
||||
assertTrue(b1.isBinary());
|
||||
BytesRef bytesRef = b1.binaryValue();
|
||||
assertEquals(bin.length, bytesRef.length);
|
||||
for (int i = 0; i < bin.length; i++) {
|
||||
assertEquals(bin[i], bytesRef.bytes[i + bytesRef.offset]);
|
||||
}
|
||||
Set<String> lazyFields = new HashSet<String>();
|
||||
lazyFields.add("bin1");
|
||||
FieldSelector sel = new SetBasedFieldSelector(new HashSet<String>(), lazyFields);
|
||||
doc2 = getDocument(reader, reader.maxDoc() - 1, sel);
|
||||
fields = doc2.getFields("bin1");
|
||||
assertNotNull(fields);
|
||||
assertEquals(1, fields.length);
|
||||
IndexableField fb1 = fields[0];
|
||||
assertTrue(fb1.binaryValue()!=null);
|
||||
bytesRef = fb1.binaryValue();
|
||||
assertEquals(bin.length, bytesRef.bytes.length);
|
||||
assertEquals(bin.length, bytesRef.length);
|
||||
for (int i = 0; i < bin.length; i++) {
|
||||
assertEquals(bin[i], bytesRef.bytes[i + bytesRef.offset]);
|
||||
}
|
||||
reader.close();
|
||||
// force optimize
|
||||
|
||||
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND).setMergePolicy(newLogMergePolicy()));
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
reader = IndexReader.open(dir, false);
|
||||
doc2 = reader.document(reader.maxDoc() - 1);
|
||||
fields = doc2.getFields("bin1");
|
||||
assertNotNull(fields);
|
||||
assertEquals(1, fields.length);
|
||||
b1 = (Field) fields[0];
|
||||
assertTrue(b1.isBinary());
|
||||
bytesRef = b1.binaryValue();
|
||||
assertEquals(bin.length, bytesRef.length);
|
||||
for (int i = 0; i < bin.length; i++) {
|
||||
assertEquals(bin[i], bytesRef.bytes[i + bytesRef.offset]);
|
||||
}
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
|
@ -1,149 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.document.FieldSelectorVisitor;
|
||||
import org.apache.lucene.document.MapFieldSelector;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestContribParallelReader extends LuceneTestCase {
|
||||
|
||||
private IndexSearcher parallel;
|
||||
private IndexSearcher single;
|
||||
private Directory dir, dir1, dir2;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
single = single(random);
|
||||
parallel = parallel(random);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
single.getIndexReader().close();
|
||||
single.close();
|
||||
parallel.getIndexReader().close();
|
||||
parallel.close();
|
||||
dir.close();
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
// Fields 1-4 indexed together:
|
||||
private IndexSearcher single(Random random) throws IOException {
|
||||
dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
Document d1 = new Document();
|
||||
d1.add(newField("f1", "v1", TextField.TYPE_STORED));
|
||||
d1.add(newField("f2", "v1", TextField.TYPE_STORED));
|
||||
d1.add(newField("f3", "v1", TextField.TYPE_STORED));
|
||||
d1.add(newField("f4", "v1", TextField.TYPE_STORED));
|
||||
w.addDocument(d1);
|
||||
Document d2 = new Document();
|
||||
d2.add(newField("f1", "v2", TextField.TYPE_STORED));
|
||||
d2.add(newField("f2", "v2", TextField.TYPE_STORED));
|
||||
d2.add(newField("f3", "v2", TextField.TYPE_STORED));
|
||||
d2.add(newField("f4", "v2", TextField.TYPE_STORED));
|
||||
w.addDocument(d2);
|
||||
w.close();
|
||||
|
||||
return new IndexSearcher(dir, false);
|
||||
}
|
||||
|
||||
// Fields 1 & 2 in one index, 3 & 4 in other, with ParallelReader:
|
||||
private IndexSearcher parallel(Random random) throws IOException {
|
||||
dir1 = getDir1(random);
|
||||
dir2 = getDir2(random);
|
||||
ParallelReader pr = new ParallelReader();
|
||||
pr.add(IndexReader.open(dir1, false));
|
||||
pr.add(IndexReader.open(dir2, false));
|
||||
return newSearcher(pr);
|
||||
}
|
||||
|
||||
private Document getDocument(IndexReader ir, int docID, FieldSelector selector) throws IOException {
|
||||
final FieldSelectorVisitor visitor = new FieldSelectorVisitor(selector);
|
||||
ir.document(docID, visitor);
|
||||
return visitor.getDocument();
|
||||
}
|
||||
|
||||
public void testDocument() throws IOException {
|
||||
Directory dir1 = getDir1(random);
|
||||
Directory dir2 = getDir2(random);
|
||||
ParallelReader pr = new ParallelReader();
|
||||
pr.add(IndexReader.open(dir1, false));
|
||||
pr.add(IndexReader.open(dir2, false));
|
||||
|
||||
Document doc11 = getDocument(pr, 0, new MapFieldSelector("f1"));
|
||||
Document doc24 = getDocument(pr, 1, new MapFieldSelector(Arrays.asList("f4")));
|
||||
Document doc223 = getDocument(pr, 1, new MapFieldSelector("f2", "f3"));
|
||||
|
||||
assertEquals(1, doc11.getFields().size());
|
||||
assertEquals(1, doc24.getFields().size());
|
||||
assertEquals(2, doc223.getFields().size());
|
||||
|
||||
assertEquals("v1", doc11.get("f1"));
|
||||
assertEquals("v2", doc24.get("f4"));
|
||||
assertEquals("v2", doc223.get("f2"));
|
||||
assertEquals("v2", doc223.get("f3"));
|
||||
pr.close();
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
}
|
||||
|
||||
private Directory getDir1(Random random) throws IOException {
|
||||
Directory dir1 = newDirectory();
|
||||
IndexWriter w1 = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
Document d1 = new Document();
|
||||
d1.add(newField("f1", "v1", TextField.TYPE_STORED));
|
||||
d1.add(newField("f2", "v1", TextField.TYPE_STORED));
|
||||
w1.addDocument(d1);
|
||||
Document d2 = new Document();
|
||||
d2.add(newField("f1", "v2", TextField.TYPE_STORED));
|
||||
d2.add(newField("f2", "v2", TextField.TYPE_STORED));
|
||||
w1.addDocument(d2);
|
||||
w1.close();
|
||||
return dir1;
|
||||
}
|
||||
|
||||
private Directory getDir2(Random random) throws IOException {
|
||||
Directory dir2 = newDirectory();
|
||||
IndexWriter w2 = new IndexWriter(dir2, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
Document d3 = new Document();
|
||||
d3.add(newField("f3", "v1", TextField.TYPE_STORED));
|
||||
d3.add(newField("f4", "v1", TextField.TYPE_STORED));
|
||||
w2.addDocument(d3);
|
||||
Document d4 = new Document();
|
||||
d4.add(newField("f3", "v2", TextField.TYPE_STORED));
|
||||
d4.add(newField("f4", "v2", TextField.TYPE_STORED));
|
||||
w2.addDocument(d4);
|
||||
w2.close();
|
||||
return dir2;
|
||||
}
|
||||
}
|
|
@ -1,144 +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.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldSelector;
|
||||
import org.apache.lucene.document.FieldSelectorResult;
|
||||
import org.apache.lucene.document.FieldSelectorVisitor;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
|
||||
/**
|
||||
* Test demonstrating EOF bug on the last field of the last doc
|
||||
* if other docs have already been accessed.
|
||||
*/
|
||||
public class TestLazyBug extends LuceneTestCase {
|
||||
|
||||
public static int NUM_DOCS = TEST_NIGHTLY ? 500 : 50;
|
||||
public static int NUM_FIELDS = TEST_NIGHTLY ? 100 : 10;
|
||||
|
||||
private static String[] data = new String[] {
|
||||
"now",
|
||||
"is the time",
|
||||
"for all good men",
|
||||
"to come to the aid",
|
||||
"of their country!",
|
||||
"this string contains big chars:{\u0111 \u0222 \u0333 \u1111 \u2222 \u3333}",
|
||||
"this string is a bigger string, mary had a little lamb, little lamb, little lamb!"
|
||||
};
|
||||
|
||||
private static Set<String> dataset = asSet(data);
|
||||
|
||||
private static String MAGIC_FIELD = "f"+(NUM_FIELDS/3);
|
||||
|
||||
private static Directory directory;
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
directory = makeIndex();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void afterClass() throws Exception {
|
||||
directory.close();
|
||||
directory = null;
|
||||
}
|
||||
|
||||
private static FieldSelector SELECTOR = new FieldSelector() {
|
||||
public FieldSelectorResult accept(String f) {
|
||||
if (f.equals(MAGIC_FIELD)) {
|
||||
return FieldSelectorResult.LOAD;
|
||||
}
|
||||
return FieldSelectorResult.LAZY_LOAD;
|
||||
}
|
||||
};
|
||||
|
||||
private static Directory makeIndex() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
try {
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
|
||||
lmp.setUseCompoundFile(false);
|
||||
for (int d = 1; d <= NUM_DOCS; d++) {
|
||||
Document doc = new Document();
|
||||
for (int f = 1; f <= NUM_FIELDS; f++ ) {
|
||||
doc.add(newField("f"+f,
|
||||
data[f % data.length]
|
||||
+ '#' + data[random.nextInt(data.length)],
|
||||
TextField.TYPE_UNSTORED));
|
||||
}
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
writer.close();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return dir;
|
||||
}
|
||||
|
||||
public void doTest(int[] docs) throws Exception {
|
||||
IndexReader reader = IndexReader.open(directory, true);
|
||||
for (int i = 0; i < docs.length; i++) {
|
||||
final FieldSelectorVisitor visitor = new FieldSelectorVisitor(SELECTOR);
|
||||
reader.document(docs[i], visitor);
|
||||
Document d = visitor.getDocument();
|
||||
d.get(MAGIC_FIELD);
|
||||
|
||||
List<IndexableField> fields = d.getFields();
|
||||
for (Iterator<IndexableField> fi = fields.iterator(); fi.hasNext(); ) {
|
||||
IndexableField f=null;
|
||||
try {
|
||||
f = fi.next();
|
||||
String fname = f.name();
|
||||
String fval = f.stringValue();
|
||||
assertNotNull(docs[i]+" FIELD: "+fname, fval);
|
||||
String[] vals = fval.split("#");
|
||||
if (!dataset.contains(vals[0]) || !dataset.contains(vals[1])) {
|
||||
fail("FIELD:"+fname+",VAL:"+fval);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new Exception(docs[i]+" WTF: "+f.name(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
|
||||
public void testLazyWorks() throws Exception {
|
||||
doTest(new int[] { NUM_DOCS-1 });
|
||||
}
|
||||
|
||||
public void testLazyAlsoWorks() throws Exception {
|
||||
doTest(new int[] { NUM_DOCS-1, NUM_DOCS/2 });
|
||||
}
|
||||
|
||||
public void testLazyBroken() throws Exception {
|
||||
doTest(new int[] { NUM_DOCS/2, NUM_DOCS-1 });
|
||||
}
|
||||
}
|
|
@ -42,7 +42,6 @@ public class TestHighFreqTerms extends LuceneTestCase {
|
|||
writer = new IndexWriter(dir, newIndexWriterConfig(random,
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false))
|
||||
.setMaxBufferedDocs(2));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
indexDocs(writer);
|
||||
reader = IndexReader.open(dir, true);
|
||||
_TestUtil.checkIndex(dir);
|
||||
|
|
|
@ -1,157 +0,0 @@
|
|||
package org.apache.lucene.search;
|
||||
/**
|
||||
* 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.LuceneTestCase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.*;
|
||||
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.io.IOException;
|
||||
|
||||
public class TestThreadSafe extends LuceneTestCase {
|
||||
Directory dir1;
|
||||
|
||||
IndexReader ir1;
|
||||
|
||||
class Thr extends Thread {
|
||||
final int iter;
|
||||
final Random rand;
|
||||
final AtomicBoolean failed;
|
||||
|
||||
// pass in random in case we want to make things reproducable
|
||||
public Thr(int iter, Random rand, AtomicBoolean failed) {
|
||||
this.iter = iter;
|
||||
this.rand = rand;
|
||||
this.failed = failed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
for (int i=0; i<iter; i++) {
|
||||
/*** future
|
||||
// pick a random index reader... a shared one, or create your own
|
||||
IndexReader ir;
|
||||
***/
|
||||
|
||||
switch(rand.nextInt(1)) {
|
||||
case 0: loadDoc(ir1); break;
|
||||
}
|
||||
|
||||
}
|
||||
} catch (Throwable th) {
|
||||
failed.set(true);
|
||||
throw new RuntimeException(th);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private Document getDocument(IndexReader ir, int docID, FieldSelector selector) throws IOException {
|
||||
final FieldSelectorVisitor visitor = new FieldSelectorVisitor(selector);
|
||||
ir.document(docID, visitor);
|
||||
return visitor.getDocument();
|
||||
}
|
||||
|
||||
void loadDoc(IndexReader ir) throws IOException {
|
||||
// beware of deleted docs in the future
|
||||
Document doc = getDocument(ir, rand.nextInt(ir.maxDoc()),
|
||||
new FieldSelector() {
|
||||
public FieldSelectorResult accept(String fieldName) {
|
||||
switch(rand.nextInt(2)) {
|
||||
case 0: return FieldSelectorResult.LAZY_LOAD;
|
||||
case 1: return FieldSelectorResult.LOAD;
|
||||
// TODO: add other options
|
||||
default: return FieldSelectorResult.LOAD;
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
for (final IndexableField f : doc ) {
|
||||
validateField(f);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
void validateField(IndexableField f) {
|
||||
String val = f.stringValue();
|
||||
if (!val.startsWith("^") || !val.endsWith("$")) {
|
||||
throw new RuntimeException("Invalid field:" + f.toString() + " val=" +val);
|
||||
}
|
||||
}
|
||||
|
||||
String[] words = "now is the time for all good men to come to the aid of their country".split(" ");
|
||||
|
||||
void buildDir(Directory dir, int nDocs, int maxFields, int maxFieldLen) throws IOException {
|
||||
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE).setMaxBufferedDocs(10));
|
||||
for (int j=0; j<nDocs; j++) {
|
||||
Document d = new Document();
|
||||
int nFields = random.nextInt(maxFields);
|
||||
for (int i=0; i<nFields; i++) {
|
||||
int flen = random.nextInt(maxFieldLen);
|
||||
StringBuilder sb = new StringBuilder("^ ");
|
||||
while (sb.length() < flen) sb.append(' ').append(words[random.nextInt(words.length)]);
|
||||
sb.append(" $");
|
||||
d.add(newField("f"+i, sb.toString(), TextField.TYPE_STORED));
|
||||
}
|
||||
iw.addDocument(d);
|
||||
}
|
||||
iw.close();
|
||||
}
|
||||
|
||||
|
||||
void doTest(int iter, int nThreads) throws Exception {
|
||||
Thr[] tarr = new Thr[nThreads];
|
||||
AtomicBoolean failed = new AtomicBoolean();
|
||||
for (int i=0; i<nThreads; i++) {
|
||||
tarr[i] = new Thr(iter, new Random(random.nextLong()), failed);
|
||||
tarr[i].start();
|
||||
}
|
||||
for (int i=0; i<nThreads; i++) {
|
||||
tarr[i].join();
|
||||
}
|
||||
assertFalse(failed.get());
|
||||
}
|
||||
|
||||
public void testLazyLoadThreadSafety() throws Exception{
|
||||
dir1 = newDirectory();
|
||||
// test w/ field sizes bigger than the buffer of an index input
|
||||
buildDir(dir1, 15, 5, 2000);
|
||||
|
||||
// do many small tests so the thread locals go away inbetween
|
||||
int num = atLeast(10);
|
||||
for (int i = 0; i < num; i++) {
|
||||
ir1 = IndexReader.open(dir1, false);
|
||||
doTest(10,10);
|
||||
ir1.close();
|
||||
}
|
||||
dir1.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -47,7 +47,6 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
|
|||
NRTCachingDirectory cachedDir = new NRTCachingDirectory(dir, 2.0, 25.0);
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
|
||||
RandomIndexWriter w = new RandomIndexWriter(random, cachedDir, conf);
|
||||
w.w.setInfoStream(VERBOSE ? System.out : null);
|
||||
final LineFileDocs docs = new LineFileDocs(random);
|
||||
final int numDocs = _TestUtil.nextInt(random, 100, 400);
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.HashSet;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
||||
/** A {@link StoredFieldVisitor} that creates a {@link
|
||||
* Document} containing all stored fields, or only specific
|
||||
|
@ -57,81 +56,54 @@ public class DocumentStoredFieldVisitor extends StoredFieldVisitor {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean binaryField(FieldInfo fieldInfo, IndexInput in, int numBytes) throws IOException {
|
||||
if (accept(fieldInfo)) {
|
||||
final byte[] b = new byte[numBytes];
|
||||
in.readBytes(b, 0, b.length);
|
||||
doc.add(new BinaryField(fieldInfo.name, b));
|
||||
} else {
|
||||
in.seek(in.getFilePointer() + numBytes);
|
||||
}
|
||||
return false;
|
||||
public void binaryField(FieldInfo fieldInfo, byte[] value, int offset, int length) throws IOException {
|
||||
doc.add(new BinaryField(fieldInfo.name, value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean stringField(FieldInfo fieldInfo, IndexInput in, int numUTF8Bytes) throws IOException {
|
||||
if (accept(fieldInfo)) {
|
||||
final byte[] b = new byte[numUTF8Bytes];
|
||||
in.readBytes(b, 0, b.length);
|
||||
final FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
ft.setIndexOptions(fieldInfo.indexOptions);
|
||||
doc.add(new Field(fieldInfo.name,
|
||||
new String(b, "UTF-8"), ft
|
||||
));
|
||||
} else {
|
||||
in.seek(in.getFilePointer() + numUTF8Bytes);
|
||||
}
|
||||
return false;
|
||||
public void stringField(FieldInfo fieldInfo, String value) throws IOException {
|
||||
final FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
ft.setIndexOptions(fieldInfo.indexOptions);
|
||||
doc.add(new Field(fieldInfo.name, value, ft));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean intField(FieldInfo fieldInfo, int value) {
|
||||
if (accept(fieldInfo)) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setIntValue(value));
|
||||
}
|
||||
return false;
|
||||
public void intField(FieldInfo fieldInfo, int value) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setIntValue(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean longField(FieldInfo fieldInfo, long value) {
|
||||
if (accept(fieldInfo)) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setLongValue(value));
|
||||
}
|
||||
return false;
|
||||
public void longField(FieldInfo fieldInfo, long value) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setLongValue(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean floatField(FieldInfo fieldInfo, float value) {
|
||||
if (accept(fieldInfo)) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setFloatValue(value));
|
||||
}
|
||||
return false;
|
||||
public void floatField(FieldInfo fieldInfo, float value) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setFloatValue(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doubleField(FieldInfo fieldInfo, double value) {
|
||||
if (accept(fieldInfo)) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setDoubleValue(value));
|
||||
}
|
||||
return false;
|
||||
public void doubleField(FieldInfo fieldInfo, double value) {
|
||||
FieldType ft = new FieldType(NumericField.TYPE_STORED);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
doc.add(new NumericField(fieldInfo.name, ft).setDoubleValue(value));
|
||||
}
|
||||
|
||||
private boolean accept(FieldInfo fieldInfo) {
|
||||
return fieldsToAdd == null || fieldsToAdd.contains(fieldInfo.name);
|
||||
@Override
|
||||
public Status needsField(FieldInfo fieldInfo) throws IOException {
|
||||
return fieldsToAdd == null || fieldsToAdd.contains(fieldInfo.name) ? Status.YES : Status.NO;
|
||||
}
|
||||
|
||||
public Document getDocument() {
|
||||
|
|
|
@ -18,10 +18,8 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.Comparator;
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -34,6 +32,7 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.QueryWrapperFilter;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/* Tracks the stream of {@link BufferedDeletes}.
|
||||
* When DocumentsWriterPerThread flushes, its buffered
|
||||
|
@ -63,22 +62,11 @@ class BufferedDeletesStream {
|
|||
// used only by assert
|
||||
private Term lastDeleteTerm;
|
||||
|
||||
private PrintStream infoStream;
|
||||
private final InfoStream infoStream;
|
||||
private final AtomicLong bytesUsed = new AtomicLong();
|
||||
private final AtomicInteger numTerms = new AtomicInteger();
|
||||
private final int messageID;
|
||||
|
||||
public BufferedDeletesStream(int messageID) {
|
||||
this.messageID = messageID;
|
||||
}
|
||||
|
||||
private synchronized void message(String message) {
|
||||
if (infoStream != null) {
|
||||
infoStream.println("BD " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void setInfoStream(PrintStream infoStream) {
|
||||
public BufferedDeletesStream(InfoStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
}
|
||||
|
||||
|
@ -101,7 +89,7 @@ class BufferedDeletesStream {
|
|||
numTerms.addAndGet(packet.numTermDeletes);
|
||||
bytesUsed.addAndGet(packet.bytesUsed);
|
||||
if (infoStream != null) {
|
||||
message("push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + deletes.size() + " totBytesUsed=" + bytesUsed.get());
|
||||
infoStream.message("BD", "push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + deletes.size() + " totBytesUsed=" + bytesUsed.get());
|
||||
}
|
||||
assert checkDeleteStats();
|
||||
return packet.delGen();
|
||||
|
@ -171,12 +159,14 @@ class BufferedDeletesStream {
|
|||
assert checkDeleteStats();
|
||||
|
||||
if (!any()) {
|
||||
message("applyDeletes: no deletes; skipping");
|
||||
if (infoStream != null) {
|
||||
infoStream.message("BD", "applyDeletes: no deletes; skipping");
|
||||
}
|
||||
return new ApplyDeletesResult(false, nextGen++, null);
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("applyDeletes: infos=" + infos + " packetCount=" + deletes.size());
|
||||
infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + deletes.size());
|
||||
}
|
||||
|
||||
List<SegmentInfo> infos2 = new ArrayList<SegmentInfo>();
|
||||
|
@ -248,7 +238,7 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("seg=" + info + " segGen=" + segGen + " segDeletes=[" + packet + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " segDeletes=[" + packet + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
}
|
||||
|
||||
if (coalescedDeletes == null) {
|
||||
|
@ -290,7 +280,7 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
}
|
||||
}
|
||||
info.setBufferedDeletesGen(nextGen);
|
||||
|
@ -301,7 +291,7 @@ class BufferedDeletesStream {
|
|||
|
||||
assert checkDeleteStats();
|
||||
if (infoStream != null) {
|
||||
message("applyDeletes took " + (System.currentTimeMillis()-t0) + " msec");
|
||||
infoStream.message("BD", "applyDeletes took " + (System.currentTimeMillis()-t0) + " msec");
|
||||
}
|
||||
// assert infos != segmentInfos || !any() : "infos=" + infos + " segmentInfos=" + segmentInfos + " any=" + any;
|
||||
|
||||
|
@ -324,7 +314,7 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + deletes.size());
|
||||
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + deletes.size());
|
||||
}
|
||||
final int limit = deletes.size();
|
||||
for(int delIDX=0;delIDX<limit;delIDX++) {
|
||||
|
@ -344,7 +334,7 @@ class BufferedDeletesStream {
|
|||
private synchronized void prune(int count) {
|
||||
if (count > 0) {
|
||||
if (infoStream != null) {
|
||||
message("pruneDeletes: prune " + count + " packets; " + (deletes.size() - count) + " packets remain");
|
||||
infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (deletes.size() - count) + " packets remain");
|
||||
}
|
||||
for(int delIDX=0;delIDX<count;delIDX++) {
|
||||
final FrozenBufferedDeletes packet = deletes.get(delIDX);
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
|
||||
|
@ -225,7 +226,10 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
* called and returned true.
|
||||
*/
|
||||
protected void message(String message) {
|
||||
writer.message("CMS: " + message);
|
||||
final InfoStream infoStream = writer.infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("CMS", message);
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void initMergeThreadPriority() {
|
||||
|
|
|
@ -44,7 +44,7 @@ import org.apache.lucene.util.IOUtils;
|
|||
final class DocFieldProcessor extends DocConsumer {
|
||||
|
||||
final DocFieldConsumer consumer;
|
||||
final StoredFieldsWriter fieldsWriter;
|
||||
final StoredFieldsConsumer fieldsWriter;
|
||||
|
||||
// Holds all fields seen in current doc
|
||||
DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1];
|
||||
|
@ -62,7 +62,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
|
||||
this.docState = docWriter.docState;
|
||||
this.consumer = consumer;
|
||||
fieldsWriter = new StoredFieldsWriter(docWriter);
|
||||
fieldsWriter = new StoredFieldsConsumer(docWriter);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/**
|
||||
* This class accepts multiple added documents and directly
|
||||
|
@ -44,7 +45,7 @@ import org.apache.lucene.store.Directory;
|
|||
* Each added document is passed to the {@link DocConsumer},
|
||||
* which in turn processes the document and interacts with
|
||||
* other consumers in the indexing chain. Certain
|
||||
* consumers, like {@link StoredFieldsWriter} and {@link
|
||||
* consumers, like {@link StoredFieldsConsumer} and {@link
|
||||
* TermVectorsTermsWriter}, digest a document and
|
||||
* immediately write bytes to the "doc store" files (ie,
|
||||
* they do not consume RAM per document, except while they
|
||||
|
@ -106,7 +107,7 @@ final class DocumentsWriter {
|
|||
|
||||
private volatile boolean closed;
|
||||
|
||||
PrintStream infoStream;
|
||||
final InfoStream infoStream;
|
||||
SimilarityProvider similarityProvider;
|
||||
|
||||
List<String> newFiles;
|
||||
|
@ -140,6 +141,7 @@ final class DocumentsWriter {
|
|||
this.codec = codec;
|
||||
this.directory = directory;
|
||||
this.indexWriter = writer;
|
||||
this.infoStream = config.getInfoStream();
|
||||
this.similarityProvider = config.getSimilarityProvider();
|
||||
this.perThreadPool = config.getIndexerThreadPool();
|
||||
this.chain = config.getIndexingChain();
|
||||
|
@ -187,14 +189,6 @@ final class DocumentsWriter {
|
|||
indexWriter.flushCount.incrementAndGet();
|
||||
}
|
||||
|
||||
synchronized void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
final Iterator<ThreadState> it = perThreadPool.getAllPerThreadsIterator();
|
||||
while (it.hasNext()) {
|
||||
it.next().perThread.setInfoStream(infoStream);
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns how many docs are currently buffered in RAM. */
|
||||
int getNumDocs() {
|
||||
return numDocsInRAM.get();
|
||||
|
@ -204,14 +198,6 @@ final class DocumentsWriter {
|
|||
return abortedFiles;
|
||||
}
|
||||
|
||||
// returns boolean for asserts
|
||||
boolean message(String message) {
|
||||
if (infoStream != null) {
|
||||
indexWriter.message("DW: " + message);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private void ensureOpen() throws AlreadyClosedException {
|
||||
if (closed) {
|
||||
throw new AlreadyClosedException("this IndexWriter is closed");
|
||||
|
@ -231,7 +217,7 @@ final class DocumentsWriter {
|
|||
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
message("DW: abort");
|
||||
infoStream.message("DW", "abort");
|
||||
}
|
||||
|
||||
final Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
|
||||
|
@ -258,14 +244,14 @@ final class DocumentsWriter {
|
|||
success = true;
|
||||
} finally {
|
||||
if (infoStream != null) {
|
||||
message("docWriter: done abort; abortedFiles=" + abortedFiles + " success=" + success);
|
||||
infoStream.message("DW", "done abort; abortedFiles=" + abortedFiles + " success=" + success);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
boolean anyChanges() {
|
||||
if (infoStream != null) {
|
||||
message("docWriter: anyChanges? numDocsInRam=" + numDocsInRAM.get()
|
||||
infoStream.message("DW", "anyChanges? numDocsInRam=" + numDocsInRAM.get()
|
||||
+ " deletes=" + anyDeletions() + " hasTickets:"
|
||||
+ ticketQueue.hasTickets() + " pendingChangesInFullFlush: "
|
||||
+ pendingChangesInCurrentFullFlush);
|
||||
|
@ -304,7 +290,7 @@ final class DocumentsWriter {
|
|||
if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
|
||||
// Help out flushing any queued DWPTs so we can un-stall:
|
||||
if (infoStream != null) {
|
||||
message("docWriter: DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
|
||||
infoStream.message("DW", "DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
|
||||
}
|
||||
do {
|
||||
// Try pick up pending threads here if possible
|
||||
|
@ -315,14 +301,14 @@ final class DocumentsWriter {
|
|||
}
|
||||
|
||||
if (infoStream != null && flushControl.anyStalledThreads()) {
|
||||
message("WARNING DocumentsWriter has stalled threads; waiting");
|
||||
infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting");
|
||||
}
|
||||
|
||||
flushControl.waitIfStalled(); // block if stalled
|
||||
} while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try help flushing
|
||||
|
||||
if (infoStream != null) {
|
||||
message("continue indexing after helpling out flushing DocumentsWriter is healthy");
|
||||
infoStream.message("DW", "continue indexing after helping out flushing DocumentsWriter is healthy");
|
||||
}
|
||||
}
|
||||
return maybeMerge;
|
||||
|
@ -481,7 +467,7 @@ final class DocumentsWriter {
|
|||
if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
|
||||
flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
|
||||
if (infoStream != null) {
|
||||
message("force apply deletes bytesUsed=" + flushControl.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*ramBufferSizeMB));
|
||||
infoStream.message("DW", "force apply deletes bytesUsed=" + flushControl.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*ramBufferSizeMB));
|
||||
}
|
||||
applyAllDeletes(deleteQueue);
|
||||
}
|
||||
|
@ -515,7 +501,7 @@ final class DocumentsWriter {
|
|||
if (bufferedDeletes != null && bufferedDeletes.any()) {
|
||||
indexWriter.publishFrozenDeletes(bufferedDeletes);
|
||||
if (infoStream != null) {
|
||||
message("flush: push buffered deletes: " + bufferedDeletes);
|
||||
infoStream.message("DW", "flush: push buffered deletes: " + bufferedDeletes);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
|
@ -542,14 +528,14 @@ final class DocumentsWriter {
|
|||
final SegmentInfo segInfo = indexWriter.prepareFlushedSegment(newSegment);
|
||||
final BufferedDeletes deletes = newSegment.segmentDeletes;
|
||||
if (infoStream != null) {
|
||||
message(Thread.currentThread().getName() + ": publishFlushedSegment seg-private deletes=" + deletes);
|
||||
infoStream.message("DW", Thread.currentThread().getName() + ": publishFlushedSegment seg-private deletes=" + deletes);
|
||||
}
|
||||
FrozenBufferedDeletes packet = null;
|
||||
if (deletes != null && deletes.any()) {
|
||||
// Segment private delete
|
||||
packet = new FrozenBufferedDeletes(deletes, true);
|
||||
if (infoStream != null) {
|
||||
message("flush: push buffered seg private deletes: " + packet);
|
||||
infoStream.message("DW", "flush: push buffered seg private deletes: " + packet);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -575,7 +561,7 @@ final class DocumentsWriter {
|
|||
throws IOException {
|
||||
final DocumentsWriterDeleteQueue flushingDeleteQueue;
|
||||
if (infoStream != null) {
|
||||
message(Thread.currentThread().getName() + " startFullFlush");
|
||||
infoStream.message("DW", Thread.currentThread().getName() + " startFullFlush");
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
|
@ -601,7 +587,7 @@ final class DocumentsWriter {
|
|||
flushControl.waitForFlush();
|
||||
if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
|
||||
if (infoStream != null) {
|
||||
message(Thread.currentThread().getName() + ": flush naked frozen global deletes");
|
||||
infoStream.message("DW", Thread.currentThread().getName() + ": flush naked frozen global deletes");
|
||||
}
|
||||
synchronized (ticketQueue) {
|
||||
ticketQueue.incTicketCount(); // first inc the ticket count - freeze opens a window for #anyChanges to fail
|
||||
|
@ -619,7 +605,7 @@ final class DocumentsWriter {
|
|||
final void finishFullFlush(boolean success) {
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
message(Thread.currentThread().getName() + " finishFullFlush success=" + success);
|
||||
infoStream.message("DW", Thread.currentThread().getName() + " finishFullFlush success=" + success);
|
||||
}
|
||||
assert setFlushingDeleteQueue(null);
|
||||
if (success) {
|
||||
|
|
|
@ -454,7 +454,7 @@ public final class DocumentsWriterFlushControl {
|
|||
|
||||
void addFlushableState(ThreadState perThread) {
|
||||
if (documentsWriter.infoStream != null) {
|
||||
documentsWriter.message("FC: " + Thread.currentThread().getName() + ": addFlushableState " + perThread.perThread);
|
||||
documentsWriter.infoStream.message("DWFC", Thread.currentThread().getName() + ": addFlushableState " + perThread.perThread);
|
||||
}
|
||||
final DocumentsWriterPerThread dwpt = perThread.perThread;
|
||||
assert perThread.isHeldByCurrentThread();
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.lucene.util.BitVector;
|
|||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.ByteBlockPool.Allocator;
|
||||
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
public class DocumentsWriterPerThread {
|
||||
|
@ -131,7 +132,7 @@ public class DocumentsWriterPerThread {
|
|||
hasAborted = aborting = true;
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
message("now abort");
|
||||
infoStream.message("DWPT", "now abort");
|
||||
}
|
||||
try {
|
||||
consumer.abort();
|
||||
|
@ -146,7 +147,7 @@ public class DocumentsWriterPerThread {
|
|||
} finally {
|
||||
aborting = false;
|
||||
if (infoStream != null) {
|
||||
message("done abort");
|
||||
infoStream.message("DWPT", "done abort");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -167,7 +168,7 @@ public class DocumentsWriterPerThread {
|
|||
boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting
|
||||
|
||||
private FieldInfos fieldInfos;
|
||||
private PrintStream infoStream;
|
||||
private final InfoStream infoStream;
|
||||
private int numDocsInRAM;
|
||||
private int flushedDocCount;
|
||||
DocumentsWriterDeleteQueue deleteQueue;
|
||||
|
@ -225,13 +226,13 @@ public class DocumentsWriterPerThread {
|
|||
// this call is synchronized on IndexWriter.segmentInfos
|
||||
segment = writer.newSegmentName();
|
||||
assert numDocsInRAM == 0;
|
||||
if (INFO_VERBOSE) {
|
||||
message(Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
|
||||
}
|
||||
|
||||
}
|
||||
if (INFO_VERBOSE) {
|
||||
message(Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
|
||||
}
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -273,12 +274,12 @@ public class DocumentsWriterPerThread {
|
|||
// this call is synchronized on IndexWriter.segmentInfos
|
||||
segment = writer.newSegmentName();
|
||||
assert numDocsInRAM == 0;
|
||||
if (INFO_VERBOSE) {
|
||||
message(Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);
|
||||
}
|
||||
}
|
||||
if (INFO_VERBOSE) {
|
||||
message(Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
|
||||
if (INFO_VERBOSE && infoStream != null) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
|
||||
}
|
||||
int docCount = 0;
|
||||
try {
|
||||
|
@ -459,12 +460,12 @@ public class DocumentsWriterPerThread {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM);
|
||||
infoStream.message("DWPT", "flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM);
|
||||
}
|
||||
|
||||
if (aborting) {
|
||||
if (infoStream != null) {
|
||||
message("flush: skip because aborting is set");
|
||||
infoStream.message("DWPT", "flush: skip because aborting is set");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -476,10 +477,10 @@ public class DocumentsWriterPerThread {
|
|||
pendingDeletes.terms.clear();
|
||||
final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
|
||||
if (infoStream != null) {
|
||||
message("new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
|
||||
message("new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
|
||||
message("flushedFiles=" + newSegment.files());
|
||||
message("flushed codec=" + newSegment.getCodec());
|
||||
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
|
||||
infoStream.message("DWPT", "new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
|
||||
infoStream.message("DWPT", "flushedFiles=" + newSegment.files());
|
||||
infoStream.message("DWPT", "flushed codec=" + newSegment.getCodec());
|
||||
}
|
||||
flushedDocCount += flushState.numDocs;
|
||||
|
||||
|
@ -495,7 +496,7 @@ public class DocumentsWriterPerThread {
|
|||
if (infoStream != null) {
|
||||
final double newSegmentSizeNoStore = newSegment.sizeInBytes(false)/1024./1024.;
|
||||
final double newSegmentSize = newSegment.sizeInBytes(true)/1024./1024.;
|
||||
message("flushed: segment=" + newSegment +
|
||||
infoStream.message("DWPT", "flushed: segment=" + newSegment +
|
||||
" ramUsed=" + nf.format(startMBUsed) + " MB" +
|
||||
" newFlushedSize=" + nf.format(newSegmentSize) + " MB" +
|
||||
" (" + nf.format(newSegmentSizeNoStore) + " MB w/o doc stores)" +
|
||||
|
@ -527,10 +528,6 @@ public class DocumentsWriterPerThread {
|
|||
return bytesUsed.get() + pendingDeletes.bytesUsed.get();
|
||||
}
|
||||
|
||||
void message(String message) {
|
||||
writer.message("DWPT: " + message);
|
||||
}
|
||||
|
||||
/* Initial chunks size of the shared byte[] blocks used to
|
||||
store postings data */
|
||||
final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK;
|
||||
|
@ -561,11 +558,6 @@ public class DocumentsWriterPerThread {
|
|||
return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, segmentSuffix, IOContext.DEFAULT);
|
||||
}
|
||||
|
||||
void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
docState.infoStream = infoStream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DocumentsWriterPerThread [pendingDeletes=" + pendingDeletes
|
||||
|
|
|
@ -64,7 +64,7 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||
control.getDeleteBytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
|
||||
control.setApplyAllDeletes();
|
||||
if (writer.infoStream != null) {
|
||||
writer.message("force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*indexWriterConfig.getRAMBufferSizeMB()));
|
||||
writer.infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*indexWriterConfig.getRAMBufferSizeMB()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -82,7 +82,7 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||
if (totalRam >= limit) {
|
||||
final DocumentsWriter writer = this.writer.get();
|
||||
if (writer.infoStream != null) {
|
||||
writer.message("flush: activeBytes=" + control.activeBytes() + " deleteBytes=" + control.getDeleteBytesUsed() + " vs limit=" + limit);
|
||||
writer.infoStream.message("FP", "flush: activeBytes=" + control.activeBytes() + " deleteBytes=" + control.getDeleteBytesUsed() + " vs limit=" + limit);
|
||||
}
|
||||
markLargestWriterPending(control, state, totalRam);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.util.Iterator;
|
|||
|
||||
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
|
||||
/**
|
||||
|
@ -123,9 +124,16 @@ public abstract class FlushPolicy {
|
|||
}
|
||||
}
|
||||
}
|
||||
assert writer.get().message(
|
||||
"set largest ram consuming thread pending on lower watermark");
|
||||
assert assertMessage("set largest ram consuming thread pending on lower watermark");
|
||||
return maxRamUsingThreadState;
|
||||
}
|
||||
|
||||
private boolean assertMessage(String s) {
|
||||
InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("FP", s);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.Map;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.NoSuchDirectoryException;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/*
|
||||
* This class keeps track of each SegmentInfos instance that
|
||||
|
@ -95,7 +96,7 @@ final class IndexFileDeleter {
|
|||
/* Commits that the IndexDeletionPolicy have decided to delete: */
|
||||
private List<CommitPoint> commitsToDelete = new ArrayList<CommitPoint>();
|
||||
|
||||
private PrintStream infoStream;
|
||||
private final InfoStream infoStream;
|
||||
private Directory directory;
|
||||
private IndexDeletionPolicy policy;
|
||||
|
||||
|
@ -109,17 +110,6 @@ final class IndexFileDeleter {
|
|||
// Used only for assert
|
||||
private final IndexWriter writer;
|
||||
|
||||
void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
if (infoStream != null) {
|
||||
message("setInfoStream deletionPolicy=" + policy);
|
||||
}
|
||||
}
|
||||
|
||||
private void message(String message) {
|
||||
infoStream.println("IFD [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
}
|
||||
|
||||
// called only from assert
|
||||
private boolean locked() {
|
||||
return writer == null || Thread.holdsLock(writer);
|
||||
|
@ -134,14 +124,14 @@ final class IndexFileDeleter {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
|
||||
PrintStream infoStream, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
InfoStream infoStream, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
this.infoStream = infoStream;
|
||||
this.writer = writer;
|
||||
|
||||
final String currentSegmentsFile = segmentInfos.getCurrentSegmentFileName();
|
||||
|
||||
if (infoStream != null) {
|
||||
message("init: current segments file is \"" + currentSegmentsFile + "\"; deletionPolicy=" + policy);
|
||||
infoStream.message("IFD", "init: current segments file is \"" + currentSegmentsFile + "\"; deletionPolicy=" + policy);
|
||||
}
|
||||
|
||||
this.policy = policy;
|
||||
|
@ -173,7 +163,7 @@ final class IndexFileDeleter {
|
|||
// it's valid (<= the max gen). Load it, then
|
||||
// incref all files it refers to:
|
||||
if (infoStream != null) {
|
||||
message("init: load commit \"" + fileName + "\"");
|
||||
infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
|
||||
}
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
try {
|
||||
|
@ -187,7 +177,7 @@ final class IndexFileDeleter {
|
|||
// doesn't. So, we catch this and handle it
|
||||
// as if the file does not exist
|
||||
if (infoStream != null) {
|
||||
message("init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
|
||||
infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
|
||||
}
|
||||
sis = null;
|
||||
} catch (IOException e) {
|
||||
|
@ -218,7 +208,7 @@ final class IndexFileDeleter {
|
|||
refresh(segmentInfo.name);
|
||||
sis = null;
|
||||
if (infoStream != null) {
|
||||
message("init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
|
||||
infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -255,7 +245,7 @@ final class IndexFileDeleter {
|
|||
throw new CorruptIndexException("failed to locate current segments_N file");
|
||||
}
|
||||
if (infoStream != null) {
|
||||
message("forced open of current segments file " + segmentInfos.getCurrentSegmentFileName());
|
||||
infoStream.message("IFD", "forced open of current segments file " + segmentInfos.getCurrentSegmentFileName());
|
||||
}
|
||||
currentCommitPoint = new CommitPoint(commitsToDelete, directory, sis);
|
||||
commits.add(currentCommitPoint);
|
||||
|
@ -273,7 +263,7 @@ final class IndexFileDeleter {
|
|||
final String fileName = entry.getKey();
|
||||
if (0 == rc.count) {
|
||||
if (infoStream != null) {
|
||||
message("init: removing unreferenced file \"" + fileName + "\"");
|
||||
infoStream.message("IFD", "init: removing unreferenced file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
}
|
||||
|
@ -313,7 +303,7 @@ final class IndexFileDeleter {
|
|||
for(int i=0;i<size;i++) {
|
||||
CommitPoint commit = commitsToDelete.get(i);
|
||||
if (infoStream != null) {
|
||||
message("deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
infoStream.message("IFD", "deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
for (final String file : commit.files) {
|
||||
decRef(file);
|
||||
|
@ -373,7 +363,7 @@ final class IndexFileDeleter {
|
|||
!fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
|
||||
// Unreferenced file, so remove it
|
||||
if (infoStream != null) {
|
||||
message("refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
|
||||
infoStream.message("IFD", "refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
}
|
||||
|
@ -415,7 +405,7 @@ final class IndexFileDeleter {
|
|||
void revisitPolicy() throws IOException {
|
||||
assert locked();
|
||||
if (infoStream != null) {
|
||||
message("now revisitPolicy");
|
||||
infoStream.message("IFD", "now revisitPolicy");
|
||||
}
|
||||
|
||||
if (commits.size() > 0) {
|
||||
|
@ -432,7 +422,7 @@ final class IndexFileDeleter {
|
|||
int size = oldDeletable.size();
|
||||
for(int i=0;i<size;i++) {
|
||||
if (infoStream != null) {
|
||||
message("delete pending file " + oldDeletable.get(i));
|
||||
infoStream.message("IFD", "delete pending file " + oldDeletable.get(i));
|
||||
}
|
||||
deleteFile(oldDeletable.get(i));
|
||||
}
|
||||
|
@ -463,7 +453,7 @@ final class IndexFileDeleter {
|
|||
assert locked();
|
||||
|
||||
if (infoStream != null) {
|
||||
message("now checkpoint \"" + segmentInfos.toString(directory) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
infoStream.message("IFD", "now checkpoint \"" + segmentInfos.toString(directory) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
}
|
||||
|
||||
// Try again now to delete any previously un-deletable
|
||||
|
@ -514,7 +504,7 @@ final class IndexFileDeleter {
|
|||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream != null && VERBOSE_REF_COUNTS) {
|
||||
message(" IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
|
||||
infoStream.message("IFD", " IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
|
||||
}
|
||||
rc.IncRef();
|
||||
}
|
||||
|
@ -530,7 +520,7 @@ final class IndexFileDeleter {
|
|||
assert locked();
|
||||
RefCount rc = getRefCount(fileName);
|
||||
if (infoStream != null && VERBOSE_REF_COUNTS) {
|
||||
message(" DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
|
||||
infoStream.message("IFD", " DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
|
||||
}
|
||||
if (0 == rc.DecRef()) {
|
||||
// This file is no longer referenced by any past
|
||||
|
@ -582,7 +572,7 @@ final class IndexFileDeleter {
|
|||
for (final String fileName: files) {
|
||||
if (!refCounts.containsKey(fileName)) {
|
||||
if (infoStream != null) {
|
||||
message("delete new file \"" + fileName + "\"");
|
||||
infoStream.message("IFD", "delete new file \"" + fileName + "\"");
|
||||
}
|
||||
deleteFile(fileName);
|
||||
}
|
||||
|
@ -594,7 +584,7 @@ final class IndexFileDeleter {
|
|||
assert locked();
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
message("delete \"" + fileName + "\"");
|
||||
infoStream.message("IFD", "delete \"" + fileName + "\"");
|
||||
}
|
||||
directory.deleteFile(fileName);
|
||||
} catch (IOException e) { // if delete fails
|
||||
|
@ -608,7 +598,7 @@ final class IndexFileDeleter {
|
|||
// the file for subsequent deletion.
|
||||
|
||||
if (infoStream != null) {
|
||||
message("unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
|
||||
infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
|
||||
}
|
||||
if (deletable == null) {
|
||||
deletable = new ArrayList<String>();
|
||||
|
|
|
@ -53,12 +53,6 @@ public final class IndexFileNames {
|
|||
/** Extension of norms file */
|
||||
public static final String NORMS_EXTENSION = "nrm";
|
||||
|
||||
/** Extension of stored fields index file */
|
||||
public static final String FIELDS_INDEX_EXTENSION = "fdx";
|
||||
|
||||
/** Extension of stored fields file */
|
||||
public static final String FIELDS_EXTENSION = "fdt";
|
||||
|
||||
/** Extension of vectors fields file */
|
||||
public static final String VECTORS_FIELDS_EXTENSION = "tvf";
|
||||
|
||||
|
@ -100,8 +94,6 @@ public final class IndexFileNames {
|
|||
COMPOUND_FILE_EXTENSION,
|
||||
COMPOUND_FILE_ENTRIES_EXTENSION,
|
||||
FIELD_INFOS_EXTENSION,
|
||||
FIELDS_INDEX_EXTENSION,
|
||||
FIELDS_EXTENSION,
|
||||
DELETES_EXTENSION,
|
||||
VECTORS_INDEX_EXTENSION,
|
||||
VECTORS_DOCUMENTS_EXTENSION,
|
||||
|
@ -116,8 +108,6 @@ public final class IndexFileNames {
|
|||
VECTORS_INDEX_EXTENSION,
|
||||
VECTORS_FIELDS_EXTENSION,
|
||||
VECTORS_DOCUMENTS_EXTENSION,
|
||||
FIELDS_INDEX_EXTENSION,
|
||||
FIELDS_EXTENSION
|
||||
};
|
||||
|
||||
public static final String[] NON_STORE_INDEX_EXTENSIONS = new String[] {
|
||||
|
@ -127,8 +117,6 @@ public final class IndexFileNames {
|
|||
|
||||
static final String COMPOUND_EXTENSIONS_NOT_CODEC[] = new String[] {
|
||||
FIELD_INFOS_EXTENSION,
|
||||
FIELDS_INDEX_EXTENSION,
|
||||
FIELDS_EXTENSION,
|
||||
};
|
||||
|
||||
/** File extensions for term vector support */
|
||||
|
@ -172,6 +160,7 @@ public final class IndexFileNames {
|
|||
* Returns true if the provided filename is one of the doc store files (ends
|
||||
* with an extension in {@link #STORE_INDEX_EXTENSIONS}).
|
||||
*/
|
||||
// TODO: this method is stupid.
|
||||
public static boolean isDocStoreFile(String fileName) {
|
||||
if (fileName.endsWith(COMPOUND_FILE_STORE_EXTENSION))
|
||||
return true;
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
import java.io.File;
|
||||
|
@ -86,30 +87,28 @@ public final class IndexUpgrader {
|
|||
}
|
||||
|
||||
private final Directory dir;
|
||||
private final PrintStream infoStream;
|
||||
private final IndexWriterConfig iwc;
|
||||
private final boolean deletePriorCommits;
|
||||
|
||||
/** Creates index upgrader on the given directory, using an {@link IndexWriter} using the given
|
||||
* {@code matchVersion}. The tool refuses to upgrade indexes with multiple commit points. */
|
||||
public IndexUpgrader(Directory dir, Version matchVersion) {
|
||||
this(dir, new IndexWriterConfig(matchVersion, null), null, false);
|
||||
this(dir, new IndexWriterConfig(matchVersion, null), false);
|
||||
}
|
||||
|
||||
/** Creates index upgrader on the given directory, using an {@link IndexWriter} using the given
|
||||
* {@code matchVersion}. You have the possibility to upgrade indexes with multiple commit points by removing
|
||||
* all older ones. If {@code infoStream} is not {@code null}, all logging output will be sent to this stream. */
|
||||
public IndexUpgrader(Directory dir, Version matchVersion, PrintStream infoStream, boolean deletePriorCommits) {
|
||||
this(dir, new IndexWriterConfig(matchVersion, null), infoStream, deletePriorCommits);
|
||||
this(dir, new IndexWriterConfig(matchVersion, null).setInfoStream(infoStream), deletePriorCommits);
|
||||
}
|
||||
|
||||
/** Creates index upgrader on the given directory, using an {@link IndexWriter} using the given
|
||||
* config. You have the possibility to upgrade indexes with multiple commit points by removing
|
||||
* all older ones. If {@code infoStream} is not {@code null}, all logging output will be sent to this stream. */
|
||||
public IndexUpgrader(Directory dir, IndexWriterConfig iwc, PrintStream infoStream, boolean deletePriorCommits) {
|
||||
* all older ones. */
|
||||
public IndexUpgrader(Directory dir, IndexWriterConfig iwc, boolean deletePriorCommits) {
|
||||
this.dir = dir;
|
||||
this.iwc = iwc;
|
||||
this.infoStream = infoStream;
|
||||
this.deletePriorCommits = deletePriorCommits;
|
||||
}
|
||||
|
||||
|
@ -131,10 +130,14 @@ public final class IndexUpgrader {
|
|||
|
||||
final IndexWriter w = new IndexWriter(dir, c);
|
||||
try {
|
||||
w.setInfoStream(infoStream);
|
||||
w.message("Upgrading all pre-" + Constants.LUCENE_MAIN_VERSION + " segments of index directory '" + dir + "' to version " + Constants.LUCENE_MAIN_VERSION + "...");
|
||||
InfoStream infoStream = c.getInfoStream();
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IndexUpgrader", "Upgrading all pre-" + Constants.LUCENE_MAIN_VERSION + " segments of index directory '" + dir + "' to version " + Constants.LUCENE_MAIN_VERSION + "...");
|
||||
}
|
||||
w.optimize();
|
||||
w.message("All segments upgraded to version " + Constants.LUCENE_MAIN_VERSION);
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IndexUpgrader", "All segments upgraded to version " + Constants.LUCENE_MAIN_VERSION);
|
||||
}
|
||||
} finally {
|
||||
w.close();
|
||||
}
|
||||
|
|
|
@ -19,12 +19,10 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
|
@ -52,6 +50,7 @@ import org.apache.lucene.store.MergeInfo;
|
|||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
|
@ -202,12 +201,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
* encoded as UTF8. If a term arrives from the analyzer
|
||||
* longer than this length, it is skipped and a message is
|
||||
* printed to infoStream, if set (see {@link
|
||||
* #setInfoStream}).
|
||||
* IndexWriterConfig#setInfoStream(InfoStream)}).
|
||||
*/
|
||||
public final static int MAX_TERM_LENGTH = DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8;
|
||||
// Used for printing messages
|
||||
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
|
||||
private int messageID = MESSAGE_ID.getAndIncrement();
|
||||
volatile private boolean hitOOM;
|
||||
|
||||
private final Directory directory; // where this index resides
|
||||
|
@ -271,9 +267,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// The PayloadProcessorProvider to use when segments are merged
|
||||
private PayloadProcessorProvider payloadProcessorProvider;
|
||||
|
||||
// for testing
|
||||
boolean anyNonBulkMerges;
|
||||
|
||||
IndexReader getReader() throws IOException {
|
||||
return getReader(true);
|
||||
}
|
||||
|
@ -343,7 +336,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
final long tStart = System.currentTimeMillis();
|
||||
|
||||
if (infoStream != null) {
|
||||
message("flush at getReader");
|
||||
infoStream.message("IW", "flush at getReader");
|
||||
}
|
||||
// Do this up front before flushing so that the readers
|
||||
// obtained during this flush are pooled, the first time
|
||||
|
@ -376,7 +369,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
maybeApplyDeletes(applyAllDeletes);
|
||||
r = new DirectoryReader(this, segmentInfos, applyAllDeletes);
|
||||
if (infoStream != null) {
|
||||
message("return reader version=" + r.getVersion() + " reader=" + r);
|
||||
infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
|
||||
}
|
||||
}
|
||||
} catch (OutOfMemoryError oom) {
|
||||
|
@ -385,7 +378,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
return null;
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
message("hit exception during NRT reader");
|
||||
infoStream.message("IW", "hit exception during NRT reader");
|
||||
}
|
||||
// Done: finish the full flush!
|
||||
docWriter.finishFullFlush(success);
|
||||
|
@ -396,7 +389,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
maybeMerge();
|
||||
}
|
||||
if (infoStream != null) {
|
||||
message("getReader took " + (System.currentTimeMillis() - tStart) + " msec");
|
||||
infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " msec");
|
||||
}
|
||||
return r;
|
||||
}
|
||||
|
@ -791,16 +784,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
ensureOpen(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Prints a message to the infoStream (if non-null),
|
||||
* prefixed with the identifying information for this
|
||||
* writer and the thread that's calling it.
|
||||
*/
|
||||
public void message(String message) {
|
||||
if (infoStream != null)
|
||||
infoStream.println("IW " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
}
|
||||
|
||||
final Codec codec; // for writing new segments
|
||||
|
||||
/**
|
||||
|
@ -832,14 +815,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
config = (IndexWriterConfig) conf.clone();
|
||||
directory = d;
|
||||
analyzer = conf.getAnalyzer();
|
||||
infoStream = defaultInfoStream;
|
||||
infoStream = conf.getInfoStream();
|
||||
mergePolicy = conf.getMergePolicy();
|
||||
mergePolicy.setIndexWriter(this);
|
||||
mergeScheduler = conf.getMergeScheduler();
|
||||
codec = conf.getCodec();
|
||||
|
||||
bufferedDeletesStream = new BufferedDeletesStream(messageID);
|
||||
bufferedDeletesStream.setInfoStream(infoStream);
|
||||
bufferedDeletesStream = new BufferedDeletesStream(infoStream);
|
||||
poolReaders = conf.getReaderPooling();
|
||||
|
||||
writeLock = directory.makeLock(WRITE_LOCK_NAME);
|
||||
|
@ -897,7 +879,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
changeCount++;
|
||||
segmentInfos.changed();
|
||||
if (infoStream != null)
|
||||
message("init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
infoStream.message("IW", "init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -906,7 +888,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// start with previous field numbers, but new FieldInfos
|
||||
globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory);
|
||||
docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
|
||||
docWriter.setInfoStream(infoStream);
|
||||
|
||||
// Default deleter (for backwards compatibility) is
|
||||
// KeepOnlyLastCommitDeleter:
|
||||
|
@ -926,7 +907,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("init: create=" + create);
|
||||
infoStream.message("IW", "init: create=" + create);
|
||||
messageState();
|
||||
}
|
||||
|
||||
|
@ -935,7 +916,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
message("init: hit exception on init; releasing write lock");
|
||||
infoStream.message("IW", "init: hit exception on init; releasing write lock");
|
||||
}
|
||||
try {
|
||||
writeLock.release();
|
||||
|
@ -963,51 +944,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
return config;
|
||||
}
|
||||
|
||||
/** If non-null, this will be the default infoStream used
|
||||
* by a newly instantiated IndexWriter.
|
||||
* @see #setInfoStream
|
||||
*/
|
||||
public static void setDefaultInfoStream(PrintStream infoStream) {
|
||||
IndexWriter.defaultInfoStream = infoStream;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current default infoStream for newly
|
||||
* instantiated IndexWriters.
|
||||
* @see #setDefaultInfoStream
|
||||
*/
|
||||
public static PrintStream getDefaultInfoStream() {
|
||||
return IndexWriter.defaultInfoStream;
|
||||
}
|
||||
|
||||
/** If non-null, information about merges, deletes and a
|
||||
* message when maxFieldLength is reached will be printed
|
||||
* to this.
|
||||
*/
|
||||
public void setInfoStream(PrintStream infoStream) throws IOException {
|
||||
ensureOpen();
|
||||
this.infoStream = infoStream;
|
||||
docWriter.setInfoStream(infoStream);
|
||||
deleter.setInfoStream(infoStream);
|
||||
bufferedDeletesStream.setInfoStream(infoStream);
|
||||
if (infoStream != null)
|
||||
messageState();
|
||||
}
|
||||
|
||||
private void messageState() throws IOException {
|
||||
message("\ndir=" + directory + "\n" +
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "\ndir=" + directory + "\n" +
|
||||
"index=" + segString() + "\n" +
|
||||
"version=" + Constants.LUCENE_VERSION + "\n" +
|
||||
config.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current infoStream in use by this writer.
|
||||
* @see #setInfoStream
|
||||
*/
|
||||
public PrintStream getInfoStream() {
|
||||
ensureOpen();
|
||||
return infoStream;
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns true if verbosing is enabled (i.e., infoStream != null). */
|
||||
|
@ -1122,7 +1065,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
message("now flush at close waitForMerges=" + waitForMerges);
|
||||
infoStream.message("IW", "now flush at close waitForMerges=" + waitForMerges);
|
||||
}
|
||||
|
||||
docWriter.close();
|
||||
|
@ -1148,14 +1091,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergeScheduler.close();
|
||||
|
||||
if (infoStream != null)
|
||||
message("now call final commit()");
|
||||
infoStream.message("IW", "now call final commit()");
|
||||
|
||||
if (!hitOOM) {
|
||||
commitInternal(null);
|
||||
}
|
||||
|
||||
if (infoStream != null)
|
||||
message("at close: " + segString());
|
||||
infoStream.message("IW", "at close: " + segString());
|
||||
// used by assert below
|
||||
final DocumentsWriter oldWriter = docWriter;
|
||||
synchronized(this) {
|
||||
|
@ -1180,7 +1123,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
notifyAll();
|
||||
if (!closed) {
|
||||
if (infoStream != null)
|
||||
message("hit exception while closing");
|
||||
infoStream.message("IW", "hit exception while closing");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1410,7 +1353,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
message("hit exception updating document");
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
}
|
||||
}
|
||||
if (anySegmentFlushed) {
|
||||
|
@ -1557,7 +1500,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream != null)
|
||||
message("hit exception updating document");
|
||||
infoStream.message("IW", "hit exception updating document");
|
||||
}
|
||||
|
||||
if (anySegmentFlushed) {
|
||||
|
@ -1619,8 +1562,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
/** If non-null, information about merges will be printed to this.
|
||||
*/
|
||||
private PrintStream infoStream;
|
||||
private static PrintStream defaultInfoStream;
|
||||
final InfoStream infoStream;
|
||||
|
||||
/**
|
||||
* Requests an "optimize" operation on an index, priming the index
|
||||
|
@ -1737,8 +1679,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
throw new IllegalArgumentException("maxNumSegments must be >= 1; got " + maxNumSegments);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("optimize: index now " + segString());
|
||||
message("now flush at optimize");
|
||||
infoStream.message("IW", "optimize: index now " + segString());
|
||||
infoStream.message("IW", "now flush at optimize");
|
||||
}
|
||||
|
||||
flush(true, true);
|
||||
|
@ -1849,7 +1791,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
flush(true, true);
|
||||
|
||||
if (infoStream != null)
|
||||
message("expungeDeletes: index now " + segString());
|
||||
infoStream.message("IW", "expungeDeletes: index now " + segString());
|
||||
|
||||
MergePolicy.MergeSpecification spec;
|
||||
|
||||
|
@ -2057,7 +1999,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
boolean success = false;
|
||||
|
||||
if (infoStream != null ) {
|
||||
message("rollback");
|
||||
infoStream.message("IW", "rollback");
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -2067,7 +2009,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null ) {
|
||||
message("rollback: done finish merges");
|
||||
infoStream.message("IW", "rollback: done finish merges");
|
||||
}
|
||||
|
||||
// Must pre-close these two, in case they increment
|
||||
|
@ -2094,7 +2036,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// once").
|
||||
segmentInfos.rollbackSegmentInfos(rollbackSegments);
|
||||
if (infoStream != null ) {
|
||||
message("rollback: infos=" + segString(segmentInfos));
|
||||
infoStream.message("IW", "rollback: infos=" + segString(segmentInfos));
|
||||
}
|
||||
|
||||
docWriter.abort();
|
||||
|
@ -2121,7 +2063,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
closing = false;
|
||||
notifyAll();
|
||||
if (infoStream != null)
|
||||
message("hit exception during rollback");
|
||||
infoStream.message("IW", "hit exception during rollback");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2174,7 +2116,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
handleOOM(oom, "deleteAll");
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
message("hit exception during deleteAll");
|
||||
infoStream.message("IW", "hit exception during deleteAll");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2187,7 +2129,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// Abort all pending & running merges:
|
||||
for (final MergePolicy.OneMerge merge : pendingMerges) {
|
||||
if (infoStream != null)
|
||||
message("now abort pending merge " + merge.segString(directory));
|
||||
infoStream.message("IW", "now abort pending merge " + merge.segString(directory));
|
||||
merge.abort();
|
||||
mergeFinish(merge);
|
||||
}
|
||||
|
@ -2195,7 +2137,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
for (final MergePolicy.OneMerge merge : runningMerges) {
|
||||
if (infoStream != null)
|
||||
message("now abort running merge " + merge.segString(directory));
|
||||
infoStream.message("IW", "now abort running merge " + merge.segString(directory));
|
||||
merge.abort();
|
||||
}
|
||||
|
||||
|
@ -2206,7 +2148,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// they are aborted.
|
||||
while(runningMerges.size() > 0) {
|
||||
if (infoStream != null)
|
||||
message("now wait for " + runningMerges.size() + " running merge to abort");
|
||||
infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge to abort");
|
||||
doWait();
|
||||
}
|
||||
|
||||
|
@ -2216,7 +2158,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
assert 0 == mergingSegments.size();
|
||||
|
||||
if (infoStream != null)
|
||||
message("all running merges have aborted");
|
||||
infoStream.message("IW", "all running merges have aborted");
|
||||
|
||||
} else {
|
||||
// waitForMerges() will ensure any running addIndexes finishes.
|
||||
|
@ -2237,7 +2179,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
public synchronized void waitForMerges() {
|
||||
ensureOpen(false);
|
||||
if (infoStream != null) {
|
||||
message("waitForMerges");
|
||||
infoStream.message("IW", "waitForMerges");
|
||||
}
|
||||
while(pendingMerges.size() > 0 || runningMerges.size() > 0) {
|
||||
doWait();
|
||||
|
@ -2247,7 +2189,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
assert 0 == mergingSegments.size();
|
||||
|
||||
if (infoStream != null) {
|
||||
message("waitForMerges done");
|
||||
infoStream.message("IW", "waitForMerges done");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2284,7 +2226,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
try {
|
||||
if (useCompoundFile(newSegment)) {
|
||||
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
|
||||
message("creating compound file " + compoundFileName);
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "creating compound file " + compoundFileName);
|
||||
}
|
||||
// Now build compound file
|
||||
final Directory cfsDir = new CompoundFileDirectory(directory, compoundFileName, context, true);
|
||||
IOException prior = null;
|
||||
|
@ -2315,7 +2259,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
newSegment.advanceDelGen();
|
||||
final String delFileName = newSegment.getDelFileName();
|
||||
if (infoStream != null) {
|
||||
message("flush: write " + delCount + " deletes to " + delFileName);
|
||||
infoStream.message("IW", "flush: write " + delCount + " deletes to " + delFileName);
|
||||
}
|
||||
boolean success2 = false;
|
||||
try {
|
||||
|
@ -2342,7 +2286,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
message("hit exception " +
|
||||
infoStream.message("IW", "hit exception " +
|
||||
"reating compound file for newly flushed segment " + newSegment.name);
|
||||
}
|
||||
|
||||
|
@ -2374,7 +2318,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// Lock order IW -> BDS
|
||||
synchronized (bufferedDeletesStream) {
|
||||
if (infoStream != null) {
|
||||
message("publishFlushedSegment");
|
||||
infoStream.message("IW", "publishFlushedSegment");
|
||||
}
|
||||
|
||||
if (globalPacket != null && globalPacket.any()) {
|
||||
|
@ -2391,7 +2335,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
nextGen = bufferedDeletesStream.getNextGen();
|
||||
}
|
||||
if (infoStream != null) {
|
||||
message("publish sets newSegment delGen=" + nextGen);
|
||||
infoStream.message("IW", "publish sets newSegment delGen=" + nextGen);
|
||||
}
|
||||
newSegment.setBufferedDeletesGen(nextGen);
|
||||
segmentInfos.add(newSegment);
|
||||
|
@ -2473,7 +2417,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
try {
|
||||
if (infoStream != null)
|
||||
message("flush at addIndexes(Directory...)");
|
||||
infoStream.message("IW", "flush at addIndexes(Directory...)");
|
||||
flush(false, true);
|
||||
|
||||
int docCount = 0;
|
||||
|
@ -2481,7 +2425,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
Comparator<String> versionComparator = StringHelper.getVersionComparator();
|
||||
for (Directory dir : dirs) {
|
||||
if (infoStream != null) {
|
||||
message("addIndexes: process directory " + dir);
|
||||
infoStream.message("IW", "addIndexes: process directory " + dir);
|
||||
}
|
||||
SegmentInfos sis = new SegmentInfos(); // read infos from dir
|
||||
sis.read(dir);
|
||||
|
@ -2495,7 +2439,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
String dsName = info.getDocStoreSegment();
|
||||
|
||||
if (infoStream != null) {
|
||||
message("addIndexes: process segment origName=" + info.name + " newName=" + newSegName + " dsName=" + dsName + " info=" + info);
|
||||
infoStream.message("IW", "addIndexes: process segment origName=" + info.name + " newName=" + newSegName + " dsName=" + dsName + " info=" + info);
|
||||
}
|
||||
|
||||
// create CFS only if the source segment is not CFS, and MP agrees it
|
||||
|
@ -2561,7 +2505,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
try {
|
||||
if (infoStream != null)
|
||||
message("flush at addIndexes(IndexReader...)");
|
||||
infoStream.message("IW", "flush at addIndexes(IndexReader...)");
|
||||
flush(false, true);
|
||||
|
||||
String mergedName = newSegmentName();
|
||||
|
@ -2572,17 +2516,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// TODO: somehow we should fix this merge so it's
|
||||
// abortable so that IW.close(false) is able to stop it
|
||||
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
|
||||
mergedName, null, payloadProcessorProvider,
|
||||
SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(),
|
||||
mergedName, MergeState.CheckAbort.NONE, payloadProcessorProvider,
|
||||
new FieldInfos(globalFieldNumberMap), codec, context);
|
||||
|
||||
for (IndexReader reader : readers) // add new indexes
|
||||
merger.add(reader);
|
||||
int docCount = merger.merge(); // merge 'em
|
||||
|
||||
final FieldInfos fieldInfos = merger.fieldInfos();
|
||||
MergeState mergeState = merger.merge(); // merge 'em
|
||||
int docCount = mergeState.mergedDocCount;
|
||||
final FieldInfos fieldInfos = mergeState.fieldInfos;
|
||||
SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
|
||||
false, merger.getCodec(),
|
||||
false, codec,
|
||||
fieldInfos);
|
||||
setDiagnostics(info, "addIndexes(IndexReader...)");
|
||||
|
||||
|
@ -2753,8 +2697,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
ensureOpen(false);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("prepareCommit: flush");
|
||||
message(" index before flush " + segString());
|
||||
infoStream.message("IW", "prepareCommit: flush");
|
||||
infoStream.message("IW", " index before flush " + segString());
|
||||
}
|
||||
|
||||
if (hitOOM) {
|
||||
|
@ -2812,7 +2756,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success && infoStream != null) {
|
||||
message("hit exception during prepareCommit");
|
||||
infoStream.message("IW", "hit exception during prepareCommit");
|
||||
}
|
||||
// Done: finish the full flush!
|
||||
docWriter.finishFullFlush(flushSuccess);
|
||||
|
@ -2896,21 +2840,21 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
private final void commitInternal(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
|
||||
|
||||
if (infoStream != null) {
|
||||
message("commit: start");
|
||||
infoStream.message("IW", "commit: start");
|
||||
}
|
||||
|
||||
synchronized(commitLock) {
|
||||
if (infoStream != null) {
|
||||
message("commit: enter lock");
|
||||
infoStream.message("IW", "commit: enter lock");
|
||||
}
|
||||
|
||||
if (pendingCommit == null) {
|
||||
if (infoStream != null) {
|
||||
message("commit: now prepare");
|
||||
infoStream.message("IW", "commit: now prepare");
|
||||
}
|
||||
prepareCommit(commitUserData);
|
||||
} else if (infoStream != null) {
|
||||
message("commit: already prepared");
|
||||
infoStream.message("IW", "commit: already prepared");
|
||||
}
|
||||
|
||||
finishCommit();
|
||||
|
@ -2922,10 +2866,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (pendingCommit != null) {
|
||||
try {
|
||||
if (infoStream != null)
|
||||
message("commit: pendingCommit != null");
|
||||
infoStream.message("IW", "commit: pendingCommit != null");
|
||||
pendingCommit.finishCommit(directory, codec);
|
||||
if (infoStream != null)
|
||||
message("commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
|
||||
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
|
||||
lastCommitChangeCount = pendingCommitChangeCount;
|
||||
segmentInfos.updateGeneration(pendingCommit);
|
||||
segmentInfos.setUserData(pendingCommit.getUserData());
|
||||
|
@ -2939,11 +2883,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
} else if (infoStream != null) {
|
||||
message("commit: pendingCommit == null; skip");
|
||||
infoStream.message("IW", "commit: pendingCommit == null; skip");
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("commit: done");
|
||||
infoStream.message("IW", "commit: done");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2985,8 +2929,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
try {
|
||||
|
||||
if (infoStream != null) {
|
||||
message(" start flush: applyAllDeletes=" + applyAllDeletes);
|
||||
message(" index before flush " + segString());
|
||||
infoStream.message("IW", " start flush: applyAllDeletes=" + applyAllDeletes);
|
||||
infoStream.message("IW", " index before flush " + segString());
|
||||
}
|
||||
final boolean anySegmentFlushed;
|
||||
|
||||
|
@ -3015,18 +2959,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
return false;
|
||||
} finally {
|
||||
if (!success && infoStream != null)
|
||||
message("hit exception during flush");
|
||||
infoStream.message("IW", "hit exception during flush");
|
||||
}
|
||||
}
|
||||
|
||||
final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException {
|
||||
if (applyAllDeletes) {
|
||||
if (infoStream != null) {
|
||||
message("apply all deletes during flush");
|
||||
infoStream.message("IW", "apply all deletes during flush");
|
||||
}
|
||||
applyAllDeletes();
|
||||
} else if (infoStream != null) {
|
||||
message("don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
|
||||
infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -3039,7 +2983,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
if (!keepFullyDeletedSegments && result.allDeleted != null) {
|
||||
if (infoStream != null) {
|
||||
message("drop 100% deleted segments: " + segString(result.allDeleted));
|
||||
infoStream.message("IW", "drop 100% deleted segments: " + segString(result.allDeleted));
|
||||
}
|
||||
for (SegmentInfo info : result.allDeleted) {
|
||||
// If a merge has already registered for this
|
||||
|
@ -3104,7 +3048,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
final List<SegmentInfo> sourceSegments = merge.segments;
|
||||
|
||||
if (infoStream != null)
|
||||
message("commitMergeDeletes " + merge.segString(directory));
|
||||
infoStream.message("IW", "commitMergeDeletes " + merge.segString(directory));
|
||||
|
||||
// Carefully merge deletes that occurred after we
|
||||
// started merging:
|
||||
|
@ -3202,7 +3146,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null)
|
||||
message("commitMerge: " + merge.segString(directory) + " index=" + segString());
|
||||
infoStream.message("IW", "commitMerge: " + merge.segString(directory) + " index=" + segString());
|
||||
|
||||
assert merge.registerDone;
|
||||
|
||||
|
@ -3214,7 +3158,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// abort this merge
|
||||
if (merge.isAborted()) {
|
||||
if (infoStream != null)
|
||||
message("commitMerge: skipping merge " + merge.segString(directory) + ": it was aborted");
|
||||
infoStream.message("IW", "commitMerge: skipping merge " + merge.segString(directory) + ": it was aborted");
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -3230,7 +3174,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
final boolean allDeleted = mergedReader.numDocs() == 0;
|
||||
|
||||
if (infoStream != null && allDeleted) {
|
||||
message("merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert"));
|
||||
infoStream.message("IW", "merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert"));
|
||||
}
|
||||
|
||||
final boolean dropSegment = allDeleted && !keepFullyDeletedSegments;
|
||||
|
@ -3241,7 +3185,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("after commit: " + segString());
|
||||
infoStream.message("IW", "after commit: " + segString());
|
||||
}
|
||||
|
||||
closeMergeReaders(merge, false);
|
||||
|
@ -3268,7 +3212,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
final private void handleMergeException(Throwable t, MergePolicy.OneMerge merge) throws IOException {
|
||||
|
||||
if (infoStream != null) {
|
||||
message("handleMergeException: merge=" + merge.segString(directory) + " exc=" + t);
|
||||
infoStream.message("IW", "handleMergeException: merge=" + merge.segString(directory) + " exc=" + t);
|
||||
}
|
||||
|
||||
// Set the exception on the merge, so if
|
||||
|
@ -3317,7 +3261,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergeInit(merge);
|
||||
|
||||
if (infoStream != null)
|
||||
message("now merge\n merge=" + merge.segString(directory) + "\n index=" + segString());
|
||||
infoStream.message("IW", "now merge\n merge=" + merge.segString(directory) + "\n index=" + segString());
|
||||
|
||||
mergeMiddle(merge);
|
||||
mergeSuccess(merge);
|
||||
|
@ -3331,7 +3275,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (!success) {
|
||||
if (infoStream != null)
|
||||
message("hit exception during merge");
|
||||
infoStream.message("IW", "hit exception during merge");
|
||||
if (merge.info != null && !segmentInfos.contains(merge.info))
|
||||
deleter.refresh(merge.info.name);
|
||||
}
|
||||
|
@ -3348,7 +3292,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
handleOOM(oom, "merge");
|
||||
}
|
||||
if (infoStream != null && merge.info != null) {
|
||||
message("merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
|
||||
infoStream.message("IW", "merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
|
||||
}
|
||||
//System.out.println(Thread.currentThread().getName() + ": merge end");
|
||||
}
|
||||
|
@ -3395,7 +3339,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
pendingMerges.add(merge);
|
||||
|
||||
if (infoStream != null)
|
||||
message("add merge to pendingMerges: " + merge.segString(directory) + " [total " + pendingMerges.size() + " pending]");
|
||||
infoStream.message("IW", "add merge to pendingMerges: " + merge.segString(directory) + " [total " + pendingMerges.size() + " pending]");
|
||||
|
||||
merge.mergeGen = mergeGen;
|
||||
merge.isExternal = isExternal;
|
||||
|
@ -3412,10 +3356,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
builder.append("]");
|
||||
// don't call mergingSegments.toString() could lead to ConcurrentModException
|
||||
// since merge updates the segments FieldInfos
|
||||
message(builder.toString());
|
||||
infoStream.message("IW", builder.toString());
|
||||
}
|
||||
for(SegmentInfo info : merge.segments) {
|
||||
message("registerMerge info=" + info);
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", "registerMerge info=" + info);
|
||||
}
|
||||
mergingSegments.add(info);
|
||||
}
|
||||
|
||||
|
@ -3435,7 +3381,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
message("hit exception in mergeInit");
|
||||
infoStream.message("IW", "hit exception in mergeInit");
|
||||
}
|
||||
mergeFinish(merge);
|
||||
}
|
||||
|
@ -3477,7 +3423,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (!keepFullyDeletedSegments && result.allDeleted != null) {
|
||||
if (infoStream != null) {
|
||||
message("drop 100% deleted segments: " + result.allDeleted);
|
||||
infoStream.message("IW", "drop 100% deleted segments: " + result.allDeleted);
|
||||
}
|
||||
for(SegmentInfo info : result.allDeleted) {
|
||||
segmentInfos.remove(info);
|
||||
|
@ -3502,7 +3448,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
setDiagnostics(merge.info, "merge", details);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("merge seg=" + merge.info.name);
|
||||
infoStream.message("IW", "merge seg=" + merge.info.name);
|
||||
}
|
||||
|
||||
assert merge.estimatedMergeBytes == 0;
|
||||
|
@ -3639,11 +3585,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
IOContext context = new IOContext(merge.getMergeInfo());
|
||||
|
||||
SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge,
|
||||
final MergeState.CheckAbort checkAbort = new MergeState.CheckAbort(merge, directory);
|
||||
SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, checkAbort,
|
||||
payloadProcessorProvider, merge.info.getFieldInfos(), codec, context);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
|
||||
infoStream.message("IW", "merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
|
||||
}
|
||||
|
||||
merge.readers = new ArrayList<SegmentReader>();
|
||||
|
@ -3677,22 +3624,21 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("merge: total " + totDocCount + " docs");
|
||||
infoStream.message("IW", "merge: total " + totDocCount + " docs");
|
||||
}
|
||||
|
||||
merge.checkAborted(directory);
|
||||
|
||||
// This is where all the work happens:
|
||||
mergedDocCount = merge.info.docCount = merger.merge();
|
||||
MergeState mergeState = merger.merge();
|
||||
mergedDocCount = merge.info.docCount = mergeState.mergedDocCount;
|
||||
|
||||
// Record which codec was used to write the segment
|
||||
merge.info.setCodec(merger.getCodec());
|
||||
merge.info.setCodec(codec);
|
||||
|
||||
if (infoStream != null) {
|
||||
message("merge codecs=" + merger.getCodec());
|
||||
message("merge store matchedCount=" + merger.getMatchedSubReaderCount() + " vs " + merge.readers.size());
|
||||
infoStream.message("IW", "merge codec=" + codec);
|
||||
}
|
||||
anyNonBulkMerges |= merger.getAnyNonBulkMerges();
|
||||
|
||||
assert mergedDocCount == totDocCount: "mergedDocCount=" + mergedDocCount + " vs " + totDocCount;
|
||||
|
||||
|
@ -3711,7 +3657,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
message("create compound file " + compoundFileName);
|
||||
infoStream.message("IW", "create compound file " + compoundFileName);
|
||||
}
|
||||
merger.createCompoundFile(compoundFileName, merge.info, new IOContext(merge.getMergeInfo()));
|
||||
success = true;
|
||||
|
@ -3730,7 +3676,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
} finally {
|
||||
if (!success) {
|
||||
if (infoStream != null) {
|
||||
message("hit exception creating compound file during merge");
|
||||
infoStream.message("IW", "hit exception creating compound file during merge");
|
||||
}
|
||||
|
||||
synchronized(this) {
|
||||
|
@ -3751,7 +3697,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (merge.isAborted()) {
|
||||
if (infoStream != null) {
|
||||
message("abort merge after building CFS");
|
||||
infoStream.message("IW", "abort merge after building CFS");
|
||||
}
|
||||
deleter.deleteFile(compoundFileName);
|
||||
return 0;
|
||||
|
@ -3762,7 +3708,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message(String.format("merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes(true)/1024./1024., merge.estimatedMergeBytes/1024/1024.));
|
||||
infoStream.message("IW", String.format("merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes(true)/1024./1024., merge.estimatedMergeBytes/1024/1024.));
|
||||
}
|
||||
|
||||
final IndexReaderWarmer mergedSegmentWarmer = config.getMergedSegmentWarmer();
|
||||
|
@ -3936,7 +3882,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
try {
|
||||
|
||||
if (infoStream != null) {
|
||||
message("startCommit(): start");
|
||||
infoStream.message("IW", "startCommit(): start");
|
||||
}
|
||||
|
||||
synchronized(this) {
|
||||
|
@ -3945,14 +3891,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (pendingCommitChangeCount == lastCommitChangeCount) {
|
||||
if (infoStream != null) {
|
||||
message(" skip startCommit(): no changes pending");
|
||||
infoStream.message("IW", " skip startCommit(): no changes pending");
|
||||
}
|
||||
deleter.decRef(toSync);
|
||||
return;
|
||||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("startCommit index=" + segString(toSync) + " changeCount=" + changeCount);
|
||||
infoStream.message("IW", "startCommit index=" + segString(toSync) + " changeCount=" + changeCount);
|
||||
}
|
||||
|
||||
assert filesExist(toSync);
|
||||
|
@ -3989,7 +3935,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
message("done all syncs");
|
||||
infoStream.message("IW", "done all syncs");
|
||||
}
|
||||
|
||||
assert testPoint("midStartCommitSuccess");
|
||||
|
@ -4004,7 +3950,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (!pendingCommitSet) {
|
||||
if (infoStream != null) {
|
||||
message("hit exception committing segments file");
|
||||
infoStream.message("IW", "hit exception committing segments file");
|
||||
}
|
||||
|
||||
// Hit exception
|
||||
|
@ -4057,7 +4003,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
private void handleOOM(OutOfMemoryError oom, String location) {
|
||||
if (infoStream != null) {
|
||||
message("hit OutOfMemoryError inside " + location);
|
||||
infoStream.message("IW", "hit OutOfMemoryError inside " + location);
|
||||
}
|
||||
hitOOM = true;
|
||||
throw oom;
|
||||
|
@ -4082,7 +4028,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
//System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
|
||||
ensureOpen();
|
||||
if (infoStream != null) {
|
||||
message("nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + " DW changes: " + docWriter.anyChanges() + " BD changes: "+bufferedDeletesStream.any());
|
||||
infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + " DW changes: " + docWriter.anyChanges() + " BD changes: "+bufferedDeletesStream.any());
|
||||
|
||||
}
|
||||
return infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any();
|
||||
|
|
|
@ -17,12 +17,16 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.PrintStream;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
|
||||
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.PrintStreamInfoStream;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/**
|
||||
|
@ -122,6 +126,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
private volatile IndexingChain indexingChain;
|
||||
private volatile IndexReaderWarmer mergedSegmentWarmer;
|
||||
private volatile Codec codec;
|
||||
private volatile InfoStream infoStream;
|
||||
private volatile MergePolicy mergePolicy;
|
||||
private volatile DocumentsWriterPerThreadPool indexerThreadPool;
|
||||
private volatile boolean readerPooling;
|
||||
|
@ -159,6 +164,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
indexingChain = DocumentsWriterPerThread.defaultIndexingChain;
|
||||
mergedSegmentWarmer = null;
|
||||
codec = Codec.getDefault();
|
||||
infoStream = InfoStream.getDefault();
|
||||
if (matchVersion.onOrAfter(Version.LUCENE_32)) {
|
||||
mergePolicy = new TieredMergePolicy();
|
||||
} else {
|
||||
|
@ -676,6 +682,30 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
public FlushPolicy getFlushPolicy() {
|
||||
return flushPolicy;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see #setInfoStream(InfoStream)
|
||||
*/
|
||||
public InfoStream getInfoStream() {
|
||||
return infoStream;
|
||||
}
|
||||
|
||||
/** If non-null, information about merges, deletes and a
|
||||
* message when maxFieldLength is reached will be printed
|
||||
* to this.
|
||||
*/
|
||||
public IndexWriterConfig setInfoStream(InfoStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method that uses {@link PrintStreamInfoStream}
|
||||
*/
|
||||
public IndexWriterConfig setInfoStream(PrintStream printStream) {
|
||||
this.infoStream = printStream == null ? null : new PrintStreamInfoStream(printStream);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
|
@ -695,6 +725,7 @@ public final class IndexWriterConfig implements Cloneable {
|
|||
sb.append("maxBufferedDocs=").append(maxBufferedDocs).append("\n");
|
||||
sb.append("mergedSegmentWarmer=").append(mergedSegmentWarmer).append("\n");
|
||||
sb.append("codec=").append(codec).append("\n");
|
||||
sb.append("infoStream=").append(infoStream == null ? "null" : infoStream.getClass().getName()).append("\n");
|
||||
sb.append("mergePolicy=").append(mergePolicy).append("\n");
|
||||
sb.append("indexerThreadPool=").append(indexerThreadPool).append("\n");
|
||||
sb.append("readerPooling=").append(readerPooling).append("\n");
|
||||
|
|
|
@ -23,6 +23,8 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/** <p>This class implements a {@link MergePolicy} that tries
|
||||
* to merge segments into levels of exponentially
|
||||
* increasing size, where each level has fewer segments than
|
||||
|
@ -104,8 +106,12 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
protected void message(String message) {
|
||||
if (verbose())
|
||||
writer.get().message("LMP: " + message);
|
||||
if (verbose()) {
|
||||
final InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("LMP", message);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** <p>Returns the number of segments that are merged at
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,14 +19,11 @@ package org.apache.lucene.index.codecs;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MergePolicy;
|
||||
import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
|
||||
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/** Holds common state used during segment merging
|
||||
*
|
||||
|
@ -45,20 +42,27 @@ public class MergeState {
|
|||
|
||||
public FieldInfos fieldInfos;
|
||||
public List<IndexReaderAndLiveDocs> readers; // Readers & liveDocs being merged
|
||||
public int readerCount; // Number of readers being merged
|
||||
public int[][] docMaps; // Maps docIDs around deletions
|
||||
public int[] docBase; // New docID base per reader
|
||||
public int mergedDocCount; // Total # merged docs
|
||||
public CheckAbort checkAbort;
|
||||
public InfoStream infoStream;
|
||||
|
||||
// Updated per field;
|
||||
public FieldInfo fieldInfo;
|
||||
|
||||
// Used to process payloads
|
||||
public boolean hasPayloadProcessorProvider;
|
||||
// TODO: this is a FactoryFactory here basically
|
||||
// and we could make a codec(wrapper) to do all of this privately so IW is uninvolved
|
||||
public PayloadProcessorProvider payloadProcessorProvider;
|
||||
public DirPayloadProcessor[] dirPayloadProcessor;
|
||||
public PayloadProcessor[] currentPayloadProcessor;
|
||||
|
||||
// TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging)
|
||||
// but is this really so expensive to compute again in different components, versus once in SM?
|
||||
public SegmentReader[] matchingSegmentReaders;
|
||||
public int matchedCount;
|
||||
|
||||
public static class CheckAbort {
|
||||
private double workCount;
|
||||
private MergePolicy.OneMerge merge;
|
||||
|
@ -83,5 +87,14 @@ public class MergeState {
|
|||
workCount = 0;
|
||||
}
|
||||
}
|
||||
|
||||
/** If you use this: IW.close(false) cannot abort your merge!
|
||||
* @lucene.internal */
|
||||
static final MergeState.CheckAbort NONE = new MergeState.CheckAbort(null, null) {
|
||||
@Override
|
||||
public void work(double units) throws MergePolicy.MergeAbortedException {
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ import org.apache.lucene.index.codecs.PerDocConsumer;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/**
|
||||
* Encapsulates all necessary state to initiate a {@link PerDocConsumer} and
|
||||
|
@ -29,7 +30,7 @@ import org.apache.lucene.util.Counter;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public class PerDocWriteState {
|
||||
public final PrintStream infoStream;
|
||||
public final InfoStream infoStream;
|
||||
public final Directory directory;
|
||||
public final String segmentName;
|
||||
public final FieldInfos fieldInfos;
|
||||
|
@ -37,7 +38,7 @@ public class PerDocWriteState {
|
|||
public final String segmentSuffix;
|
||||
public final IOContext context;
|
||||
|
||||
public PerDocWriteState(PrintStream infoStream, Directory directory,
|
||||
public PerDocWriteState(InfoStream infoStream, Directory directory,
|
||||
String segmentName, FieldInfos fieldInfos, Counter bytesUsed,
|
||||
String segmentSuffix, IOContext context) {
|
||||
this.infoStream = infoStream;
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsReader;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -55,7 +55,7 @@ final class SegmentCoreReaders {
|
|||
|
||||
private final SegmentReader owner;
|
||||
|
||||
FieldsReader fieldsReaderOrig;
|
||||
StoredFieldsReader fieldsReaderOrig;
|
||||
TermVectorsReader termVectorsReaderOrig;
|
||||
CompoundFileDirectory cfsReader;
|
||||
CompoundFileDirectory storeCFSReader;
|
||||
|
@ -110,7 +110,7 @@ final class SegmentCoreReaders {
|
|||
return termVectorsReaderOrig;
|
||||
}
|
||||
|
||||
synchronized FieldsReader getFieldsReaderOrig() {
|
||||
synchronized StoredFieldsReader getFieldsReaderOrig() {
|
||||
return fieldsReaderOrig;
|
||||
}
|
||||
|
||||
|
@ -166,14 +166,8 @@ final class SegmentCoreReaders {
|
|||
}
|
||||
|
||||
final String storesSegment = si.getDocStoreSegment();
|
||||
fieldsReaderOrig = si.getCodec().fieldsFormat().fieldsReader(storeDir, storesSegment, fieldInfos, context,
|
||||
si.getDocStoreOffset(), si.docCount);
|
||||
|
||||
// Verify two sources of "maxDoc" agree:
|
||||
if (si.getDocStoreOffset() == -1 && fieldsReaderOrig.size() != si.docCount) {
|
||||
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + fieldsReaderOrig.size() + " but segmentInfo shows " + si.docCount);
|
||||
}
|
||||
|
||||
fieldsReaderOrig = si.getCodec().storedFieldsFormat().fieldsReader(storeDir, si, fieldInfos, context);
|
||||
|
||||
if (si.getHasVectors()) { // open term vector files only as needed
|
||||
termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, context, si.getDocStoreOffset(), si.docCount);
|
||||
}
|
||||
|
|
|
@ -285,6 +285,8 @@ public final class SegmentInfo implements Cloneable {
|
|||
* store files otherwise.
|
||||
*/
|
||||
public long sizeInBytes(boolean includeDocStores) throws IOException {
|
||||
// TODO: based on how this is used, can't we just forget about all this docstore crap?
|
||||
// its really an abstraction violation into the codec
|
||||
if (includeDocStores) {
|
||||
if (sizeInBytesWithStore != -1) {
|
||||
return sizeInBytesWithStore;
|
||||
|
@ -293,7 +295,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
for (final String fileName : files()) {
|
||||
// We don't count bytes used by a shared doc store
|
||||
// against this segment
|
||||
if (docStoreOffset == -1 || !IndexFileNames.isDocStoreFile(fileName)) {
|
||||
if (docStoreOffset == -1 || !isDocStoreFile(fileName)) {
|
||||
sum += dir.fileLength(fileName);
|
||||
}
|
||||
}
|
||||
|
@ -305,7 +307,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
long sum = 0;
|
||||
for (final String fileName : files()) {
|
||||
if (IndexFileNames.isDocStoreFile(fileName)) {
|
||||
if (isDocStoreFile(fileName)) {
|
||||
continue;
|
||||
}
|
||||
sum += dir.fileLength(fileName);
|
||||
|
@ -314,6 +316,13 @@ public final class SegmentInfo implements Cloneable {
|
|||
return sizeInBytesNoStore;
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: a little messy, but sizeInBytes above that uses this is the real problem.
|
||||
private boolean isDocStoreFile(String fileName) throws IOException {
|
||||
Set<String> docStoreFiles = new HashSet<String>();
|
||||
codec.storedFieldsFormat().files(dir, this, docStoreFiles);
|
||||
return IndexFileNames.isDocStoreFile(fileName) || docStoreFiles.contains(fileName);
|
||||
}
|
||||
|
||||
public boolean getHasVectors() throws IOException {
|
||||
return hasVectors == CHECK_FIELDINFO ? getFieldInfos().hasVectors() : hasVectors == YES;
|
||||
|
@ -634,11 +643,10 @@ public final class SegmentInfo implements Cloneable {
|
|||
// We are sharing doc stores (stored fields, term
|
||||
// vectors) with other segments
|
||||
assert docStoreSegment != null;
|
||||
// TODO: push this out into preflex fieldsFormat?
|
||||
if (docStoreIsCompoundFile) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION));
|
||||
} else {
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
|
||||
if (getHasVectors()) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
|
@ -646,8 +654,6 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
}
|
||||
} else if (!useCompoundFile) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_EXTENSION));
|
||||
if (getHasVectors()) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
|
|
|
@ -23,23 +23,20 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
import org.apache.lucene.index.MergePolicy.MergeAbortedException;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsReader;
|
||||
import org.apache.lucene.index.codecs.FieldsWriter;
|
||||
import org.apache.lucene.index.codecs.MergeState;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
/**
|
||||
|
@ -51,52 +48,33 @@ import org.apache.lucene.util.ReaderUtil;
|
|||
* @see #add
|
||||
*/
|
||||
final class SegmentMerger {
|
||||
private Directory directory;
|
||||
private String segment;
|
||||
private int termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
|
||||
|
||||
private List<MergeState.IndexReaderAndLiveDocs> readers = new ArrayList<MergeState.IndexReaderAndLiveDocs>();
|
||||
private final FieldInfos fieldInfos;
|
||||
|
||||
private int mergedDocs;
|
||||
|
||||
private final MergeState.CheckAbort checkAbort;
|
||||
private final Directory directory;
|
||||
private final String segment;
|
||||
private final int termIndexInterval;
|
||||
|
||||
/** Maximum number of contiguous documents to bulk-copy
|
||||
when merging stored fields */
|
||||
when merging term vectors */
|
||||
private final static int MAX_RAW_MERGE_DOCS = 4192;
|
||||
|
||||
private Codec codec;
|
||||
private SegmentWriteState segmentWriteState;
|
||||
|
||||
private PayloadProcessorProvider payloadProcessorProvider;
|
||||
private final Codec codec;
|
||||
|
||||
private IOContext context;
|
||||
private final IOContext context;
|
||||
|
||||
private final MergeState mergeState = new MergeState();
|
||||
|
||||
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, Codec codec, IOContext context) {
|
||||
this.payloadProcessorProvider = payloadProcessorProvider;
|
||||
SegmentMerger(InfoStream infoStream, Directory dir, int termIndexInterval, String name, MergeState.CheckAbort checkAbort, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, Codec codec, IOContext context) {
|
||||
mergeState.infoStream = infoStream;
|
||||
mergeState.readers = new ArrayList<MergeState.IndexReaderAndLiveDocs>();
|
||||
mergeState.fieldInfos = fieldInfos;
|
||||
mergeState.checkAbort = checkAbort;
|
||||
mergeState.payloadProcessorProvider = payloadProcessorProvider;
|
||||
directory = dir;
|
||||
segment = name;
|
||||
this.fieldInfos = fieldInfos;
|
||||
if (merge != null) {
|
||||
checkAbort = new MergeState.CheckAbort(merge, directory);
|
||||
} else {
|
||||
checkAbort = new MergeState.CheckAbort(null, null) {
|
||||
@Override
|
||||
public void work(double units) throws MergeAbortedException {
|
||||
// do nothing
|
||||
}
|
||||
};
|
||||
}
|
||||
this.termIndexInterval = termIndexInterval;
|
||||
this.codec = codec;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
public FieldInfos fieldInfos() {
|
||||
return fieldInfos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add an IndexReader to the collection of readers that are to be merged
|
||||
* @param reader
|
||||
|
@ -106,7 +84,7 @@ final class SegmentMerger {
|
|||
new ReaderUtil.Gather(reader) {
|
||||
@Override
|
||||
protected void add(int base, IndexReader r) {
|
||||
readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs()));
|
||||
mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs()));
|
||||
}
|
||||
}.run();
|
||||
} catch (IOException ioe) {
|
||||
|
@ -116,7 +94,7 @@ final class SegmentMerger {
|
|||
}
|
||||
|
||||
final void add(SegmentReader reader, Bits liveDocs) {
|
||||
readers.add(new MergeState.IndexReaderAndLiveDocs(reader, liveDocs));
|
||||
mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(reader, liveDocs));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -125,25 +103,36 @@ final class SegmentMerger {
|
|||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
final int merge() throws CorruptIndexException, IOException {
|
||||
final MergeState merge() throws CorruptIndexException, IOException {
|
||||
// NOTE: it's important to add calls to
|
||||
// checkAbort.work(...) if you make any changes to this
|
||||
// method that will spend alot of time. The frequency
|
||||
// of this check impacts how long
|
||||
// IndexWriter.close(false) takes to actually stop the
|
||||
// threads.
|
||||
|
||||
final int numReaders = mergeState.readers.size();
|
||||
// Remap docIDs
|
||||
mergeState.docMaps = new int[numReaders][];
|
||||
mergeState.docBase = new int[numReaders];
|
||||
mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[numReaders];
|
||||
mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
|
||||
|
||||
mergedDocs = mergeFields();
|
||||
mergeTerms();
|
||||
mergePerDoc();
|
||||
mergeFieldInfos();
|
||||
setMatchingSegmentReaders();
|
||||
mergeState.mergedDocCount = mergeFields();
|
||||
|
||||
final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, segment, mergeState.fieldInfos, mergeState.mergedDocCount, termIndexInterval, codec, null, context);
|
||||
mergeTerms(segmentWriteState);
|
||||
mergePerDoc(segmentWriteState);
|
||||
mergeNorms();
|
||||
|
||||
if (fieldInfos.hasVectors()) {
|
||||
mergeVectors();
|
||||
if (mergeState.fieldInfos.hasVectors()) {
|
||||
mergeVectors(segmentWriteState);
|
||||
}
|
||||
// write FIS once merge is done. IDV might change types or drops fields
|
||||
fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
|
||||
return mergedDocs;
|
||||
mergeState.fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
|
||||
return mergeState;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -165,7 +154,7 @@ final class SegmentMerger {
|
|||
assert !IndexFileNames.isSeparateNormsFile(file)
|
||||
: "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
|
||||
directory.copy(cfsDir, file, file, context);
|
||||
checkAbort.work(directory.fileLength(file));
|
||||
mergeState.checkAbort.work(directory.fileLength(file));
|
||||
}
|
||||
} finally {
|
||||
cfsDir.close();
|
||||
|
@ -186,45 +175,63 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
|
||||
private SegmentReader[] matchingSegmentReaders;
|
||||
private int[] rawDocLengths;
|
||||
private int[] rawDocLengths2;
|
||||
private int matchedCount;
|
||||
|
||||
public int getMatchedSubReaderCount() {
|
||||
return matchedCount;
|
||||
}
|
||||
|
||||
private void setMatchingSegmentReaders() {
|
||||
// If the i'th reader is a SegmentReader and has
|
||||
// identical fieldName -> number mapping, then this
|
||||
// array will be non-null at position i:
|
||||
int numReaders = readers.size();
|
||||
matchingSegmentReaders = new SegmentReader[numReaders];
|
||||
int numReaders = mergeState.readers.size();
|
||||
mergeState.matchingSegmentReaders = new SegmentReader[numReaders];
|
||||
|
||||
// If this reader is a SegmentReader, and all of its
|
||||
// field name -> number mappings match the "merged"
|
||||
// FieldInfos, then we can do a bulk copy of the
|
||||
// stored fields:
|
||||
for (int i = 0; i < numReaders; i++) {
|
||||
MergeState.IndexReaderAndLiveDocs reader = readers.get(i);
|
||||
MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(i);
|
||||
if (reader.reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader.reader;
|
||||
boolean same = true;
|
||||
FieldInfos segmentFieldInfos = segmentReader.fieldInfos();
|
||||
for (FieldInfo fi : segmentFieldInfos) {
|
||||
same = fieldInfos.fieldName(fi.number).equals(fi.name);
|
||||
same = mergeState.fieldInfos.fieldName(fi.number).equals(fi.name);
|
||||
}
|
||||
if (same) {
|
||||
matchingSegmentReaders[i] = segmentReader;
|
||||
matchedCount++;
|
||||
mergeState.matchingSegmentReaders[i] = segmentReader;
|
||||
mergeState.matchedCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Used for bulk-reading raw bytes for stored fields
|
||||
rawDocLengths = new int[MAX_RAW_MERGE_DOCS];
|
||||
rawDocLengths2 = new int[MAX_RAW_MERGE_DOCS];
|
||||
if (mergeState.infoStream != null) {
|
||||
mergeState.infoStream.message("SM", "merge store matchedCount=" + mergeState.matchedCount + " vs " + mergeState.readers.size());
|
||||
if (mergeState.matchedCount != mergeState.readers.size()) {
|
||||
mergeState.infoStream.message("SM", "" + (mergeState.readers.size() - mergeState.matchedCount) + " non-bulk merges");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void mergeFieldInfos() throws IOException {
|
||||
for (MergeState.IndexReaderAndLiveDocs readerAndLiveDocs : mergeState.readers) {
|
||||
final IndexReader reader = readerAndLiveDocs.reader;
|
||||
if (reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader;
|
||||
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
|
||||
for (FieldInfo fi : readerFieldInfos) {
|
||||
mergeState.fieldInfos.add(fi);
|
||||
}
|
||||
} else {
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION), true, true, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_OFFSET), true, false, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR), true, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.OMIT_POSITIONS), false, false, false, false, IndexOptions.DOCS_AND_FREQS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS), false, false, false, false, IndexOptions.DOCS_ONLY);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
mergeState.fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
|
||||
mergeState.fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.DOC_VALUES), false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -234,147 +241,15 @@ final class SegmentMerger {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
private int mergeFields() throws CorruptIndexException, IOException {
|
||||
for (MergeState.IndexReaderAndLiveDocs readerAndLiveDocs : readers) {
|
||||
final IndexReader reader = readerAndLiveDocs.reader;
|
||||
if (reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader;
|
||||
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
|
||||
for (FieldInfo fi : readerFieldInfos) {
|
||||
fieldInfos.add(fi);
|
||||
}
|
||||
} else {
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION), true, true, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_OFFSET), true, false, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR), true, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_POSITIONS), false, false, false, false, IndexOptions.DOCS_AND_FREQS);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS), false, false, false, false, IndexOptions.DOCS_ONLY);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
|
||||
fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.DOC_VALUES), false);
|
||||
}
|
||||
}
|
||||
|
||||
int docCount = 0;
|
||||
|
||||
setMatchingSegmentReaders();
|
||||
final FieldsWriter fieldsWriter = codec.fieldsFormat().fieldsWriter(directory, segment, context);
|
||||
final StoredFieldsWriter fieldsWriter = codec.storedFieldsFormat().fieldsWriter(directory, segment, context);
|
||||
try {
|
||||
int idx = 0;
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : readers) {
|
||||
final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
|
||||
FieldsReader matchingFieldsReader = null;
|
||||
if (matchingSegmentReader != null) {
|
||||
final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
|
||||
if (fieldsReader != null) {
|
||||
matchingFieldsReader = fieldsReader;
|
||||
}
|
||||
}
|
||||
if (reader.liveDocs != null) {
|
||||
docCount += copyFieldsWithDeletions(fieldsWriter,
|
||||
reader, matchingFieldsReader);
|
||||
} else {
|
||||
docCount += copyFieldsNoDeletions(fieldsWriter,
|
||||
reader, matchingFieldsReader);
|
||||
}
|
||||
}
|
||||
docCount = fieldsWriter.merge(mergeState);
|
||||
} finally {
|
||||
fieldsWriter.close();
|
||||
}
|
||||
|
||||
final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
|
||||
final long fdxFileLength = directory.fileLength(fileName);
|
||||
|
||||
if (4+((long) docCount)*8 != fdxFileLength)
|
||||
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
|
||||
// we detect that the bug has struck, here, and
|
||||
// throw an exception to prevent the corruption from
|
||||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
|
||||
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codec, null, context);
|
||||
|
||||
return docCount;
|
||||
}
|
||||
|
||||
private int copyFieldsWithDeletions(final FieldsWriter fieldsWriter, final MergeState.IndexReaderAndLiveDocs reader,
|
||||
final FieldsReader matchingFieldsReader)
|
||||
throws IOException, MergeAbortedException, CorruptIndexException {
|
||||
int docCount = 0;
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
assert liveDocs != null;
|
||||
if (matchingFieldsReader != null) {
|
||||
// We can bulk-copy because the fieldInfos are "congruent"
|
||||
for (int j = 0; j < maxDoc;) {
|
||||
if (!liveDocs.get(j)) {
|
||||
// skip deleted docs
|
||||
++j;
|
||||
continue;
|
||||
}
|
||||
// We can optimize this case (doing a bulk byte copy) since the field
|
||||
// numbers are identical
|
||||
int start = j, numDocs = 0;
|
||||
do {
|
||||
j++;
|
||||
numDocs++;
|
||||
if (j >= maxDoc) break;
|
||||
if (!liveDocs.get(j)) {
|
||||
j++;
|
||||
break;
|
||||
}
|
||||
} while(numDocs < MAX_RAW_MERGE_DOCS);
|
||||
|
||||
IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, start, numDocs);
|
||||
fieldsWriter.addRawDocuments(stream, rawDocLengths, numDocs);
|
||||
docCount += numDocs;
|
||||
checkAbort.work(300 * numDocs);
|
||||
}
|
||||
} else {
|
||||
for (int j = 0; j < maxDoc; j++) {
|
||||
if (!liveDocs.get(j)) {
|
||||
// skip deleted docs
|
||||
continue;
|
||||
}
|
||||
// TODO: this could be more efficient using
|
||||
// FieldVisitor instead of loading/writing entire
|
||||
// doc; ie we just have to renumber the field number
|
||||
// on the fly?
|
||||
// NOTE: it's very important to first assign to doc then pass it to
|
||||
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
|
||||
Document doc = reader.reader.document(j);
|
||||
fieldsWriter.addDocument(doc, fieldInfos);
|
||||
docCount++;
|
||||
checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
return docCount;
|
||||
}
|
||||
|
||||
private int copyFieldsNoDeletions(final FieldsWriter fieldsWriter, final MergeState.IndexReaderAndLiveDocs reader,
|
||||
final FieldsReader matchingFieldsReader)
|
||||
throws IOException, MergeAbortedException, CorruptIndexException {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
int docCount = 0;
|
||||
if (matchingFieldsReader != null) {
|
||||
// We can bulk-copy because the fieldInfos are "congruent"
|
||||
while (docCount < maxDoc) {
|
||||
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
|
||||
IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, docCount, len);
|
||||
fieldsWriter.addRawDocuments(stream, rawDocLengths, len);
|
||||
docCount += len;
|
||||
checkAbort.work(300 * len);
|
||||
}
|
||||
} else {
|
||||
for (; docCount < maxDoc; docCount++) {
|
||||
// NOTE: it's very important to first assign to doc then pass it to
|
||||
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
|
||||
Document doc = reader.reader.document(docCount);
|
||||
fieldsWriter.addDocument(doc, fieldInfos);
|
||||
checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
return docCount;
|
||||
}
|
||||
|
||||
|
@ -382,13 +257,15 @@ final class SegmentMerger {
|
|||
* Merge the TermVectors from each of the segments into the new one.
|
||||
* @throws IOException
|
||||
*/
|
||||
private final void mergeVectors() throws IOException {
|
||||
TermVectorsWriter termVectorsWriter = new TermVectorsWriter(directory, segment, fieldInfos, context);
|
||||
|
||||
private final void mergeVectors(SegmentWriteState segmentWriteState) throws IOException {
|
||||
TermVectorsWriter termVectorsWriter = new TermVectorsWriter(directory, segment, mergeState.fieldInfos, context);
|
||||
// Used for bulk-reading raw bytes for term vectors
|
||||
int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
|
||||
int rawDocLengths2[] = new int[MAX_RAW_MERGE_DOCS];
|
||||
try {
|
||||
int idx = 0;
|
||||
for (final MergeState.IndexReaderAndLiveDocs reader : readers) {
|
||||
final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
|
||||
for (final MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
|
||||
TermVectorsReader matchingVectorsReader = null;
|
||||
if (matchingSegmentReader != null) {
|
||||
TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();
|
||||
|
@ -399,9 +276,9 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
if (reader.liveDocs != null) {
|
||||
copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader);
|
||||
copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
|
||||
} else {
|
||||
copyVectorsNoDeletions(termVectorsWriter, matchingVectorsReader, reader);
|
||||
copyVectorsNoDeletions(termVectorsWriter, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
|
@ -410,6 +287,7 @@ final class SegmentMerger {
|
|||
|
||||
final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
final long tvxSize = directory.fileLength(fileName);
|
||||
final int mergedDocs = segmentWriteState.numDocs;
|
||||
|
||||
if (4+((long) mergedDocs)*16 != tvxSize)
|
||||
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
|
||||
|
@ -422,7 +300,9 @@ final class SegmentMerger {
|
|||
|
||||
private void copyVectorsWithDeletions(final TermVectorsWriter termVectorsWriter,
|
||||
final TermVectorsReader matchingVectorsReader,
|
||||
final MergeState.IndexReaderAndLiveDocs reader)
|
||||
final MergeState.IndexReaderAndLiveDocs reader,
|
||||
int rawDocLengths[],
|
||||
int rawDocLengths2[])
|
||||
throws IOException, MergeAbortedException {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
|
@ -449,7 +329,7 @@ final class SegmentMerger {
|
|||
|
||||
matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
|
||||
termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
|
||||
checkAbort.work(300 * numDocs);
|
||||
mergeState.checkAbort.work(300 * numDocs);
|
||||
}
|
||||
} else {
|
||||
for (int docNum = 0; docNum < maxDoc; docNum++) {
|
||||
|
@ -462,14 +342,16 @@ final class SegmentMerger {
|
|||
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
|
||||
TermFreqVector[] vectors = reader.reader.getTermFreqVectors(docNum);
|
||||
termVectorsWriter.addAllDocVectors(vectors);
|
||||
checkAbort.work(300);
|
||||
mergeState.checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void copyVectorsNoDeletions(final TermVectorsWriter termVectorsWriter,
|
||||
final TermVectorsReader matchingVectorsReader,
|
||||
final MergeState.IndexReaderAndLiveDocs reader)
|
||||
final MergeState.IndexReaderAndLiveDocs reader,
|
||||
int rawDocLengths[],
|
||||
int rawDocLengths2[])
|
||||
throws IOException, MergeAbortedException {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
if (matchingVectorsReader != null) {
|
||||
|
@ -480,7 +362,7 @@ final class SegmentMerger {
|
|||
matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, docCount, len);
|
||||
termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, len);
|
||||
docCount += len;
|
||||
checkAbort.work(300 * len);
|
||||
mergeState.checkAbort.work(300 * len);
|
||||
}
|
||||
} else {
|
||||
for (int docNum = 0; docNum < maxDoc; docNum++) {
|
||||
|
@ -488,27 +370,18 @@ final class SegmentMerger {
|
|||
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
|
||||
TermFreqVector[] vectors = reader.reader.getTermFreqVectors(docNum);
|
||||
termVectorsWriter.addAllDocVectors(vectors);
|
||||
checkAbort.work(300);
|
||||
mergeState.checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Codec getCodec() {
|
||||
assert segmentWriteState != null;
|
||||
return segmentWriteState.codec;
|
||||
}
|
||||
|
||||
private final void mergeTerms() throws CorruptIndexException, IOException {
|
||||
|
||||
// Let CodecProvider decide which codec will be used to write
|
||||
// the new segment:
|
||||
|
||||
private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
|
||||
int docBase = 0;
|
||||
|
||||
final List<Fields> fields = new ArrayList<Fields>();
|
||||
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
|
||||
|
||||
for(MergeState.IndexReaderAndLiveDocs r : readers) {
|
||||
for(MergeState.IndexReaderAndLiveDocs r : mergeState.readers) {
|
||||
final Fields f = r.reader.fields();
|
||||
final int maxDoc = r.reader.maxDoc();
|
||||
if (f != null) {
|
||||
|
@ -518,30 +391,15 @@ final class SegmentMerger {
|
|||
docBase += maxDoc;
|
||||
}
|
||||
|
||||
// we may gather more readers than mergeState.readerCount
|
||||
mergeState = new MergeState();
|
||||
mergeState.readers = readers;
|
||||
mergeState.readerCount = readers.size();
|
||||
mergeState.fieldInfos = fieldInfos;
|
||||
mergeState.mergedDocCount = mergedDocs;
|
||||
|
||||
// Remap docIDs
|
||||
mergeState.docMaps = new int[mergeState.readerCount][];
|
||||
mergeState.docBase = new int[mergeState.readerCount];
|
||||
mergeState.hasPayloadProcessorProvider = payloadProcessorProvider != null;
|
||||
mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[mergeState.readerCount];
|
||||
mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[mergeState.readerCount];
|
||||
mergeState.checkAbort = checkAbort;
|
||||
final int numReaders = mergeState.readers.size();
|
||||
|
||||
docBase = 0;
|
||||
int inputDocBase = 0;
|
||||
|
||||
for(int i=0;i<mergeState.readerCount;i++) {
|
||||
for(int i=0;i<numReaders;i++) {
|
||||
|
||||
final MergeState.IndexReaderAndLiveDocs reader = readers.get(i);
|
||||
final MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(i);
|
||||
|
||||
mergeState.docBase[i] = docBase;
|
||||
inputDocBase += reader.reader.maxDoc();
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
if (reader.liveDocs != null) {
|
||||
int delCount = 0;
|
||||
|
@ -562,8 +420,8 @@ final class SegmentMerger {
|
|||
docBase += maxDoc;
|
||||
}
|
||||
|
||||
if (payloadProcessorProvider != null) {
|
||||
mergeState.dirPayloadProcessor[i] = payloadProcessorProvider.getDirProcessor(reader.reader.directory());
|
||||
if (mergeState.payloadProcessorProvider != null) {
|
||||
mergeState.dirPayloadProcessor[i] = mergeState.payloadProcessorProvider.getDirProcessor(reader.reader.directory());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -583,7 +441,7 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
|
||||
private void mergePerDoc() throws IOException {
|
||||
private void mergePerDoc(SegmentWriteState segmentWriteState) throws IOException {
|
||||
final PerDocConsumer docsConsumer = codec.docValuesFormat()
|
||||
.docsConsumer(new PerDocWriteState(segmentWriteState));
|
||||
// TODO: remove this check when 3.x indexes are no longer supported
|
||||
|
@ -603,25 +461,18 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
private MergeState mergeState;
|
||||
|
||||
public boolean getAnyNonBulkMerges() {
|
||||
assert matchedCount <= readers.size();
|
||||
return matchedCount != readers.size();
|
||||
}
|
||||
|
||||
private void mergeNorms() throws IOException {
|
||||
IndexOutput output = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
for (FieldInfo fi : mergeState.fieldInfos) {
|
||||
if (fi.isIndexed && !fi.omitNorms) {
|
||||
if (output == null) {
|
||||
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION), context);
|
||||
output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
|
||||
}
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : readers) {
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
byte normBuffer[] = reader.reader.norms(fi.name);
|
||||
if (normBuffer == null) {
|
||||
|
@ -643,7 +494,7 @@ final class SegmentMerger {
|
|||
}
|
||||
}
|
||||
}
|
||||
checkAbort.work(maxDoc);
|
||||
mergeState.checkAbort.work(maxDoc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.codecs.FieldsReader;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
|
@ -47,7 +47,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
|
||||
private SegmentInfo si;
|
||||
private final ReaderContext readerContext = new AtomicReaderContext(this);
|
||||
CloseableThreadLocal<FieldsReader> fieldsReaderLocal = new FieldsReaderLocal();
|
||||
CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new FieldsReaderLocal();
|
||||
CloseableThreadLocal<TermVectorsReader> termVectorsLocal = new CloseableThreadLocal<TermVectorsReader>();
|
||||
|
||||
volatile BitVector liveDocs;
|
||||
|
@ -74,9 +74,9 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
/**
|
||||
* Sets the initial value
|
||||
*/
|
||||
private class FieldsReaderLocal extends CloseableThreadLocal<FieldsReader> {
|
||||
private class FieldsReaderLocal extends CloseableThreadLocal<StoredFieldsReader> {
|
||||
@Override
|
||||
protected FieldsReader initialValue() {
|
||||
protected StoredFieldsReader initialValue() {
|
||||
return core.getFieldsReaderOrig().clone();
|
||||
}
|
||||
}
|
||||
|
@ -368,7 +368,8 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
hasChanges = false;
|
||||
}
|
||||
|
||||
FieldsReader getFieldsReader() {
|
||||
/** @lucene.internal */
|
||||
public StoredFieldsReader getFieldsReader() {
|
||||
return fieldsReaderLocal.get();
|
||||
}
|
||||
|
||||
|
|
|
@ -17,18 +17,17 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.PrintStream;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SegmentWriteState {
|
||||
public final PrintStream infoStream;
|
||||
public final InfoStream infoStream;
|
||||
public final Directory directory;
|
||||
public final String segmentName;
|
||||
public final FieldInfos fieldInfos;
|
||||
|
@ -56,7 +55,7 @@ public class SegmentWriteState {
|
|||
|
||||
public final IOContext context;
|
||||
|
||||
public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
|
||||
public SegmentWriteState(InfoStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
|
||||
int numDocs, int termIndexInterval, Codec codec, BufferedDeletes segDeletes, IOContext context) {
|
||||
this.infoStream = infoStream;
|
||||
this.segDeletes = segDeletes;
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
||||
/**
|
||||
* Expert: provides a low-level means of accessing the stored field
|
||||
|
@ -35,54 +34,39 @@ import org.apache.lucene.store.IndexInput;
|
|||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
public class StoredFieldVisitor {
|
||||
/** Process a binary field. Note that if you want to
|
||||
* skip the field you must seek the IndexInput
|
||||
* (e.g., call <code>in.seek(numUTF8Bytes + in.getFilePointer()</code>)
|
||||
*
|
||||
* <p>Return true to stop loading fields. */
|
||||
public boolean binaryField(FieldInfo fieldInfo, IndexInput in, int numBytes) throws IOException {
|
||||
in.seek(in.getFilePointer() + numBytes);
|
||||
return false;
|
||||
public abstract class StoredFieldVisitor {
|
||||
/** Process a binary field. */
|
||||
public void binaryField(FieldInfo fieldInfo, byte[] value, int offset, int length) throws IOException {
|
||||
}
|
||||
|
||||
/** Process a string field by reading numUTF8Bytes.
|
||||
* Note that if you want to skip the field you must
|
||||
* seek the IndexInput as if you had read numBytes by
|
||||
* (e.g., call <code>in.seek(numUTF8Bytes + in.getFilePointer()</code>)
|
||||
*
|
||||
* <p>Return true to stop loading fields. */
|
||||
public boolean stringField(FieldInfo fieldInfo, IndexInput in, int numUTF8Bytes) throws IOException {
|
||||
in.seek(in.getFilePointer() + numUTF8Bytes);
|
||||
return false;
|
||||
/** Process a string field */
|
||||
public void stringField(FieldInfo fieldInfo, String value) throws IOException {
|
||||
}
|
||||
|
||||
/** Process a int numeric field.
|
||||
*
|
||||
* <p>Return true to stop loading fields. */
|
||||
public boolean intField(FieldInfo fieldInfo, int value) throws IOException {
|
||||
return false;
|
||||
/** Process a int numeric field. */
|
||||
public void intField(FieldInfo fieldInfo, int value) throws IOException {
|
||||
}
|
||||
|
||||
/** Process a long numeric field.
|
||||
*
|
||||
* <p>Return true to stop loading fields. */
|
||||
public boolean longField(FieldInfo fieldInfo, long value) throws IOException {
|
||||
return false;
|
||||
/** Process a long numeric field. */
|
||||
public void longField(FieldInfo fieldInfo, long value) throws IOException {
|
||||
}
|
||||
|
||||
/** Process a float numeric field.
|
||||
*
|
||||
* <p>Return true to stop loading fields. */
|
||||
public boolean floatField(FieldInfo fieldInfo, float value) throws IOException {
|
||||
return false;
|
||||
/** Process a float numeric field. */
|
||||
public void floatField(FieldInfo fieldInfo, float value) throws IOException {
|
||||
}
|
||||
|
||||
/** Process a double numeric field.
|
||||
*
|
||||
* <p>Return true to stop loading fields. */
|
||||
public boolean doubleField(FieldInfo fieldInfo, double value) throws IOException {
|
||||
return false;
|
||||
/** Process a double numeric field. */
|
||||
public void doubleField(FieldInfo fieldInfo, double value) throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public abstract Status needsField(FieldInfo fieldInfo) throws IOException;
|
||||
|
||||
public static enum Status {
|
||||
/** yes, i want the field */
|
||||
YES,
|
||||
/** no, i do not */
|
||||
NO,
|
||||
/** stop loading fields for this document entirely */
|
||||
STOP
|
||||
}
|
||||
}
|
|
@ -20,15 +20,15 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldsWriter;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/** This is a DocFieldConsumer that writes stored fields. */
|
||||
final class StoredFieldsWriter {
|
||||
final class StoredFieldsConsumer {
|
||||
|
||||
FieldsWriter fieldsWriter;
|
||||
StoredFieldsWriter fieldsWriter;
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
int lastDocID;
|
||||
|
||||
|
@ -37,7 +37,7 @@ final class StoredFieldsWriter {
|
|||
final DocumentsWriterPerThread.DocState docState;
|
||||
final Codec codec;
|
||||
|
||||
public StoredFieldsWriter(DocumentsWriterPerThread docWriter) {
|
||||
public StoredFieldsConsumer(DocumentsWriterPerThread docWriter) {
|
||||
this.docWriter = docWriter;
|
||||
this.docState = docWriter.docState;
|
||||
this.codec = docWriter.codec;
|
||||
|
@ -68,20 +68,19 @@ final class StoredFieldsWriter {
|
|||
}
|
||||
|
||||
if (fieldsWriter != null) {
|
||||
fieldsWriter.close();
|
||||
fieldsWriter = null;
|
||||
lastDocID = 0;
|
||||
|
||||
String fieldsIdxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
|
||||
if (4 + ((long) state.numDocs) * 8 != state.directory.fileLength(fieldsIdxName)) {
|
||||
throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
|
||||
try {
|
||||
fieldsWriter.finish(state.numDocs);
|
||||
} finally {
|
||||
fieldsWriter.close();
|
||||
fieldsWriter = null;
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void initFieldsWriter(IOContext context) throws IOException {
|
||||
if (fieldsWriter == null) {
|
||||
fieldsWriter = codec.fieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegment(), context);
|
||||
fieldsWriter = codec.storedFieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegment(), context);
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
@ -103,7 +102,7 @@ final class StoredFieldsWriter {
|
|||
// We must "catch up" for all docs before us
|
||||
// that had no stored fields:
|
||||
while(lastDocID < docID) {
|
||||
fieldsWriter.skipDocument();
|
||||
fieldsWriter.startDocument(0);
|
||||
lastDocID++;
|
||||
}
|
||||
}
|
|
@ -26,6 +26,8 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/**
|
||||
* Merges segments of approximately equal size, subject to
|
||||
* an allowed number of segments per tier. This is similar
|
||||
|
@ -648,7 +650,10 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
|
||||
private void message(String message) {
|
||||
if (verbose()) {
|
||||
writer.get().message("TMP: " + message);
|
||||
final InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("TMP", message);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -151,8 +152,12 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
private void message(String message) {
|
||||
if (verbose())
|
||||
writer.get().message("UPGMP: " + message);
|
||||
if (verbose()) {
|
||||
final InfoStream infoStream = writer.get().infoStream;
|
||||
if (infoStream != null) {
|
||||
infoStream.message("UPGMP", message);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -45,7 +45,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
postingsFormat().files(dir, info, "", files);
|
||||
//TODO: not yet fieldsFormat().files(dir, info, files);
|
||||
storedFieldsFormat().files(dir, info, files);
|
||||
docValuesFormat().files(dir, info, files);
|
||||
}
|
||||
|
||||
|
@ -55,9 +55,10 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
/** Encodes/decodes docvalues */
|
||||
public abstract DocValuesFormat docValuesFormat();
|
||||
|
||||
/** Encodes/decodes stored fields, term vectors, fieldinfos */
|
||||
public abstract FieldsFormat fieldsFormat();
|
||||
/** Encodes/decodes stored fields */
|
||||
public abstract StoredFieldsFormat storedFieldsFormat();
|
||||
|
||||
/** Encodes/decodes segments file */
|
||||
public abstract SegmentInfosFormat segmentInfosFormat();
|
||||
|
||||
/** looks up a codec by name */
|
||||
|
|
|
@ -69,7 +69,7 @@ public class DefaultSegmentInfosReader extends SegmentInfosReader {
|
|||
}
|
||||
|
||||
try {
|
||||
DefaultFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
|
||||
DefaultStoredFieldsReader.checkCodeVersion(dir, si.getDocStoreSegment());
|
||||
} finally {
|
||||
// If we opened the directory, close it
|
||||
if (dir != directory) dir.close();
|
||||
|
|
|
@ -26,23 +26,22 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public class DefaultFieldsFormat extends FieldsFormat {
|
||||
public class DefaultStoredFieldsFormat extends StoredFieldsFormat {
|
||||
|
||||
@Override
|
||||
public FieldsReader fieldsReader(Directory directory, String segment,
|
||||
FieldInfos fn, IOContext context, int docStoreOffset, int size)
|
||||
throws IOException {
|
||||
return new DefaultFieldsReader(directory, segment, fn, context, docStoreOffset, size);
|
||||
public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si,
|
||||
FieldInfos fn, IOContext context) throws IOException {
|
||||
return new DefaultStoredFieldsReader(directory, si, fn, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsWriter fieldsWriter(Directory directory, String segment,
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, String segment,
|
||||
IOContext context) throws IOException {
|
||||
return new DefaultFieldsWriter(directory, segment, context);
|
||||
return new DefaultStoredFieldsWriter(directory, segment, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
// TODO!
|
||||
DefaultStoredFieldsReader.files(dir, info, files);
|
||||
}
|
||||
}
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.index.FieldReaderException;
|
|||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -35,6 +36,8 @@ import org.apache.lucene.util.CloseableThreadLocal;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Class responsible for access to stored document fields.
|
||||
|
@ -43,7 +46,7 @@ import java.io.Closeable;
|
|||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public final class DefaultFieldsReader extends FieldsReader implements Cloneable, Closeable {
|
||||
public final class DefaultStoredFieldsReader extends StoredFieldsReader implements Cloneable, Closeable {
|
||||
private final static int FORMAT_SIZE = 4;
|
||||
|
||||
private final FieldInfos fieldInfos;
|
||||
|
@ -75,22 +78,22 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
* clones are called (eg, currently SegmentReader manages
|
||||
* this logic). */
|
||||
@Override
|
||||
public DefaultFieldsReader clone() {
|
||||
public DefaultStoredFieldsReader clone() {
|
||||
ensureOpen();
|
||||
return new DefaultFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, cloneableFieldsStream, cloneableIndexStream);
|
||||
return new DefaultStoredFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, cloneableFieldsStream, cloneableIndexStream);
|
||||
}
|
||||
|
||||
/** Verifies that the code version which wrote the segment is supported. */
|
||||
public static void checkCodeVersion(Directory dir, String segment) throws IOException {
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", DefaultFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", DefaultStoredFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
IndexInput idxStream = dir.openInput(indexStreamFN, IOContext.DEFAULT);
|
||||
|
||||
try {
|
||||
int format = idxStream.readInt();
|
||||
if (format < DefaultFieldsWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(idxStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
|
||||
if (format > DefaultFieldsWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(idxStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
|
||||
if (format < DefaultStoredFieldsWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(idxStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
|
||||
if (format > DefaultStoredFieldsWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(idxStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
|
||||
} finally {
|
||||
idxStream.close();
|
||||
}
|
||||
|
@ -98,7 +101,7 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
}
|
||||
|
||||
// Used only by clone
|
||||
private DefaultFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
|
||||
private DefaultStoredFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
|
||||
IndexInput cloneableFieldsStream, IndexInput cloneableIndexStream) {
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.numTotalDocs = numTotalDocs;
|
||||
|
@ -110,27 +113,26 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
fieldsStream = (IndexInput) cloneableFieldsStream.clone();
|
||||
indexStream = (IndexInput) cloneableIndexStream.clone();
|
||||
}
|
||||
|
||||
public DefaultFieldsReader(Directory d, String segment, FieldInfos fn) throws IOException {
|
||||
this(d, segment, fn, IOContext.DEFAULT, -1, 0);
|
||||
}
|
||||
|
||||
public DefaultFieldsReader(Directory d, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException {
|
||||
public DefaultStoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
|
||||
final String segment = si.getDocStoreSegment();
|
||||
final int docStoreOffset = si.getDocStoreOffset();
|
||||
final int size = si.docCount;
|
||||
boolean success = false;
|
||||
isOriginal = true;
|
||||
try {
|
||||
fieldInfos = fn;
|
||||
|
||||
cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", DefaultFieldsWriter.FIELDS_EXTENSION), context);
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", DefaultFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", DefaultStoredFieldsWriter.FIELDS_EXTENSION), context);
|
||||
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", DefaultStoredFieldsWriter.FIELDS_INDEX_EXTENSION);
|
||||
cloneableIndexStream = d.openInput(indexStreamFN, context);
|
||||
|
||||
format = cloneableIndexStream.readInt();
|
||||
|
||||
if (format < DefaultFieldsWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(cloneableIndexStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
|
||||
if (format > DefaultFieldsWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(cloneableIndexStream, format, DefaultFieldsWriter.FORMAT_MINIMUM, DefaultFieldsWriter.FORMAT_CURRENT);
|
||||
if (format < DefaultStoredFieldsWriter.FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(cloneableIndexStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
|
||||
if (format > DefaultStoredFieldsWriter.FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(cloneableIndexStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
|
||||
|
||||
fieldsStream = (IndexInput) cloneableFieldsStream.clone();
|
||||
|
||||
|
@ -147,6 +149,10 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
} else {
|
||||
this.docStoreOffset = 0;
|
||||
this.size = (int) (indexSize >> 3);
|
||||
// Verify two sources of "maxDoc" agree:
|
||||
if (this.size != si.docCount) {
|
||||
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.docCount);
|
||||
}
|
||||
}
|
||||
|
||||
indexStream = (IndexInput) cloneableIndexStream.clone();
|
||||
|
@ -208,41 +214,72 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
|
||||
|
||||
int bits = fieldsStream.readByte() & 0xFF;
|
||||
assert bits <= (DefaultFieldsWriter.FIELD_IS_NUMERIC_MASK | DefaultFieldsWriter.FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
|
||||
assert bits <= (DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_MASK | DefaultStoredFieldsWriter.FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
|
||||
|
||||
final boolean binary = (bits & DefaultFieldsWriter.FIELD_IS_BINARY) != 0;
|
||||
final int numeric = bits & DefaultFieldsWriter.FIELD_IS_NUMERIC_MASK;
|
||||
switch(visitor.needsField(fieldInfo)) {
|
||||
case YES:
|
||||
readField(visitor, fieldInfo, bits);
|
||||
break;
|
||||
case NO:
|
||||
skipField(bits);
|
||||
break;
|
||||
case STOP:
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static final Charset UTF8 = Charset.forName("UTF-8");
|
||||
|
||||
final boolean doStop;
|
||||
if (binary) {
|
||||
final int numBytes = fieldsStream.readVInt();
|
||||
doStop = visitor.binaryField(fieldInfo, fieldsStream, numBytes);
|
||||
} else if (numeric != 0) {
|
||||
switch(numeric) {
|
||||
case DefaultFieldsWriter.FIELD_IS_NUMERIC_INT:
|
||||
doStop = visitor.intField(fieldInfo, fieldsStream.readInt());
|
||||
break;
|
||||
case DefaultFieldsWriter.FIELD_IS_NUMERIC_LONG:
|
||||
doStop = visitor.longField(fieldInfo, fieldsStream.readLong());
|
||||
break;
|
||||
case DefaultFieldsWriter.FIELD_IS_NUMERIC_FLOAT:
|
||||
doStop = visitor.floatField(fieldInfo, Float.intBitsToFloat(fieldsStream.readInt()));
|
||||
break;
|
||||
case DefaultFieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
|
||||
doStop = visitor.doubleField(fieldInfo, Double.longBitsToDouble(fieldsStream.readLong()));
|
||||
break;
|
||||
private void readField(StoredFieldVisitor visitor, FieldInfo info, int bits) throws IOException {
|
||||
final int numeric = bits & DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_MASK;
|
||||
if (numeric != 0) {
|
||||
switch(numeric) {
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_INT:
|
||||
visitor.intField(info, fieldsStream.readInt());
|
||||
return;
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_LONG:
|
||||
visitor.longField(info, fieldsStream.readLong());
|
||||
return;
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_FLOAT:
|
||||
visitor.floatField(info, Float.intBitsToFloat(fieldsStream.readInt()));
|
||||
return;
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
|
||||
visitor.doubleField(info, Double.longBitsToDouble(fieldsStream.readLong()));
|
||||
return;
|
||||
default:
|
||||
throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
final int length = fieldsStream.readVInt();
|
||||
byte bytes[] = new byte[length];
|
||||
fieldsStream.readBytes(bytes, 0, length);
|
||||
if ((bits & DefaultStoredFieldsWriter.FIELD_IS_BINARY) != 0) {
|
||||
visitor.binaryField(info, bytes, 0, bytes.length);
|
||||
} else {
|
||||
// Text:
|
||||
final int numUTF8Bytes = fieldsStream.readVInt();
|
||||
doStop = visitor.stringField(fieldInfo, fieldsStream, numUTF8Bytes);
|
||||
visitor.stringField(info, new String(bytes, 0, bytes.length, UTF8));
|
||||
}
|
||||
|
||||
if (doStop) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
private void skipField(int bits) throws IOException {
|
||||
final int numeric = bits & DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_MASK;
|
||||
if (numeric != 0) {
|
||||
switch(numeric) {
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_INT:
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_FLOAT:
|
||||
fieldsStream.readInt();
|
||||
return;
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_LONG:
|
||||
case DefaultStoredFieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
|
||||
fieldsStream.readLong();
|
||||
return;
|
||||
default:
|
||||
throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
|
||||
}
|
||||
} else {
|
||||
final int length = fieldsStream.readVInt();
|
||||
fieldsStream.seek(fieldsStream.getFilePointer() + length);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -271,4 +308,19 @@ public final class DefaultFieldsReader extends FieldsReader implements Cloneable
|
|||
|
||||
return fieldsStream;
|
||||
}
|
||||
|
||||
// TODO: split into PreFlexFieldsReader so it can handle this shared docstore crap?
|
||||
// only preflex segments refer to these?
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
if (!info.getDocStoreIsCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", DefaultStoredFieldsWriter.FIELDS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", DefaultStoredFieldsWriter.FIELDS_EXTENSION));
|
||||
}
|
||||
} else {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", DefaultStoredFieldsWriter.FIELDS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", DefaultStoredFieldsWriter.FIELDS_EXTENSION));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,18 +18,23 @@ package org.apache.lucene.index.codecs;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.SegmentReader;
|
||||
import org.apache.lucene.index.MergePolicy.MergeAbortedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/** @lucene.experimental */
|
||||
public final class DefaultFieldsWriter extends FieldsWriter {
|
||||
public final class DefaultStoredFieldsWriter extends StoredFieldsWriter {
|
||||
// NOTE: bit 0 is free here! You can steal it!
|
||||
static final int FIELD_IS_BINARY = 1 << 1;
|
||||
|
||||
|
@ -61,17 +66,19 @@ public final class DefaultFieldsWriter extends FieldsWriter {
|
|||
// when removing support for old versions, leave the last supported version here
|
||||
static final int FORMAT_MINIMUM = FORMAT_LUCENE_3_0_NO_COMPRESSED_FIELDS;
|
||||
|
||||
// TODO: remove from IndexFileNames
|
||||
public static final String FIELDS_EXTENSION = IndexFileNames.FIELDS_EXTENSION;
|
||||
public static final String FIELDS_INDEX_EXTENSION = IndexFileNames.FIELDS_INDEX_EXTENSION;
|
||||
/** Extension of stored fields file */
|
||||
public static final String FIELDS_EXTENSION = "fdt";
|
||||
|
||||
/** Extension of stored fields index file */
|
||||
public static final String FIELDS_INDEX_EXTENSION = "fdx";
|
||||
|
||||
// If null - we were supplied with streams, if notnull - we manage them ourselves
|
||||
private Directory directory;
|
||||
private String segment;
|
||||
private IndexOutput fieldsStream;
|
||||
private IndexOutput indexStream;
|
||||
|
||||
DefaultFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
public DefaultStoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
assert directory != null;
|
||||
this.directory = directory;
|
||||
this.segment = segment;
|
||||
|
||||
|
@ -91,17 +98,6 @@ public final class DefaultFieldsWriter extends FieldsWriter {
|
|||
}
|
||||
}
|
||||
|
||||
DefaultFieldsWriter(IndexOutput fdx, IndexOutput fdt) {
|
||||
directory = null;
|
||||
segment = null;
|
||||
fieldsStream = fdt;
|
||||
indexStream = fdx;
|
||||
}
|
||||
|
||||
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
|
||||
|
@ -111,35 +107,26 @@ public final class DefaultFieldsWriter extends FieldsWriter {
|
|||
fieldsStream.writeVInt(numStoredFields);
|
||||
}
|
||||
|
||||
public void skipDocument() throws IOException {
|
||||
indexStream.writeLong(fieldsStream.getFilePointer());
|
||||
fieldsStream.writeVInt(0);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
if (directory != null) {
|
||||
try {
|
||||
IOUtils.close(fieldsStream, indexStream);
|
||||
} finally {
|
||||
fieldsStream = indexStream = null;
|
||||
}
|
||||
try {
|
||||
IOUtils.close(fieldsStream, indexStream);
|
||||
} finally {
|
||||
fieldsStream = indexStream = null;
|
||||
}
|
||||
}
|
||||
|
||||
public void abort() {
|
||||
if (directory != null) {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -220,21 +207,128 @@ public final class DefaultFieldsWriter extends FieldsWriter {
|
|||
assert fieldsStream.getFilePointer() == position;
|
||||
}
|
||||
|
||||
public final void addDocument(Iterable<? extends IndexableField> doc, FieldInfos fieldInfos) throws IOException {
|
||||
indexStream.writeLong(fieldsStream.getFilePointer());
|
||||
|
||||
int storedCount = 0;
|
||||
for (IndexableField field : doc) {
|
||||
if (field.fieldType().stored()) {
|
||||
storedCount++;
|
||||
@Override
|
||||
public void finish(int numDocs) throws IOException {
|
||||
if (4+((long) numDocs)*8 != indexStream.getFilePointer())
|
||||
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
|
||||
// we detect that the bug has struck, here, and
|
||||
// throw an exception to prevent the corruption from
|
||||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + numDocs + " but fdx file size is " + indexStream.getFilePointer() + " file=" + indexStream.toString() + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int merge(MergeState mergeState) throws IOException {
|
||||
int docCount = 0;
|
||||
// Used for bulk-reading raw bytes for stored fields
|
||||
int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
|
||||
int idx = 0;
|
||||
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
|
||||
DefaultStoredFieldsReader matchingFieldsReader = null;
|
||||
if (matchingSegmentReader != null) {
|
||||
final StoredFieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
|
||||
// we can only bulk-copy if the matching reader is also a DefaultFieldsReader
|
||||
if (fieldsReader != null && fieldsReader instanceof DefaultStoredFieldsReader) {
|
||||
matchingFieldsReader = (DefaultStoredFieldsReader) fieldsReader;
|
||||
}
|
||||
}
|
||||
|
||||
if (reader.liveDocs != null) {
|
||||
docCount += copyFieldsWithDeletions(mergeState,
|
||||
reader, matchingFieldsReader, rawDocLengths);
|
||||
} else {
|
||||
docCount += copyFieldsNoDeletions(mergeState,
|
||||
reader, matchingFieldsReader, rawDocLengths);
|
||||
}
|
||||
}
|
||||
fieldsStream.writeVInt(storedCount);
|
||||
finish(docCount);
|
||||
return docCount;
|
||||
}
|
||||
|
||||
for (IndexableField field : doc) {
|
||||
if (field.fieldType().stored()) {
|
||||
writeField(fieldInfos.fieldNumber(field.name()), field);
|
||||
/** Maximum number of contiguous documents to bulk-copy
|
||||
when merging stored fields */
|
||||
private final static int MAX_RAW_MERGE_DOCS = 4192;
|
||||
|
||||
private int copyFieldsWithDeletions(MergeState mergeState, final MergeState.IndexReaderAndLiveDocs reader,
|
||||
final DefaultStoredFieldsReader matchingFieldsReader, int rawDocLengths[])
|
||||
throws IOException, MergeAbortedException, CorruptIndexException {
|
||||
int docCount = 0;
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
assert liveDocs != null;
|
||||
if (matchingFieldsReader != null) {
|
||||
// We can bulk-copy because the fieldInfos are "congruent"
|
||||
for (int j = 0; j < maxDoc;) {
|
||||
if (!liveDocs.get(j)) {
|
||||
// skip deleted docs
|
||||
++j;
|
||||
continue;
|
||||
}
|
||||
// We can optimize this case (doing a bulk byte copy) since the field
|
||||
// numbers are identical
|
||||
int start = j, numDocs = 0;
|
||||
do {
|
||||
j++;
|
||||
numDocs++;
|
||||
if (j >= maxDoc) break;
|
||||
if (!liveDocs.get(j)) {
|
||||
j++;
|
||||
break;
|
||||
}
|
||||
} while(numDocs < MAX_RAW_MERGE_DOCS);
|
||||
|
||||
IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, start, numDocs);
|
||||
addRawDocuments(stream, rawDocLengths, numDocs);
|
||||
docCount += numDocs;
|
||||
mergeState.checkAbort.work(300 * numDocs);
|
||||
}
|
||||
} else {
|
||||
for (int j = 0; j < maxDoc; j++) {
|
||||
if (!liveDocs.get(j)) {
|
||||
// skip deleted docs
|
||||
continue;
|
||||
}
|
||||
// TODO: this could be more efficient using
|
||||
// FieldVisitor instead of loading/writing entire
|
||||
// doc; ie we just have to renumber the field number
|
||||
// on the fly?
|
||||
// NOTE: it's very important to first assign to doc then pass it to
|
||||
// fieldsWriter.addDocument; see LUCENE-1282
|
||||
Document doc = reader.reader.document(j);
|
||||
addDocument(doc, mergeState.fieldInfos);
|
||||
docCount++;
|
||||
mergeState.checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
return docCount;
|
||||
}
|
||||
|
||||
private int copyFieldsNoDeletions(MergeState mergeState, final MergeState.IndexReaderAndLiveDocs reader,
|
||||
final DefaultStoredFieldsReader matchingFieldsReader, int rawDocLengths[])
|
||||
throws IOException, MergeAbortedException, CorruptIndexException {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
int docCount = 0;
|
||||
if (matchingFieldsReader != null) {
|
||||
// We can bulk-copy because the fieldInfos are "congruent"
|
||||
while (docCount < maxDoc) {
|
||||
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
|
||||
IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, docCount, len);
|
||||
addRawDocuments(stream, rawDocLengths, len);
|
||||
docCount += len;
|
||||
mergeState.checkAbort.work(300 * len);
|
||||
}
|
||||
} else {
|
||||
for (; docCount < maxDoc; docCount++) {
|
||||
// NOTE: it's very important to first assign to doc then pass it to
|
||||
// fieldsWriter.addDocument; see LUCENE-1282
|
||||
Document doc = reader.reader.document(docCount);
|
||||
addDocument(doc, mergeState.fieldInfos);
|
||||
mergeState.checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
return docCount;
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@ import java.io.IOException;
|
|||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.index.values.Writer;
|
||||
|
@ -87,13 +88,13 @@ public abstract class DocValuesConsumer {
|
|||
public abstract void files(Collection<String> files) throws IOException;
|
||||
|
||||
/**
|
||||
* Merges the given {@link org.apache.lucene.index.codecs.MergeState} into
|
||||
* Merges the given {@link org.apache.lucene.index.MergeState} into
|
||||
* this {@link DocValuesConsumer}.
|
||||
*
|
||||
* @param mergeState
|
||||
* the state to merge
|
||||
* @param docValues docValues array containing one instance per reader (
|
||||
* {@link org.apache.lucene.index.codecs.MergeState#readers}) or <code>null</code> if the reader has
|
||||
* {@link org.apache.lucene.index.MergeState#readers}) or <code>null</code> if the reader has
|
||||
* no {@link IndexDocValues} instance.
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
|
@ -102,7 +103,7 @@ public abstract class DocValuesConsumer {
|
|||
assert mergeState != null;
|
||||
boolean hasMerged = false;
|
||||
for(int readerIDX=0;readerIDX<mergeState.readers.size();readerIDX++) {
|
||||
final org.apache.lucene.index.codecs.MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(readerIDX);
|
||||
final org.apache.lucene.index.MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(readerIDX);
|
||||
if (docValues[readerIDX] != null) {
|
||||
hasMerged = true;
|
||||
merge(new Writer.SingleSubMergeState(docValues[readerIDX], mergeState.docBase[readerIDX], reader.reader.maxDoc(),
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs;
|
|||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
|
||||
import java.io.IOException;
|
||||
|
|
|
@ -1,44 +0,0 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
|
||||
* use this file except in compliance with the License. You may obtain a copy of
|
||||
* the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
public abstract class FieldsWriter implements Closeable {
|
||||
|
||||
public abstract void addDocument(Iterable<? extends IndexableField> doc, FieldInfos fieldInfos) throws IOException;
|
||||
|
||||
/** Bulk write a contiguous series of documents. The
|
||||
* lengths array is the length (in bytes) of each raw
|
||||
* document. The stream IndexInput is the
|
||||
* fieldsStream from which we should bulk-copy all
|
||||
* bytes. */
|
||||
public abstract void addRawDocuments(IndexInput stream, int[] lengths, int numDocs) throws IOException;
|
||||
|
||||
public abstract void startDocument(int numStoredFields) throws IOException;
|
||||
|
||||
public abstract void skipDocument() throws IOException;
|
||||
|
||||
public abstract void writeField(int fieldNumber, IndexableField field) throws IOException;
|
||||
|
||||
public abstract void abort();
|
||||
}
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index.codecs;
|
|||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.MultiDocsAndPositionsEnum;
|
||||
import java.io.IOException;
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.index.codecs;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.MultiDocsEnum;
|
||||
import java.io.IOException;
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.IndexDocValues;
|
||||
import org.apache.lucene.index.values.TypePromoter;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
|
|
@ -26,10 +26,10 @@ import org.apache.lucene.store.IOContext;
|
|||
*/
|
||||
|
||||
/**
|
||||
* Controls the format of stored fields/termvectors/...
|
||||
* Controls the format of stored fields
|
||||
*/
|
||||
public abstract class FieldsFormat {
|
||||
public abstract FieldsReader fieldsReader(Directory directory, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException;
|
||||
public abstract FieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException;
|
||||
public abstract class StoredFieldsFormat {
|
||||
public abstract StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException;
|
||||
public abstract StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
}
|
|
@ -5,7 +5,6 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
|
@ -23,17 +22,18 @@ import org.apache.lucene.store.IndexInput;
|
|||
* the License.
|
||||
*/
|
||||
|
||||
public abstract class FieldsReader implements Cloneable, Closeable {
|
||||
/**
|
||||
* Codec API for reading stored fields:
|
||||
*
|
||||
* You need to implement {@link #visitDocument(int, StoredFieldVisitor)} to
|
||||
* read the stored fields for a document, implement {@link #clone()} (creating
|
||||
* clones of any IndexInputs used, etc), and {@link #close()}
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class StoredFieldsReader implements Cloneable, Closeable {
|
||||
|
||||
/** Visit the stored fields for document <code>n</code> */
|
||||
public abstract void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException;
|
||||
|
||||
/** Returns the length in bytes of each raw document in a
|
||||
* contiguous range of length numDocs starting with
|
||||
* startDocID. Returns the IndexInput (the fieldStream),
|
||||
* already seeked to the starting point for startDocID.*/
|
||||
public abstract IndexInput rawDocs(int[] lengths, int startDocID, int numDocs) throws IOException;
|
||||
|
||||
public abstract int size();
|
||||
|
||||
public abstract FieldsReader clone();
|
||||
public abstract StoredFieldsReader clone();
|
||||
}
|
|
@ -0,0 +1,117 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/**
|
||||
* Copyright 2004 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
|
||||
* use this file except in compliance with the License. You may obtain a copy of
|
||||
* the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Codec API for writing stored fields:
|
||||
* <p>
|
||||
* <ol>
|
||||
* <li>For every document, {@link #startDocument(int)} is called,
|
||||
* informing the Codec how many fields will be written.
|
||||
* <li>{@link #writeField(int, IndexableField)} is called for
|
||||
* each field in the document.
|
||||
* <li>After all documents have been written, {@link #finish(int)}
|
||||
* is called for verification/sanity-checks.
|
||||
* <li>Finally the writer is closed ({@link #close()})
|
||||
* </ol>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class StoredFieldsWriter implements Closeable {
|
||||
|
||||
/** Called before writing the stored fields of the document.
|
||||
* {@link #writeField(int, IndexableField)} will be called
|
||||
* <code>numStoredFields</code> times. Note that this is
|
||||
* called even if the document has no stored fields, in
|
||||
* this case <code>numStoredFields</code> will be zero. */
|
||||
public abstract void startDocument(int numStoredFields) throws IOException;
|
||||
|
||||
/** Writes a single stored field. */
|
||||
public abstract void writeField(int fieldNumber, IndexableField field) throws IOException;
|
||||
|
||||
/** Aborts writing entirely, implementation should remove
|
||||
* any partially-written files, etc. */
|
||||
public abstract void abort();
|
||||
|
||||
/** Called before {@link #close()}, passing in the number
|
||||
* of documents that were written. Note that this is
|
||||
* intentionally redundant (equivalent to the number of
|
||||
* calls to {@link #startDocument(int)}, but a Codec should
|
||||
* check that this is the case to detect the JRE bug described
|
||||
* in LUCENE-1282. */
|
||||
public abstract void finish(int numDocs) throws IOException;
|
||||
|
||||
/** Merges in the stored fields from the readers in
|
||||
* <code>mergeState</code>. The default implementation skips
|
||||
* over deleted documents, and uses {@link #startDocument(int)},
|
||||
* {@link #writeField(int, IndexableField)}, and {@link #finish(int)},
|
||||
* returning the number of documents that were written.
|
||||
* Implementations can override this method for more sophisticated
|
||||
* merging (bulk-byte copying, etc). */
|
||||
public int merge(MergeState mergeState) throws IOException {
|
||||
int docCount = 0;
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
for (int i = 0; i < maxDoc; i++) {
|
||||
if (liveDocs != null && !liveDocs.get(i)) {
|
||||
// skip deleted docs
|
||||
continue;
|
||||
}
|
||||
// TODO: this could be more efficient using
|
||||
// FieldVisitor instead of loading/writing entire
|
||||
// doc; ie we just have to renumber the field number
|
||||
// on the fly?
|
||||
// NOTE: it's very important to first assign to doc then pass it to
|
||||
// fieldsWriter.addDocument; see LUCENE-1282
|
||||
Document doc = reader.reader.document(i);
|
||||
addDocument(doc, mergeState.fieldInfos);
|
||||
docCount++;
|
||||
mergeState.checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
finish(docCount);
|
||||
return docCount;
|
||||
}
|
||||
|
||||
/** sugar method for startDocument() + writeField() for every stored field in the document */
|
||||
protected final void addDocument(Iterable<? extends IndexableField> doc, FieldInfos fieldInfos) throws IOException {
|
||||
int storedCount = 0;
|
||||
for (IndexableField field : doc) {
|
||||
if (field.fieldType().stored()) {
|
||||
storedCount++;
|
||||
}
|
||||
}
|
||||
|
||||
startDocument(storedCount);
|
||||
|
||||
for (IndexableField field : doc) {
|
||||
if (field.fieldType().stored()) {
|
||||
writeField(fieldInfos.fieldNumber(field.name()), field);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.MultiDocsEnum;
|
||||
import org.apache.lucene.index.MultiDocsAndPositionsEnum;
|
||||
|
@ -103,8 +104,8 @@ public abstract class TermsConsumer {
|
|||
if (postingsEnumIn != null) {
|
||||
postingsEnum.reset(postingsEnumIn);
|
||||
// set PayloadProcessor
|
||||
if (mergeState.hasPayloadProcessorProvider) {
|
||||
for (int i = 0; i < mergeState.readerCount; i++) {
|
||||
if (mergeState.payloadProcessorProvider != null) {
|
||||
for (int i = 0; i < mergeState.readers.size(); i++) {
|
||||
if (mergeState.dirPayloadProcessor[i] != null) {
|
||||
mergeState.currentPayloadProcessor[i] = mergeState.dirPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name, term);
|
||||
}
|
||||
|
|
|
@ -24,10 +24,10 @@ import org.apache.lucene.index.PerDocWriteState;
|
|||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsFormat;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
|
@ -45,7 +45,7 @@ public class Lucene3xCodec extends Codec {
|
|||
private final PostingsFormat postingsFormat = new Lucene3xPostingsFormat();
|
||||
|
||||
// TODO: this should really be a different impl
|
||||
private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
|
||||
private final StoredFieldsFormat fieldsFormat = new DefaultStoredFieldsFormat();
|
||||
|
||||
// TODO: this should really be a different impl
|
||||
// also if we want preflex to *really* be read-only it should throw exception for the writer?
|
||||
|
@ -79,7 +79,7 @@ public class Lucene3xCodec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldsFormat fieldsFormat() {
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,17 +19,17 @@ package org.apache.lucene.index.codecs.lucene40;
|
|||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsFormat;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 4.0 index format, with configurable per-field postings formats
|
||||
* and using {@link DefaultFieldsFormat} for stored fields and {@link
|
||||
* and using {@link DefaultStoredFieldsFormat} for stored fields and {@link
|
||||
* DefaultDocValuesFormat} for doc values.
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
@ -38,7 +38,7 @@ import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
|
|||
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
|
||||
// (it writes a minor version, etc).
|
||||
public class Lucene40Codec extends Codec {
|
||||
private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
|
||||
private final StoredFieldsFormat fieldsFormat = new DefaultStoredFieldsFormat();
|
||||
private final DocValuesFormat docValuesFormat = new DefaultDocValuesFormat();
|
||||
private final SegmentInfosFormat infosFormat = new DefaultSegmentInfosFormat();
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
|
@ -53,7 +53,7 @@ public class Lucene40Codec extends Codec {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldsFormat fieldsFormat() {
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,66 @@
|
|||
package org.apache.lucene.index.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
|
||||
/**
|
||||
* plain text index format.
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class SimpleTextCodec extends Codec {
|
||||
private final PostingsFormat postings = new SimpleTextPostingsFormat();
|
||||
private final StoredFieldsFormat storedFields = new SimpleTextStoredFieldsFormat();
|
||||
|
||||
// TODO: need a plain-text impl
|
||||
private final DocValuesFormat docValues = new DefaultDocValuesFormat();
|
||||
// TODO: need a plain-text impl
|
||||
private final SegmentInfosFormat segmentInfos = new DefaultSegmentInfosFormat();
|
||||
|
||||
public SimpleTextCodec() {
|
||||
super("SimpleText");
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsFormat postingsFormat() {
|
||||
return postings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat docValuesFormat() {
|
||||
return docValues;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return storedFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentInfosFormat segmentInfosFormat() {
|
||||
return segmentInfos;
|
||||
}
|
||||
}
|
|
@ -49,8 +49,6 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
private final IndexInput in;
|
||||
private final FieldInfos fieldInfos;
|
||||
|
||||
final static byte NEWLINE = SimpleTextFieldsWriter.NEWLINE;
|
||||
final static byte ESCAPE = SimpleTextFieldsWriter.ESCAPE;
|
||||
final static BytesRef END = SimpleTextFieldsWriter.END;
|
||||
final static BytesRef FIELD = SimpleTextFieldsWriter.FIELD;
|
||||
final static BytesRef TERM = SimpleTextFieldsWriter.TERM;
|
||||
|
@ -65,27 +63,6 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
fieldInfos = state.fieldInfos;
|
||||
}
|
||||
|
||||
static void readLine(IndexInput in, BytesRef scratch) throws IOException {
|
||||
int upto = 0;
|
||||
while(true) {
|
||||
byte b = in.readByte();
|
||||
if (scratch.bytes.length == upto) {
|
||||
scratch.grow(1+upto);
|
||||
}
|
||||
if (b == ESCAPE) {
|
||||
scratch.bytes[upto++] = in.readByte();
|
||||
} else {
|
||||
if (b == NEWLINE) {
|
||||
break;
|
||||
} else {
|
||||
scratch.bytes[upto++] = b;
|
||||
}
|
||||
}
|
||||
}
|
||||
scratch.offset = 0;
|
||||
scratch.length = upto;
|
||||
}
|
||||
|
||||
private class SimpleTextFieldsEnum extends FieldsEnum {
|
||||
private final IndexInput in;
|
||||
private final BytesRef scratch = new BytesRef(10);
|
||||
|
@ -98,7 +75,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
@Override
|
||||
public String next() throws IOException {
|
||||
while(true) {
|
||||
readLine(in, scratch);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
if (scratch.equals(END)) {
|
||||
current = null;
|
||||
return null;
|
||||
|
@ -292,7 +269,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
int termFreq = 0;
|
||||
while(true) {
|
||||
final long lineStart = in.getFilePointer();
|
||||
readLine(in, scratch);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
if (scratch.startsWith(DOC)) {
|
||||
if (!first && (liveDocs == null || liveDocs.get(docID))) {
|
||||
in.seek(lineStart);
|
||||
|
@ -379,7 +356,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
long posStart = 0;
|
||||
while(true) {
|
||||
final long lineStart = in.getFilePointer();
|
||||
readLine(in, scratch);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
if (scratch.startsWith(DOC)) {
|
||||
if (!first && (liveDocs == null || liveDocs.get(docID))) {
|
||||
nextDocStart = lineStart;
|
||||
|
@ -419,12 +396,12 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
|
||||
@Override
|
||||
public int nextPosition() throws IOException {
|
||||
readLine(in, scratch);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert scratch.startsWith(POS): "got line=" + scratch.utf8ToString();
|
||||
UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+POS.length, scratch.length-POS.length, scratchUTF16_2);
|
||||
final int pos = ArrayUtil.parseInt(scratchUTF16_2.chars, 0, scratchUTF16_2.length);
|
||||
final long fp = in.getFilePointer();
|
||||
readLine(in, scratch);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
if (scratch.startsWith(PAYLOAD)) {
|
||||
final int len = scratch.length - PAYLOAD.length;
|
||||
if (scratch2.bytes.length < len) {
|
||||
|
@ -498,7 +475,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
|
|||
long totalTermFreq = 0;
|
||||
OpenBitSet visitedDocs = new OpenBitSet();
|
||||
while(true) {
|
||||
readLine(in, scratch);
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
if (scratch.equals(END) || scratch.startsWith(FIELD)) {
|
||||
if (lastDocsStart != -1) {
|
||||
b.add(lastTerm, new PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>(lastDocsStart,
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index.codecs.simpletext;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.index.codecs.PostingsConsumer;
|
||||
|
@ -35,8 +34,6 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
|
|||
|
||||
private final IndexOutput out;
|
||||
private final BytesRef scratch = new BytesRef(10);
|
||||
final static byte NEWLINE = 10;
|
||||
final static byte ESCAPE = 92;
|
||||
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
final static BytesRef FIELD = new BytesRef("field ");
|
||||
|
@ -52,29 +49,22 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
|
|||
}
|
||||
|
||||
private void write(String s) throws IOException {
|
||||
UnicodeUtil.UTF16toUTF8(s, 0, s.length(), scratch);
|
||||
write(scratch);
|
||||
SimpleTextUtil.write(out, s, scratch);
|
||||
}
|
||||
|
||||
private void write(BytesRef b) throws IOException {
|
||||
for(int i=0;i<b.length;i++) {
|
||||
final byte bx = b.bytes[b.offset+i];
|
||||
if (bx == NEWLINE || bx == ESCAPE) {
|
||||
out.writeByte(ESCAPE);
|
||||
}
|
||||
out.writeByte(bx);
|
||||
}
|
||||
SimpleTextUtil.write(out, b);
|
||||
}
|
||||
|
||||
private void newline() throws IOException {
|
||||
out.writeByte(NEWLINE);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsConsumer addField(FieldInfo field) throws IOException {
|
||||
write(FIELD);
|
||||
write(field.name);
|
||||
out.writeByte(NEWLINE);
|
||||
newline();
|
||||
return new SimpleTextTermsWriter(field);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,53 @@
|
|||
package org.apache.lucene.index.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* 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.Set;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/**
|
||||
* plain text stored fields format.
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextStoredFieldsFormat extends StoredFieldsFormat {
|
||||
|
||||
@Override
|
||||
public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
|
||||
return new SimpleTextStoredFieldsReader(directory, si, fn, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
return new SimpleTextStoredFieldsWriter(directory, segment, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextStoredFieldsReader.files(dir, info, files);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,197 @@
|
|||
package org.apache.lucene.index.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
import static org.apache.lucene.index.codecs.simpletext.SimpleTextStoredFieldsWriter.*;
|
||||
|
||||
/**
|
||||
* reads plaintext stored fields
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextStoredFieldsReader extends StoredFieldsReader {
|
||||
private ArrayList<Long> offsets; /* docid -> offset in .fld file */
|
||||
private IndexInput in;
|
||||
private BytesRef scratch = new BytesRef();
|
||||
private CharsRef scratchUTF16 = new CharsRef();
|
||||
private final FieldInfos fieldInfos;
|
||||
|
||||
public SimpleTextStoredFieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
|
||||
this.fieldInfos = fn;
|
||||
boolean success = false;
|
||||
try {
|
||||
in = directory.openInput(IndexFileNames.segmentFileName(si.name, "", SimpleTextStoredFieldsWriter.FIELDS_EXTENSION), context);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
close();
|
||||
}
|
||||
}
|
||||
readIndex();
|
||||
}
|
||||
|
||||
// used by clone
|
||||
SimpleTextStoredFieldsReader(ArrayList<Long> offsets, IndexInput in, FieldInfos fieldInfos) {
|
||||
this.offsets = offsets;
|
||||
this.in = in;
|
||||
this.fieldInfos = fieldInfos;
|
||||
}
|
||||
|
||||
// we don't actually write a .fdx-like index, instead we read the
|
||||
// stored fields file in entirety up-front and save the offsets
|
||||
// so we can seek to the documents later.
|
||||
private void readIndex() throws IOException {
|
||||
offsets = new ArrayList<Long>();
|
||||
while (!scratch.equals(END)) {
|
||||
readLine();
|
||||
if (scratch.startsWith(DOC)) {
|
||||
offsets.add(in.getFilePointer());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
|
||||
in.seek(offsets.get(n));
|
||||
readLine();
|
||||
assert scratch.startsWith(NUM);
|
||||
int numFields = parseIntAt(NUM.length);
|
||||
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
readLine();
|
||||
assert scratch.startsWith(FIELD);
|
||||
int fieldNumber = parseIntAt(FIELD.length);
|
||||
FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
|
||||
readLine();
|
||||
assert scratch.startsWith(NAME);
|
||||
readLine();
|
||||
assert scratch.startsWith(TYPE);
|
||||
|
||||
final BytesRef type;
|
||||
if (equalsAt(TYPE_STRING, scratch, TYPE.length)) {
|
||||
type = TYPE_STRING;
|
||||
} else if (equalsAt(TYPE_BINARY, scratch, TYPE.length)) {
|
||||
type = TYPE_BINARY;
|
||||
} else if (equalsAt(TYPE_INT, scratch, TYPE.length)) {
|
||||
type = TYPE_INT;
|
||||
} else if (equalsAt(TYPE_LONG, scratch, TYPE.length)) {
|
||||
type = TYPE_LONG;
|
||||
} else if (equalsAt(TYPE_FLOAT, scratch, TYPE.length)) {
|
||||
type = TYPE_FLOAT;
|
||||
} else if (equalsAt(TYPE_DOUBLE, scratch, TYPE.length)) {
|
||||
type = TYPE_DOUBLE;
|
||||
} else {
|
||||
throw new RuntimeException("unknown field type");
|
||||
}
|
||||
|
||||
switch (visitor.needsField(fieldInfo)) {
|
||||
case YES:
|
||||
readField(type, fieldInfo, visitor);
|
||||
break;
|
||||
case NO:
|
||||
readLine();
|
||||
assert scratch.startsWith(VALUE);
|
||||
break;
|
||||
case STOP: return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void readField(BytesRef type, FieldInfo fieldInfo, StoredFieldVisitor visitor) throws IOException {
|
||||
readLine();
|
||||
assert scratch.startsWith(VALUE);
|
||||
if (type == TYPE_STRING) {
|
||||
visitor.stringField(fieldInfo, new String(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length));
|
||||
} else if (type == TYPE_BINARY) {
|
||||
// TODO: who owns the bytes?
|
||||
byte[] copy = new byte[scratch.length-VALUE.length];
|
||||
System.arraycopy(scratch.bytes, scratch.offset+VALUE.length, copy, 0, copy.length);
|
||||
visitor.binaryField(fieldInfo, copy, 0, copy.length);
|
||||
} else if (type == TYPE_INT) {
|
||||
UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
|
||||
visitor.intField(fieldInfo, Integer.parseInt(scratchUTF16.toString()));
|
||||
} else if (type == TYPE_LONG) {
|
||||
UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
|
||||
visitor.longField(fieldInfo, Long.parseLong(scratchUTF16.toString()));
|
||||
} else if (type == TYPE_FLOAT) {
|
||||
UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
|
||||
visitor.floatField(fieldInfo, Float.parseFloat(scratchUTF16.toString()));
|
||||
} else if (type == TYPE_DOUBLE) {
|
||||
UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, scratchUTF16);
|
||||
visitor.doubleField(fieldInfo, Double.parseDouble(scratchUTF16.toString()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoredFieldsReader clone() {
|
||||
if (in == null) {
|
||||
throw new AlreadyClosedException("this FieldsReader is closed");
|
||||
}
|
||||
return new SimpleTextStoredFieldsReader(offsets, (IndexInput) in.clone(), fieldInfos);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(in);
|
||||
} finally {
|
||||
in = null;
|
||||
offsets = null;
|
||||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", SimpleTextStoredFieldsWriter.FIELDS_EXTENSION));
|
||||
}
|
||||
|
||||
private void readLine() throws IOException {
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
}
|
||||
|
||||
private int parseIntAt(int offset) throws IOException {
|
||||
UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+offset, scratch.length-offset, scratchUTF16);
|
||||
return ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
|
||||
}
|
||||
|
||||
private boolean equalsAt(BytesRef a, BytesRef b, int bOffset) {
|
||||
return a.length == b.length - bOffset &&
|
||||
ArrayUtil.equals(a.bytes, a.offset, b.bytes, b.offset + bOffset, b.length - bOffset);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,201 @@
|
|||
package org.apache.lucene.index.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* 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.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Writes plain-text stored fields.
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
|
||||
private int numDocsWritten = 0;
|
||||
private final Directory directory;
|
||||
private final String segment;
|
||||
private IndexOutput out;
|
||||
|
||||
final static String FIELDS_EXTENSION = "fld";
|
||||
|
||||
final static BytesRef TYPE_STRING = new BytesRef("string");
|
||||
final static BytesRef TYPE_BINARY = new BytesRef("binary");
|
||||
final static BytesRef TYPE_INT = new BytesRef("int");
|
||||
final static BytesRef TYPE_LONG = new BytesRef("long");
|
||||
final static BytesRef TYPE_FLOAT = new BytesRef("float");
|
||||
final static BytesRef TYPE_DOUBLE = new BytesRef("double");
|
||||
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
final static BytesRef DOC = new BytesRef("doc ");
|
||||
final static BytesRef NUM = new BytesRef(" numfields ");
|
||||
final static BytesRef FIELD = new BytesRef(" field ");
|
||||
final static BytesRef NAME = new BytesRef(" name ");
|
||||
final static BytesRef TYPE = new BytesRef(" type ");
|
||||
final static BytesRef VALUE = new BytesRef(" value ");
|
||||
|
||||
private final BytesRef scratch = new BytesRef();
|
||||
|
||||
public SimpleTextStoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
this.directory = directory;
|
||||
this.segment = segment;
|
||||
boolean success = false;
|
||||
try {
|
||||
out = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startDocument(int numStoredFields) throws IOException {
|
||||
write(DOC);
|
||||
write(Integer.toString(numDocsWritten));
|
||||
newLine();
|
||||
|
||||
write(NUM);
|
||||
write(Integer.toString(numStoredFields));
|
||||
newLine();
|
||||
|
||||
numDocsWritten++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeField(int fieldNumber, IndexableField field) throws IOException {
|
||||
write(FIELD);
|
||||
write(Integer.toString(fieldNumber));
|
||||
newLine();
|
||||
|
||||
write(NAME);
|
||||
write(field.name());
|
||||
newLine();
|
||||
|
||||
write(TYPE);
|
||||
if (field.numeric()) {
|
||||
switch (field.numericDataType()) {
|
||||
case INT:
|
||||
write(TYPE_INT);
|
||||
newLine();
|
||||
|
||||
write(VALUE);
|
||||
write(Integer.toString(field.numericValue().intValue()));
|
||||
newLine();
|
||||
|
||||
break;
|
||||
case LONG:
|
||||
write(TYPE_LONG);
|
||||
newLine();
|
||||
|
||||
write(VALUE);
|
||||
write(Long.toString(field.numericValue().longValue()));
|
||||
newLine();
|
||||
|
||||
break;
|
||||
case FLOAT:
|
||||
write(TYPE_FLOAT);
|
||||
newLine();
|
||||
|
||||
write(VALUE);
|
||||
write(Float.toString(field.numericValue().floatValue()));
|
||||
newLine();
|
||||
|
||||
break;
|
||||
case DOUBLE:
|
||||
write(TYPE_DOUBLE);
|
||||
newLine();
|
||||
|
||||
write(VALUE);
|
||||
write(Double.toString(field.numericValue().doubleValue()));
|
||||
newLine();
|
||||
|
||||
break;
|
||||
default:
|
||||
assert false : "Should never get here";
|
||||
}
|
||||
} else {
|
||||
BytesRef bytes = field.binaryValue();
|
||||
if (bytes != null) {
|
||||
write(TYPE_BINARY);
|
||||
newLine();
|
||||
|
||||
write(VALUE);
|
||||
write(bytes);
|
||||
newLine();
|
||||
} else {
|
||||
write(TYPE_STRING);
|
||||
newLine();
|
||||
|
||||
write(VALUE);
|
||||
write(field.stringValue());
|
||||
newLine();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int numDocs) throws IOException {
|
||||
if (numDocsWritten != numDocs) {
|
||||
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + numDocs
|
||||
+ " but only saw " + numDocsWritten + " file=" + out.toString() + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
write(END);
|
||||
newLine();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(out);
|
||||
} finally {
|
||||
out = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void write(String s) throws IOException {
|
||||
SimpleTextUtil.write(out, s, scratch);
|
||||
}
|
||||
|
||||
private void write(BytesRef bytes) throws IOException {
|
||||
SimpleTextUtil.write(out, bytes);
|
||||
}
|
||||
|
||||
private void newLine() throws IOException {
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
package org.apache.lucene.index.codecs.simpletext;
|
||||
|
||||
/**
|
||||
* 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.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
class SimpleTextUtil {
|
||||
public final static byte NEWLINE = 10;
|
||||
public final static byte ESCAPE = 92;
|
||||
|
||||
public static void write(DataOutput out, String s, BytesRef scratch) throws IOException {
|
||||
UnicodeUtil.UTF16toUTF8(s, 0, s.length(), scratch);
|
||||
write(out, scratch);
|
||||
}
|
||||
|
||||
public static void write(DataOutput out, BytesRef b) throws IOException {
|
||||
for(int i=0;i<b.length;i++) {
|
||||
final byte bx = b.bytes[b.offset+i];
|
||||
if (bx == NEWLINE || bx == ESCAPE) {
|
||||
out.writeByte(ESCAPE);
|
||||
}
|
||||
out.writeByte(bx);
|
||||
}
|
||||
}
|
||||
|
||||
public static void writeNewline(DataOutput out) throws IOException {
|
||||
out.writeByte(NEWLINE);
|
||||
}
|
||||
|
||||
public static void readLine(DataInput in, BytesRef scratch) throws IOException {
|
||||
int upto = 0;
|
||||
while(true) {
|
||||
byte b = in.readByte();
|
||||
if (scratch.bytes.length == upto) {
|
||||
scratch.grow(1+upto);
|
||||
}
|
||||
if (b == ESCAPE) {
|
||||
scratch.bytes[upto++] = in.readByte();
|
||||
} else {
|
||||
if (b == NEWLINE) {
|
||||
break;
|
||||
} else {
|
||||
scratch.bytes[upto++] = b;
|
||||
}
|
||||
}
|
||||
}
|
||||
scratch.offset = 0;
|
||||
scratch.length = upto;
|
||||
}
|
||||
}
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.codecs.MergeState;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
|
||||
|
|
|
@ -22,8 +22,8 @@ import java.util.Collections;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.codecs.MergeState;
|
||||
import org.apache.lucene.index.codecs.MergeState.IndexReaderAndLiveDocs;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.MergeState.IndexReaderAndLiveDocs;
|
||||
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
|
||||
import org.apache.lucene.index.values.IndexDocValues.Source;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.codecs.MergeState;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
|
||||
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
|
||||
import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
|
||||
|
|
|
@ -509,6 +509,32 @@ public final class ArrayUtil {
|
|||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
// Since Arrays.equals doesn't implement offsets for equals
|
||||
/**
|
||||
* See if two array slices are the same.
|
||||
*
|
||||
* @param left The left array to compare
|
||||
* @param offsetLeft The offset into the array. Must be positive
|
||||
* @param right The right array to compare
|
||||
* @param offsetRight the offset into the right array. Must be positive
|
||||
* @param length The length of the section of the array to compare
|
||||
* @return true if the two arrays, starting at their respective offsets, are equal
|
||||
*
|
||||
* @see java.util.Arrays#equals(byte[], byte[])
|
||||
*/
|
||||
public static boolean equals(byte[] left, int offsetLeft, byte[] right, int offsetRight, int length) {
|
||||
if ((offsetLeft + length <= left.length) && (offsetRight + length <= right.length)) {
|
||||
for (int i = 0; i < length; i++) {
|
||||
if (left[offsetLeft + i] != right[offsetRight + i]) {
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/* DISABLE THIS FOR NOW: This has performance problems until Java creates intrinsics for Class#getComponentType() and Array.newInstance()
|
||||
public static <T> T[] grow(T[] array, int minSize) {
|
||||
|
|
|
@ -0,0 +1,47 @@
|
|||
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.
|
||||
*/
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/** @lucene.internal */
|
||||
public abstract class InfoStream implements Closeable {
|
||||
// Used for printing messages
|
||||
private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
|
||||
protected final int messageID = MESSAGE_ID.getAndIncrement();
|
||||
|
||||
/** prints a message */
|
||||
public abstract void message(String component, String message);
|
||||
|
||||
private static InfoStream defaultInfoStream;
|
||||
|
||||
/** The default infoStream (possibly null) used
|
||||
* by a newly instantiated classes.
|
||||
* @see #setDefault */
|
||||
public static InfoStream getDefault() {
|
||||
return defaultInfoStream;
|
||||
}
|
||||
|
||||
/** Sets the default infoStream (possibly null) used
|
||||
* by a newly instantiated classes.
|
||||
* @see #setDefault */
|
||||
public static void setDefault(InfoStream infoStream) {
|
||||
defaultInfoStream = infoStream;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Date;
|
||||
|
||||
/**
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class PrintStreamInfoStream extends InfoStream {
|
||||
private final PrintStream stream;
|
||||
|
||||
public PrintStreamInfoStream(PrintStream stream) {
|
||||
this.stream = stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void message(String component, String message) {
|
||||
stream.println(component + " " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (!isSystemStream()) {
|
||||
stream.close();
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isSystemStream() {
|
||||
return stream == System.out || stream == System.err;
|
||||
}
|
||||
}
|
|
@ -15,3 +15,4 @@
|
|||
|
||||
org.apache.lucene.index.codecs.lucene40.Lucene40Codec
|
||||
org.apache.lucene.index.codecs.lucene3x.Lucene3xCodec
|
||||
org.apache.lucene.index.codecs.simpletext.SimpleTextCodec
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.lucene.util._TestUtil;
|
|||
public class RandomCodec extends Lucene40Codec {
|
||||
/** shuffled list of postingsformats to use for new mappings */
|
||||
private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
|
||||
/** memorized field->postingsformat mappings */
|
||||
// note: we have to sync this map even though its just for debugging/toString,
|
||||
// otherwise DWPT's .toString() calls that iterate over the map can
|
||||
// cause concurrentmodificationexception if indexwriter's infostream is on
|
||||
|
|
|
@ -100,7 +100,6 @@ public class RandomIndexWriter implements Closeable {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("RIW config=" + w.getConfig());
|
||||
System.out.println("codec default=" + codec.getName());
|
||||
w.setInfoStream(System.out);
|
||||
}
|
||||
/* TODO: find some what to make that random...
|
||||
* This must be fixed across all fixed bytes
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.FailOnNonBulkMergesInfoStream;
|
||||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.NamedThreadFactory;
|
||||
|
@ -435,7 +436,8 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
|
|||
final File tempDir = _TestUtil.getTempDir(testName);
|
||||
dir = newFSDirectory(tempDir);
|
||||
((MockDirectoryWrapper) dir).setCheckIndexOnClose(false); // don't double-checkIndex, we do it ourselves.
|
||||
final IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
|
||||
final IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).
|
||||
setInfoStream(new FailOnNonBulkMergesInfoStream());
|
||||
|
||||
if (LuceneTestCase.TEST_NIGHTLY) {
|
||||
// newIWConfig makes smallish max seg size, which
|
||||
|
@ -479,9 +481,6 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
|
|||
});
|
||||
|
||||
writer = new IndexWriter(dir, conf);
|
||||
if (VERBOSE) {
|
||||
writer.setInfoStream(System.out);
|
||||
}
|
||||
_TestUtil.reduceOpenFiles(writer);
|
||||
|
||||
final ExecutorService es = random.nextBoolean() ? null : Executors.newCachedThreadPool(new NamedThreadFactory(testName));
|
||||
|
@ -616,7 +615,6 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
|
|||
|
||||
assertEquals("index=" + writer.segString() + " addCount=" + addCount + " delCount=" + delCount, addCount.get() - delCount.get(), writer.numDocs());
|
||||
|
||||
assertFalse(writer.anyNonBulkMerges);
|
||||
doClose();
|
||||
writer.close(false);
|
||||
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/** Hackidy-Häck-Hack to cause a test to fail on non-bulk merges */
|
||||
// TODO: we should probably be a wrapper so verbose still works...
|
||||
public class FailOnNonBulkMergesInfoStream extends InfoStream {
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void message(String component, String message) {
|
||||
assert !message.contains("non-bulk merges");
|
||||
}
|
||||
}
|
|
@ -52,6 +52,7 @@ import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
|
|||
import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
|
||||
import org.apache.lucene.index.codecs.preflexrw.PreFlexRWPostingsFormat;
|
||||
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
|
||||
import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
import org.apache.lucene.search.FieldCache.CacheEntry;
|
||||
|
@ -110,6 +111,8 @@ public abstract class LuceneTestCase extends Assert {
|
|||
* expected to print any messages.
|
||||
*/
|
||||
public static final boolean VERBOSE = Boolean.getBoolean("tests.verbose");
|
||||
|
||||
public static final boolean INFOSTREAM = Boolean.parseBoolean(System.getProperty("tests.infostream", Boolean.toString(VERBOSE)));
|
||||
|
||||
/** Use this constant when creating Analyzers and any other version-dependent stuff.
|
||||
* <p><b>NOTE:</b> Change this when development starts for new Lucene version:
|
||||
|
@ -139,6 +142,8 @@ public abstract class LuceneTestCase extends Assert {
|
|||
// by default we randomly pick a different codec for
|
||||
// each test case (non-J4 tests) and each test class (J4
|
||||
// tests)
|
||||
/** Gets the codec to run tests with. */
|
||||
public static final String TEST_CODEC = System.getProperty("tests.codec", "random");
|
||||
/** Gets the postingsFormat to run tests with. */
|
||||
public static final String TEST_POSTINGSFORMAT = System.getProperty("tests.postingsformat", "random");
|
||||
/** Gets the locale to run tests with */
|
||||
|
@ -216,6 +221,8 @@ public abstract class LuceneTestCase extends Assert {
|
|||
// default codec
|
||||
private static Codec savedCodec;
|
||||
|
||||
private static InfoStream savedInfoStream;
|
||||
|
||||
private static SimilarityProvider similarityProvider;
|
||||
|
||||
private static Locale locale;
|
||||
|
@ -265,13 +272,30 @@ public abstract class LuceneTestCase extends Assert {
|
|||
System.out.println("Loaded postingsFormat: '" + postingsFormat + "': " + PostingsFormat.forName(postingsFormat).getClass().getName());
|
||||
}
|
||||
}
|
||||
|
||||
savedInfoStream = InfoStream.getDefault();
|
||||
if (INFOSTREAM) {
|
||||
// consume random for consistency
|
||||
random.nextBoolean();
|
||||
InfoStream.setDefault(new PrintStreamInfoStream(System.out));
|
||||
} else {
|
||||
if (random.nextBoolean()) {
|
||||
InfoStream.setDefault(new NullInfoStream());
|
||||
}
|
||||
}
|
||||
|
||||
PREFLEX_IMPERSONATION_IS_ACTIVE = false;
|
||||
savedCodec = Codec.getDefault();
|
||||
final Codec codec;
|
||||
if ("Lucene3x".equals(TEST_POSTINGSFORMAT) || ("random".equals(TEST_POSTINGSFORMAT) && random.nextInt(4) == 0)) { // preflex-only setup
|
||||
int randomVal = random.nextInt(10);
|
||||
|
||||
if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal <= 3)) { // preflex-only setup
|
||||
codec = new PreFlexRWCodec();
|
||||
PREFLEX_IMPERSONATION_IS_ACTIVE = true;
|
||||
} else if ("SimpleText".equals(TEST_CODEC) || "random".equals(TEST_CODEC) && randomVal == 9) {
|
||||
codec = new SimpleTextCodec();
|
||||
} else if (!"random".equals(TEST_CODEC)) {
|
||||
codec = Codec.forName(TEST_CODEC);
|
||||
} else if ("random".equals(TEST_POSTINGSFORMAT)) {
|
||||
codec = new RandomCodec(random, useNoMemoryExpensiveCodec);
|
||||
} else {
|
||||
|
@ -342,6 +366,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
|
||||
String codecDescription = Codec.getDefault().toString();
|
||||
Codec.setDefault(savedCodec);
|
||||
InfoStream.setDefault(savedInfoStream);
|
||||
Locale.setDefault(savedLocale);
|
||||
TimeZone.setDefault(savedTimeZone);
|
||||
System.clearProperty("solr.solr.home");
|
||||
|
@ -1273,6 +1298,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
// extra params that were overridden needed to reproduce the command
|
||||
private static String reproduceWithExtraParams() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
if (!TEST_CODEC.equals("random")) sb.append(" -Dtests.codec=").append(TEST_CODEC);
|
||||
if (!TEST_POSTINGSFORMAT.equals("random")) sb.append(" -Dtests.postingsformat=").append(TEST_POSTINGSFORMAT);
|
||||
if (!TEST_LOCALE.equals("random")) sb.append(" -Dtests.locale=").append(TEST_LOCALE);
|
||||
if (!TEST_TIMEZONE.equals("random")) sb.append(" -Dtests.timezone=").append(TEST_TIMEZONE);
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Prints nothing. Just to make sure tests pass w/ and without infostream
|
||||
* without actually making noise.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class NullInfoStream extends InfoStream {
|
||||
@Override
|
||||
public void message(String component, String message) {
|
||||
assert component != null;
|
||||
assert message != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
}
|
|
@ -155,6 +155,9 @@ public class _TestUtil {
|
|||
System.out.println(bos.toString());
|
||||
throw new RuntimeException("CheckIndex failed");
|
||||
} else {
|
||||
if (LuceneTestCase.INFOSTREAM) {
|
||||
System.out.println(bos.toString());
|
||||
}
|
||||
return indexStatus;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -45,7 +45,6 @@ public class TestDemo extends LuceneTestCase {
|
|||
// To store an index on disk, use this instead:
|
||||
//Directory directory = FSDirectory.open("/tmp/testindex");
|
||||
RandomIndexWriter iwriter = new RandomIndexWriter(random, directory, analyzer);
|
||||
iwriter.w.setInfoStream(VERBOSE ? System.out : null);
|
||||
Document doc = new Document();
|
||||
String longTerm = "longtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongtermlongterm";
|
||||
String text = "This is the text to be indexed. " + longTerm;
|
||||
|
|
|
@ -68,7 +68,6 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
setCodec(new CustomPerFieldCodec()).
|
||||
setMergePolicy(newLogMergePolicy(3))
|
||||
);
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
Document doc = new Document();
|
||||
// uses default codec:
|
||||
doc.add(newField("field1", "this field uses the standard codec as the test", TextField.TYPE_UNSTORED));
|
||||
|
|
|
@ -87,7 +87,6 @@ public class TestSearchForDuplicates extends LuceneTestCase {
|
|||
IndexWriter writer = new IndexWriter(directory, conf);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now build index MAX_DOCS=" + MAX_DOCS);
|
||||
writer.setInfoStream(System.out);
|
||||
}
|
||||
|
||||
for (int j = 0; j < MAX_DOCS; j++) {
|
||||
|
|
|
@ -168,7 +168,6 @@ public class Test2BTerms extends LuceneTestCase {
|
|||
.setMergePolicy(newLogMergePolicy(false, 10))
|
||||
.setOpenMode(IndexWriterConfig.OpenMode.CREATE));
|
||||
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
MergePolicy mp = w.getConfig().getMergePolicy();
|
||||
if (mp instanceof LogByteSizeMergePolicy) {
|
||||
// 1 petabyte:
|
||||
|
|
|
@ -32,10 +32,10 @@ import org.apache.lucene.document.TextField;
|
|||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldsFormat;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
|
||||
|
@ -69,7 +69,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
writer = newWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random))
|
||||
.setOpenMode(OpenMode.CREATE));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
// add 100 documents
|
||||
addDocs(writer, 100);
|
||||
assertEquals(100, writer.maxDoc());
|
||||
|
@ -171,7 +170,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
setUpDirs(dir, aux);
|
||||
IndexWriter writer = newWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
writer.addIndexes(aux);
|
||||
|
||||
// Adds 10 docs, then replaces them with another 10
|
||||
|
@ -465,7 +463,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
setMaxBufferedDocs(100).
|
||||
setMergePolicy(newLogMergePolicy(10))
|
||||
);
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
writer.addIndexes(aux);
|
||||
assertEquals(30, writer.maxDoc());
|
||||
assertEquals(3, writer.getSegmentCount());
|
||||
|
@ -655,7 +652,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
dir2 = newDirectory();
|
||||
writer2 = new IndexWriter(dir2, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
writer2.setInfoStream(VERBOSE ? System.out : null);
|
||||
writer2.commit();
|
||||
|
||||
|
||||
|
@ -773,7 +769,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
final int NUM_ITER = TEST_NIGHTLY ? 15 : 5;
|
||||
final int NUM_COPY = 3;
|
||||
CommitAndAddIndexes c = new CommitAndAddIndexes(NUM_COPY);
|
||||
c.writer2.setInfoStream(VERBOSE ? System.out : null);
|
||||
c.launchThreads(NUM_ITER);
|
||||
|
||||
for(int i=0;i<100;i++)
|
||||
|
@ -898,9 +893,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
final int NUM_COPY = 50;
|
||||
CommitAndAddIndexes3 c = new CommitAndAddIndexes3(NUM_COPY);
|
||||
if (VERBOSE) {
|
||||
c.writer2.setInfoStream(System.out);
|
||||
}
|
||||
c.launchThreads(-1);
|
||||
|
||||
Thread.sleep(_TestUtil.nextInt(random, 10, 500));
|
||||
|
@ -1088,7 +1080,6 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
lmp.setUseCompoundFile(true);
|
||||
lmp.setNoCFSRatio(1.0); // Force creation of CFS
|
||||
IndexWriter w3 = new IndexWriter(dir, conf);
|
||||
w3.setInfoStream(VERBOSE ? System.out : null);
|
||||
w3.addIndexes(readers);
|
||||
w3.close();
|
||||
// we should now see segments_X,
|
||||
|
@ -1172,8 +1163,8 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldsFormat fieldsFormat() {
|
||||
return new DefaultFieldsFormat();
|
||||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return new DefaultStoredFieldsFormat();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -131,7 +131,6 @@ public class TestAtomicUpdate extends LuceneTestCase {
|
|||
.setMaxBufferedDocs(7);
|
||||
((TieredMergePolicy) conf.getMergePolicy()).setMaxMergeAtOnce(3);
|
||||
IndexWriter writer = new MockIndexWriter(directory, conf);
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
// Establish a base index of 100 docs:
|
||||
for(int i=0;i<100;i++) {
|
||||
|
|
|
@ -191,7 +191,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
|
||||
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
w.optimize();
|
||||
w.close();
|
||||
|
||||
|
@ -351,7 +350,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
Directory dir = newFSDirectory(oldIndexDir);
|
||||
// open writer
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
// add 10 docs
|
||||
for(int i=0;i<10;i++) {
|
||||
addDoc(writer, 35+i);
|
||||
|
@ -756,7 +754,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
_TestUtil.unzip(getDataFile("index." + name + ".zip"), oldIndxeDir);
|
||||
Directory dir = newFSDirectory(oldIndxeDir);
|
||||
|
||||
new IndexUpgrader(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null), VERBOSE ? System.out : null, false)
|
||||
new IndexUpgrader(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null), false)
|
||||
.upgrade();
|
||||
|
||||
checkAllSegmentsUpgraded(dir);
|
||||
|
@ -798,14 +796,13 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, null)
|
||||
.setMergePolicy(mp);
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
w.setInfoStream(VERBOSE ? System.out : null);
|
||||
w.addIndexes(ramDir);
|
||||
w.close(false);
|
||||
|
||||
// determine count of segments in modified index
|
||||
final int origSegCount = getNumberOfSegments(dir);
|
||||
|
||||
new IndexUpgrader(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null), VERBOSE ? System.out : null, false)
|
||||
new IndexUpgrader(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null), false)
|
||||
.upgrade();
|
||||
|
||||
final int segCount = checkAllSegmentsUpgraded(dir);
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
@ -609,7 +610,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
|
||||
final int termIndexInterval = _TestUtil.nextInt(random, 13, 27);
|
||||
final Codec codec = Codec.getDefault();
|
||||
final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codec, null, newIOContext(random));
|
||||
final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codec, null, newIOContext(random));
|
||||
|
||||
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
|
||||
Arrays.sort(fields);
|
||||
|
|
|
@ -74,7 +74,6 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
directory.failOn(failure);
|
||||
|
||||
IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMaxBufferedDocs(2));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
Document doc = new Document();
|
||||
Field idField = newField("id", "", StringField.TYPE_STORED);
|
||||
doc.add(idField);
|
||||
|
@ -133,7 +132,6 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random))
|
||||
.setMergePolicy(mp));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
Document doc = new Document();
|
||||
Field idField = newField("id", "", StringField.TYPE_STORED);
|
||||
|
@ -172,7 +170,6 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random))
|
||||
.setMaxBufferedDocs(2));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
for(int iter=0;iter<7;iter++) {
|
||||
if (VERBOSE) {
|
||||
|
@ -192,7 +189,6 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
|
|||
writer = new IndexWriter(directory, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random))
|
||||
.setOpenMode(OpenMode.APPEND).setMaxBufferedDocs(2));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
}
|
||||
|
||||
writer.close();
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.lucene.document.FieldType;
|
|||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.FailOnNonBulkMergesInfoStream;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -250,9 +251,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
|
|||
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(
|
||||
new LogByteSizeMergePolicy()));
|
||||
new LogByteSizeMergePolicy()).setInfoStream(new FailOnNonBulkMergesInfoStream()));
|
||||
writer.optimize();
|
||||
assertFalse(" field numbers got mixed up", writer.anyNonBulkMerges);
|
||||
writer.close();
|
||||
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
|
|
|
@ -335,7 +335,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
System.out.println("TEST: open writer for optimize");
|
||||
}
|
||||
writer = new IndexWriter(dir, conf);
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
}
|
||||
|
@ -645,7 +644,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
|
|||
((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
|
||||
}
|
||||
writer = new IndexWriter(dir, conf);
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
for(int j=0;j<17;j++) {
|
||||
addDoc(writer);
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.lucene.index.codecs.Codec;
|
|||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
|
@ -198,16 +199,17 @@ public class TestDoc extends LuceneTestCase {
|
|||
SegmentReader r1 = SegmentReader.get(true, si1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
|
||||
SegmentReader r2 = SegmentReader.get(true, si2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
|
||||
|
||||
SegmentMerger merger = new SegmentMerger(si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, null, null, new FieldInfos(), Codec.getDefault(), context);
|
||||
final Codec codec = Codec.getDefault();
|
||||
SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, MergeState.CheckAbort.NONE, null, new FieldInfos(), codec, context);
|
||||
|
||||
merger.add(r1);
|
||||
merger.add(r2);
|
||||
merger.merge();
|
||||
MergeState mergeState = merger.merge();
|
||||
r1.close();
|
||||
r2.close();
|
||||
final FieldInfos fieldInfos = merger.fieldInfos();
|
||||
final FieldInfos fieldInfos = mergeState.fieldInfos;
|
||||
final SegmentInfo info = new SegmentInfo(merged, si1.docCount + si2.docCount, si1.dir,
|
||||
false, merger.getCodec(), fieldInfos);
|
||||
false, codec, fieldInfos);
|
||||
|
||||
if (useCompoundFile) {
|
||||
Collection<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info, newIOContext(random));
|
||||
|
|
|
@ -199,7 +199,6 @@ public class TestDocsAndPositions extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random, dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
writer.w.setInfoStream(VERBOSE ? System.out : null);
|
||||
int numDocs = atLeast(49);
|
||||
int max = 15678;
|
||||
int term = random.nextInt(max);
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
|
|||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
import org.apache.lucene.util.FailOnNonBulkMergesInfoStream;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
|
@ -98,9 +99,8 @@ public class TestGlobalFieldNumbers extends LuceneTestCase {
|
|||
}
|
||||
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)).setInfoStream(new FailOnNonBulkMergesInfoStream()));
|
||||
writer.optimize();
|
||||
assertFalse(" field numbers got mixed up", writer.anyNonBulkMerges);
|
||||
writer.close();
|
||||
assertFNXFiles(dir, "_2.fnx");
|
||||
|
||||
|
@ -139,9 +139,8 @@ public class TestGlobalFieldNumbers extends LuceneTestCase {
|
|||
assertFNXFiles(dir, "_2.fnx");
|
||||
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)).setInfoStream(new FailOnNonBulkMergesInfoStream()));
|
||||
writer.optimize();
|
||||
assertFalse(" field numbers got mixed up", writer.anyNonBulkMerges);
|
||||
writer.close();
|
||||
assertFNXFiles(dir, "_2.fnx");
|
||||
|
||||
|
@ -187,9 +186,8 @@ public class TestGlobalFieldNumbers extends LuceneTestCase {
|
|||
assertFNXFiles(dir, "_2.fnx");
|
||||
}
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)).setInfoStream(new FailOnNonBulkMergesInfoStream()));
|
||||
writer.optimize();
|
||||
assertFalse(" field numbers got mixed up", writer.anyNonBulkMerges);
|
||||
writer.close();
|
||||
assertFNXFiles(dir, "_2.fnx");
|
||||
dir.close();
|
||||
|
|
|
@ -54,8 +54,6 @@ public class TestIndexFileDeleter extends LuceneTestCase {
|
|||
setMergePolicy(mergePolicy)
|
||||
);
|
||||
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
|
||||
int i;
|
||||
for(i=0;i<35;i++) {
|
||||
addDoc(writer, i);
|
||||
|
@ -149,7 +147,9 @@ public class TestIndexFileDeleter extends LuceneTestCase {
|
|||
copyFile(dir, "segments_2", "segments_1");
|
||||
|
||||
// Create a bogus cfs file shadowing a non-cfs segment:
|
||||
assertTrue(dir.fileExists("_3.fdt"));
|
||||
|
||||
// TODO: assert is bogus (relies upon codec-specific filenames)
|
||||
assertTrue(dir.fileExists("_3.fdt") || dir.fileExists("_3.fld"));
|
||||
assertTrue(!dir.fileExists("_3.cfs"));
|
||||
copyFile(dir, "_1.cfs", "_3.cfs");
|
||||
|
||||
|
|
|
@ -88,7 +88,6 @@ public class TestIndexReader extends LuceneTestCase
|
|||
writer = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random)).setOpenMode(
|
||||
OpenMode.APPEND).setMaxBufferedDocs(2));
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
for(int i=0;i<7;i++)
|
||||
addDocumentWithFields(writer);
|
||||
writer.close();
|
||||
|
@ -1363,7 +1362,6 @@ public class TestIndexReader extends LuceneTestCase
|
|||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(3);
|
||||
writer.setInfoStream(VERBOSE ? System.out : null);
|
||||
writer.addDocument(new Document());
|
||||
writer.commit();
|
||||
writer.addDocument(new Document());
|
||||
|
|
|
@ -121,7 +121,6 @@ public class TestIndexReaderCloneNorms extends LuceneTestCase {
|
|||
setMaxBufferedDocs(5).
|
||||
setMergePolicy(newLogMergePolicy(3))
|
||||
);
|
||||
iw.setInfoStream(VERBOSE ? System.out : null);
|
||||
iw.addIndexes(dir1, dir2);
|
||||
iw.optimize();
|
||||
iw.close();
|
||||
|
@ -317,7 +316,6 @@ public class TestIndexReaderCloneNorms extends LuceneTestCase {
|
|||
lmp.setMergeFactor(3);
|
||||
lmp.setUseCompoundFile(compound);
|
||||
IndexWriter iw = new IndexWriter(dir, conf);
|
||||
iw.setInfoStream(VERBOSE ? System.out : null);
|
||||
for (int i = 0; i < ndocs; i++) {
|
||||
iw.addDocument(newDoc());
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue