1 package org.apache.lucene.search;
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
11 * http://www.apache.org/licenses/LICENSE-2.0
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.
20 import java.io.Closeable;
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Collections;
24 import java.util.List;
25 import java.util.concurrent.ConcurrentHashMap;
26 import java.util.concurrent.TimeUnit;
28 import org.apache.lucene.index.IndexReader; // javadocs
29 import org.apache.lucene.search.NRTManager; // javadocs
30 import org.apache.lucene.search.IndexSearcher; // javadocs
31 import org.apache.lucene.store.AlreadyClosedException;
32 import org.apache.lucene.util.IOUtils;
35 * Keeps track of current plus old IndexSearchers, closing
36 * the old ones once they have timed out.
41 * SearcherLifetimeManager mgr = new SearcherLifetimeManager();
44 * Per search-request, if it's a "new" search request, then
45 * obtain the latest searcher you have (for example, by
46 * using {@link SearcherManager} or {@link NRTManager}), and
47 * then record this searcher:
50 * // Record the current searcher, and save the returend
51 * // token into user's search results (eg as a hidden
52 * // HTML form field):
53 * long token = mgr.record(searcher);
56 * When a follow-up search arrives, for example the user
57 * clicks next page, drills down/up, etc., take the token
58 * that you saved from the previous search and:
61 * // If possible, obtain the same searcher as the last
63 * IndexSearcher searcher = mgr.acquire(token);
64 * if (searcher != null) {
65 * // Searcher is still here
69 * mgr.release(searcher);
70 * // Do not use searcher after this!
74 * // Searcher was pruned -- notify user session timed
75 * // out, or, pull fresh searcher again
79 * Finally, in a separate thread, ideally the same thread
80 * that's periodically reopening your searchers, you should
81 * periodically prune old searchers:
84 * mgr.prune(new PruneByAge(600.0));
87 * <p><b>NOTE</b>: keeping many searchers around means
88 * you'll use more resources (open files, RAM) than a single
89 * searcher. However, as long as you are using {@link
90 * IndexReader#openIfChanged}, the searchers will usually
91 * share almost all segments and the added resource usage is
92 * contained. When a large merge has completed, and
93 * you reopen, because that is a large change, the new
94 * searcher will use higher additional RAM than other
95 * searchers; but large merges don't complete very often and
96 * it's unlikely you'll hit two of them in your expiration
97 * window. Still you should budget plenty of heap in the
98 * JVM to have a good safety margin.
100 * @lucene.experimental
103 public class SearcherLifetimeManager implements Closeable {
105 private static class SearcherTracker implements Comparable<SearcherTracker>, Closeable {
106 public final IndexSearcher searcher;
107 public final long recordTimeSec;
108 public final long version;
110 public SearcherTracker(IndexSearcher searcher) {
111 this.searcher = searcher;
112 version = searcher.getIndexReader().getVersion();
113 searcher.getIndexReader().incRef();
114 // Use nanoTime not currentTimeMillis since it [in
115 // theory] reduces risk from clock shift
116 recordTimeSec = TimeUnit.NANOSECONDS.toSeconds(System.nanoTime());
119 // Newer searchers are sort before older ones:
120 public int compareTo(SearcherTracker other) {
121 // Be defensive: cannot subtract since it could
122 // technically overflow long, though, we'd never hit
124 if (recordTimeSec < other.recordTimeSec) {
126 } else if (other.recordTimeSec < recordTimeSec) {
133 public synchronized void close() throws IOException {
134 searcher.getIndexReader().decRef();
138 private volatile boolean closed;
140 // TODO: we could get by w/ just a "set"; need to have
141 // Tracker hash by its version and have compareTo(Long)
142 // compare to its version
143 private final ConcurrentHashMap<Long,SearcherTracker> searchers = new ConcurrentHashMap<Long,SearcherTracker>();
145 private void ensureOpen() {
147 throw new AlreadyClosedException("this SearcherLifetimeManager instance is closed");
151 /** Records that you are now using this IndexSearcher.
152 * Always call this when you've obtained a possibly new
153 * {@link IndexSearcher}, for example from one of the
154 * <code>get</code> methods in {@link NRTManager} or {@link
155 * SearcherManager}. It's fine if you already passed the
156 * same searcher to this method before.
158 * <p>This returns the long token that you can later pass
159 * to {@link #acquire} to retrieve the same IndexSearcher.
160 * You should record this long token in the search results
161 * sent to your user, such that if the user performs a
162 * follow-on action (clicks next page, drills down, etc.)
163 * the token is returned. */
164 public long record(IndexSearcher searcher) throws IOException {
166 // TODO: we don't have to use IR.getVersion to track;
167 // could be risky (if it's buggy); we could get better
168 // bug isolation if we assign our own private ID:
169 final long version = searcher.getIndexReader().getVersion();
170 SearcherTracker tracker = searchers.get(version);
171 if (tracker == null) {
172 tracker = new SearcherTracker(searcher);
173 if (searchers.putIfAbsent(version, tracker) != null) {
174 // Another thread beat us -- must decRef to undo
175 // incRef done by SearcherTracker ctor:
178 } else if (tracker.searcher != searcher) {
179 throw new IllegalArgumentException("the provided searcher has the same underlying reader version yet the searcher instance differs from before (new=" + searcher + " vs old=" + tracker.searcher);
185 /** Retrieve a previously recorded {@link IndexSearcher}, if it
186 * has not yet been closed
188 * <p><b>NOTE</b>: this may return null when the
189 * requested searcher has already timed out. When this
190 * happens you should notify your user that their session
191 * timed out and that they'll have to restart their
194 * <p>If this returns a non-null result, you must match
195 * later call {@link #release} on this searcher, best
196 * from a finally clause. */
197 public IndexSearcher acquire(long version) {
199 final SearcherTracker tracker = searchers.get(version);
200 if (tracker != null &&
201 tracker.searcher.getIndexReader().tryIncRef()) {
202 return tracker.searcher;
208 /** Release a searcher previously obtained from {@link
211 * <p><b>NOTE</b>: it's fine to call this after close. */
212 public void release(IndexSearcher s) throws IOException {
213 s.getIndexReader().decRef();
216 /** See {@link #prune}. */
217 public interface Pruner {
218 /** Return true if this searcher should be removed.
219 * @param ageSec how long ago this searcher was
220 * recorded vs the most recently recorded
222 * @param searcher Searcher
224 public boolean doPrune(int ageSec, IndexSearcher searcher);
227 /** Simple pruner that drops any searcher older by
228 * more than the specified seconds, than the newest
230 public final static class PruneByAge implements Pruner {
231 private final int maxAgeSec;
233 public PruneByAge(int maxAgeSec) {
235 throw new IllegalArgumentException("maxAgeSec must be > 0 (got " + maxAgeSec + ")");
237 this.maxAgeSec = maxAgeSec;
240 public boolean doPrune(int ageSec, IndexSearcher searcher) {
241 return ageSec > maxAgeSec;
245 /** Calls provided {@link Pruner} to prune entries. The
246 * entries are passed to the Pruner in sorted (newest to
247 * oldest IndexSearcher) order.
249 * <p><b>NOTE</b>: you must peridiocally call this, ideally
250 * from the same background thread that opens new
252 public synchronized void prune(Pruner pruner) throws IOException {
253 // Cannot just pass searchers.values() to ArrayList ctor
254 // (not thread-safe since the values can change while
255 // ArrayList is init'ing itself); must instead iterate
257 final List<SearcherTracker> trackers = new ArrayList<SearcherTracker>();
258 for(SearcherTracker tracker : searchers.values()) {
259 trackers.add(tracker);
261 Collections.sort(trackers);
262 final long newestSec = trackers.isEmpty() ? 0L : trackers.get(0).recordTimeSec;
263 for (SearcherTracker tracker: trackers) {
264 final int ageSec = (int) (newestSec - tracker.recordTimeSec);
266 if (pruner.doPrune(ageSec, tracker.searcher)) {
267 searchers.remove(tracker.version);
273 /** Close this to future searching; any searches still in
274 * process in other threads won't be affected, and they
275 * should still call {@link #release} after they are
278 * <p><b>NOTE: you must ensure no other threads are
279 * calling {@link #record} while you call close();
280 * otherwise it's possible not all searcher references
282 public synchronized void close() throws IOException {
284 final List<SearcherTracker> toClose = new ArrayList<SearcherTracker>(searchers.values());
286 // Remove up front in case exc below, so we don't
287 // over-decRef on double-close:
288 for(SearcherTracker tracker : toClose) {
289 searchers.remove(tracker.version);
292 IOUtils.close(toClose);
294 // Make some effort to catch mis-use:
295 if (searchers.size() != 0) {
296 throw new IllegalStateException("another thread called record while this SearcherLifetimeManager instance was being closed; not all searchers were closed");