Lucene: don't commit the index after every write

When constructing a SearcherManager with an IndexWriter, as SubIndex
does, the returned searcher has immediate access to the unflushed
writes, so there is no need to commit after every write to make the
writes visible in the running process. This allows us to "decouple
durability to hardware/OS crashes from visibility of changes to a new
IndexReader" [1].

Since we don't know exactly when changes will get committed to disk,
start a thread to periodically refresh the SearcherManagers. The 100ms
poll time is a bit of a guess, but [1] says even 50ms should not be a
significant load.

[1] http://blog.mikemccandless.com/2011/11/near-real-time-readers-with-lucenes.html

Change-Id: I1d413e8334057e1b04f9d6414635a2d1c53507f4
This commit is contained in:
Dave Borowitz
2013-06-21 11:32:27 -07:00
parent 9c179db224
commit 2aacdd0e7b
3 changed files with 46 additions and 11 deletions

View File

@@ -86,6 +86,7 @@ public class LuceneChangeIndex implements ChangeIndex, LifecycleListener {
public static final String CHANGES_OPEN = "changes_open";
public static final String CHANGES_CLOSED = "changes_closed";
private final RefreshThread refreshThread;
private final FillArgs fillArgs;
private final boolean readOnly;
private final SubIndex openIndex;
@@ -93,6 +94,7 @@ public class LuceneChangeIndex implements ChangeIndex, LifecycleListener {
LuceneChangeIndex(SitePaths sitePaths, FillArgs fillArgs, boolean readOnly)
throws IOException {
this.refreshThread = new RefreshThread();
this.fillArgs = fillArgs;
this.readOnly = readOnly;
openIndex = new SubIndex(new File(sitePaths.index_dir, CHANGES_OPEN));
@@ -101,11 +103,12 @@ public class LuceneChangeIndex implements ChangeIndex, LifecycleListener {
@Override
public void start() {
// Do nothing.
refreshThread.start();
}
@Override
public void stop() {
refreshThread.halt();
openIndex.close();
closedIndex.close();
}
@@ -342,4 +345,35 @@ public class LuceneChangeIndex implements ChangeIndex, LifecycleListener {
private static IllegalArgumentException badFieldType(FieldType<?> t) {
return new IllegalArgumentException("unknown index field type " + t);
}
private class RefreshThread extends Thread {
private boolean stop;
@Override
public void run() {
while (!stop) {
openIndex.maybeRefresh();
closedIndex.maybeRefresh();
synchronized (this) {
try {
wait(100);
} catch (InterruptedException e) {
log.warn("error refreshing index searchers", e);
}
}
}
}
void halt() {
synchronized (this) {
stop = true;
notify();
}
try {
join();
} catch (InterruptedException e) {
log.warn("error stopping refresh thread", e);
}
}
}
}

View File

@@ -34,8 +34,7 @@ import java.io.IOException;
/** Piece of the change index that is implemented as a separate Lucene index. */
class SubIndex {
private static final Logger log =
LoggerFactory.getLogger(LuceneChangeIndex.class);
private static final Logger log = LoggerFactory.getLogger(SubIndex.class);
private final Directory dir;
private final IndexWriter writer;
@@ -57,7 +56,7 @@ class SubIndex {
log.warn("error closing Lucene searcher", e);
}
try {
writer.close(true);
writer.close();
} catch (IOException e) {
log.warn("error closing Lucene writer", e);
}
@@ -70,17 +69,14 @@ class SubIndex {
void insert(Document doc) throws IOException {
writer.addDocument(doc);
commit();
}
void replace(Term term, Document doc) throws IOException {
writer.updateDocument(term, doc);
commit();
}
void delete(Term term) throws IOException {
writer.deleteDocuments(term);
commit();
}
IndexSearcher acquire() throws IOException {
@@ -91,8 +87,11 @@ class SubIndex {
searcherManager.release(searcher);
}
private void commit() throws IOException {
writer.commit();
searcherManager.maybeRefresh();
void maybeRefresh() {
try {
searcherManager.maybeRefresh();
} catch (IOException e) {
log.warn("error refreshing indexer", e);
}
}
}