pylucene 3.5.0-3
[pylucene.git] / lucene-java-3.5.0 / lucene / src / java / org / apache / lucene / index / DirectoryReader.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.FileNotFoundException;
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Arrays;
24 import java.util.Collection;
25 import java.util.Collections;
26 import java.util.HashMap;
27 import java.util.HashSet;
28 import java.util.List;
29
30 import java.util.Map;
31 import java.util.Set;
32 import java.util.concurrent.ConcurrentHashMap;
33
34 import org.apache.lucene.document.Document;
35 import org.apache.lucene.document.FieldSelector;
36 import org.apache.lucene.search.Similarity;
37 import org.apache.lucene.store.Directory;
38 import org.apache.lucene.store.Lock;
39 import org.apache.lucene.store.LockObtainFailedException;
40 import org.apache.lucene.util.MapBackedSet;
41
42 /** 
43  * An IndexReader which reads indexes with multiple segments.
44  */
45 class DirectoryReader extends IndexReader implements Cloneable {
46   protected Directory directory;
47   protected boolean readOnly;
48
49   IndexWriter writer;
50
51   private IndexDeletionPolicy deletionPolicy;
52   private Lock writeLock;
53   private final SegmentInfos segmentInfos;
54   private boolean stale;
55   private final int termInfosIndexDivisor;
56
57   private boolean rollbackHasChanges;
58
59   private SegmentReader[] subReaders;
60   private int[] starts;                           // 1st docno for each segment
61   private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
62   private int maxDoc = 0;
63   private int numDocs = -1;
64   private boolean hasDeletions = false;
65
66   // Max version in index as of when we opened; this can be
67   // > our current segmentInfos version in case we were
68   // opened on a past IndexCommit:
69   private long maxIndexVersion;
70
71   private final boolean applyAllDeletes;
72
73   static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly,
74                           final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
75     return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
76       @Override
77       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
78         SegmentInfos infos = new SegmentInfos();
79         infos.read(directory, segmentFileName);
80         if (readOnly)
81           return new ReadOnlyDirectoryReader(directory, infos, deletionPolicy, termInfosIndexDivisor, null);
82         else
83           return new DirectoryReader(directory, infos, deletionPolicy, false, termInfosIndexDivisor, null);
84       }
85     }.run(commit);
86   }
87
88   /** Construct reading the named set of readers. */
89   DirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor,
90                   Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
91     this.directory = directory;
92     this.readOnly = readOnly;
93     this.segmentInfos = sis;
94     this.deletionPolicy = deletionPolicy;
95     this.termInfosIndexDivisor = termInfosIndexDivisor;
96
97     if (readerFinishedListeners == null) {
98       this.readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
99     } else {
100       this.readerFinishedListeners = readerFinishedListeners;
101     }
102     applyAllDeletes = false;
103
104     // To reduce the chance of hitting FileNotFound
105     // (and having to retry), we open segments in
106     // reverse because IndexWriter merges & deletes
107     // the newest segments first.
108
109     SegmentReader[] readers = new SegmentReader[sis.size()];
110     for (int i = sis.size()-1; i >= 0; i--) {
111       boolean success = false;
112       try {
113         readers[i] = SegmentReader.get(readOnly, sis.info(i), termInfosIndexDivisor);
114         readers[i].readerFinishedListeners = this.readerFinishedListeners;
115         success = true;
116       } finally {
117         if (!success) {
118           // Close all readers we had opened:
119           for(i++;i<sis.size();i++) {
120             try {
121               readers[i].close();
122             } catch (Throwable ignore) {
123               // keep going - we want to clean up as much as possible
124             }
125           }
126         }
127       }
128     }
129
130     initialize(readers);
131   }
132
133   // Used by near real-time search
134   DirectoryReader(IndexWriter writer, SegmentInfos infos, int termInfosIndexDivisor, boolean applyAllDeletes) throws IOException {
135     this.directory = writer.getDirectory();
136     this.readOnly = true;
137     this.applyAllDeletes = applyAllDeletes;       // saved for reopen
138
139     this.termInfosIndexDivisor = termInfosIndexDivisor;
140     readerFinishedListeners = writer.getReaderFinishedListeners();
141
142     // IndexWriter synchronizes externally before calling
143     // us, which ensures infos will not change; so there's
144     // no need to process segments in reverse order
145     final int numSegments = infos.size();
146
147     List<SegmentReader> readers = new ArrayList<SegmentReader>();
148     final Directory dir = writer.getDirectory();
149
150     segmentInfos = (SegmentInfos) infos.clone();
151     int infosUpto = 0;
152     for (int i=0;i<numSegments;i++) {
153       boolean success = false;
154       try {
155         final SegmentInfo info = infos.info(i);
156         assert info.dir == dir;
157         final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, true, termInfosIndexDivisor);
158         if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
159           reader.readerFinishedListeners = readerFinishedListeners;
160           readers.add(reader);
161           infosUpto++;
162         } else {
163           reader.close();
164           segmentInfos.remove(infosUpto);
165         }
166         success = true;
167       } finally {
168         if (!success) {
169           // Close all readers we had opened:
170           for(SegmentReader reader : readers) {
171             try {
172               reader.close();
173             } catch (Throwable ignore) {
174               // keep going - we want to clean up as much as possible
175             }
176           }
177         }
178       }
179     }
180
181     this.writer = writer;
182
183     initialize(readers.toArray(new SegmentReader[readers.size()]));
184   }
185
186   /** This constructor is only used for {@link #doOpenIfChanged()} */
187   DirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[] oldStarts,
188                   Map<String,byte[]> oldNormsCache, boolean readOnly, boolean doClone, int termInfosIndexDivisor,
189                   Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
190     this.directory = directory;
191     this.readOnly = readOnly;
192     this.segmentInfos = infos;
193     this.termInfosIndexDivisor = termInfosIndexDivisor;
194     assert readerFinishedListeners != null;
195     this.readerFinishedListeners = readerFinishedListeners;
196     applyAllDeletes = false;
197
198     // we put the old SegmentReaders in a map, that allows us
199     // to lookup a reader using its segment name
200     Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
201
202     if (oldReaders != null) {
203       // create a Map SegmentName->SegmentReader
204       for (int i = 0; i < oldReaders.length; i++) {
205         segmentReaders.put(oldReaders[i].getSegmentName(), Integer.valueOf(i));
206       }
207     }
208     
209     SegmentReader[] newReaders = new SegmentReader[infos.size()];
210     
211     // remember which readers are shared between the old and the re-opened
212     // DirectoryReader - we have to incRef those readers
213     boolean[] readerShared = new boolean[infos.size()];
214     
215     for (int i = infos.size() - 1; i>=0; i--) {
216       // find SegmentReader for this segment
217       Integer oldReaderIndex = segmentReaders.get(infos.info(i).name);
218       if (oldReaderIndex == null) {
219         // this is a new segment, no old SegmentReader can be reused
220         newReaders[i] = null;
221       } else {
222         // there is an old reader for this segment - we'll try to reopen it
223         newReaders[i] = oldReaders[oldReaderIndex.intValue()];
224       }
225
226       boolean success = false;
227       try {
228         SegmentReader newReader;
229         if (newReaders[i] == null || infos.info(i).getUseCompoundFile() != newReaders[i].getSegmentInfo().getUseCompoundFile()) {
230
231           // We should never see a totally new segment during cloning
232           assert !doClone;
233
234           // this is a new reader; in case we hit an exception we can close it safely
235           newReader = SegmentReader.get(readOnly, infos.info(i), termInfosIndexDivisor);
236           newReader.readerFinishedListeners = readerFinishedListeners;
237           readerShared[i] = false;
238           newReaders[i] = newReader;
239         } else {
240           newReader = newReaders[i].reopenSegment(infos.info(i), doClone, readOnly);
241           if (newReader == null) {
242             // this reader will be shared between the old and the new one,
243             // so we must incRef it
244             readerShared[i] = true;
245             newReaders[i].incRef();
246           } else {
247             assert newReader.readerFinishedListeners == readerFinishedListeners;
248             readerShared[i] = false;
249             // Steal ref returned to us by reopenSegment:
250             newReaders[i] = newReader;
251           }
252         }
253         success = true;
254       } finally {
255         if (!success) {
256           for (i++; i < infos.size(); i++) {
257             if (newReaders[i] != null) {
258               try {
259                 if (!readerShared[i]) {
260                   // this is a new subReader that is not used by the old one,
261                   // we can close it
262                   newReaders[i].close();
263                 } else {
264                   // this subReader is also used by the old reader, so instead
265                   // closing we must decRef it
266                   newReaders[i].decRef();
267                 }
268               } catch (IOException ignore) {
269                 // keep going - we want to clean up as much as possible
270               }
271             }
272           }
273         }
274       }
275     }    
276     
277     // initialize the readers to calculate maxDoc before we try to reuse the old normsCache
278     initialize(newReaders);
279     
280     // try to copy unchanged norms from the old normsCache to the new one
281     if (oldNormsCache != null) {
282       for (Map.Entry<String,byte[]> entry: oldNormsCache.entrySet()) {
283         String field = entry.getKey();
284         if (!hasNorms(field)) {
285           continue;
286         }
287
288         byte[] oldBytes = entry.getValue();
289
290         byte[] bytes = new byte[maxDoc()];
291
292         for (int i = 0; i < subReaders.length; i++) {
293           Integer oldReaderIndex = segmentReaders.get(subReaders[i].getSegmentName());
294
295           // this SegmentReader was not re-opened, we can copy all of its norms 
296           if (oldReaderIndex != null &&
297                (oldReaders[oldReaderIndex.intValue()] == subReaders[i] 
298                  || oldReaders[oldReaderIndex.intValue()].norms.get(field) == subReaders[i].norms.get(field))) {
299             // we don't have to synchronize here: either this constructor is called from a SegmentReader,
300             // in which case no old norms cache is present, or it is called from MultiReader.reopen(),
301             // which is synchronized
302             System.arraycopy(oldBytes, oldStarts[oldReaderIndex.intValue()], bytes, starts[i], starts[i+1] - starts[i]);
303           } else {
304             subReaders[i].norms(field, bytes, starts[i]);
305           }
306         }
307
308         normsCache.put(field, bytes);      // update cache
309       }
310     }
311   }
312
313   /** {@inheritDoc} */
314   @Override
315   public String toString() {
316     final StringBuilder buffer = new StringBuilder();
317     if (hasChanges) {
318       buffer.append("*");
319     }
320     buffer.append(getClass().getSimpleName());
321     buffer.append('(');
322     final String segmentsFile = segmentInfos.getCurrentSegmentFileName();
323     if (segmentsFile != null) {
324       buffer.append(segmentsFile);
325     }
326     if (writer != null) {
327       buffer.append(":nrt");
328     }
329     for(int i=0;i<subReaders.length;i++) {
330       buffer.append(' ');
331       buffer.append(subReaders[i]);
332     }
333     buffer.append(')');
334     return buffer.toString();
335   }
336
337   private void initialize(SegmentReader[] subReaders) throws IOException {
338     this.subReaders = subReaders;
339     starts = new int[subReaders.length + 1];    // build starts array
340     for (int i = 0; i < subReaders.length; i++) {
341       starts[i] = maxDoc;
342       maxDoc += subReaders[i].maxDoc();      // compute maxDocs
343
344       if (subReaders[i].hasDeletions())
345         hasDeletions = true;
346     }
347     starts[subReaders.length] = maxDoc;
348
349     if (!readOnly) {
350       maxIndexVersion = SegmentInfos.readCurrentVersion(directory);
351     }
352   }
353
354   @Override
355   public final synchronized Object clone() {
356     try {
357       return clone(readOnly); // Preserve current readOnly
358     } catch (Exception ex) {
359       throw new RuntimeException(ex);
360     }
361   }
362
363   @Override
364   public final synchronized IndexReader clone(boolean openReadOnly) throws CorruptIndexException, IOException {
365     // doOpenIfChanged calls ensureOpen
366     DirectoryReader newReader = doOpenIfChanged((SegmentInfos) segmentInfos.clone(), true, openReadOnly);
367
368     if (this != newReader) {
369       newReader.deletionPolicy = deletionPolicy;
370     }
371     newReader.writer = writer;
372     // If we're cloning a non-readOnly reader, move the
373     // writeLock (if there is one) to the new reader:
374     if (!openReadOnly && writeLock != null) {
375       // In near real-time search, reader is always readonly
376       assert writer == null;
377       newReader.writeLock = writeLock;
378       newReader.hasChanges = hasChanges;
379       newReader.hasDeletions = hasDeletions;
380       writeLock = null;
381       hasChanges = false;
382     }
383     assert newReader.readerFinishedListeners != null;
384
385     return newReader;
386   }
387
388   @Override
389   protected final IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
390     // Preserve current readOnly
391     return doOpenIfChanged(readOnly, null);
392   }
393
394   @Override
395   protected final IndexReader doOpenIfChanged(boolean openReadOnly) throws CorruptIndexException, IOException {
396     return doOpenIfChanged(openReadOnly, null);
397   }
398
399   @Override
400   protected final IndexReader doOpenIfChanged(final IndexCommit commit) throws CorruptIndexException, IOException {
401     return doOpenIfChanged(true, commit);
402   }
403
404   @Override
405   protected final IndexReader doOpenIfChanged(IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
406     if (writer == this.writer && applyAllDeletes == this.applyAllDeletes) {
407       return doOpenIfChanged();
408     } else {    
409       return super.doOpenIfChanged(writer, applyAllDeletes);
410     }
411   }
412
413   private final IndexReader doOpenFromWriter(boolean openReadOnly, IndexCommit commit) throws CorruptIndexException, IOException {
414     assert readOnly;
415
416     if (!openReadOnly) {
417       throw new IllegalArgumentException("a reader obtained from IndexWriter.getReader() can only be reopened with openReadOnly=true (got false)");
418     }
419
420     if (commit != null) {
421       throw new IllegalArgumentException("a reader obtained from IndexWriter.getReader() cannot currently accept a commit");
422     }
423
424     if (writer.nrtIsCurrent(segmentInfos)) {
425       return null;
426     }
427
428     IndexReader reader = writer.getReader(applyAllDeletes);
429
430     // If in fact no changes took place, return null:
431     if (reader.getVersion() == segmentInfos.getVersion()) {
432       reader.decRef();
433       return null;
434     }
435
436     reader.readerFinishedListeners = readerFinishedListeners;
437     return reader;
438   }
439
440   private IndexReader doOpenIfChanged(final boolean openReadOnly, IndexCommit commit) throws CorruptIndexException, IOException {
441     ensureOpen();
442
443     assert commit == null || openReadOnly;
444
445     // If we were obtained by writer.getReader(), re-ask the
446     // writer to get a new reader.
447     if (writer != null) {
448       return doOpenFromWriter(openReadOnly, commit);
449     } else {
450       return doOpenNoWriter(openReadOnly, commit);
451     }
452   }
453
454   private synchronized IndexReader doOpenNoWriter(final boolean openReadOnly, IndexCommit commit) throws CorruptIndexException, IOException {
455
456     if (commit == null) {
457       if (hasChanges) {
458         // We have changes, which means we are not readOnly:
459         assert readOnly == false;
460         // and we hold the write lock:
461         assert writeLock != null;
462         // so no other writer holds the write lock, which
463         // means no changes could have been done to the index:
464         assert isCurrent();
465
466         if (openReadOnly) {
467           return clone(openReadOnly);
468         } else {
469           return null;
470         }
471       } else if (isCurrent()) {
472         if (openReadOnly != readOnly) {
473           // Just fallback to clone
474           return clone(openReadOnly);
475         } else {
476           return null;
477         }
478       }
479     } else {
480       if (directory != commit.getDirectory()) {
481         throw new IOException("the specified commit does not match the specified Directory");
482       }
483       if (segmentInfos != null && commit.getSegmentsFileName().equals(segmentInfos.getCurrentSegmentFileName())) {
484         if (readOnly != openReadOnly) {
485           // Just fallback to clone
486           return clone(openReadOnly);
487         } else {
488           return null;
489         }
490       }
491     }
492
493     return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
494       @Override
495       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
496         SegmentInfos infos = new SegmentInfos();
497         infos.read(directory, segmentFileName);
498         return doOpenIfChanged(infos, false, openReadOnly);
499       }
500     }.run(commit);
501   }
502
503   private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
504     DirectoryReader reader;
505     if (openReadOnly) {
506       reader = new ReadOnlyDirectoryReader(directory, infos, subReaders, starts, normsCache, doClone, termInfosIndexDivisor, readerFinishedListeners);
507     } else {
508       reader = new DirectoryReader(directory, infos, subReaders, starts, normsCache, false, doClone, termInfosIndexDivisor, readerFinishedListeners);
509     }
510     return reader;
511   }
512
513   /** Version number when this IndexReader was opened. */
514   @Override
515   public long getVersion() {
516     ensureOpen();
517     return segmentInfos.getVersion();
518   }
519
520   @Override
521   public TermFreqVector[] getTermFreqVectors(int n) throws IOException {
522     ensureOpen();
523     int i = readerIndex(n);        // find segment num
524     return subReaders[i].getTermFreqVectors(n - starts[i]); // dispatch to segment
525   }
526
527   @Override
528   public TermFreqVector getTermFreqVector(int n, String field)
529       throws IOException {
530     ensureOpen();
531     int i = readerIndex(n);        // find segment num
532     return subReaders[i].getTermFreqVector(n - starts[i], field);
533   }
534
535
536   @Override
537   public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
538     ensureOpen();
539     int i = readerIndex(docNumber);        // find segment num
540     subReaders[i].getTermFreqVector(docNumber - starts[i], field, mapper);
541   }
542
543   @Override
544   public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
545     ensureOpen();
546     int i = readerIndex(docNumber);        // find segment num
547     subReaders[i].getTermFreqVector(docNumber - starts[i], mapper);
548   }
549
550   @Deprecated
551   @Override
552   public boolean isOptimized() {
553     ensureOpen();
554     return segmentInfos.size() == 1 && !hasDeletions();
555   }
556
557   @Override
558   public int numDocs() {
559     // Don't call ensureOpen() here (it could affect performance)
560
561     // NOTE: multiple threads may wind up init'ing
562     // numDocs... but that's harmless
563     if (numDocs == -1) {        // check cache
564       int n = 0;                // cache miss--recompute
565       for (int i = 0; i < subReaders.length; i++)
566         n += subReaders[i].numDocs();      // sum from readers
567       numDocs = n;
568     }
569     return numDocs;
570   }
571
572   @Override
573   public int maxDoc() {
574     // Don't call ensureOpen() here (it could affect performance)
575     return maxDoc;
576   }
577
578   // inherit javadoc
579   @Override
580   public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
581     ensureOpen();
582     int i = readerIndex(n);                          // find segment num
583     return subReaders[i].document(n - starts[i], fieldSelector);    // dispatch to segment reader
584   }
585
586   @Override
587   public boolean isDeleted(int n) {
588     // Don't call ensureOpen() here (it could affect performance)
589     final int i = readerIndex(n);                           // find segment num
590     return subReaders[i].isDeleted(n - starts[i]);    // dispatch to segment reader
591   }
592
593   @Override
594   public boolean hasDeletions() {
595     ensureOpen();
596     return hasDeletions;
597   }
598
599   @Override
600   protected void doDelete(int n) throws CorruptIndexException, IOException {
601     numDocs = -1;                             // invalidate cache
602     int i = readerIndex(n);                   // find segment num
603     subReaders[i].deleteDocument(n - starts[i]);      // dispatch to segment reader
604     hasDeletions = true;
605   }
606
607   @Override
608   protected void doUndeleteAll() throws CorruptIndexException, IOException {
609     for (int i = 0; i < subReaders.length; i++)
610       subReaders[i].undeleteAll();
611
612     hasDeletions = false;
613     numDocs = -1;                                 // invalidate cache
614   }
615
616   private int readerIndex(int n) {    // find reader for doc n:
617     return readerIndex(n, this.starts, this.subReaders.length);
618   }
619   
620   final static int readerIndex(int n, int[] starts, int numSubReaders) {    // find reader for doc n:
621     int lo = 0;                                      // search starts array
622     int hi = numSubReaders - 1;                  // for first element less
623
624     while (hi >= lo) {
625       int mid = (lo + hi) >>> 1;
626       int midValue = starts[mid];
627       if (n < midValue)
628         hi = mid - 1;
629       else if (n > midValue)
630         lo = mid + 1;
631       else {                                      // found a match
632         while (mid+1 < numSubReaders && starts[mid+1] == midValue) {
633           mid++;                                  // scan to last match
634         }
635         return mid;
636       }
637     }
638     return hi;
639   }
640
641   @Override
642   public boolean hasNorms(String field) throws IOException {
643     ensureOpen();
644     for (int i = 0; i < subReaders.length; i++) {
645       if (subReaders[i].hasNorms(field)) return true;
646     }
647     return false;
648   }
649
650   @Override
651   public synchronized byte[] norms(String field) throws IOException {
652     ensureOpen();
653     byte[] bytes = normsCache.get(field);
654     if (bytes != null)
655       return bytes;          // cache hit
656     if (!hasNorms(field))
657       return null;
658
659     bytes = new byte[maxDoc()];
660     for (int i = 0; i < subReaders.length; i++)
661       subReaders[i].norms(field, bytes, starts[i]);
662     normsCache.put(field, bytes);      // update cache
663     return bytes;
664   }
665
666   @Override
667   public synchronized void norms(String field, byte[] result, int offset)
668     throws IOException {
669     ensureOpen();
670     byte[] bytes = normsCache.get(field);
671     if (bytes==null && !hasNorms(field)) {
672       Arrays.fill(result, offset, result.length, Similarity.getDefault().encodeNormValue(1.0f));
673     } else if (bytes != null) {                           // cache hit
674       System.arraycopy(bytes, 0, result, offset, maxDoc());
675     } else {
676       for (int i = 0; i < subReaders.length; i++) {      // read from segments
677         subReaders[i].norms(field, result, offset + starts[i]);
678       }
679     }
680   }
681
682   @Override
683   protected void doSetNorm(int n, String field, byte value)
684     throws CorruptIndexException, IOException {
685     synchronized (normsCache) {
686       normsCache.remove(field);                         // clear cache      
687     }
688     int i = readerIndex(n);                           // find segment num
689     subReaders[i].setNorm(n-starts[i], field, value); // dispatch
690   }
691
692   @Override
693   public TermEnum terms() throws IOException {
694     ensureOpen();
695     if (subReaders.length == 1) {
696       // Optimize single segment case:
697       return subReaders[0].terms();
698     } else {
699       return new MultiTermEnum(this, subReaders, starts, null);
700     }
701   }
702
703   @Override
704   public TermEnum terms(Term term) throws IOException {
705     ensureOpen();
706     if (subReaders.length == 1) {
707       // Optimize single segment case:
708       return subReaders[0].terms(term);
709     } else {
710       return new MultiTermEnum(this, subReaders, starts, term);
711     }
712   }
713
714   @Override
715   public int docFreq(Term t) throws IOException {
716     ensureOpen();
717     int total = 0;          // sum freqs in segments
718     for (int i = 0; i < subReaders.length; i++)
719       total += subReaders[i].docFreq(t);
720     return total;
721   }
722
723   @Override
724   public TermDocs termDocs() throws IOException {
725     ensureOpen();
726     if (subReaders.length == 1) {
727       // Optimize single segment case:
728       return subReaders[0].termDocs();
729     } else {
730       return new MultiTermDocs(this, subReaders, starts);
731     }
732   }
733
734   @Override
735   public TermDocs termDocs(Term term) throws IOException {
736     ensureOpen();
737     if (subReaders.length == 1) {
738       // Optimize single segment case:
739       return subReaders[0].termDocs(term);
740     } else {
741       return super.termDocs(term);
742     }
743   }
744
745   @Override
746   public TermPositions termPositions() throws IOException {
747     ensureOpen();
748     if (subReaders.length == 1) {
749       // Optimize single segment case:
750       return subReaders[0].termPositions();
751     } else {
752       return new MultiTermPositions(this, subReaders, starts);
753     }
754   }
755
756   /**
757    * Tries to acquire the WriteLock on this directory. this method is only valid if this IndexReader is directory
758    * owner.
759    *
760    * @throws StaleReaderException  if the index has changed since this reader was opened
761    * @throws CorruptIndexException if the index is corrupt
762    * @throws org.apache.lucene.store.LockObtainFailedException
763    *                               if another writer has this index open (<code>write.lock</code> could not be
764    *                               obtained)
765    * @throws IOException           if there is a low-level IO error
766    */
767   @Override
768   protected void acquireWriteLock() throws StaleReaderException, CorruptIndexException, LockObtainFailedException, IOException {
769
770     if (readOnly) {
771       // NOTE: we should not reach this code w/ the core
772       // IndexReader classes; however, an external subclass
773       // of IndexReader could reach this.
774       ReadOnlySegmentReader.noWrite();
775     }
776
777     if (segmentInfos != null) {
778       ensureOpen();
779       if (stale)
780         throw new StaleReaderException("IndexReader out of date and no longer valid for delete, undelete, or setNorm operations");
781
782       if (writeLock == null) {
783         Lock writeLock = directory.makeLock(IndexWriter.WRITE_LOCK_NAME);
784         if (!writeLock.obtain(IndexWriterConfig.WRITE_LOCK_TIMEOUT)) // obtain write lock
785           throw new LockObtainFailedException("Index locked for write: " + writeLock);
786         this.writeLock = writeLock;
787
788         // we have to check whether index has changed since this reader was opened.
789         // if so, this reader is no longer valid for
790         // deletion
791         if (SegmentInfos.readCurrentVersion(directory) > maxIndexVersion) {
792           stale = true;
793           this.writeLock.release();
794           this.writeLock = null;
795           throw new StaleReaderException("IndexReader out of date and no longer valid for delete, undelete, or setNorm operations");
796         }
797       }
798     }
799   }
800
801   /**
802    * Commit changes resulting from delete, undeleteAll, or setNorm operations
803    * <p/>
804    * If an exception is hit, then either no changes or all changes will have been committed to the index (transactional
805    * semantics).
806    *
807    * @throws IOException if there is a low-level IO error
808    */
809   @Override
810   protected void doCommit(Map<String,String> commitUserData) throws IOException {
811     if (hasChanges) {
812       segmentInfos.setUserData(commitUserData);
813       // Default deleter (for backwards compatibility) is
814       // KeepOnlyLastCommitDeleter:
815       IndexFileDeleter deleter = new IndexFileDeleter(directory,
816                                                       deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
817                                                       segmentInfos, null, null);
818       segmentInfos.updateGeneration(deleter.getLastSegmentInfos());
819       segmentInfos.changed();
820
821       // Checkpoint the state we are about to change, in
822       // case we have to roll back:
823       startCommit();
824
825       final List<SegmentInfo> rollbackSegments = segmentInfos.createBackupSegmentInfos(false);
826
827       boolean success = false;
828       try {
829         for (int i = 0; i < subReaders.length; i++)
830           subReaders[i].commit();
831
832         // Remove segments that contain only 100% deleted
833         // docs:
834         segmentInfos.pruneDeletedSegments();
835
836         // Sync all files we just wrote
837         directory.sync(segmentInfos.files(directory, false));
838         segmentInfos.commit(directory);
839         success = true;
840       } finally {
841
842         if (!success) {
843
844           // Rollback changes that were made to
845           // SegmentInfos but failed to get [fully]
846           // committed.  This way this reader instance
847           // remains consistent (matched to what's
848           // actually in the index):
849           rollbackCommit();
850
851           // Recompute deletable files & remove them (so
852           // partially written .del files, etc, are
853           // removed):
854           deleter.refresh();
855
856           // Restore all SegmentInfos (in case we pruned some)
857           segmentInfos.rollbackSegmentInfos(rollbackSegments);
858         }
859       }
860
861       // Have the deleter remove any now unreferenced
862       // files due to this commit:
863       deleter.checkpoint(segmentInfos, true);
864       deleter.close();
865
866       maxIndexVersion = segmentInfos.getVersion();
867
868       if (writeLock != null) {
869         writeLock.release();  // release write lock
870         writeLock = null;
871       }
872     }
873     hasChanges = false;
874   }
875
876   void startCommit() {
877     rollbackHasChanges = hasChanges;
878     for (int i = 0; i < subReaders.length; i++) {
879       subReaders[i].startCommit();
880     }
881   }
882
883   void rollbackCommit() {
884     hasChanges = rollbackHasChanges;
885     for (int i = 0; i < subReaders.length; i++) {
886       subReaders[i].rollbackCommit();
887     }
888   }
889
890   @Override
891   public Map<String,String> getCommitUserData() {
892     ensureOpen();
893     return segmentInfos.getUserData();
894   }
895
896   @Override
897   public boolean isCurrent() throws CorruptIndexException, IOException {
898     ensureOpen();
899     if (writer == null || writer.isClosed()) {
900       // we loaded SegmentInfos from the directory
901       return SegmentInfos.readCurrentVersion(directory) == segmentInfos.getVersion();
902     } else {
903       return writer.nrtIsCurrent(segmentInfos);
904     }
905   }
906
907   @Override
908   protected synchronized void doClose() throws IOException {
909     IOException ioe = null;
910     normsCache = null;
911     for (int i = 0; i < subReaders.length; i++) {
912       // try to close each reader, even if an exception is thrown
913       try {
914         subReaders[i].decRef();
915       } catch (IOException e) {
916         if (ioe == null) ioe = e;
917       }
918     }
919
920     if (writer != null) {
921       // Since we just closed, writer may now be able to
922       // delete unused files:
923       writer.deletePendingFiles();
924     }
925
926     // throw the first exception
927     if (ioe != null) throw ioe;
928   }
929
930   @Override
931   public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
932     ensureOpen();
933     return getFieldNames(fieldNames, this.subReaders);
934   }
935   
936   static Collection<String> getFieldNames (IndexReader.FieldOption fieldNames, IndexReader[] subReaders) {
937     // maintain a unique set of field names
938     Set<String> fieldSet = new HashSet<String>();
939     for (IndexReader reader : subReaders) {
940       Collection<String> names = reader.getFieldNames(fieldNames);
941       fieldSet.addAll(names);
942     }
943     return fieldSet;
944   } 
945   
946   @Override
947   public IndexReader[] getSequentialSubReaders() {
948     return subReaders;
949   }
950
951   /** Returns the directory this index resides in. */
952   @Override
953   public Directory directory() {
954     // Don't ensureOpen here -- in certain cases, when a
955     // cloned/reopened reader needs to commit, it may call
956     // this method on the closed original reader
957     return directory;
958   }
959
960   @Override
961   public int getTermInfosIndexDivisor() {
962     ensureOpen();
963     return termInfosIndexDivisor;
964   }
965
966   /**
967    * Expert: return the IndexCommit that this reader has opened.
968    * <p/>
969    * @lucene.experimental
970    */
971   @Override
972   public IndexCommit getIndexCommit() throws IOException {
973     ensureOpen();
974     return new ReaderCommit(segmentInfos, directory);
975   }
976
977   /** @see org.apache.lucene.index.IndexReader#listCommits */
978   public static Collection<IndexCommit> listCommits(Directory dir) throws IOException {
979     final String[] files = dir.listAll();
980
981     List<IndexCommit> commits = new ArrayList<IndexCommit>();
982
983     SegmentInfos latest = new SegmentInfos();
984     latest.read(dir);
985     final long currentGen = latest.getGeneration();
986
987     commits.add(new ReaderCommit(latest, dir));
988
989     for(int i=0;i<files.length;i++) {
990
991       final String fileName = files[i];
992
993       if (fileName.startsWith(IndexFileNames.SEGMENTS) &&
994           !fileName.equals(IndexFileNames.SEGMENTS_GEN) &&
995           SegmentInfos.generationFromSegmentsFileName(fileName) < currentGen) {
996
997         SegmentInfos sis = new SegmentInfos();
998         try {
999           // IOException allowed to throw there, in case
1000           // segments_N is corrupt
1001           sis.read(dir, fileName);
1002         } catch (FileNotFoundException fnfe) {
1003           // LUCENE-948: on NFS (and maybe others), if
1004           // you have writers switching back and forth
1005           // between machines, it's very likely that the
1006           // dir listing will be stale and will claim a
1007           // file segments_X exists when in fact it
1008           // doesn't.  So, we catch this and handle it
1009           // as if the file does not exist
1010           sis = null;
1011         }
1012
1013         if (sis != null)
1014           commits.add(new ReaderCommit(sis, dir));
1015       }
1016     }
1017
1018     // Ensure that the commit points are sorted in ascending order.
1019     Collections.sort(commits);
1020
1021     return commits;
1022   }
1023
1024   private static final class ReaderCommit extends IndexCommit {
1025     private String segmentsFileName;
1026     Collection<String> files;
1027     Directory dir;
1028     long generation;
1029     long version;
1030     final Map<String,String> userData;
1031     private final int segmentCount;
1032
1033     ReaderCommit(SegmentInfos infos, Directory dir) throws IOException {
1034       segmentsFileName = infos.getCurrentSegmentFileName();
1035       this.dir = dir;
1036       userData = infos.getUserData();
1037       files = Collections.unmodifiableCollection(infos.files(dir, true));
1038       version = infos.getVersion();
1039       generation = infos.getGeneration();
1040       segmentCount = infos.size();
1041     }
1042
1043     @Override
1044     public String toString() {
1045       return "DirectoryReader.ReaderCommit(" + segmentsFileName + ")";
1046     }
1047
1048     @Override
1049     public int getSegmentCount() {
1050       return segmentCount;
1051     }
1052
1053     @Override
1054     public String getSegmentsFileName() {
1055       return segmentsFileName;
1056     }
1057
1058     @Override
1059     public Collection<String> getFileNames() {
1060       return files;
1061     }
1062
1063     @Override
1064     public Directory getDirectory() {
1065       return dir;
1066     }
1067
1068     @Override
1069     public long getVersion() {
1070       return version;
1071     }
1072
1073     @Override
1074     public long getGeneration() {
1075       return generation;
1076     }
1077
1078     @Override
1079     public boolean isDeleted() {
1080       return false;
1081     }
1082
1083     @Override
1084     public Map<String,String> getUserData() {
1085       return userData;
1086     }
1087
1088     @Override
1089     public void delete() {
1090       throw new UnsupportedOperationException("This IndexCommit does not support deletions");
1091     }
1092   }
1093
1094   static class MultiTermEnum extends TermEnum {
1095     IndexReader topReader; // used for matching TermEnum to TermDocs
1096     private SegmentMergeQueue queue;
1097   
1098     private Term term;
1099     private int docFreq;
1100     final SegmentMergeInfo[] matchingSegments; // null terminated array of matching segments
1101
1102     public MultiTermEnum(IndexReader topReader, IndexReader[] readers, int[] starts, Term t)
1103       throws IOException {
1104       this.topReader = topReader;
1105       queue = new SegmentMergeQueue(readers.length);
1106       matchingSegments = new SegmentMergeInfo[readers.length+1];
1107       for (int i = 0; i < readers.length; i++) {
1108         IndexReader reader = readers[i];
1109         TermEnum termEnum;
1110   
1111         if (t != null) {
1112           termEnum = reader.terms(t);
1113         } else
1114           termEnum = reader.terms();
1115   
1116         SegmentMergeInfo smi = new SegmentMergeInfo(starts[i], termEnum, reader);
1117         smi.ord = i;
1118         if (t == null ? smi.next() : termEnum.term() != null)
1119           queue.add(smi);          // initialize queue
1120         else
1121           smi.close();
1122       }
1123   
1124       if (t != null && queue.size() > 0) {
1125         next();
1126       }
1127     }
1128   
1129     @Override
1130     public boolean next() throws IOException {
1131       for (int i=0; i<matchingSegments.length; i++) {
1132         SegmentMergeInfo smi = matchingSegments[i];
1133         if (smi==null) break;
1134         if (smi.next())
1135           queue.add(smi);
1136         else
1137           smi.close(); // done with segment
1138       }
1139       
1140       int numMatchingSegments = 0;
1141       matchingSegments[0] = null;
1142
1143       SegmentMergeInfo top = queue.top();
1144
1145       if (top == null) {
1146         term = null;
1147         return false;
1148       }
1149   
1150       term = top.term;
1151       docFreq = 0;
1152   
1153       while (top != null && term.compareTo(top.term) == 0) {
1154         matchingSegments[numMatchingSegments++] = top;
1155         queue.pop();
1156         docFreq += top.termEnum.docFreq();    // increment freq
1157         top = queue.top();
1158       }
1159
1160       matchingSegments[numMatchingSegments] = null;
1161       return true;
1162     }
1163   
1164     @Override
1165     public Term term() {
1166       return term;
1167     }
1168   
1169     @Override
1170     public int docFreq() {
1171       return docFreq;
1172     }
1173   
1174     @Override
1175     public void close() throws IOException {
1176       queue.close();
1177     }
1178   }
1179
1180   static class MultiTermDocs implements TermDocs {
1181     IndexReader topReader;  // used for matching TermEnum to TermDocs
1182     protected IndexReader[] readers;
1183     protected int[] starts;
1184     protected Term term;
1185   
1186     protected int base = 0;
1187     protected int pointer = 0;
1188   
1189     private TermDocs[] readerTermDocs;
1190     protected TermDocs current;              // == readerTermDocs[pointer]
1191
1192     private MultiTermEnum tenum;  // the term enum used for seeking... can be null
1193     int matchingSegmentPos;  // position into the matching segments from tenum
1194     SegmentMergeInfo smi;     // current segment mere info... can be null
1195
1196     public MultiTermDocs(IndexReader topReader, IndexReader[] r, int[] s) {
1197       this.topReader = topReader;
1198       readers = r;
1199       starts = s;
1200   
1201       readerTermDocs = new TermDocs[r.length];
1202     }
1203
1204     public int doc() {
1205       return base + current.doc();
1206     }
1207     public int freq() {
1208       return current.freq();
1209     }
1210   
1211     public void seek(Term term) {
1212       this.term = term;
1213       this.base = 0;
1214       this.pointer = 0;
1215       this.current = null;
1216       this.tenum = null;
1217       this.smi = null;
1218       this.matchingSegmentPos = 0;
1219     }
1220   
1221     public void seek(TermEnum termEnum) throws IOException {
1222       seek(termEnum.term());
1223       if (termEnum instanceof MultiTermEnum) {
1224         tenum = (MultiTermEnum)termEnum;
1225         if (topReader != tenum.topReader)
1226           tenum = null;
1227       }
1228     }
1229   
1230     public boolean next() throws IOException {
1231       for(;;) {
1232         if (current!=null && current.next()) {
1233           return true;
1234         }
1235         else if (pointer < readers.length) {
1236           if (tenum != null) {
1237             smi = tenum.matchingSegments[matchingSegmentPos++];
1238             if (smi==null) {
1239               pointer = readers.length;
1240               return false;
1241             }
1242             pointer = smi.ord;
1243           }
1244           base = starts[pointer];
1245           current = termDocs(pointer++);
1246         } else {
1247           return false;
1248         }
1249       }
1250     }
1251   
1252     /** Optimized implementation. */
1253     public int read(final int[] docs, final int[] freqs) throws IOException {
1254       while (true) {
1255         while (current == null) {
1256           if (pointer < readers.length) {      // try next segment
1257             if (tenum != null) {
1258               smi = tenum.matchingSegments[matchingSegmentPos++];
1259               if (smi==null) {
1260                 pointer = readers.length;
1261                 return 0;
1262               }
1263               pointer = smi.ord;
1264             }
1265             base = starts[pointer];
1266             current = termDocs(pointer++);
1267           } else {
1268             return 0;
1269           }
1270         }
1271         int end = current.read(docs, freqs);
1272         if (end == 0) {          // none left in segment
1273           current = null;
1274         } else {            // got some
1275           final int b = base;        // adjust doc numbers
1276           for (int i = 0; i < end; i++)
1277            docs[i] += b;
1278           return end;
1279         }
1280       }
1281     }
1282   
1283    /* A Possible future optimization could skip entire segments */ 
1284     public boolean skipTo(int target) throws IOException {
1285       for(;;) {
1286         if (current != null && current.skipTo(target-base)) {
1287           return true;
1288         } else if (pointer < readers.length) {
1289           if (tenum != null) {
1290             SegmentMergeInfo smi = tenum.matchingSegments[matchingSegmentPos++];
1291             if (smi==null) {
1292               pointer = readers.length;
1293               return false;
1294             }
1295             pointer = smi.ord;
1296           }
1297           base = starts[pointer];
1298           current = termDocs(pointer++);
1299         } else
1300           return false;
1301       }
1302     }
1303   
1304     private TermDocs termDocs(int i) throws IOException {
1305       TermDocs result = readerTermDocs[i];
1306       if (result == null)
1307         result = readerTermDocs[i] = termDocs(readers[i]);
1308       if (smi != null) {
1309         assert(smi.ord == i);
1310         assert(smi.termEnum.term().equals(term));
1311         result.seek(smi.termEnum);
1312       } else {
1313         result.seek(term);
1314       }
1315       return result;
1316     }
1317   
1318     protected TermDocs termDocs(IndexReader reader)
1319       throws IOException {
1320       return term==null ? reader.termDocs(null) : reader.termDocs();
1321     }
1322   
1323     public void close() throws IOException {
1324       for (int i = 0; i < readerTermDocs.length; i++) {
1325         if (readerTermDocs[i] != null)
1326           readerTermDocs[i].close();
1327       }
1328     }
1329   }
1330
1331   static class MultiTermPositions extends MultiTermDocs implements TermPositions {
1332     public MultiTermPositions(IndexReader topReader, IndexReader[] r, int[] s) {
1333       super(topReader,r,s);
1334     }
1335   
1336     @Override
1337     protected TermDocs termDocs(IndexReader reader) throws IOException {
1338       return reader.termPositions();
1339     }
1340   
1341     public int nextPosition() throws IOException {
1342       return ((TermPositions)current).nextPosition();
1343     }
1344     
1345     public int getPayloadLength() {
1346       return ((TermPositions)current).getPayloadLength();
1347     }
1348      
1349     public byte[] getPayload(byte[] data, int offset) throws IOException {
1350       return ((TermPositions)current).getPayload(data, offset);
1351     }
1352   
1353   
1354     // TODO: Remove warning after API has been finalized
1355     public boolean isPayloadAvailable() {
1356       return ((TermPositions) current).isPayloadAvailable();
1357     }
1358   }
1359 }