LUCENE-5618, LUCENE-5636: write each DocValues update in a separate file; stop referencing old fieldInfos files

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1596570 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Shai Erera 2014-05-21 13:18:18 +00:00
parent d1e607f5df
commit c863a7251d
16 changed files with 547 additions and 332 deletions

View File

@ -200,6 +200,10 @@ Bug fixes
if the underlying TermsEnum supports ord() and the insertion point would
be at the end. (Robert Muir)
* LUCENE-5618, LUCENE-5636: SegmentReader referenced unneeded files following
doc-values updates. Now doc-values field updates are written in separate file
per field. (Shai Erera, Robert Muir)
Test Framework
* LUCENE-5622: Fail tests if they print over the given limit of bytes to

View File

@ -148,12 +148,10 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
// check should be: infos.fieldInfo(fieldNumber) != null, which incorporates negative check
// but docvalues updates are currently buggy here (loading extra stuff, etc): LUCENE-5616
if (fieldNumber < 0) {
if (infos.fieldInfo(fieldNumber) == null) {
// trickier to validate more: because we re-use for norms, because we use multiple entries
// for "composite" types like sortedset, etc.
throw new CorruptIndexException("Invalid field number: " + fieldNumber + ", input=" + meta);
throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");
}
int fieldType = meta.readByte();
if (fieldType == NUMBER) {

View File

@ -57,6 +57,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.packed.BlockPackedReader;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
import org.apache.lucene.util.packed.PackedInts;
@ -72,13 +73,29 @@ public class Lucene45DocValuesProducer extends DocValuesProducer implements Clos
private final IndexInput data;
private final int maxDoc;
private final int version;
// We need this for pre-4.9 indexes which recorded multiple fields' DocValues
// updates under the same generation, and therefore the passed FieldInfos may
// not include all the fields that are encoded in this generation. In that
// case, we are more lenient about the fields we read and the passed-in
// FieldInfos.
@Deprecated
private final boolean lenientFieldInfoCheck;
// memory-resident structures
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
/** expert: instantiates a new reader */
@SuppressWarnings("deprecation")
protected Lucene45DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
Version ver;
try {
ver = Version.parseLeniently(state.segmentInfo.getVersion());
} catch (IllegalArgumentException e) {
ver = null;
}
lenientFieldInfoCheck = ver == null || !ver.onOrAfter(Version.LUCENE_4_9);
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
// read in the entries from the metadata file.
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
@ -185,9 +202,7 @@ public class Lucene45DocValuesProducer extends DocValuesProducer implements Clos
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
// check should be: infos.fieldInfo(fieldNumber) != null, which incorporates negative check
// but docvalues updates are currently buggy here (loading extra stuff, etc): LUCENE-5616
if (fieldNumber < 0) {
if ((lenientFieldInfoCheck && fieldNumber < 0) || (!lenientFieldInfoCheck && infos.fieldInfo(fieldNumber) == null)) {
// trickier to validate more: because we re-use for norms, because we use multiple entries
// for "composite" types like sortedset, etc.
throw new CorruptIndexException("Invalid field number: " + fieldNumber + " (resource=" + meta + ")");

View File

@ -416,7 +416,7 @@ class DocumentsWriterPerThread {
pendingUpdates.terms.clear();
segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L);
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L, -1L);
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : flushState.delCountOnFlush) + " deleted docs");
infoStream.message("DWPT", "new segment has " +

View File

@ -2532,7 +2532,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
}
}
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L);
SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
info.setFiles(new HashSet<>(trackingDir.getCreatedFiles()));
trackingDir.getCreatedFiles().clear();
@ -2610,7 +2610,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
info.info.getUseCompoundFile(), info.info.getCodec(),
info.info.getDiagnostics());
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(), info.getFieldInfosGen());
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo,
info.getDelCount(), info.getDelGen(), info.getFieldInfosGen(),
info.getDocValuesGen());
Set<String> segFiles = new HashSet<>();
@ -3733,7 +3735,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
details.put("mergeFactor", Integer.toString(merge.segments.size()));
setDiagnostics(si, SOURCE_MERGE, details);
merge.setInfo(new SegmentCommitInfo(si, 0, -1L, -1L));
merge.setInfo(new SegmentCommitInfo(si, 0, -1L, -1L, -1L));
// System.out.println("[" + Thread.currentThread().getName() + "] IW._mergeInit: " + segString(merge.segments) + " into " + si);

View File

@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.NumericDocValuesField;
@ -38,7 +39,6 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
// Used by IndexWriter to hold open SegmentReaders (for
@ -294,6 +294,170 @@ class ReadersAndUpdates {
return true;
}
@SuppressWarnings("synthetic-access")
private void handleNumericDVUpdates(FieldInfos infos, Map<String,NumericDocValuesFieldUpdates> updates,
Directory dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
for (Entry<String,NumericDocValuesFieldUpdates> e : updates.entrySet()) {
final String field = e.getKey();
final NumericDocValuesFieldUpdates fieldUpdates = e.getValue();
final long nextDocValuesGen = info.getNextDocValuesGen();
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.getDocCount();
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
final FieldInfo fieldInfo = infos.fieldInfo(field);
assert fieldInfo != null;
fieldInfo.setDocValuesGen(nextDocValuesGen);
final FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { fieldInfo });
// separately also track which files were created for this gen
final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
// write the numeric updates to a new gen'd docvalues file
fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
final NumericDocValues currentValues = reader.getNumericDocValues(field);
final Bits docsWithField = reader.getDocsWithField(field);
final int maxDoc = reader.maxDoc();
final NumericDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
@Override
public Iterator<Number> iterator() {
updatesIter.reset();
return new Iterator<Number>() {
int curDoc = -1;
int updateDoc = updatesIter.nextDoc();
@Override
public boolean hasNext() {
return curDoc < maxDoc - 1;
}
@Override
public Number next() {
if (++curDoc >= maxDoc) {
throw new NoSuchElementException("no more documents to return values for");
}
if (curDoc == updateDoc) { // this document has an updated value
Long value = updatesIter.value(); // either null (unset value) or updated value
updateDoc = updatesIter.nextDoc(); // prepare for next round
return value;
} else {
// no update for this document
assert curDoc < updateDoc;
if (currentValues != null && docsWithField.get(curDoc)) {
// only read the current value if the document had a value before
return currentValues.get(curDoc);
} else {
return null;
}
}
}
@Override
public void remove() {
throw new UnsupportedOperationException("this iterator does not support removing elements");
}
};
}
});
}
info.advanceDocValuesGen();
assert !fieldFiles.containsKey(fieldInfo.number);
fieldFiles.put(fieldInfo.number, trackingDir.getCreatedFiles());
}
}
@SuppressWarnings("synthetic-access")
private void handleBinaryDVUpdates(FieldInfos infos, Map<String,BinaryDocValuesFieldUpdates> updates,
TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
for (Entry<String,BinaryDocValuesFieldUpdates> e : updates.entrySet()) {
final String field = e.getKey();
final BinaryDocValuesFieldUpdates fieldUpdates = e.getValue();
final long nextDocValuesGen = info.getNextDocValuesGen();
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.getDocCount();
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
final FieldInfo fieldInfo = infos.fieldInfo(field);
assert fieldInfo != null;
fieldInfo.setDocValuesGen(nextDocValuesGen);
final FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { fieldInfo });
// separately also track which files were created for this gen
final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
try (final DocValuesConsumer fieldsConsumer = dvFormat.fieldsConsumer(state)) {
// write the binary updates to a new gen'd docvalues file
fieldsConsumer.addBinaryField(fieldInfo, new Iterable<BytesRef>() {
final BinaryDocValues currentValues = reader.getBinaryDocValues(field);
final Bits docsWithField = reader.getDocsWithField(field);
final int maxDoc = reader.maxDoc();
final BinaryDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
@Override
public Iterator<BytesRef> iterator() {
updatesIter.reset();
return new Iterator<BytesRef>() {
int curDoc = -1;
int updateDoc = updatesIter.nextDoc();
BytesRef scratch = new BytesRef();
@Override
public boolean hasNext() {
return curDoc < maxDoc - 1;
}
@Override
public BytesRef next() {
if (++curDoc >= maxDoc) {
throw new NoSuchElementException("no more documents to return values for");
}
if (curDoc == updateDoc) { // this document has an updated value
BytesRef value = updatesIter.value(); // either null (unset value) or updated value
updateDoc = updatesIter.nextDoc(); // prepare for next round
return value;
} else {
// no update for this document
assert curDoc < updateDoc;
if (currentValues != null && docsWithField.get(curDoc)) {
// only read the current value if the document had a value before
currentValues.get(curDoc, scratch);
return scratch;
} else {
return null;
}
}
}
@Override
public void remove() {
throw new UnsupportedOperationException("this iterator does not support removing elements");
}
};
}
});
}
info.advanceDocValuesGen();
assert !fieldFiles.containsKey(fieldInfo.number);
fieldFiles.put(fieldInfo.number, trackingDir.getCreatedFiles());
}
}
private Set<String> writeFieldInfosGen(FieldInfos fieldInfos, Directory dir, DocValuesFormat dvFormat,
FieldInfosFormat infosFormat) throws IOException {
final long nextFieldInfosGen = info.getNextFieldInfosGen();
final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
// we write approximately that many bytes (based on Lucene46DVF):
// HEADER + FOOTER: 40
// 90 bytes per-field (over estimating long name and attributes map)
final long estInfosSize = 40 + 90 * fieldInfos.size();
final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
// separately also track which files were created for this gen
final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
infosFormat.getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, infosContext);
info.advanceFieldInfosGen();
return trackingDir.getCreatedFiles();
}
// Writes field updates (new _X_N updates files) to the directory
public synchronized void writeFieldUpdates(Directory dir, DocValuesFieldUpdates.Container dvUpdates) throws IOException {
@ -307,6 +471,8 @@ class ReadersAndUpdates {
// it:
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
final Map<Integer,Set<String>> newDVFiles = new HashMap<>();
Set<String> fieldInfosFiles = null;
FieldInfos fieldInfos = null;
boolean success = false;
try {
@ -341,147 +507,16 @@ class ReadersAndUpdates {
}
fieldInfos = builder.finish();
final long nextFieldInfosGen = info.getNextFieldInfosGen();
final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
final long estUpdatesSize = dvUpdates.ramBytesPerDoc() * info.info.getDocCount();
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.getDocCount(), estUpdatesSize));
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, updatesContext, segmentSuffix);
final DocValuesFormat docValuesFormat = codec.docValuesFormat();
final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
boolean fieldsConsumerSuccess = false;
try {
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeFieldUpdates: applying numeric updates; seg=" + info + " updates=" + numericFieldUpdates);
for (Entry<String,NumericDocValuesFieldUpdates> e : dvUpdates.numericDVUpdates.entrySet()) {
final String field = e.getKey();
final NumericDocValuesFieldUpdates fieldUpdates = e.getValue();
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
assert fieldInfo != null;
fieldInfo.setDocValuesGen(nextFieldInfosGen);
// write the numeric updates to a new gen'd docvalues file
fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
final NumericDocValues currentValues = reader.getNumericDocValues(field);
final Bits docsWithField = reader.getDocsWithField(field);
final int maxDoc = reader.maxDoc();
final NumericDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
@Override
public Iterator<Number> iterator() {
updatesIter.reset();
return new Iterator<Number>() {
int curDoc = -1;
int updateDoc = updatesIter.nextDoc();
@Override
public boolean hasNext() {
return curDoc < maxDoc - 1;
}
@Override
public Number next() {
if (++curDoc >= maxDoc) {
throw new NoSuchElementException("no more documents to return values for");
}
if (curDoc == updateDoc) { // this document has an updated value
Long value = updatesIter.value(); // either null (unset value) or updated value
updateDoc = updatesIter.nextDoc(); // prepare for next round
return value;
} else {
// no update for this document
assert curDoc < updateDoc;
if (currentValues != null && docsWithField.get(curDoc)) {
// only read the current value if the document had a value before
return currentValues.get(curDoc);
} else {
return null;
}
}
}
@Override
public void remove() {
throw new UnsupportedOperationException("this iterator does not support removing elements");
}
};
}
});
}
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " updates=" + dvUpdates.binaryDVUpdates);
for (Entry<String,BinaryDocValuesFieldUpdates> e : dvUpdates.binaryDVUpdates.entrySet()) {
final String field = e.getKey();
final BinaryDocValuesFieldUpdates dvFieldUpdates = e.getValue();
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
assert fieldInfo != null;
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " f=" + dvFieldUpdates + ", updates=" + dvFieldUpdates);
fieldInfo.setDocValuesGen(nextFieldInfosGen);
// write the numeric updates to a new gen'd docvalues file
fieldsConsumer.addBinaryField(fieldInfo, new Iterable<BytesRef>() {
final BinaryDocValues currentValues = reader.getBinaryDocValues(field);
final Bits docsWithField = reader.getDocsWithField(field);
final int maxDoc = reader.maxDoc();
final BinaryDocValuesFieldUpdates.Iterator updatesIter = dvFieldUpdates.iterator();
@Override
public Iterator<BytesRef> iterator() {
updatesIter.reset();
return new Iterator<BytesRef>() {
int curDoc = -1;
int updateDoc = updatesIter.nextDoc();
BytesRef scratch = new BytesRef();
@Override
public boolean hasNext() {
return curDoc < maxDoc - 1;
}
@Override
public BytesRef next() {
if (++curDoc >= maxDoc) {
throw new NoSuchElementException("no more documents to return values for");
}
if (curDoc == updateDoc) { // this document has an updated value
BytesRef value = updatesIter.value(); // either null (unset value) or updated value
updateDoc = updatesIter.nextDoc(); // prepare for next round
return value;
} else {
// no update for this document
assert curDoc < updateDoc;
if (currentValues != null && docsWithField.get(curDoc)) {
// only read the current value if the document had a value before
currentValues.get(curDoc, scratch);
return scratch;
} else {
return null;
}
}
}
@Override
public void remove() {
throw new UnsupportedOperationException("this iterator does not support removing elements");
}
};
}
});
}
// we write approximately that many bytes (based on Lucene46DVF):
// HEADER + FOOTER: 40
// 90 bytes per-field (over estimating long name and attributes map)
final long estInfosSize = 40 + 90 * fieldInfos.size();
final IOContext infosContext = new IOContext(new FlushInfo(info.info.getDocCount(), estInfosSize));
codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, infosContext);
fieldsConsumerSuccess = true;
} finally {
if (fieldsConsumerSuccess) {
fieldsConsumer.close();
} else {
IOUtils.closeWhileHandlingException(fieldsConsumer);
}
}
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeFieldUpdates: applying numeric updates; seg=" + info + " updates=" + numericFieldUpdates);
handleNumericDVUpdates(fieldInfos, dvUpdates.numericDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " updates=" + dvUpdates.binaryDVUpdates);
handleBinaryDVUpdates(fieldInfos, dvUpdates.binaryDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: write fieldInfos; seg=" + info);
fieldInfosFiles = writeFieldInfosGen(fieldInfos, trackingDir, docValuesFormat, codec.fieldInfosFormat());
} finally {
if (reader != this.reader) {
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: closeReader " + reader);
@ -492,9 +527,10 @@ class ReadersAndUpdates {
success = true;
} finally {
if (!success) {
// Advance only the nextWriteFieldInfosGen so that a 2nd
// attempt to write will write to a new file
// Advance only the nextWriteFieldInfosGen and nextWriteDocValuesGen, so
// that a 2nd attempt to write will write to a new file
info.advanceNextWriteFieldInfosGen();
info.advanceNextWriteDocValuesGen();
// Delete any partially created file(s):
for (String fileName : trackingDir.getCreatedFiles()) {
@ -507,7 +543,6 @@ class ReadersAndUpdates {
}
}
info.advanceFieldInfosGen();
// copy all the updates to mergingUpdates, so they can later be applied to the merged segment
if (isMerging) {
for (Entry<String,NumericDocValuesFieldUpdates> e : dvUpdates.numericDVUpdates.entrySet()) {
@ -528,22 +563,21 @@ class ReadersAndUpdates {
}
}
// create a new map, keeping only the gens that are in use
Map<Long,Set<String>> genUpdatesFiles = info.getUpdatesFiles();
Map<Long,Set<String>> newGenUpdatesFiles = new HashMap<>();
final long fieldInfosGen = info.getFieldInfosGen();
for (FieldInfo fi : fieldInfos) {
long dvGen = fi.getDocValuesGen();
if (dvGen != -1 && !newGenUpdatesFiles.containsKey(dvGen)) {
if (dvGen == fieldInfosGen) {
newGenUpdatesFiles.put(fieldInfosGen, trackingDir.getCreatedFiles());
} else {
newGenUpdatesFiles.put(dvGen, genUpdatesFiles.get(dvGen));
}
// writing field updates succeeded
assert fieldInfosFiles != null;
info.setFieldInfosFiles(fieldInfosFiles);
// update the doc-values updates files. the files map each field to its set
// of files, hence we copy from the existing map all fields w/ updates that
// were not updated in this session, and add new mappings for fields that
// were updated now.
assert !newDVFiles.isEmpty();
for (Entry<Integer,Set<String>> e : info.getDocValuesUpdatesFiles().entrySet()) {
if (!newDVFiles.containsKey(e.getKey())) {
newDVFiles.put(e.getKey(), e.getValue());
}
}
info.setGenUpdatesFiles(newGenUpdatesFiles);
info.setDocValuesUpdatesFiles(newDVFiles);
// wrote new files, should checkpoint()
writer.checkpoint();

View File

@ -51,11 +51,27 @@ public class SegmentCommitInfo {
// Generation number of the FieldInfos (-1 if there are no updates)
private long fieldInfosGen;
// Normally 1 + fieldInfosGen, unless an exception was hit on last attempt to
// Normally 1+fieldInfosGen, unless an exception was hit on last attempt to
// write
private long nextWriteFieldInfosGen;
// Generation number of the DocValues (-1 if there are no updates)
private long docValuesGen;
// Normally 1+dvGen, unless an exception was hit on last attempt to
// write
private long nextWriteDocValuesGen;
// Track the per-field DocValues update files
private final Map<Integer,Set<String>> dvUpdatesFiles = new HashMap<>();
// TODO should we add .files() to FieldInfosFormat, like we have on
// LiveDocsFormat?
// track the fieldInfos update files
private final Set<String> fieldInfosFiles = new HashSet<>();
// Track the per-generation updates files
@Deprecated
private final Map<Long,Set<String>> genUpdatesFiles = new HashMap<>();
private volatile long sizeInBytes = -1;
@ -71,36 +87,53 @@ public class SegmentCommitInfo {
* deletion generation number (used to name deletion files)
* @param fieldInfosGen
* FieldInfos generation number (used to name field-infos files)
**/
public SegmentCommitInfo(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) {
* @param docValuesGen
* DocValues generation number (used to name doc-values updates files)
*/
public SegmentCommitInfo(SegmentInfo info, int delCount, long delGen, long fieldInfosGen, long docValuesGen) {
this.info = info;
this.delCount = delCount;
this.delGen = delGen;
if (delGen == -1) {
nextWriteDelGen = 1;
} else {
nextWriteDelGen = delGen+1;
}
this.nextWriteDelGen = delGen == -1 ? 1 : delGen + 1;
this.fieldInfosGen = fieldInfosGen;
if (fieldInfosGen == -1) {
nextWriteFieldInfosGen = 1;
} else {
nextWriteFieldInfosGen = fieldInfosGen + 1;
}
this.nextWriteFieldInfosGen = fieldInfosGen == -1 ? 1 : fieldInfosGen + 1;
this.docValuesGen = docValuesGen;
this.nextWriteDocValuesGen = docValuesGen == -1 ? 1 : docValuesGen + 1;
}
/** Returns the per generation updates files. */
public Map<Long,Set<String>> getUpdatesFiles() {
return Collections.unmodifiableMap(genUpdatesFiles);
}
/** Sets the updates file names per generation. Does not deep clone the map. */
/**
* Sets the updates file names per generation. Does not deep clone the map.
*
* @deprecated required to support 4.6-4.8 indexes.
*/
@Deprecated
public void setGenUpdatesFiles(Map<Long,Set<String>> genUpdatesFiles) {
this.genUpdatesFiles.clear();
this.genUpdatesFiles.putAll(genUpdatesFiles);
}
/** Returns the per-field DocValues updates files. */
public Map<Integer,Set<String>> getDocValuesUpdatesFiles() {
return Collections.unmodifiableMap(dvUpdatesFiles);
}
/** Sets the DocValues updates file names, per field number. Does not deep clone the map. */
public void setDocValuesUpdatesFiles(Map<Integer,Set<String>> dvUpdatesFiles) {
this.dvUpdatesFiles.clear();
this.dvUpdatesFiles.putAll(dvUpdatesFiles);
}
/** Returns the FieldInfos file names. */
public Set<String> getFieldInfosFiles() {
return Collections.unmodifiableSet(fieldInfosFiles);
}
/** Sets the FieldInfos file names. */
public void setFieldInfosFiles(Set<String> fieldInfosFiles) {
this.fieldInfosFiles.clear();
this.fieldInfosFiles.addAll(fieldInfosFiles);
}
/** Called when we succeed in writing deletes */
void advanceDelGen() {
delGen = nextWriteDelGen;
@ -129,6 +162,21 @@ public class SegmentCommitInfo {
void advanceNextWriteFieldInfosGen() {
nextWriteFieldInfosGen++;
}
/** Called when we succeed in writing a new DocValues generation. */
void advanceDocValuesGen() {
docValuesGen = nextWriteDocValuesGen;
nextWriteDocValuesGen = docValuesGen + 1;
sizeInBytes = -1;
}
/**
* Called if there was an exception while writing a new generation of
* DocValues, so that we don't try to write to the same file more than once.
*/
void advanceNextWriteDocValuesGen() {
nextWriteDocValuesGen++;
}
/** Returns total size in bytes of all files for this
* segment. */
@ -155,11 +203,20 @@ public class SegmentCommitInfo {
// Must separately add any live docs files:
info.getCodec().liveDocsFormat().files(this, files);
// Must separately add any field updates files
// Must separately add any per-gen updates files. This can go away when we
// get rid of genUpdatesFiles (6.0)
for (Set<String> updateFiles : genUpdatesFiles.values()) {
files.addAll(updateFiles);
}
// must separately add any field updates files
for (Set<String> updatefiles : dvUpdatesFiles.values()) {
files.addAll(updatefiles);
}
// must separately add fieldInfos files
files.addAll(fieldInfosFiles);
return files;
}
@ -200,6 +257,19 @@ public class SegmentCommitInfo {
return fieldInfosGen;
}
/** Returns the next available generation number of the DocValues files. */
public long getNextDocValuesGen() {
return nextWriteDocValuesGen;
}
/**
* Returns the generation number of the DocValues file or -1 if there are no
* doc-values updates yet.
*/
public long getDocValuesGen() {
return docValuesGen;
}
/**
* Returns the next available generation number
* of the live docs file.
@ -239,6 +309,9 @@ public class SegmentCommitInfo {
if (fieldInfosGen != -1) {
s += ":fieldInfosGen=" + fieldInfosGen;
}
if (docValuesGen != -1) {
s += ":dvGen=" + docValuesGen;
}
return s;
}
@ -249,19 +322,27 @@ public class SegmentCommitInfo {
@Override
public SegmentCommitInfo clone() {
SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen);
SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, docValuesGen);
// Not clear that we need to carry over nextWriteDelGen
// (i.e. do we ever clone after a failed write and
// before the next successful write?), but just do it to
// be safe:
other.nextWriteDelGen = nextWriteDelGen;
other.nextWriteFieldInfosGen = nextWriteFieldInfosGen;
other.nextWriteDocValuesGen = nextWriteDocValuesGen;
// deep clone
for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
other.genUpdatesFiles.put(e.getKey(), new HashSet<>(e.getValue()));
}
// deep clone
for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
other.dvUpdatesFiles.put(e.getKey(), new HashSet<>(e.getValue()));
}
other.fieldInfosFiles.addAll(fieldInfosFiles);
return other;
}
}

View File

@ -38,7 +38,7 @@ final class SegmentDocValues {
private final Map<Long,RefCount<DocValuesProducer>> genDVProducers = new HashMap<>();
private RefCount<DocValuesProducer> newDocValuesProducer(SegmentCommitInfo si, IOContext context, Directory dir,
DocValuesFormat dvFormat, final Long gen, List<FieldInfo> infos) throws IOException {
DocValuesFormat dvFormat, final Long gen, FieldInfos infos) throws IOException {
Directory dvDir = dir;
String segmentSuffix = "";
if (gen.longValue() != -1) {
@ -47,7 +47,7 @@ final class SegmentDocValues {
}
// set SegmentReadState to list only the fields that are relevant to that gen
SegmentReadState srs = new SegmentReadState(dvDir, si.info, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])), context, segmentSuffix);
SegmentReadState srs = new SegmentReadState(dvDir, si.info, infos, context, segmentSuffix);
return new RefCount<DocValuesProducer>(dvFormat.fieldsProducer(srs)) {
@SuppressWarnings("synthetic-access")
@Override
@ -62,7 +62,7 @@ final class SegmentDocValues {
/** Returns the {@link DocValuesProducer} for the given generation. */
synchronized DocValuesProducer getDocValuesProducer(long gen, SegmentCommitInfo si, IOContext context, Directory dir,
DocValuesFormat dvFormat, List<FieldInfo> infos) throws IOException {
DocValuesFormat dvFormat, FieldInfos infos) throws IOException {
RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
if (dvp == null) {
dvp = newDocValuesProducer(si, context, dir, dvFormat, gen, infos);

View File

@ -33,6 +33,7 @@ import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.store.ChecksumIndexInput;
@ -44,68 +45,79 @@ import org.apache.lucene.store.NoSuchDirectoryException;
import org.apache.lucene.util.IOUtils;
/**
* A collection of segmentInfo objects with methods for operating on
* those segments in relation to the file system.
* A collection of segmentInfo objects with methods for operating on those
* segments in relation to the file system.
* <p>
* The active segments in the index are stored in the segment info file,
* <tt>segments_N</tt>. There may be one or more <tt>segments_N</tt> files in the
* index; however, the one with the largest generation is the active one (when
* older segments_N files are present it's because they temporarily cannot be
* deleted, or, a writer is in the process of committing, or a custom
* {@link org.apache.lucene.index.IndexDeletionPolicy IndexDeletionPolicy}
* is in use). This file lists each segment by name and has details about the
* codec and generation of deletes.
* <tt>segments_N</tt>. There may be one or more <tt>segments_N</tt> files in
* the index; however, the one with the largest generation is the active one
* (when older segments_N files are present it's because they temporarily cannot
* be deleted, or, a writer is in the process of committing, or a custom
* {@link org.apache.lucene.index.IndexDeletionPolicy IndexDeletionPolicy} is in
* use). This file lists each segment by name and has details about the codec
* and generation of deletes.
* </p>
* <p>
* There is also a file <tt>segments.gen</tt>. This file contains the current
* generation (the <tt>_N</tt> in <tt>segments_N</tt>) of the index. This is
* used only as a fallback in case the current generation cannot be accurately
* determined by directory listing alone (as is the case for some NFS clients
* with time-based directory cache expiration). This file simply contains an
* {@link DataOutput#writeInt Int32} version header (
* {@link #FORMAT_SEGMENTS_GEN_CURRENT}), followed by the generation recorded as
* {@link DataOutput#writeLong Int64}, written twice.
* </p>
* <p>There is also a file <tt>segments.gen</tt>. This file contains
* the current generation (the <tt>_N</tt> in <tt>segments_N</tt>) of the index.
* This is used only as a fallback in case the current generation cannot be
* accurately determined by directory listing alone (as is the case for some NFS
* clients with time-based directory cache expiration). This file simply contains
* an {@link DataOutput#writeInt Int32} version header
* ({@link #FORMAT_SEGMENTS_GEN_CURRENT}), followed by the
* generation recorded as {@link DataOutput#writeLong Int64}, written twice.</p>
* <p>
* Files:
* <ul>
* <li><tt>segments.gen</tt>: GenHeader, Generation, Generation, Footer
* <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount,
* &lt;SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles&gt;<sup>SegCount</sup>,
* CommitUserData, Footer
* <li><tt>segments.gen</tt>: GenHeader, Generation, Generation, Footer
* <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount, &lt;SegName,
* SegCodec, DelGen, DeletionCount, FieldInfosGen, DocValuesGen,
* UpdatesFiles&gt;<sup>SegCount</sup>, CommitUserData, Footer
* </ul>
* </p>
* Data types:
* <p>
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>GenHeader, NameCounter, SegCount, DeletionCount --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>Generation, Version, DelGen, Checksum, FieldInfosGen --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>SegName, SegCodec --&gt; {@link DataOutput#writeString String}</li>
* <li>CommitUserData --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* <li>UpdatesFiles --&gt; {@link DataOutput#writeStringSet(Set) Set&lt;String&gt;}</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>GenHeader, NameCounter, SegCount, DeletionCount --&gt;
* {@link DataOutput#writeInt Int32}</li>
* <li>Generation, Version, DelGen, Checksum, FieldInfosGen, DocValuesGen --&gt;
* {@link DataOutput#writeLong Int64}</li>
* <li>SegName, SegCodec --&gt; {@link DataOutput#writeString String}</li>
* <li>CommitUserData --&gt; {@link DataOutput#writeStringStringMap
* Map&lt;String,String&gt;}</li>
* <li>UpdatesFiles --&gt; Map&lt;{@link DataOutput#writeInt Int32},
* {@link DataOutput#writeStringSet(Set) Set&lt;String&gt;}&gt;</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* </p>
* Field Descriptions:
* <p>
* <ul>
* <li>Version counts how often the index has been changed by adding or deleting
* documents.</li>
* <li>NameCounter is used to generate names for new segment files.</li>
* <li>SegName is the name of the segment, and is used as the file name prefix for
* all of the files that compose the segment's index.</li>
* <li>DelGen is the generation count of the deletes file. If this is -1,
* there are no deletes. Anything above zero means there are deletes
* stored by {@link LiveDocsFormat}.</li>
* <li>DeletionCount records the number of deleted documents in this segment.</li>
* <li>SegCodec is the {@link Codec#getName() name} of the Codec that encoded
* this segment.</li>
* <li>CommitUserData stores an optional user-supplied opaque
* Map&lt;String,String&gt; that was passed to
* {@link IndexWriter#setCommitData(java.util.Map)}.</li>
* <li>FieldInfosGen is the generation count of the fieldInfos file. If this is -1,
* there are no updates to the fieldInfos in that segment. Anything above zero
* means there are updates to fieldInfos stored by {@link FieldInfosFormat}.</li>
* <li>UpdatesFiles stores the list of files that were updated in that segment.</li>
* <li>Version counts how often the index has been changed by adding or deleting
* documents.</li>
* <li>NameCounter is used to generate names for new segment files.</li>
* <li>SegName is the name of the segment, and is used as the file name prefix
* for all of the files that compose the segment's index.</li>
* <li>DelGen is the generation count of the deletes file. If this is -1, there
* are no deletes. Anything above zero means there are deletes stored by
* {@link LiveDocsFormat}.</li>
* <li>DeletionCount records the number of deleted documents in this segment.</li>
* <li>SegCodec is the {@link Codec#getName() name} of the Codec that encoded
* this segment.</li>
* <li>CommitUserData stores an optional user-supplied opaque
* Map&lt;String,String&gt; that was passed to
* {@link IndexWriter#setCommitData(java.util.Map)}.</li>
* <li>FieldInfosGen is the generation count of the fieldInfos file. If this is
* -1, there are no updates to the fieldInfos in that segment. Anything above
* zero means there are updates to fieldInfos stored by {@link FieldInfosFormat}
* .</li>
* <li>DocValuesGen is the generation count of the updatable DocValues. If this
* is -1, there are no updates to DocValues in that segment. Anything above zero
* means there are updates to DocValues stored by {@link DocValuesFormat}.</li>
* <li>UpdatesFiles stores the set of files that were updated in that segment
* per field.</li>
* </ul>
* </p>
*
@ -121,6 +133,9 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
/** The file format version for the segments_N codec header, since 4.8+ */
public static final int VERSION_48 = 2;
/** The file format version for the segments_N codec header, since 4.9+ */
public static final int VERSION_49 = 3;
// Used for the segments.gen file only!
// Whenever you add a new format, make it 1 smaller (negative version logic)!
@ -330,7 +345,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
}
// 4.0+
int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_48);
int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_49);
version = input.readLong();
counter = input.readInt();
int numSegments = input.readInt();
@ -352,19 +367,45 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
if (format >= VERSION_46) {
fieldInfosGen = input.readLong();
}
SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen);
long dvGen = -1;
if (format >= VERSION_49) {
dvGen = input.readLong();
} else {
dvGen = fieldInfosGen;
}
SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
if (format >= VERSION_46) {
int numGensUpdatesFiles = input.readInt();
final Map<Long,Set<String>> genUpdatesFiles;
if (numGensUpdatesFiles == 0) {
genUpdatesFiles = Collections.emptyMap();
} else {
genUpdatesFiles = new HashMap<>(numGensUpdatesFiles);
for (int i = 0; i < numGensUpdatesFiles; i++) {
genUpdatesFiles.put(input.readLong(), input.readStringSet());
if (format < VERSION_49) {
// Recorded per-generation files, which were buggy (see
// LUCENE-5636). We need to read and keep them so we continue to
// reference those files. Unfortunately it means that the files will
// be referenced even if the fields are updated again, until the
// segment is merged.
final int numGensUpdatesFiles = input.readInt();
final Map<Long,Set<String>> genUpdatesFiles;
if (numGensUpdatesFiles == 0) {
genUpdatesFiles = Collections.emptyMap();
} else {
genUpdatesFiles = new HashMap<>(numGensUpdatesFiles);
for (int i = 0; i < numGensUpdatesFiles; i++) {
genUpdatesFiles.put(input.readLong(), input.readStringSet());
}
}
siPerCommit.setGenUpdatesFiles(genUpdatesFiles);
} else {
siPerCommit.setFieldInfosFiles(input.readStringSet());
final Map<Integer,Set<String>> dvUpdateFiles;
final int numDVFields = input.readInt();
if (numDVFields == 0) {
dvUpdateFiles = Collections.emptyMap();
} else {
dvUpdateFiles = new HashMap<>(numDVFields);
for (int i = 0; i < numDVFields; i++) {
dvUpdateFiles.put(input.readInt(), input.readStringSet());
}
}
siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
}
siPerCommit.setGenUpdatesFiles(genUpdatesFiles);
}
add(siPerCommit);
}
@ -429,7 +470,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
try {
segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
CodecUtil.writeHeader(segnOutput, "segments", VERSION_48);
CodecUtil.writeHeader(segnOutput, "segments", VERSION_49);
segnOutput.writeLong(version);
segnOutput.writeInt(counter); // write counter
segnOutput.writeInt(size()); // write infos
@ -444,10 +485,12 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
}
segnOutput.writeInt(delCount);
segnOutput.writeLong(siPerCommit.getFieldInfosGen());
final Map<Long,Set<String>> genUpdatesFiles = siPerCommit.getUpdatesFiles();
segnOutput.writeInt(genUpdatesFiles.size());
for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
segnOutput.writeLong(e.getKey());
segnOutput.writeLong(siPerCommit.getDocValuesGen());
segnOutput.writeStringSet(siPerCommit.getFieldInfosFiles());
final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
segnOutput.writeInt(dvUpdatesFiles.size());
for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
segnOutput.writeInt(e.getKey());
segnOutput.writeStringSet(e.getValue());
}
assert si.dir == directory;

View File

@ -23,7 +23,6 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map.Entry;
import java.util.Map;
import java.util.Set;
@ -41,6 +40,7 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.Version;
/**
* IndexReader implementation over a single segment.
@ -112,7 +112,7 @@ public final class SegmentReader extends AtomicReader {
liveDocs = null;
}
numDocs = si.info.getDocCount() - si.getDelCount();
if (fieldInfos.hasDocValues()) {
initDocValuesProducers(codec);
}
@ -175,24 +175,88 @@ public final class SegmentReader extends AtomicReader {
}
// initialize the per-field DocValuesProducer
@SuppressWarnings("deprecation")
private void initDocValuesProducers(Codec codec) throws IOException {
final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
final DocValuesFormat dvFormat = codec.docValuesFormat();
final Map<Long,List<FieldInfo>> genInfos = getGenInfos();
// System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gens=" + genInfos.keySet());
// TODO: can we avoid iterating over fieldinfos several times and creating maps of all this stuff if dv updates do not exist?
for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
Long gen = e.getKey();
List<FieldInfo> infos = e.getValue();
DocValuesProducer dvp = segDocValues.getDocValuesProducer(gen, si, IOContext.READ, dir, dvFormat, infos);
dvGens.add(gen);
for (FieldInfo fi : infos) {
if (!si.hasFieldUpdates()) {
// simple case, no DocValues updates
final DocValuesProducer dvp = segDocValues.getDocValuesProducer(-1L, si, IOContext.READ, dir, dvFormat, fieldInfos);
dvGens.add(-1L);
dvProducers.add(dvp);
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
assert fi.getDocValuesGen() == -1;
dvProducersByField.put(fi.name, dvp);
}
dvProducers.add(dvp);
return;
}
Version ver;
try {
ver = Version.parseLeniently(si.info.getVersion());
} catch (IllegalArgumentException e) {
// happened in TestBackwardsCompatibility on a 4.0.0.2 index (no matching
// Version constant), anyway it's a pre-4.9 index.
ver = null;
}
if (ver != null && ver.onOrAfter(Version.LUCENE_4_9)) {
DocValuesProducer baseProducer = null;
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
long docValuesGen = fi.getDocValuesGen();
if (docValuesGen == -1) {
if (baseProducer == null) {
// System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
// the base producer gets all the fields, so the Codec can validate properly
baseProducer = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
dvGens.add(docValuesGen);
dvProducers.add(baseProducer);
}
// System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
dvProducersByField.put(fi.name, baseProducer);
} else {
assert !dvGens.contains(docValuesGen);
// System.out.println("[" + Thread.currentThread().getName() + "] SR.initDocValuesProducers: segInfo=" + si + "; gen=" + docValuesGen + "; field=" + fi.name);
final DocValuesProducer dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(new FieldInfo[] { fi }));
dvGens.add(docValuesGen);
dvProducers.add(dvp);
dvProducersByField.put(fi.name, dvp);
}
}
} else {
// For pre-4.9 indexes, especially with doc-values updates, multiple
// FieldInfos could belong to the same dvGen. Therefore need to make sure
// we initialize each DocValuesProducer once per gen.
Map<Long,List<FieldInfo>> genInfos = new HashMap<>();
for (FieldInfo fi : fieldInfos) {
if (!fi.hasDocValues()) continue;
List<FieldInfo> genFieldInfos = genInfos.get(fi.getDocValuesGen());
if (genFieldInfos == null) {
genFieldInfos = new ArrayList<>();
genInfos.put(fi.getDocValuesGen(), genFieldInfos);
}
genFieldInfos.add(fi);
}
for (Map.Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
long docValuesGen = e.getKey();
List<FieldInfo> infos = e.getValue();
final DocValuesProducer dvp;
if (docValuesGen == -1) {
// we need to send all FieldInfos to gen=-1, but later we need to
// record the DVP only for the "true" gen=-1 fields (not updated)
dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, fieldInfos);
} else {
dvp = segDocValues.getDocValuesProducer(docValuesGen, si, IOContext.READ, dir, dvFormat, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])));
}
dvGens.add(docValuesGen);
dvProducers.add(dvp);
for (FieldInfo fi : infos) {
dvProducersByField.put(fi.name, dvp);
}
}
}
}
@ -229,24 +293,6 @@ public final class SegmentReader extends AtomicReader {
}
}
// returns a gen->List<FieldInfo> mapping. Fields without DV updates have gen=-1
private Map<Long,List<FieldInfo>> getGenInfos() {
final Map<Long,List<FieldInfo>> genInfos = new HashMap<>();
for (FieldInfo fi : fieldInfos) {
if (fi.getDocValuesType() == null) {
continue;
}
long gen = fi.getDocValuesGen();
List<FieldInfo> infos = genInfos.get(gen);
if (infos == null) {
infos = new ArrayList<>();
genInfos.put(gen, infos);
}
infos.add(fi);
}
return genInfos;
}
@Override
public Bits getLiveDocs() {
ensureOpen();

View File

@ -1285,27 +1285,23 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
Document doc = new Document();
doc.add(new StringField("id", "d0", Store.NO));
doc.add(new BinaryDocValuesField("f", toBytes(1L)));
doc.add(new BinaryDocValuesField("f1", toBytes(1L)));
doc.add(new BinaryDocValuesField("f2", toBytes(1L)));
writer.addDocument(doc);
// create first gen of update files
writer.updateBinaryDocValue(new Term("id", "d0"), "f", toBytes(2L));
writer.commit();
int numFiles = dir.listAll().length;
DirectoryReader r = DirectoryReader.open(dir);
BytesRef scratch = new BytesRef();
assertEquals(2L, getValue(r.leaves().get(0).reader().getBinaryDocValues("f"), 0, scratch));
r.close();
// create second gen of update files, first gen should be deleted
writer.updateBinaryDocValue(new Term("id", "d0"), "f", toBytes(5L));
writer.commit();
assertEquals(numFiles, dir.listAll().length);
r = DirectoryReader.open(dir);
assertEquals(5L, getValue(r.leaves().get(0).reader().getBinaryDocValues("f"), 0, scratch));
r.close();
// update each field twice to make sure all unneeded files are deleted
for (String f : new String[] { "f1", "f2" }) {
writer.updateBinaryDocValue(new Term("id", "d0"), f, toBytes(2L));
writer.commit();
int numFiles = dir.listAll().length;
// update again, number of files shouldn't change (old field's gen is
// removed)
writer.updateBinaryDocValue(new Term("id", "d0"), f, toBytes(3L));
writer.commit();
assertEquals(numFiles, dir.listAll().length);
}
writer.shutdown();
dir.close();

View File

@ -44,7 +44,6 @@ import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
/** JUnit adaptation of an older test case DocTest. */
@ -240,7 +239,7 @@ public class TestDoc extends LuceneTestCase {
}
}
return new SegmentCommitInfo(info, 0, -1L, -1L);
return new SegmentCommitInfo(info, 0, -1L, -1L, -1L);
}

View File

@ -325,7 +325,7 @@ public class TestIndexWriterThreadsToSegments extends LuceneTestCase {
segSeen.add(segName);
SegmentInfo si = new Lucene46SegmentInfoFormat().getSegmentInfoReader().read(dir, segName, IOContext.DEFAULT);
si.setCodec(codec);
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1);
SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1, -1);
SegmentReader sr = new SegmentReader(sci, IOContext.DEFAULT);
try {
thread0Count += sr.docFreq(new Term("field", "threadID0"));

View File

@ -1264,27 +1264,24 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
Document doc = new Document();
doc.add(new StringField("id", "d0", Store.NO));
doc.add(new NumericDocValuesField("f", 1L));
doc.add(new NumericDocValuesField("f1", 1L));
doc.add(new NumericDocValuesField("f2", 1L));
writer.addDocument(doc);
// create first gen of update files
writer.updateNumericDocValue(new Term("id", "d0"), "f", 2L);
writer.commit();
int numFiles = dir.listAll().length;
DirectoryReader r = DirectoryReader.open(dir);
assertEquals(2L, r.leaves().get(0).reader().getNumericDocValues("f").get(0));
r.close();
// update each field twice to make sure all unneeded files are deleted
for (String f : new String[] { "f1", "f2" }) {
writer.updateNumericDocValue(new Term("id", "d0"), f, 2L);
writer.commit();
int numFiles = dir.listAll().length;
// update again, number of files shouldn't change (old field's gen is
// removed)
writer.updateNumericDocValue(new Term("id", "d0"), f, 3L);
writer.commit();
assertEquals(numFiles, dir.listAll().length);
}
// create second gen of update files, first gen should be deleted
writer.updateNumericDocValue(new Term("id", "d0"), "f", 5L);
writer.commit();
assertEquals(numFiles, dir.listAll().length);
r = DirectoryReader.open(dir);
assertEquals(5L, r.leaves().get(0).reader().getNumericDocValues("f").get(0));
r.close();
writer.shutdown();
dir.close();
}
@ -1455,7 +1452,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
writer.updateNumericDocValue(new Term("id", "doc-0"), "val", 100L);
DirectoryReader reader = DirectoryReader.open(writer, true); // flush
assertEquals(0, cachingDir.listCachedFiles().length);
for (String f : cachingDir.listAll()) System.out.println(f + " " + cachingDir.fileLength(f));
IOUtils.close(reader, writer, cachingDir);
}

View File

@ -30,7 +30,6 @@ import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TestUtil;
public class TestSegmentMerger extends LuceneTestCase {
//The variables for the new merged segment
@ -91,7 +90,7 @@ public class TestSegmentMerger extends LuceneTestCase {
SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo(
new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged,
false, codec, null),
0, -1L, -1L),
0, -1L, -1L, -1L),
newIOContext(random()));
assertTrue(mergedReader != null);
assertTrue(mergedReader.numDocs() == 2);

View File

@ -140,7 +140,9 @@ public class IndexSplitter {
// Same info just changing the dir:
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(),
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics());
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen()));
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
infoPerCommit.getDocValuesGen()));
// now copy files over
Collection<String> files = infoPerCommit.files();
for (final String srcName : files) {