Return-Path: X-Original-To: apmail-lucene-commits-archive@www.apache.org Delivered-To: apmail-lucene-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 4FD5611A97 for ; Wed, 21 May 2014 16:03:52 +0000 (UTC) Received: (qmail 12413 invoked by uid 500); 21 May 2014 16:03:52 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 12406 invoked by uid 99); 21 May 2014 16:03:52 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 21 May 2014 16:03:52 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 21 May 2014 16:03:44 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 7EFE3238896F; Wed, 21 May 2014 16:03:18 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1596608 [1/2] - in /lucene/dev/branches/lucene4236: ./ dev-tools/ dev-tools/idea/lucene/spatial/ dev-tools/idea/solr/contrib/analysis-extras/ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/ l... Date: Wed, 21 May 2014 16:03:17 -0000 To: commits@lucene.apache.org From: rmuir@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140521160318.7EFE3238896F@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: rmuir Date: Wed May 21 16:03:15 2014 New Revision: 1596608 URL: http://svn.apache.org/r1596608 Log: merge trunk Added: lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java - copied unchanged from r1596607, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java Modified: lucene/dev/branches/lucene4236/ (props changed) lucene/dev/branches/lucene4236/dev-tools/ (props changed) lucene/dev/branches/lucene4236/dev-tools/idea/lucene/spatial/spatial.iml lucene/dev/branches/lucene4236/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml lucene/dev/branches/lucene4236/lucene/ (props changed) lucene/dev/branches/lucene4236/lucene/CHANGES.txt (contents, props changed) lucene/dev/branches/lucene4236/lucene/codecs/ (props changed) lucene/dev/branches/lucene4236/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java lucene/dev/branches/lucene4236/lucene/common-build.xml (contents, props changed) lucene/dev/branches/lucene4236/lucene/core/ (props changed) lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestDoc.java lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java lucene/dev/branches/lucene4236/lucene/memory/ (props changed) lucene/dev/branches/lucene4236/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java lucene/dev/branches/lucene4236/lucene/misc/ (props changed) lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java lucene/dev/branches/lucene4236/lucene/test-framework/ (props changed) lucene/dev/branches/lucene4236/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java lucene/dev/branches/lucene4236/solr/ (props changed) lucene/dev/branches/lucene4236/solr/core/ (props changed) lucene/dev/branches/lucene4236/solr/core/src/java/org/apache/solr/search/Insanity.java lucene/dev/branches/lucene4236/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java lucene/dev/branches/lucene4236/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java Modified: lucene/dev/branches/lucene4236/dev-tools/idea/lucene/spatial/spatial.iml URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/dev-tools/idea/lucene/spatial/spatial.iml?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/dev-tools/idea/lucene/spatial/spatial.iml (original) +++ lucene/dev/branches/lucene4236/dev-tools/idea/lucene/spatial/spatial.iml Wed May 21 16:03:15 2014 @@ -25,6 +25,7 @@ + Modified: lucene/dev/branches/lucene4236/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml (original) +++ lucene/dev/branches/lucene4236/dev-tools/idea/solr/contrib/analysis-extras/analysis-extras.iml Wed May 21 16:03:15 2014 @@ -45,5 +45,6 @@ + Modified: lucene/dev/branches/lucene4236/lucene/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/CHANGES.txt?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/CHANGES.txt (original) +++ lucene/dev/branches/lucene4236/lucene/CHANGES.txt Wed May 21 16:03:15 2014 @@ -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 Modified: lucene/dev/branches/lucene4236/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (original) +++ lucene/dev/branches/lucene4236/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java Wed May 21 16:03:15 2014 @@ -125,8 +125,7 @@ public class SimpleTextFieldInfosReader } infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, - omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(atts)); - infos[i].setDocValuesGen(dvGen); + omitNorms, storePayloads, indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(atts)); } SimpleTextUtil.checkFooter(input); Modified: lucene/dev/branches/lucene4236/lucene/common-build.xml URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/common-build.xml?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/common-build.xml (original) +++ lucene/dev/branches/lucene4236/lucene/common-build.xml Wed May 21 16:03:15 2014 @@ -2202,7 +2202,7 @@ ${ant.project.name}.test.dependencies=${ - >> 4) & 0x0F)); final Map attributes = input.readStringStringMap(); infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, - omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes)); + omitNorms, storePayloads, indexOptions, docValuesType, normsType, -1, Collections.unmodifiableMap(attributes)); } CodecUtil.checkEOF(input); Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java Wed May 21 16:03:15 2014 @@ -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 e 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 addressInstances = new HashMap<>(); private final Map 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 e 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 + ")"); Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java Wed May 21 16:03:15 2014 @@ -89,8 +89,7 @@ final class Lucene46FieldInfosReader ext final long dvGen = input.readLong(); final Map attributes = input.readStringStringMap(); infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, - omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes)); - infos[i].setDocValuesGen(dvGen); + omitNorms, storePayloads, indexOptions, docValuesType, normsType, dvGen, Collections.unmodifiableMap(attributes)); } if (codecVersion >= Lucene46FieldInfosFormat.FORMAT_CHECKSUM) { Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Wed May 21 16:03:15 2014 @@ -416,9 +416,9 @@ 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.segmentInfo.getDocCount() - flushState.delCountOnFlush)) + " deleted docs"); + infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : flushState.delCountOnFlush) + " deleted docs"); infoStream.message("DWPT", "new segment has " + (flushState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " + (flushState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " + Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Wed May 21 16:03:15 2014 @@ -47,7 +47,7 @@ public final class FieldInfo { private Map attributes; - private long dvGen = -1; // the DocValues generation of this field + private long dvGen; /** * Controls how much information is stored in the postings lists. @@ -121,7 +121,7 @@ public final class FieldInfo { */ public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, - Map attributes) { + long dvGen, Map attributes) { this.name = name; this.indexed = indexed; this.number = number; @@ -139,6 +139,7 @@ public final class FieldInfo { this.indexOptions = null; this.normType = null; } + this.dvGen = dvGen; this.attributes = attributes; assert checkConsistency(); } @@ -158,6 +159,10 @@ public final class FieldInfo { // Cannot store payloads unless positions are indexed: assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !this.storePayloads; } + + if (dvGen != -1) { + assert docValueType != null; + } return true; } @@ -221,8 +226,9 @@ public final class FieldInfo { } /** Sets the docValues generation of this field. */ - public void setDocValuesGen(long dvGen) { + void setDocValuesGen(long dvGen) { this.dvGen = dvGen; + assert checkConsistency(); } /** Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java Wed May 21 16:03:15 2014 @@ -302,7 +302,7 @@ public class FieldInfos implements Itera // before then we'll get the same name and number, // else we'll allocate a new one: final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, docValues); - fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType, null); + fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType, -1, null); assert !byName.containsKey(fi.name); assert globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name, fi.getDocValuesType()); byName.put(fi.name, fi); Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Wed May 21 16:03:15 2014 @@ -2532,7 +2532,7 @@ public class IndexWriter implements Clos } } - 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 Clos 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 segFiles = new HashSet<>(); @@ -3733,7 +3735,7 @@ public class IndexWriter implements Clos 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); Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java Wed May 21 16:03:15 2014 @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.Atomi 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 updates, + Directory dir, DocValuesFormat dvFormat, final SegmentReader reader, Map> fieldFiles) throws IOException { + for (Entry 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() { + 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 iterator() { + updatesIter.reset(); + return new Iterator() { + + 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 updates, + TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader, Map> fieldFiles) throws IOException { + for (Entry 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() { + 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 iterator() { + updatesIter.reset(); + return new Iterator() { + + 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 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> newDVFiles = new HashMap<>(); + Set 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 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() { - 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 iterator() { - updatesIter.reset(); - return new Iterator() { - - 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"); - } - }; - } - }); - } - + handleNumericDVUpdates(fieldInfos, dvUpdates.numericDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles); + // System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " updates=" + dvUpdates.binaryDVUpdates); - for (Entry 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() { - 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 iterator() { - updatesIter.reset(); - return new Iterator() { - - int curDoc = -1; - int updateDoc = updatesIter.nextDoc(); - BytesRef scratch = new BytesRef(); - - @Override - public boolean hasNext() { - return curDoc < maxDoc - 1; - } + handleBinaryDVUpdates(fieldInfos, dvUpdates.binaryDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles); - @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() + "] 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 e : dvUpdates.numericDVUpdates.entrySet()) { @@ -528,22 +563,21 @@ class ReadersAndUpdates { } } - // create a new map, keeping only the gens that are in use - Map> genUpdatesFiles = info.getUpdatesFiles(); - Map> 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> 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(); Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java Wed May 21 16:03:15 2014 @@ -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> dvUpdatesFiles = new HashMap<>(); + + // TODO should we add .files() to FieldInfosFormat, like we have on + // LiveDocsFormat? + // track the fieldInfos update files + private final Set fieldInfosFiles = new HashSet<>(); + // Track the per-generation updates files + @Deprecated private final Map> 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> 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> genUpdatesFiles) { this.genUpdatesFiles.clear(); this.genUpdatesFiles.putAll(genUpdatesFiles); } + /** Returns the per-field DocValues updates files. */ + public Map> getDocValuesUpdatesFiles() { + return Collections.unmodifiableMap(dvUpdatesFiles); + } + + /** Sets the DocValues updates file names, per field number. Does not deep clone the map. */ + public void setDocValuesUpdatesFiles(Map> dvUpdatesFiles) { + this.dvUpdatesFiles.clear(); + this.dvUpdatesFiles.putAll(dvUpdatesFiles); + } + + /** Returns the FieldInfos file names. */ + public Set getFieldInfosFiles() { + return Collections.unmodifiableSet(fieldInfosFiles); + } + + /** Sets the FieldInfos file names. */ + public void setFieldInfosFiles(Set 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 updateFiles : genUpdatesFiles.values()) { files.addAll(updateFiles); } + // must separately add any field updates files + for (Set 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> e : genUpdatesFiles.entrySet()) { other.genUpdatesFiles.put(e.getKey(), new HashSet<>(e.getValue())); } + // deep clone + for (Entry> e : dvUpdatesFiles.entrySet()) { + other.dvUpdatesFiles.put(e.getKey(), new HashSet<>(e.getValue())); + } + + other.fieldInfosFiles.addAll(fieldInfosFiles); + return other; } } Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java Wed May 21 16:03:15 2014 @@ -38,7 +38,7 @@ final class SegmentDocValues { private final Map> genDVProducers = new HashMap<>(); private RefCount newDocValuesProducer(SegmentCommitInfo si, IOContext context, Directory dir, - DocValuesFormat dvFormat, final Long gen, List 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(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 infos) throws IOException { + DocValuesFormat dvFormat, FieldInfos infos) throws IOException { RefCount dvp = genDVProducers.get(gen); if (dvp == null) { dvp = newDocValuesProducer(si, context, dir, dvFormat, gen, infos); Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Wed May 21 16:03:15 2014 @@ -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.NoSuchDir 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. *

* The active segments in the index are stored in the segment info file, - * segments_N. There may be one or more segments_N 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. + * segments_N. There may be one or more segments_N 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. + *

+ *

+ * There is also a file segments.gen. This file contains the current + * generation (the _N in segments_N) 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. *

- *

There is also a file segments.gen. This file contains - * the current generation (the _N in segments_N) 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.

*

* Files: *

    - *
  • segments.gen: GenHeader, Generation, Generation, Footer - *
  • segments_N: Header, Version, NameCounter, SegCount, - * <SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles>SegCount, - * CommitUserData, Footer + *
  • segments.gen: GenHeader, Generation, Generation, Footer + *
  • segments_N: Header, Version, NameCounter, SegCount, <SegName, + * SegCodec, DelGen, DeletionCount, FieldInfosGen, DocValuesGen, + * UpdatesFiles>SegCount, CommitUserData, Footer *
*

* Data types: *

*

    - *
  • Header --> {@link CodecUtil#writeHeader CodecHeader}
  • - *
  • GenHeader, NameCounter, SegCount, DeletionCount --> {@link DataOutput#writeInt Int32}
  • - *
  • Generation, Version, DelGen, Checksum, FieldInfosGen --> {@link DataOutput#writeLong Int64}
  • - *
  • SegName, SegCodec --> {@link DataOutput#writeString String}
  • - *
  • CommitUserData --> {@link DataOutput#writeStringStringMap Map<String,String>}
  • - *
  • UpdatesFiles --> {@link DataOutput#writeStringSet(Set) Set<String>}
  • - *
  • Footer --> {@link CodecUtil#writeFooter CodecFooter}
  • + *
  • Header --> {@link CodecUtil#writeHeader CodecHeader}
  • + *
  • GenHeader, NameCounter, SegCount, DeletionCount --> + * {@link DataOutput#writeInt Int32}
  • + *
  • Generation, Version, DelGen, Checksum, FieldInfosGen, DocValuesGen --> + * {@link DataOutput#writeLong Int64}
  • + *
  • SegName, SegCodec --> {@link DataOutput#writeString String}
  • + *
  • CommitUserData --> {@link DataOutput#writeStringStringMap + * Map<String,String>}
  • + *
  • UpdatesFiles --> Map<{@link DataOutput#writeInt Int32}, + * {@link DataOutput#writeStringSet(Set) Set<String>}>
  • + *
  • Footer --> {@link CodecUtil#writeFooter CodecFooter}
  • *
*

* Field Descriptions: *

*

    - *
  • Version counts how often the index has been changed by adding or deleting - * documents.
  • - *
  • NameCounter is used to generate names for new segment files.
  • - *
  • 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.
  • - *
  • 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}.
  • - *
  • DeletionCount records the number of deleted documents in this segment.
  • - *
  • SegCodec is the {@link Codec#getName() name} of the Codec that encoded - * this segment.
  • - *
  • CommitUserData stores an optional user-supplied opaque - * Map<String,String> that was passed to - * {@link IndexWriter#setCommitData(java.util.Map)}.
  • - *
  • 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}.
  • - *
  • UpdatesFiles stores the list of files that were updated in that segment.
  • + *
  • Version counts how often the index has been changed by adding or deleting + * documents.
  • + *
  • NameCounter is used to generate names for new segment files.
  • + *
  • 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.
  • + *
  • 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}.
  • + *
  • DeletionCount records the number of deleted documents in this segment.
  • + *
  • SegCodec is the {@link Codec#getName() name} of the Codec that encoded + * this segment.
  • + *
  • CommitUserData stores an optional user-supplied opaque + * Map<String,String> that was passed to + * {@link IndexWriter#setCommitData(java.util.Map)}.
  • + *
  • 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} + * .
  • + *
  • 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}.
  • + *
  • UpdatesFiles stores the set of files that were updated in that segment + * per field.
  • *
*

* @@ -121,6 +133,9 @@ public final class SegmentInfos implemen /** 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 implemen 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 implemen 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> genUpdatesFiles; - if (numGensUpdatesFiles == 0) { - genUpdatesFiles = Collections.emptyMap(); + 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> 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 { - genUpdatesFiles = new HashMap<>(numGensUpdatesFiles); - for (int i = 0; i < numGensUpdatesFiles; i++) { - genUpdatesFiles.put(input.readLong(), input.readStringSet()); + siPerCommit.setFieldInfosFiles(input.readStringSet()); + final Map> 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 implemen 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 implemen } segnOutput.writeInt(delCount); segnOutput.writeLong(siPerCommit.getFieldInfosGen()); - final Map> genUpdatesFiles = siPerCommit.getUpdatesFiles(); - segnOutput.writeInt(genUpdatesFiles.size()); - for (Entry> e : genUpdatesFiles.entrySet()) { - segnOutput.writeLong(e.getKey()); + segnOutput.writeLong(siPerCommit.getDocValuesGen()); + segnOutput.writeStringSet(siPerCommit.getFieldInfosFiles()); + final Map> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles(); + segnOutput.writeInt(dvUpdatesFiles.size()); + for (Entry> e : dvUpdatesFiles.entrySet()) { + segnOutput.writeInt(e.getKey()); segnOutput.writeStringSet(e.getValue()); } assert si.dir == directory; Modified: lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Wed May 21 16:03:15 2014 @@ -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 liveDocs = null; } numDocs = si.info.getDocCount() - si.getDelCount(); - + if (fieldInfos.hasDocValues()) { initDocValuesProducers(codec); } @@ -175,24 +175,88 @@ public final class SegmentReader extends } // 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> 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> e : genInfos.entrySet()) { - Long gen = e.getKey(); - List 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> genInfos = new HashMap<>(); + for (FieldInfo fi : fieldInfos) { + if (!fi.hasDocValues()) continue; + List genFieldInfos = genInfos.get(fi.getDocValuesGen()); + if (genFieldInfos == null) { + genFieldInfos = new ArrayList<>(); + genInfos.put(fi.getDocValuesGen(), genFieldInfos); + } + genFieldInfos.add(fi); + } + + for (Map.Entry> e : genInfos.entrySet()) { + long docValuesGen = e.getKey(); + List 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 } } - // returns a gen->List mapping. Fields without DV updates have gen=-1 - private Map> getGenInfos() { - final Map> genInfos = new HashMap<>(); - for (FieldInfo fi : fieldInfos) { - if (fi.getDocValuesType() == null) { - continue; - } - long gen = fi.getDocValuesGen(); - List infos = genInfos.get(gen); - if (infos == null) { - infos = new ArrayList<>(); - genInfos.put(gen, infos); - } - infos.add(fi); - } - return genInfos; - } - @Override public Bits getLiveDocs() { ensureOpen(); Modified: lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java Wed May 21 16:03:15 2014 @@ -1285,27 +1285,23 @@ public class TestBinaryDocValuesUpdates 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(); Modified: lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestDoc.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Wed May 21 16:03:15 2014 @@ -44,7 +44,6 @@ import org.apache.lucene.store.TrackingD 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 LuceneTestC } } - return new SegmentCommitInfo(info, 0, -1L, -1L); + return new SegmentCommitInfo(info, 0, -1L, -1L, -1L); } Modified: lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java Wed May 21 16:03:15 2014 @@ -325,7 +325,7 @@ public class TestIndexWriterThreadsToSeg 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")); Modified: lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java Wed May 21 16:03:15 2014 @@ -1264,27 +1264,24 @@ public class TestNumericDocValuesUpdates 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 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); } Modified: lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (original) +++ lucene/dev/branches/lucene4236/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java Wed May 21 16:03:15 2014 @@ -30,7 +30,6 @@ import org.apache.lucene.util.FixedBitSe 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 L 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); Modified: lucene/dev/branches/lucene4236/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original) +++ lucene/dev/branches/lucene4236/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Wed May 21 16:03:15 2014 @@ -448,7 +448,7 @@ public class MemoryIndex { if (!fieldInfos.containsKey(fieldName)) { fieldInfos.put(fieldName, - new FieldInfo(fieldName, true, fieldInfos.size(), false, false, false, this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS , null, null, null)); + new FieldInfo(fieldName, true, fieldInfos.size(), false, false, false, this.storeOffsets ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS , null, null, -1, null)); } TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class); PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class); Modified: lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (original) +++ lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java Wed May 21 16:03:15 2014 @@ -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 files = infoPerCommit.files(); for (final String srcName : files) { Modified: lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java?rev=1596608&r1=1596607&r2=1596608&view=diff ============================================================================== --- lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java (original) +++ lucene/dev/branches/lucene4236/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java Wed May 21 16:03:15 2014 @@ -215,7 +215,7 @@ public class UninvertingReader extends F } } filteredInfos.add(new FieldInfo(fi.name, fi.isIndexed(), fi.number, fi.hasVectors(), fi.omitsNorms(), - fi.hasPayloads(), fi.getIndexOptions(), type, fi.getNormType(), null)); + fi.hasPayloads(), fi.getIndexOptions(), type, fi.getNormType(), -1, null)); } fieldInfos = new FieldInfos(filteredInfos.toArray(new FieldInfo[filteredInfos.size()])); }