One overarching question here: I understand, with Ocean, that you'd expect to be re-opening IndexReaders very frequently (after each add or delete) to be "real-time". But: wouldn't you also expect to be aggressively closing the old ones as well (ie, after the in-flight searches have finished with them)? Ie I would think you would not have a great many IndexReaders (SegmentReaders) open at a time.
More stuff below: Jason Rutherglen <[EMAIL PROTECTED]> wrote: > I've been looking more at how to improve the IndexReader.document call. > There are a few options. I implemented the IndexReader.documents call which > has the down side of not being backward compatible. Is this the new Documents class you proposed? Is the thinking that each instance of Documents would only last for one search? Ie, I do a search, then I call documents() to get a Documents instance, I interact with that to load all my documents, then I close it? > Probably the only way > to achieve both ends is the threadlocal as I noticed term vectors does the > same thing. This raises the issue of too many file descriptors for term > vectors if there are many reopens, does it not? Actually, when you clone a TermVectorsReader, which then clones the 3 IndexInputs, for FSDirectory this does not result in opening additional file descriptors. Instead, the cloned IndexInputs share the same RandomAccessFile instance, and synchronize on it so that no two can be reading from the file at once. Of course, this means there's still contention since all threads must share the same RandomAccessFile instance (but see LUCENE-753 as Yonik suggested). I think the best way to eventually solve this is to use asynchronous IO (JSR 203, to be in Java 7). If N threads want to load M documents each (to show their page of results) then you really want the OS to see all M*N requests at once so that the IO system can best schedule things. Modern hard drives, and I believe the high-speed SSDs as well, have substantial concurrency available, so to utilize that you really want to get the full queue down to devices. But this solution is quite a ways off! To "emulate" asynchronous IO, we should be able to allow multiple threads to access the same file at once, each with their own private RandomAccessFile instance. But of course we can't generally afford that today because we'd quickly run out of file descriptors. Maybe what we need is a new layer, under oal.store, which would manage when to create a new file descriptor, per thread, and when not to. This layer would be responsible for keeping total # descriptors under a certain limit, but would otherwise be free to go up to that limit if it seemed like there was contention. Not sure if there would be enough gains to make this worthwhile... > It would seem that copying > the reference to termVectorsLocal on reopens would help with this. If this > is amenable then the same could be done for fieldsReader with a > fieldsReaderThreadLocal. I agree, we should be copying this when we copy fieldsReader over. (And the same with termVectorsReader if we take this same approach). Can you include that in your new patch as well? (Or, under a new issue). I'm losing track of all these changes! > IndexReader.document as it is is really a lame duck. The > IndexReader.document call being synchronized at the top level drags down the > performance of systems that store data in Lucene. A single file descriptor > for all threads on an index that is constantly returning results with fields > is a serious problem. Users are always complaining about this issue and now > I know why. > > This should be a separate issue from IndexReader.clone. Agreed. Mike --------------------------------------------------------------------- To unsubscribe, e-mail: [EMAIL PROTECTED] For additional commands, e-mail: [EMAIL PROTECTED]