pylucene 3.5.0-3
[pylucene.git] / lucene-java-3.5.0 / lucene / src / java / org / apache / lucene / index / SegmentMerger.java
1 package org.apache.lucene.index;
2
3 /**
4  * Licensed to the Apache Software Foundation (ASF) under one or more
5  * contributor license agreements.  See the NOTICE file distributed with
6  * this work for additional information regarding copyright ownership.
7  * The ASF licenses this file to You under the Apache License, Version 2.0
8  * (the "License"); you may not use this file except in compliance with
9  * the License.  You may obtain a copy of the License at
10  *
11  *     http://www.apache.org/licenses/LICENSE-2.0
12  *
13  * Unless required by applicable law or agreed to in writing, software
14  * distributed under the License is distributed on an "AS IS" BASIS,
15  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16  * See the License for the specific language governing permissions and
17  * limitations under the License.
18  */
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.Collection;
23 import java.util.List;
24
25 import org.apache.lucene.document.Document;
26 import org.apache.lucene.index.FieldInfo.IndexOptions;
27 import org.apache.lucene.index.IndexReader.FieldOption;
28 import org.apache.lucene.index.MergePolicy.MergeAbortedException;
29 import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
30 import org.apache.lucene.store.Directory;
31 import org.apache.lucene.store.IndexInput;
32 import org.apache.lucene.store.IndexOutput;
33 import org.apache.lucene.util.IOUtils;
34 import org.apache.lucene.util.ReaderUtil;
35
36 /**
37  * The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
38  * into a single Segment.  After adding the appropriate readers, call the merge method to combine the 
39  * segments.
40  * 
41  * @see #merge
42  * @see #add
43  */
44 final class SegmentMerger {
45   private Directory directory;
46   private String segment;
47   private int termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
48
49   private List<IndexReader> readers = new ArrayList<IndexReader>();
50   private final FieldInfos fieldInfos;
51   
52   private int mergedDocs;
53
54   private final CheckAbort checkAbort;
55
56   /** Maximum number of contiguous documents to bulk-copy
57       when merging stored fields */
58   private final static int MAX_RAW_MERGE_DOCS = 4192;
59
60   private SegmentWriteState segmentWriteState;
61
62   private final PayloadProcessorProvider payloadProcessorProvider;
63   
64   SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
65     this.payloadProcessorProvider = payloadProcessorProvider;
66     directory = dir;
67     this.fieldInfos = fieldInfos;
68     segment = name;
69     if (merge != null) {
70       checkAbort = new CheckAbort(merge, directory);
71     } else {
72       checkAbort = new CheckAbort(null, null) {
73         @Override
74         public void work(double units) throws MergeAbortedException {
75           // do nothing
76         }
77       };
78     }
79     this.termIndexInterval = termIndexInterval;
80   }
81
82   public FieldInfos fieldInfos() {
83     return fieldInfos;
84   }
85
86   /**
87    * Add an IndexReader to the collection of readers that are to be merged
88    * @param reader
89    */
90   final void add(IndexReader reader) {
91     ReaderUtil.gatherSubReaders(readers, reader);
92   }
93
94   /**
95    * Merges the readers specified by the {@link #add} method into the directory passed to the constructor
96    * @return The number of documents that were merged
97    * @throws CorruptIndexException if the index is corrupt
98    * @throws IOException if there is a low-level IO error
99    */
100   final int merge() throws CorruptIndexException, IOException {
101     // NOTE: it's important to add calls to
102     // checkAbort.work(...) if you make any changes to this
103     // method that will spend alot of time.  The frequency
104     // of this check impacts how long
105     // IndexWriter.close(false) takes to actually stop the
106     // threads.
107
108     mergedDocs = mergeFields();
109     mergeTerms();
110     mergeNorms();
111
112     if (fieldInfos.hasVectors())
113       mergeVectors();
114
115     return mergedDocs;
116   }
117
118   /**
119    * NOTE: this method creates a compound file for all files returned by
120    * info.files(). While, generally, this may include separate norms and
121    * deletion files, this SegmentInfo must not reference such files when this
122    * method is called, because they are not allowed within a compound file.
123    */
124   final Collection<String> createCompoundFile(String fileName, final SegmentInfo info)
125           throws IOException {
126     // Now merge all added files
127     Collection<String> files = info.files();
128     CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, checkAbort);
129     for (String file : files) {
130       assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION) 
131                 : ".del file is not allowed in .cfs: " + file;
132       assert !IndexFileNames.isSeparateNormsFile(file)
133                 : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
134       cfsWriter.addFile(file);
135     }
136     
137     // Perform the merge
138     cfsWriter.close();
139    
140     return files;
141   }
142
143   private static void addIndexed(IndexReader reader, FieldInfos fInfos,
144       Collection<String> names, boolean storeTermVectors,
145       boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
146       boolean storePayloads, IndexOptions indexOptions)
147       throws IOException {
148     for (String field : names) {
149       fInfos.add(field, true, storeTermVectors,
150           storePositionWithTermVector, storeOffsetWithTermVector, !reader
151               .hasNorms(field), storePayloads, indexOptions);
152     }
153   }
154
155   private SegmentReader[] matchingSegmentReaders;
156   private int[] rawDocLengths;
157   private int[] rawDocLengths2;
158   private int matchedCount;
159
160   public int getMatchedSubReaderCount() {
161     return matchedCount;
162   }
163
164   private void setMatchingSegmentReaders() {
165     // If the i'th reader is a SegmentReader and has
166     // identical fieldName -> number mapping, then this
167     // array will be non-null at position i:
168     int numReaders = readers.size();
169     matchingSegmentReaders = new SegmentReader[numReaders];
170
171     // If this reader is a SegmentReader, and all of its
172     // field name -> number mappings match the "merged"
173     // FieldInfos, then we can do a bulk copy of the
174     // stored fields:
175     for (int i = 0; i < numReaders; i++) {
176       IndexReader reader = readers.get(i);
177       if (reader instanceof SegmentReader) {
178         SegmentReader segmentReader = (SegmentReader) reader;
179         boolean same = true;
180         FieldInfos segmentFieldInfos = segmentReader.fieldInfos();
181         int numFieldInfos = segmentFieldInfos.size();
182         for (int j = 0; j < numFieldInfos; j++) {
183           if (!fieldInfos.fieldName(j).equals(segmentFieldInfos.fieldName(j))) {
184             same = false;
185             break;
186           }
187         }
188         if (same) {
189           matchingSegmentReaders[i] = segmentReader;
190           matchedCount++;
191         }
192       }
193     }
194
195     // Used for bulk-reading raw bytes for stored fields
196     rawDocLengths = new int[MAX_RAW_MERGE_DOCS];
197     rawDocLengths2 = new int[MAX_RAW_MERGE_DOCS];
198   }
199
200   /**
201    * 
202    * @return The number of documents in all of the readers
203    * @throws CorruptIndexException if the index is corrupt
204    * @throws IOException if there is a low-level IO error
205    */
206   private int mergeFields() throws CorruptIndexException, IOException {
207
208     for (IndexReader reader : readers) {
209       if (reader instanceof SegmentReader) {
210         SegmentReader segmentReader = (SegmentReader) reader;
211         FieldInfos readerFieldInfos = segmentReader.fieldInfos();
212         int numReaderFieldInfos = readerFieldInfos.size();
213         for (int j = 0; j < numReaderFieldInfos; j++) {
214           fieldInfos.add(readerFieldInfos.fieldInfo(j));
215         }
216       } else {
217         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
218         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION), true, true, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
219         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_OFFSET), true, false, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
220         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR), true, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
221         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_POSITIONS), false, false, false, false, IndexOptions.DOCS_AND_FREQS);
222         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS), false, false, false, false, IndexOptions.DOCS_ONLY);
223         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
224         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
225         fieldInfos.add(reader.getFieldNames(FieldOption.UNINDEXED), false);
226       }
227     }
228     fieldInfos.write(directory, segment + ".fnm");
229
230     int docCount = 0;
231
232     setMatchingSegmentReaders();
233
234     final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
235
236     try {
237       int idx = 0;
238       for (IndexReader reader : readers) {
239         final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
240         FieldsReader matchingFieldsReader = null;
241         if (matchingSegmentReader != null) {
242           final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
243           if (fieldsReader != null && fieldsReader.canReadRawDocs()) {
244             matchingFieldsReader = fieldsReader;
245           }
246         }
247         if (reader.hasDeletions()) {
248           docCount += copyFieldsWithDeletions(fieldsWriter,
249                                               reader, matchingFieldsReader);
250         } else {
251           docCount += copyFieldsNoDeletions(fieldsWriter,
252                                             reader, matchingFieldsReader);
253         }
254       }
255     } finally {
256       fieldsWriter.close();
257     }
258
259     final String fileName = IndexFileNames.segmentFileName(segment, IndexFileNames.FIELDS_INDEX_EXTENSION);
260     final long fdxFileLength = directory.fileLength(fileName);
261
262     if (4+((long) docCount)*8 != fdxFileLength)
263       // This is most likely a bug in Sun JRE 1.6.0_04/_05;
264       // we detect that the bug has struck, here, and
265       // throw an exception to prevent the corruption from
266       // entering the index.  See LUCENE-1282 for
267       // details.
268       throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
269
270     segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, null);
271     return docCount;
272   }
273
274   private int copyFieldsWithDeletions(final FieldsWriter fieldsWriter, final IndexReader reader,
275                                       final FieldsReader matchingFieldsReader)
276     throws IOException, MergeAbortedException, CorruptIndexException {
277     int docCount = 0;
278     final int maxDoc = reader.maxDoc();
279     if (matchingFieldsReader != null) {
280       // We can bulk-copy because the fieldInfos are "congruent"
281       for (int j = 0; j < maxDoc;) {
282         if (reader.isDeleted(j)) {
283           // skip deleted docs
284           ++j;
285           continue;
286         }
287         // We can optimize this case (doing a bulk byte copy) since the field 
288         // numbers are identical
289         int start = j, numDocs = 0;
290         do {
291           j++;
292           numDocs++;
293           if (j >= maxDoc) break;
294           if (reader.isDeleted(j)) {
295             j++;
296             break;
297           }
298         } while(numDocs < MAX_RAW_MERGE_DOCS);
299         
300         IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, start, numDocs);
301         fieldsWriter.addRawDocuments(stream, rawDocLengths, numDocs);
302         docCount += numDocs;
303         checkAbort.work(300 * numDocs);
304       }
305     } else {
306       for (int j = 0; j < maxDoc; j++) {
307         if (reader.isDeleted(j)) {
308           // skip deleted docs
309           continue;
310         }
311         // NOTE: it's very important to first assign to doc then pass it to
312         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
313         Document doc = reader.document(j);
314         fieldsWriter.addDocument(doc);
315         docCount++;
316         checkAbort.work(300);
317       }
318     }
319     return docCount;
320   }
321
322   private int copyFieldsNoDeletions(final FieldsWriter fieldsWriter, final IndexReader reader,
323                                     final FieldsReader matchingFieldsReader)
324     throws IOException, MergeAbortedException, CorruptIndexException {
325     final int maxDoc = reader.maxDoc();
326     int docCount = 0;
327     if (matchingFieldsReader != null) {
328       // We can bulk-copy because the fieldInfos are "congruent"
329       while (docCount < maxDoc) {
330         int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
331         IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, docCount, len);
332         fieldsWriter.addRawDocuments(stream, rawDocLengths, len);
333         docCount += len;
334         checkAbort.work(300 * len);
335       }
336     } else {
337       for (; docCount < maxDoc; docCount++) {
338         // NOTE: it's very important to first assign to doc then pass it to
339         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
340         Document doc = reader.document(docCount);
341         fieldsWriter.addDocument(doc);
342         checkAbort.work(300);
343       }
344     }
345     return docCount;
346   }
347
348   /**
349    * Merge the TermVectors from each of the segments into the new one.
350    * @throws IOException
351    */
352   private final void mergeVectors() throws IOException {
353     TermVectorsWriter termVectorsWriter = 
354       new TermVectorsWriter(directory, segment, fieldInfos);
355
356     try {
357       int idx = 0;
358       for (final IndexReader reader : readers) {
359         final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
360         TermVectorsReader matchingVectorsReader = null;
361         if (matchingSegmentReader != null) {
362           TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();
363
364           // If the TV* files are an older format then they cannot read raw docs:
365           if (vectorsReader != null && vectorsReader.canReadRawDocs()) {
366             matchingVectorsReader = vectorsReader;
367           }
368         }
369         if (reader.hasDeletions()) {
370           copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader);
371         } else {
372           copyVectorsNoDeletions(termVectorsWriter, matchingVectorsReader, reader);
373           
374         }
375       }
376     } finally {
377       termVectorsWriter.close();
378     }
379
380     final String fileName = IndexFileNames.segmentFileName(segment, IndexFileNames.VECTORS_INDEX_EXTENSION);
381     final long tvxSize = directory.fileLength(fileName);
382
383     if (4+((long) mergedDocs)*16 != tvxSize)
384       // This is most likely a bug in Sun JRE 1.6.0_04/_05;
385       // we detect that the bug has struck, here, and
386       // throw an exception to prevent the corruption from
387       // entering the index.  See LUCENE-1282 for
388       // details.
389       throw new RuntimeException("mergeVectors produced an invalid result: mergedDocs is " + mergedDocs + " but tvx size is " + tvxSize + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
390   }
391
392   private void copyVectorsWithDeletions(final TermVectorsWriter termVectorsWriter,
393                                         final TermVectorsReader matchingVectorsReader,
394                                         final IndexReader reader)
395     throws IOException, MergeAbortedException {
396     final int maxDoc = reader.maxDoc();
397     if (matchingVectorsReader != null) {
398       // We can bulk-copy because the fieldInfos are "congruent"
399       for (int docNum = 0; docNum < maxDoc;) {
400         if (reader.isDeleted(docNum)) {
401           // skip deleted docs
402           ++docNum;
403           continue;
404         }
405         // We can optimize this case (doing a bulk byte copy) since the field 
406         // numbers are identical
407         int start = docNum, numDocs = 0;
408         do {
409           docNum++;
410           numDocs++;
411           if (docNum >= maxDoc) break;
412           if (reader.isDeleted(docNum)) {
413             docNum++;
414             break;
415           }
416         } while(numDocs < MAX_RAW_MERGE_DOCS);
417         
418         matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
419         termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
420         checkAbort.work(300 * numDocs);
421       }
422     } else {
423       for (int docNum = 0; docNum < maxDoc; docNum++) {
424         if (reader.isDeleted(docNum)) {
425           // skip deleted docs
426           continue;
427         }
428         
429         // NOTE: it's very important to first assign to vectors then pass it to
430         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
431         TermFreqVector[] vectors = reader.getTermFreqVectors(docNum);
432         termVectorsWriter.addAllDocVectors(vectors);
433         checkAbort.work(300);
434       }
435     }
436   }
437   
438   private void copyVectorsNoDeletions(final TermVectorsWriter termVectorsWriter,
439                                       final TermVectorsReader matchingVectorsReader,
440                                       final IndexReader reader)
441       throws IOException, MergeAbortedException {
442     final int maxDoc = reader.maxDoc();
443     if (matchingVectorsReader != null) {
444       // We can bulk-copy because the fieldInfos are "congruent"
445       int docCount = 0;
446       while (docCount < maxDoc) {
447         int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
448         matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, docCount, len);
449         termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, len);
450         docCount += len;
451         checkAbort.work(300 * len);
452       }
453     } else {
454       for (int docNum = 0; docNum < maxDoc; docNum++) {
455         // NOTE: it's very important to first assign to vectors then pass it to
456         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
457         TermFreqVector[] vectors = reader.getTermFreqVectors(docNum);
458         termVectorsWriter.addAllDocVectors(vectors);
459         checkAbort.work(300);
460       }
461     }
462   }
463
464   private SegmentMergeQueue queue = null;
465
466   private final void mergeTerms() throws CorruptIndexException, IOException {
467
468     final FormatPostingsFieldsConsumer fieldsConsumer = new FormatPostingsFieldsWriter(segmentWriteState, fieldInfos);
469
470     try {
471       queue = new SegmentMergeQueue(readers.size());
472
473       mergeTermInfos(fieldsConsumer);
474
475     } finally {
476       try {
477         fieldsConsumer.finish();
478       } finally {
479         if (queue != null) {
480           queue.close();
481         }
482       }
483     }
484   }
485
486   IndexOptions indexOptions;
487
488   private final void mergeTermInfos(final FormatPostingsFieldsConsumer consumer) throws CorruptIndexException, IOException {
489     int base = 0;
490     final int readerCount = readers.size();
491     for (int i = 0; i < readerCount; i++) {
492       IndexReader reader = readers.get(i);
493       TermEnum termEnum = reader.terms();
494       SegmentMergeInfo smi = new SegmentMergeInfo(base, termEnum, reader);
495       if (payloadProcessorProvider != null) {
496         smi.dirPayloadProcessor = payloadProcessorProvider.getDirProcessor(reader.directory());
497       }
498       int[] docMap  = smi.getDocMap();
499       if (docMap != null) {
500         if (docMaps == null) {
501           docMaps = new int[readerCount][];
502         }
503         docMaps[i] = docMap;
504       }
505       
506       base += reader.numDocs();
507
508       assert reader.numDocs() == reader.maxDoc() - smi.delCount;
509
510       if (smi.next())
511         queue.add(smi);                           // initialize queue
512       else
513         smi.close();
514     }
515
516     SegmentMergeInfo[] match = new SegmentMergeInfo[readers.size()];
517
518     String currentField = null;
519     FormatPostingsTermsConsumer termsConsumer = null;
520
521     while (queue.size() > 0) {
522       int matchSize = 0;                          // pop matching terms
523       match[matchSize++] = queue.pop();
524       Term term = match[0].term;
525       SegmentMergeInfo top = queue.top();
526
527       while (top != null && term.compareTo(top.term) == 0) {
528         match[matchSize++] =  queue.pop();
529         top =  queue.top();
530       }
531
532       if (currentField != term.field) {
533         currentField = term.field;
534         if (termsConsumer != null)
535           termsConsumer.finish();
536         final FieldInfo fieldInfo = fieldInfos.fieldInfo(currentField);
537         termsConsumer = consumer.addField(fieldInfo);
538         indexOptions = fieldInfo.indexOptions;
539       }
540
541       int df = appendPostings(termsConsumer, match, matchSize);           // add new TermInfo
542       checkAbort.work(df/3.0);
543
544       while (matchSize > 0) {
545         SegmentMergeInfo smi = match[--matchSize];
546         if (smi.next())
547           queue.add(smi);                         // restore queue
548         else
549           smi.close();                            // done with a segment
550       }
551     }
552   }
553
554   private byte[] payloadBuffer;
555   private int[][] docMaps;
556
557   /** Process postings from multiple segments all positioned on the
558    *  same term. Writes out merged entries into freqOutput and
559    *  the proxOutput streams.
560    *
561    * @param smis array of segments
562    * @param n number of cells in the array actually occupied
563    * @return number of documents across all segments where this term was found
564    * @throws CorruptIndexException if the index is corrupt
565    * @throws IOException if there is a low-level IO error
566    */
567   private final int appendPostings(final FormatPostingsTermsConsumer termsConsumer, SegmentMergeInfo[] smis, int n)
568         throws CorruptIndexException, IOException {
569
570     final FormatPostingsDocsConsumer docConsumer = termsConsumer.addTerm(smis[0].term.text);
571     int df = 0;
572     for (int i = 0; i < n; i++) {
573       SegmentMergeInfo smi = smis[i];
574       TermPositions postings = smi.getPositions();
575       assert postings != null;
576       int base = smi.base;
577       int[] docMap = smi.getDocMap();
578       postings.seek(smi.termEnum);
579
580       PayloadProcessor payloadProcessor = null;
581       if (smi.dirPayloadProcessor != null) {
582         payloadProcessor = smi.dirPayloadProcessor.getProcessor(smi.term);
583       }
584
585       while (postings.next()) {
586         df++;
587         int doc = postings.doc();
588         if (docMap != null)
589           doc = docMap[doc];                      // map around deletions
590         doc += base;                              // convert to merged space
591
592         final int freq = postings.freq();
593         final FormatPostingsPositionsConsumer posConsumer = docConsumer.addDoc(doc, freq);
594
595         if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
596           for (int j = 0; j < freq; j++) {
597             final int position = postings.nextPosition();
598             int payloadLength = postings.getPayloadLength();
599             if (payloadLength > 0) {
600               if (payloadBuffer == null || payloadBuffer.length < payloadLength)
601                 payloadBuffer = new byte[payloadLength];
602               postings.getPayload(payloadBuffer, 0);
603               if (payloadProcessor != null) {
604                 payloadBuffer = payloadProcessor.processPayload(payloadBuffer, 0, payloadLength);
605                 payloadLength = payloadProcessor.payloadLength();
606               }
607             }
608             posConsumer.addPosition(position, payloadBuffer, 0, payloadLength);
609           }
610           posConsumer.finish();
611         }
612       }
613     }
614     docConsumer.finish();
615
616     return df;
617   }
618
619   public boolean getAnyNonBulkMerges() {
620     assert matchedCount <= readers.size();
621     return matchedCount != readers.size();
622   }
623
624   private void mergeNorms() throws IOException {
625     // get needed buffer size by finding the largest segment
626     int bufferSize = 0;
627     for (IndexReader reader : readers) {
628       bufferSize = Math.max(bufferSize, reader.maxDoc());
629     }
630     
631     byte[] normBuffer = null;
632     IndexOutput output = null;
633     boolean success = false;
634     try {
635       int numFieldInfos = fieldInfos.size();
636       for (int i = 0; i < numFieldInfos; i++) {
637         FieldInfo fi = fieldInfos.fieldInfo(i);
638         if (fi.isIndexed && !fi.omitNorms) {
639           if (output == null) { 
640             output = directory.createOutput(IndexFileNames.segmentFileName(segment, IndexFileNames.NORMS_EXTENSION));
641             output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
642           }
643           if (normBuffer == null) {
644             normBuffer = new byte[bufferSize];
645           }
646           for (IndexReader reader : readers) {
647             final int maxDoc = reader.maxDoc();
648             reader.norms(fi.name, normBuffer, 0);
649             if (!reader.hasDeletions()) {
650               //optimized case for segments without deleted docs
651               output.writeBytes(normBuffer, maxDoc);
652             } else {
653               // this segment has deleted docs, so we have to
654               // check for every doc if it is deleted or not
655               for (int k = 0; k < maxDoc; k++) {
656                 if (!reader.isDeleted(k)) {
657                   output.writeByte(normBuffer[k]);
658                 }
659               }
660             }
661             checkAbort.work(maxDoc);
662           }
663         }
664       }
665       success = true;
666     } finally {
667       if (success) {
668         IOUtils.close(output);
669       } else {
670         IOUtils.closeWhileHandlingException(output);
671       }
672     }
673   }
674
675   static class CheckAbort {
676     private double workCount;
677     private MergePolicy.OneMerge merge;
678     private Directory dir;
679     public CheckAbort(MergePolicy.OneMerge merge, Directory dir) {
680       this.merge = merge;
681       this.dir = dir;
682     }
683
684     /**
685      * Records the fact that roughly units amount of work
686      * have been done since this method was last called.
687      * When adding time-consuming code into SegmentMerger,
688      * you should test different values for units to ensure
689      * that the time in between calls to merge.checkAborted
690      * is up to ~ 1 second.
691      */
692     public void work(double units) throws MergePolicy.MergeAbortedException {
693       workCount += units;
694       if (workCount >= 10000.0) {
695         merge.checkAborted(dir);
696         workCount = 0;
697       }
698     }
699   }
700   
701 }