lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From rm...@apache.org
Subject svn commit: r1198811 - in /lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index: MergeState.java SegmentMerger.java codecs/TermsConsumer.java
Date Mon, 07 Nov 2011 16:58:09 GMT
Author: rmuir
Date: Mon Nov  7 16:58:08 2011
New Revision: 1198811

URL: http://svn.apache.org/viewvc?rev=1198811&view=rev
Log:
LUCENE-2621: start building up mergeState in SegmentMerger instead of copying everything to
it in mergeTerms

Modified:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/MergeState.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/MergeState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/MergeState.java?rev=1198811&r1=1198810&r2=1198811&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/MergeState.java
(original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/MergeState.java
Mon Nov  7 16:58:08 2011
@@ -42,7 +42,6 @@ public class MergeState {
 
   public FieldInfos fieldInfos;
   public List<IndexReaderAndLiveDocs> readers;    // Readers & liveDocs being merged
-  public int readerCount;                         // Number of readers being merged
   public int[][] docMaps;                         // Maps docIDs around deletions
   public int[] docBase;                           // New docID base per reader
   public int mergedDocCount;                      // Total # merged docs

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1198811&r1=1198810&r2=1198811&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
(original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
Mon Nov  7 16:58:08 2011
@@ -55,11 +55,6 @@ final class SegmentMerger {
   private final String segment;
   private final int termIndexInterval;
 
-  private final List<MergeState.IndexReaderAndLiveDocs> readers = new ArrayList<MergeState.IndexReaderAndLiveDocs>();
-  private final FieldInfos fieldInfos;
-
-  private final MergeState.CheckAbort checkAbort;
-
   /** Maximum number of contiguous documents to bulk-copy
       when merging stored fields */
   private final static int MAX_RAW_MERGE_DOCS = 4192;
@@ -69,18 +64,20 @@ final class SegmentMerger {
   private final PayloadProcessorProvider payloadProcessorProvider;
   
   private final IOContext context;
-  private final InfoStream infoStream;
+  
+  private final MergeState mergeState = new MergeState();
 
   SegmentMerger(InfoStream infoStream, Directory dir, int termIndexInterval, String name,
MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos
fieldInfos, Codec codec, IOContext context) {
-    this.infoStream = infoStream;
+    mergeState.infoStream = infoStream;
+    mergeState.readers = new ArrayList<MergeState.IndexReaderAndLiveDocs>();
+    mergeState.fieldInfos = fieldInfos;
     this.payloadProcessorProvider = payloadProcessorProvider;
     directory = dir;
     segment = name;
-    this.fieldInfos = fieldInfos;
     if (merge != null) {
-      checkAbort = new MergeState.CheckAbort(merge, directory);
+      mergeState.checkAbort = new MergeState.CheckAbort(merge, directory);
     } else {
-      checkAbort = new MergeState.CheckAbort(null, null) {
+      mergeState.checkAbort = new MergeState.CheckAbort(null, null) {
         @Override
         public void work(double units) throws MergeAbortedException {
           // do nothing
@@ -101,7 +98,7 @@ final class SegmentMerger {
       new ReaderUtil.Gather(reader) {
         @Override
         protected void add(int base, IndexReader r) {
-          readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs()));
+          mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs()));
         }
       }.run();
     } catch (IOException ioe) {
@@ -111,7 +108,7 @@ final class SegmentMerger {
   }
 
   final void add(SegmentReader reader, Bits liveDocs) {
-    readers.add(new MergeState.IndexReaderAndLiveDocs(reader, liveDocs));
+    mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(reader, liveDocs));
   }
 
   /**
@@ -127,19 +124,29 @@ final class SegmentMerger {
     // of this check impacts how long
     // IndexWriter.close(false) takes to actually stop the
     // threads.
+    
+    final int numReaders = mergeState.readers.size();
+    // Remap docIDs
+    mergeState.docMaps = new int[numReaders][];
+    mergeState.docBase = new int[numReaders];
+    mergeState.hasPayloadProcessorProvider = payloadProcessorProvider != null;
+    mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[numReaders];
+    mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
+
     mergeFieldInfos();
     setMatchingSegmentReaders();
-    final int mergedDocs = mergeFields();
-    final SegmentWriteState segmentWriteState = new SegmentWriteState(infoStream, directory,
segment, fieldInfos, mergedDocs, termIndexInterval, codec, null, context);
+    mergeState.mergedDocCount = mergeFields();
+
+    final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream,
directory, segment, mergeState.fieldInfos, mergeState.mergedDocCount, termIndexInterval, codec,
null, context);
     mergeTerms(segmentWriteState);
     mergePerDoc(segmentWriteState);
     mergeNorms();
 
-    if (fieldInfos.hasVectors()) {
+    if (mergeState.fieldInfos.hasVectors()) {
       mergeVectors(segmentWriteState);
     }
     // write FIS once merge is done. IDV might change types or drops fields
-    fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
+    mergeState.fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
     return mergeState;
   }
 
@@ -162,7 +169,7 @@ final class SegmentMerger {
         assert !IndexFileNames.isSeparateNormsFile(file) 
                   : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
         directory.copy(cfsDir, file, file, context);
-        checkAbort.work(directory.fileLength(file));
+        mergeState.checkAbort.work(directory.fileLength(file));
       }
     } finally {
       cfsDir.close();
@@ -192,7 +199,7 @@ final class SegmentMerger {
     // If the i'th reader is a SegmentReader and has
     // identical fieldName -> number mapping, then this
     // array will be non-null at position i:
-    int numReaders = readers.size();
+    int numReaders = mergeState.readers.size();
     matchingSegmentReaders = new SegmentReader[numReaders];
 
     // If this reader is a SegmentReader, and all of its
@@ -200,13 +207,13 @@ final class SegmentMerger {
     // FieldInfos, then we can do a bulk copy of the
     // stored fields:
     for (int i = 0; i < numReaders; i++) {
-      MergeState.IndexReaderAndLiveDocs reader = readers.get(i);
+      MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(i);
       if (reader.reader instanceof SegmentReader) {
         SegmentReader segmentReader = (SegmentReader) reader.reader;
         boolean same = true;
         FieldInfos segmentFieldInfos = segmentReader.fieldInfos();
         for (FieldInfo fi : segmentFieldInfos) {
-          same = fieldInfos.fieldName(fi.number).equals(fi.name);
+          same = mergeState.fieldInfos.fieldName(fi.number).equals(fi.name);
         }
         if (same) {
           matchingSegmentReaders[i] = segmentReader;
@@ -218,34 +225,34 @@ final class SegmentMerger {
     // Used for bulk-reading raw bytes for stored fields
     rawDocLengths = new int[MAX_RAW_MERGE_DOCS];
     rawDocLengths2 = new int[MAX_RAW_MERGE_DOCS];
-    if (infoStream != null) {
-      infoStream.message("SM", "merge store matchedCount=" + matchedCount + " vs " + readers.size());
-      if (matchedCount != readers.size()) {
-        infoStream.message("SM", "" + (readers.size() - matchedCount) + " non-bulk merges");
+    if (mergeState.infoStream != null) {
+      mergeState.infoStream.message("SM", "merge store matchedCount=" + matchedCount + "
vs " + mergeState.readers.size());
+      if (matchedCount != mergeState.readers.size()) {
+        mergeState.infoStream.message("SM", "" + (mergeState.readers.size() - matchedCount)
+ " non-bulk merges");
       }
     }
   }
 
   private void mergeFieldInfos() throws IOException {
-    for (MergeState.IndexReaderAndLiveDocs readerAndLiveDocs : readers) {
+    for (MergeState.IndexReaderAndLiveDocs readerAndLiveDocs : mergeState.readers) {
       final IndexReader reader = readerAndLiveDocs.reader;
       if (reader instanceof SegmentReader) {
         SegmentReader segmentReader = (SegmentReader) reader;
         FieldInfos readerFieldInfos = segmentReader.fieldInfos();
         for (FieldInfo fi : readerFieldInfos) {
-          fieldInfos.add(fi);
+          mergeState.fieldInfos.add(fi);
         }
       } else {
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET),
true, true, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION),
true, true, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_OFFSET),
true, false, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR), true,
false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_POSITIONS),
false, false, false, false, IndexOptions.DOCS_AND_FREQS);
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS),
false, false, false, false, IndexOptions.DOCS_ONLY);
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS),
false, false, false, true, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-        addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false,
false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-        fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
-        fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.DOC_VALUES), false);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET),
true, true, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION),
true, true, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_OFFSET),
true, false, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR),
true, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.OMIT_POSITIONS),
false, false, false, false, IndexOptions.DOCS_AND_FREQS);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS),
false, false, false, false, IndexOptions.DOCS_ONLY);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS),
false, false, false, true, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+        addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.INDEXED),
false, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+        mergeState.fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
+        mergeState.fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.DOC_VALUES), false);
       }
     }
   }
@@ -262,7 +269,7 @@ final class SegmentMerger {
     final FieldsWriter fieldsWriter = codec.fieldsFormat().fieldsWriter(directory, segment,
context);
     try {
       int idx = 0;
-      for (MergeState.IndexReaderAndLiveDocs reader : readers) {
+      for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
         final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
         FieldsReader matchingFieldsReader = null;
         if (matchingSegmentReader != null) {
@@ -318,7 +325,7 @@ final class SegmentMerger {
         IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, start, numDocs);
         fieldsWriter.addRawDocuments(stream, rawDocLengths, numDocs);
         docCount += numDocs;
-        checkAbort.work(300 * numDocs);
+        mergeState.checkAbort.work(300 * numDocs);
       }
     } else {
       for (int j = 0; j < maxDoc; j++) {
@@ -333,9 +340,9 @@ final class SegmentMerger {
         // NOTE: it's very important to first assign to doc then pass it to
         // fieldsWriter.addDocument; see LUCENE-1282
         Document doc = reader.reader.document(j);
-        fieldsWriter.addDocument(doc, fieldInfos);
+        fieldsWriter.addDocument(doc, mergeState.fieldInfos);
         docCount++;
-        checkAbort.work(300);
+        mergeState.checkAbort.work(300);
       }
     }
     return docCount;
@@ -353,15 +360,15 @@ final class SegmentMerger {
         IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, docCount, len);
         fieldsWriter.addRawDocuments(stream, rawDocLengths, len);
         docCount += len;
-        checkAbort.work(300 * len);
+        mergeState.checkAbort.work(300 * len);
       }
     } else {
       for (; docCount < maxDoc; docCount++) {
         // NOTE: it's very important to first assign to doc then pass it to
         // fieldsWriter.addDocument; see LUCENE-1282
         Document doc = reader.reader.document(docCount);
-        fieldsWriter.addDocument(doc, fieldInfos);
-        checkAbort.work(300);
+        fieldsWriter.addDocument(doc, mergeState.fieldInfos);
+        mergeState.checkAbort.work(300);
       }
     }
     return docCount;
@@ -372,11 +379,11 @@ final class SegmentMerger {
    * @throws IOException
    */
   private final void mergeVectors(SegmentWriteState segmentWriteState) throws IOException
{
-    TermVectorsWriter termVectorsWriter = new TermVectorsWriter(directory, segment, fieldInfos,
context);
+    TermVectorsWriter termVectorsWriter = new TermVectorsWriter(directory, segment, mergeState.fieldInfos,
context);
 
     try {
       int idx = 0;
-      for (final MergeState.IndexReaderAndLiveDocs reader : readers) {
+      for (final MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
         final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
         TermVectorsReader matchingVectorsReader = null;
         if (matchingSegmentReader != null) {
@@ -439,7 +446,7 @@ final class SegmentMerger {
 
         matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
         termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2,
numDocs);
-        checkAbort.work(300 * numDocs);
+        mergeState.checkAbort.work(300 * numDocs);
       }
     } else {
       for (int docNum = 0; docNum < maxDoc; docNum++) {
@@ -452,7 +459,7 @@ final class SegmentMerger {
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         TermFreqVector[] vectors = reader.reader.getTermFreqVectors(docNum);
         termVectorsWriter.addAllDocVectors(vectors);
-        checkAbort.work(300);
+        mergeState.checkAbort.work(300);
       }
     }
   }
@@ -470,7 +477,7 @@ final class SegmentMerger {
         matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, docCount, len);
         termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2,
len);
         docCount += len;
-        checkAbort.work(300 * len);
+        mergeState.checkAbort.work(300 * len);
       }
     } else {
       for (int docNum = 0; docNum < maxDoc; docNum++) {
@@ -478,7 +485,7 @@ final class SegmentMerger {
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         TermFreqVector[] vectors = reader.reader.getTermFreqVectors(docNum);
         termVectorsWriter.addAllDocVectors(vectors);
-        checkAbort.work(300);
+        mergeState.checkAbort.work(300);
       }
     }
   }
@@ -489,7 +496,7 @@ final class SegmentMerger {
     final List<Fields> fields = new ArrayList<Fields>();
     final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
 
-    for(MergeState.IndexReaderAndLiveDocs r : readers) {
+    for(MergeState.IndexReaderAndLiveDocs r : mergeState.readers) {
       final Fields f = r.reader.fields();
       final int maxDoc = r.reader.maxDoc();
       if (f != null) {
@@ -499,27 +506,13 @@ final class SegmentMerger {
       docBase += maxDoc;
     }
 
-    // we may gather more readers than mergeState.readerCount
-    mergeState = new MergeState();
-    mergeState.infoStream = infoStream;
-    mergeState.readers = readers;
-    mergeState.readerCount = readers.size();
-    mergeState.fieldInfos = fieldInfos;
-    mergeState.mergedDocCount = segmentWriteState.numDocs;
-
-    // Remap docIDs
-    mergeState.docMaps = new int[mergeState.readerCount][];
-    mergeState.docBase = new int[mergeState.readerCount];
-    mergeState.hasPayloadProcessorProvider = payloadProcessorProvider != null;
-    mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[mergeState.readerCount];
-    mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[mergeState.readerCount];
-    mergeState.checkAbort = checkAbort;
+    final int numReaders = mergeState.readers.size();
 
     docBase = 0;
 
-    for(int i=0;i<mergeState.readerCount;i++) {
+    for(int i=0;i<numReaders;i++) {
 
-      final MergeState.IndexReaderAndLiveDocs reader = readers.get(i);
+      final MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(i);
 
       mergeState.docBase[i] = docBase;
       final int maxDoc = reader.reader.maxDoc();
@@ -583,20 +576,18 @@ final class SegmentMerger {
         }
       }
   }
-  
-  private MergeState mergeState;
 
   private void mergeNorms() throws IOException {
     IndexOutput output = null;
     boolean success = false;
     try {
-      for (FieldInfo fi : fieldInfos) {
+      for (FieldInfo fi : mergeState.fieldInfos) {
         if (fi.isIndexed && !fi.omitNorms) {
           if (output == null) {
             output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION),
context);
             output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
           }
-          for (MergeState.IndexReaderAndLiveDocs reader : readers) {
+          for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
             final int maxDoc = reader.reader.maxDoc();
             byte normBuffer[] = reader.reader.norms(fi.name);
             if (normBuffer == null) {
@@ -618,7 +609,7 @@ final class SegmentMerger {
                 }
               }
             }
-            checkAbort.work(maxDoc);
+            mergeState.checkAbort.work(maxDoc);
           }
         }
       }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java?rev=1198811&r1=1198810&r2=1198811&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java
(original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java
Mon Nov  7 16:58:08 2011
@@ -105,7 +105,7 @@ public abstract class TermsConsumer {
           postingsEnum.reset(postingsEnumIn);
           // set PayloadProcessor
           if (mergeState.hasPayloadProcessorProvider) {
-            for (int i = 0; i < mergeState.readerCount; i++) {
+            for (int i = 0; i < mergeState.readers.size(); i++) {
               if (mergeState.dirPayloadProcessor[i] != null) {
                 mergeState.currentPayloadProcessor[i] = mergeState.dirPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name,
term);
               }



Mime
View raw message