mirror of https://github.com/apache/lucene.git
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:
parent
d1e607f5df
commit
c863a7251d
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 + ")");
|
||||
|
|
|
@ -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 " +
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
* <SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles><sup>SegCount</sup>,
|
||||
* CommitUserData, Footer
|
||||
* <li><tt>segments.gen</tt>: GenHeader, Generation, Generation, Footer
|
||||
* <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount, <SegName,
|
||||
* SegCodec, DelGen, DeletionCount, FieldInfosGen, DocValuesGen,
|
||||
* UpdatesFiles><sup>SegCount</sup>, CommitUserData, Footer
|
||||
* </ul>
|
||||
* </p>
|
||||
* Data types:
|
||||
* <p>
|
||||
* <ul>
|
||||
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
|
||||
* <li>GenHeader, NameCounter, SegCount, DeletionCount --> {@link DataOutput#writeInt Int32}</li>
|
||||
* <li>Generation, Version, DelGen, Checksum, FieldInfosGen --> {@link DataOutput#writeLong Int64}</li>
|
||||
* <li>SegName, SegCodec --> {@link DataOutput#writeString String}</li>
|
||||
* <li>CommitUserData --> {@link DataOutput#writeStringStringMap Map<String,String>}</li>
|
||||
* <li>UpdatesFiles --> {@link DataOutput#writeStringSet(Set) Set<String>}</li>
|
||||
* <li>Footer --> {@link CodecUtil#writeFooter CodecFooter}</li>
|
||||
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
|
||||
* <li>GenHeader, NameCounter, SegCount, DeletionCount -->
|
||||
* {@link DataOutput#writeInt Int32}</li>
|
||||
* <li>Generation, Version, DelGen, Checksum, FieldInfosGen, DocValuesGen -->
|
||||
* {@link DataOutput#writeLong Int64}</li>
|
||||
* <li>SegName, SegCodec --> {@link DataOutput#writeString String}</li>
|
||||
* <li>CommitUserData --> {@link DataOutput#writeStringStringMap
|
||||
* Map<String,String>}</li>
|
||||
* <li>UpdatesFiles --> Map<{@link DataOutput#writeInt Int32},
|
||||
* {@link DataOutput#writeStringSet(Set) Set<String>}></li>
|
||||
* <li>Footer --> {@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<String,String> 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<String,String> 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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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"));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
Loading…
Reference in New Issue