Decouple locking from Directory: LUCENE-635

git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@437897 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Yonik Seeley 2006-08-29 01:13:10 +00:00
parent 9f374d9202
commit 254945aa32
12 changed files with 2115 additions and 105 deletions

View File

@ -66,6 +66,17 @@ API Changes
not related to Lucene.
(Simon Willnauer via Daniel Naber)
8. LUCENE-635: Decoupling locking implementation from Directory
implementation. Added set/getLockFactory to Directory and moved
all locking code into subclasses of abstract class LockFactory.
FSDirectory and RAMDirectory still default to their prior locking
implementations, but now you can mix & match, for example using
SingleInstanceLockFactory (ie, in memory locking) locking with an
FSDirectory. Note that now you must call setDisableLocks before
the instantiation a FSDirectory if you wish to disable locking
for that Directory.
(Michael McCandless, Jeff Patterson via Yonik Seeley)
Bug fixes
1. Fixed the web application demo (built with "ant war-demo") which

View File

@ -0,0 +1,916 @@
package org.apache.lucene.index;
/**
* Copyright 2004 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.search.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.RAMDirectory;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.util.Vector;
/**
An IndexWriter creates and maintains an index.
The third argument to the
<a href="#IndexWriter(org.apache.lucene.store.Directory, org.apache.lucene.analysis.Analyzer, boolean)"><b>constructor</b></a>
determines whether a new index is created, or whether an existing index is
opened for the addition of new documents.
In either case, documents are added with the <a
href="#addDocument(org.apache.lucene.document.Document)"><b>addDocument</b></a> method.
When finished adding documents, <a href="#close()"><b>close</b></a> should be called.
<p>If an index will not have more documents added for a while and optimal search
performance is desired, then the <a href="#optimize()"><b>optimize</b></a>
method should be called before the index is closed.
<p>Opening an IndexWriter creates a lock file for the directory in use. Trying to open
another IndexWriter on the same directory will lead to an IOException. The IOException
is also thrown if an IndexReader on the same directory is used to delete documents
from the index.
@see IndexModifier IndexModifier supports the important methods of IndexWriter plus deletion
*/
public class IndexWriter {
/**
* Default value for the write lock timeout (1,000).
* @see #setDefaultWriteLockTimeout
*/
public static long WRITE_LOCK_TIMEOUT = 1000;
private long writeLockTimeout = WRITE_LOCK_TIMEOUT;
/**
* Default value for the commit lock timeout (10,000).
* @see #setDefaultCommitLockTimeout
*/
public static long COMMIT_LOCK_TIMEOUT = 10000;
private long commitLockTimeout = COMMIT_LOCK_TIMEOUT;
public static final String WRITE_LOCK_NAME = "write.lock";
public static final String COMMIT_LOCK_NAME = "commit.lock";
/**
* Default value is 10. Change using {@link #setMergeFactor(int)}.
*/
public final static int DEFAULT_MERGE_FACTOR = 10;
/**
* Default value is 10. Change using {@link #setMaxBufferedDocs(int)}.
*/
public final static int DEFAULT_MAX_BUFFERED_DOCS = 10;
/**
* Default value is {@link Integer#MAX_VALUE}. Change using {@link #setMaxMergeDocs(int)}.
*/
public final static int DEFAULT_MAX_MERGE_DOCS = Integer.MAX_VALUE;
/**
* Default value is 10,000. Change using {@link #setMaxFieldLength(int)}.
*/
public final static int DEFAULT_MAX_FIELD_LENGTH = 10000;
/**
* Default value is 128. Change using {@link #setTermIndexInterval(int)}.
*/
public final static int DEFAULT_TERM_INDEX_INTERVAL = 128;
private Directory directory; // where this index resides
private Analyzer analyzer; // how to analyze text
private Similarity similarity = Similarity.getDefault(); // how to normalize
private SegmentInfos segmentInfos = new SegmentInfos(); // the segments
private final Directory ramDirectory = new RAMDirectory(); // for temp segs
private int singleDocSegmentsCount = 0; // for speeding decision on merge candidates
private Lock writeLock;
private int termIndexInterval = DEFAULT_TERM_INDEX_INTERVAL;
/** Use compound file setting. Defaults to true, minimizing the number of
* files used. Setting this to false may improve indexing performance, but
* may also cause file handle problems.
*/
private boolean useCompoundFile = true;
private boolean closeDir;
/** Get the current setting of whether to use the compound file format.
* Note that this just returns the value you set with setUseCompoundFile(boolean)
* or the default. You cannot use this to query the status of an existing index.
* @see #setUseCompoundFile(boolean)
*/
public boolean getUseCompoundFile() {
return useCompoundFile;
}
/** Setting to turn on usage of a compound file. When on, multiple files
* for each segment are merged into a single file once the segment creation
* is finished. This is done regardless of what directory is in use.
*/
public void setUseCompoundFile(boolean value) {
useCompoundFile = value;
}
/** Expert: Set the Similarity implementation used by this IndexWriter.
*
* @see Similarity#setDefault(Similarity)
*/
public void setSimilarity(Similarity similarity) {
this.similarity = similarity;
}
/** Expert: Return the Similarity implementation used by this IndexWriter.
*
* <p>This defaults to the current value of {@link Similarity#getDefault()}.
*/
public Similarity getSimilarity() {
return this.similarity;
}
/** Expert: Set the interval between indexed terms. Large values cause less
* memory to be used by IndexReader, but slow random-access to terms. Small
* values cause more memory to be used by an IndexReader, and speed
* random-access to terms.
*
* This parameter determines the amount of computation required per query
* term, regardless of the number of documents that contain that term. In
* particular, it is the maximum number of other terms that must be
* scanned before a term is located and its frequency and position information
* may be processed. In a large index with user-entered query terms, query
* processing time is likely to be dominated not by term lookup but rather
* by the processing of frequency and positional data. In a small index
* or when many uncommon query terms are generated (e.g., by wildcard
* queries) term lookup may become a dominant cost.
*
* In particular, <code>numUniqueTerms/interval</code> terms are read into
* memory by an IndexReader, and, on average, <code>interval/2</code> terms
* must be scanned for each random term access.
*
* @see #DEFAULT_TERM_INDEX_INTERVAL
*/
public void setTermIndexInterval(int interval) {
this.termIndexInterval = interval;
}
/** Expert: Return the interval between indexed terms.
*
* @see #setTermIndexInterval(int)
*/
public int getTermIndexInterval() { return termIndexInterval; }
/**
* Constructs an IndexWriter for the index in <code>path</code>.
* Text will be analyzed with <code>a</code>. If <code>create</code>
* is true, then a new, empty index will be created in
* <code>path</code>, replacing the index already there, if any.
*
* @param path the path to the index directory
* @param a the analyzer to use
* @param create <code>true</code> to create the index or overwrite
* the existing one; <code>false</code> to append to the existing
* index
* @throws IOException if the directory cannot be read/written to, or
* if it does not exist, and <code>create</code> is
* <code>false</code>
*/
public IndexWriter(String path, Analyzer a, boolean create)
throws IOException {
this(FSDirectory.getDirectory(path, create), a, create, true);
}
/**
* Constructs an IndexWriter for the index in <code>path</code>.
* Text will be analyzed with <code>a</code>. If <code>create</code>
* is true, then a new, empty index will be created in
* <code>path</code>, replacing the index already there, if any.
*
* @param path the path to the index directory
* @param a the analyzer to use
* @param create <code>true</code> to create the index or overwrite
* the existing one; <code>false</code> to append to the existing
* index
* @throws IOException if the directory cannot be read/written to, or
* if it does not exist, and <code>create</code> is
* <code>false</code>
*/
public IndexWriter(File path, Analyzer a, boolean create)
throws IOException {
this(FSDirectory.getDirectory(path, create), a, create, true);
}
/**
* Constructs an IndexWriter for the index in <code>d</code>.
* Text will be analyzed with <code>a</code>. If <code>create</code>
* is true, then a new, empty index will be created in
* <code>d</code>, replacing the index already there, if any.
*
* @param d the index directory
* @param a the analyzer to use
* @param create <code>true</code> to create the index or overwrite
* the existing one; <code>false</code> to append to the existing
* index
* @throws IOException if the directory cannot be read/written to, or
* if it does not exist, and <code>create</code> is
* <code>false</code>
*/
public IndexWriter(Directory d, Analyzer a, boolean create)
throws IOException {
this(d, a, create, false);
}
private IndexWriter(Directory d, Analyzer a, final boolean create, boolean closeDir)
throws IOException {
this.closeDir = closeDir;
directory = d;
analyzer = a;
Lock writeLock = directory.makeLock(IndexWriter.WRITE_LOCK_NAME);
if (!writeLock.obtain(writeLockTimeout)) // obtain write lock
throw new IOException("Index locked for write: " + writeLock);
this.writeLock = writeLock; // save it
synchronized (directory) { // in- & inter-process sync
new Lock.With(directory.makeLock(IndexWriter.COMMIT_LOCK_NAME), commitLockTimeout) {
public Object doBody() throws IOException {
if (create)
segmentInfos.write(directory);
else
segmentInfos.read(directory);
return null;
}
}.run();
}
}
/** Determines the largest number of documents ever merged by addDocument().
* Small values (e.g., less than 10,000) are best for interactive indexing,
* as this limits the length of pauses while indexing to a few seconds.
* Larger values are best for batched indexing and speedier searches.
*
* <p>The default value is {@link Integer#MAX_VALUE}.
*/
public void setMaxMergeDocs(int maxMergeDocs) {
this.maxMergeDocs = maxMergeDocs;
}
/**
* @see #setMaxMergeDocs
*/
public int getMaxMergeDocs() {
return maxMergeDocs;
}
/**
* The maximum number of terms that will be indexed for a single field in a
* document. This limits the amount of memory required for indexing, so that
* collections with very large files will not crash the indexing process by
* running out of memory.<p/>
* Note that this effectively truncates large documents, excluding from the
* index terms that occur further in the document. If you know your source
* documents are large, be sure to set this value high enough to accomodate
* the expected size. If you set it to Integer.MAX_VALUE, then the only limit
* is your memory, but you should anticipate an OutOfMemoryError.<p/>
* By default, no more than 10,000 terms will be indexed for a field.
*/
public void setMaxFieldLength(int maxFieldLength) {
this.maxFieldLength = maxFieldLength;
}
/**
* @see #setMaxFieldLength
*/
public int getMaxFieldLength() {
return maxFieldLength;
}
/** Determines the minimal number of documents required before the buffered
* in-memory documents are merging and a new Segment is created.
* Since Documents are merged in a {@link org.apache.lucene.store.RAMDirectory},
* large value gives faster indexing. At the same time, mergeFactor limits
* the number of files open in a FSDirectory.
*
* <p> The default value is 10.
*
* @throws IllegalArgumentException if maxBufferedDocs is smaller than 2
*/
public void setMaxBufferedDocs(int maxBufferedDocs) {
if (maxBufferedDocs < 2)
throw new IllegalArgumentException("maxBufferedDocs must at least be 2");
this.minMergeDocs = maxBufferedDocs;
}
/**
* @see #setMaxBufferedDocs
*/
public int getMaxBufferedDocs() {
return minMergeDocs;
}
/** Determines how often segment indices are merged by addDocument(). With
* smaller values, less RAM is used while indexing, and searches on
* unoptimized indices are faster, but indexing speed is slower. With larger
* values, more RAM is used during indexing, and while searches on unoptimized
* indices are slower, indexing is faster. Thus larger values (> 10) are best
* for batch index creation, and smaller values (< 10) for indices that are
* interactively maintained.
*
* <p>This must never be less than 2. The default value is 10.
*/
public void setMergeFactor(int mergeFactor) {
if (mergeFactor < 2)
throw new IllegalArgumentException("mergeFactor cannot be less than 2");
this.mergeFactor = mergeFactor;
}
/**
* @see #setMergeFactor
*/
public int getMergeFactor() {
return mergeFactor;
}
/** If non-null, information about merges and a message when
* maxFieldLength is reached will be printed to this.
*/
public void setInfoStream(PrintStream infoStream) {
this.infoStream = infoStream;
}
/**
* @see #setInfoStream
*/
public PrintStream getInfoStream() {
return infoStream;
}
/**
* Sets the maximum time to wait for a commit lock (in milliseconds) for this instance of IndexWriter. @see
* @see #setDefaultCommitLockTimeout to change the default value for all instances of IndexWriter.
*/
public void setCommitLockTimeout(long commitLockTimeout) {
this.commitLockTimeout = commitLockTimeout;
}
/**
* @see #setCommitLockTimeout
*/
public long getCommitLockTimeout() {
return commitLockTimeout;
}
/**
* Sets the default (for any instance of IndexWriter) maximum time to wait for a commit lock (in milliseconds)
*/
public static void setDefaultCommitLockTimeout(long commitLockTimeout) {
IndexWriter.COMMIT_LOCK_TIMEOUT = commitLockTimeout;
}
/**
* @see #setDefaultCommitLockTimeout
*/
public static long getDefaultCommitLockTimeout() {
return IndexWriter.COMMIT_LOCK_TIMEOUT;
}
/**
* Sets the maximum time to wait for a write lock (in milliseconds) for this instance of IndexWriter. @see
* @see #setDefaultWriteLockTimeout to change the default value for all instances of IndexWriter.
*/
public void setWriteLockTimeout(long writeLockTimeout) {
this.writeLockTimeout = writeLockTimeout;
}
/**
* @see #setWriteLockTimeout
*/
public long getWriteLockTimeout() {
return writeLockTimeout;
}
/**
* Sets the default (for any instance of IndexWriter) maximum time to wait for a write lock (in
* milliseconds).
*/
public static void setDefaultWriteLockTimeout(long writeLockTimeout) {
IndexWriter.WRITE_LOCK_TIMEOUT = writeLockTimeout;
}
/**
* @see #setDefaultWriteLockTimeout
*/
public static long getDefaultWriteLockTimeout() {
return IndexWriter.WRITE_LOCK_TIMEOUT;
}
/** Flushes all changes to an index and closes all associated files. */
public synchronized void close() throws IOException {
flushRamSegments();
// testInvariants();
ramDirectory.close();
if (writeLock != null) {
writeLock.release(); // release write lock
writeLock = null;
}
if(closeDir)
directory.close();
}
/** Release the write lock, if needed. */
protected void finalize() throws IOException {
if (writeLock != null) {
writeLock.release(); // release write lock
writeLock = null;
}
}
/** Returns the Directory used by this index. */
public Directory getDirectory() {
return directory;
}
/** Returns the analyzer used by this index. */
public Analyzer getAnalyzer() {
return analyzer;
}
/** Returns the number of documents currently in this index. */
public synchronized int docCount() {
int count = 0;
for (int i = 0; i < segmentInfos.size(); i++) {
SegmentInfo si = segmentInfos.info(i);
count += si.docCount;
}
return count;
}
/**
* The maximum number of terms that will be indexed for a single field in a
* document. This limits the amount of memory required for indexing, so that
* collections with very large files will not crash the indexing process by
* running out of memory.<p/>
* Note that this effectively truncates large documents, excluding from the
* index terms that occur further in the document. If you know your source
* documents are large, be sure to set this value high enough to accomodate
* the expected size. If you set it to Integer.MAX_VALUE, then the only limit
* is your memory, but you should anticipate an OutOfMemoryError.<p/>
* By default, no more than 10,000 terms will be indexed for a field.
*
*/
private int maxFieldLength = DEFAULT_MAX_FIELD_LENGTH;
/**
* Adds a document to this index. If the document contains more than
* {@link #setMaxFieldLength(int)} terms for a given field, the remainder are
* discarded.
*/
public void addDocument(Document doc) throws IOException {
addDocument(doc, analyzer);
}
/**
* Adds a document to this index, using the provided analyzer instead of the
* value of {@link #getAnalyzer()}. If the document contains more than
* {@link #setMaxFieldLength(int)} terms for a given field, the remainder are
* discarded.
*/
public void addDocument(Document doc, Analyzer analyzer) throws IOException {
DocumentWriter dw =
new DocumentWriter(ramDirectory, analyzer, this);
dw.setInfoStream(infoStream);
String segmentName = newSegmentName();
dw.addDocument(segmentName, doc);
synchronized (this) {
segmentInfos.addElement(new SegmentInfo(segmentName, 1, ramDirectory));
singleDocSegmentsCount++;
maybeMergeSegments();
}
// testInvariants();
}
final int getSegmentsCounter(){
return segmentInfos.counter;
}
private final synchronized String newSegmentName() {
return "_" + Integer.toString(segmentInfos.counter++, Character.MAX_RADIX);
}
/** Determines how often segment indices are merged by addDocument(). With
* smaller values, less RAM is used while indexing, and searches on
* unoptimized indices are faster, but indexing speed is slower. With larger
* values, more RAM is used during indexing, and while searches on unoptimized
* indices are slower, indexing is faster. Thus larger values (> 10) are best
* for batch index creation, and smaller values (< 10) for indices that are
* interactively maintained.
*
* <p>This must never be less than 2. The default value is {@link #DEFAULT_MERGE_FACTOR}.
*/
private int mergeFactor = DEFAULT_MERGE_FACTOR;
/** Determines the minimal number of documents required before the buffered
* in-memory documents are merging and a new Segment is created.
* Since Documents are merged in a {@link org.apache.lucene.store.RAMDirectory},
* large value gives faster indexing. At the same time, mergeFactor limits
* the number of files open in a FSDirectory.
*
* <p> The default value is {@link #DEFAULT_MAX_BUFFERED_DOCS}.
*/
private int minMergeDocs = DEFAULT_MAX_BUFFERED_DOCS;
/** Determines the largest number of documents ever merged by addDocument().
* Small values (e.g., less than 10,000) are best for interactive indexing,
* as this limits the length of pauses while indexing to a few seconds.
* Larger values are best for batched indexing and speedier searches.
*
* <p>The default value is {@link #DEFAULT_MAX_MERGE_DOCS}.
*/
private int maxMergeDocs = DEFAULT_MAX_MERGE_DOCS;
/** If non-null, information about merges will be printed to this.
*/
private PrintStream infoStream = null;
/** Merges all segments together into a single segment, optimizing an index
for search. */
public synchronized void optimize() throws IOException {
flushRamSegments();
while (segmentInfos.size() > 1 ||
(segmentInfos.size() == 1 &&
(SegmentReader.hasDeletions(segmentInfos.info(0)) ||
segmentInfos.info(0).dir != directory ||
(useCompoundFile &&
(!SegmentReader.usesCompoundFile(segmentInfos.info(0)) ||
SegmentReader.hasSeparateNorms(segmentInfos.info(0))))))) {
int minSegment = segmentInfos.size() - mergeFactor;
mergeSegments(minSegment < 0 ? 0 : minSegment);
}
// testInvariants();
}
/** Merges all segments from an array of indexes into this index.
*
* <p>This may be used to parallelize batch indexing. A large document
* collection can be broken into sub-collections. Each sub-collection can be
* indexed in parallel, on a different thread, process or machine. The
* complete index can then be created by merging sub-collection indexes
* with this method.
*
* <p>After this completes, the index is optimized. */
public synchronized void addIndexes(Directory[] dirs)
throws IOException {
optimize(); // start with zero or 1 seg
int start = segmentInfos.size();
for (int i = 0; i < dirs.length; i++) {
SegmentInfos sis = new SegmentInfos(); // read infos from dir
sis.read(dirs[i]);
for (int j = 0; j < sis.size(); j++) {
segmentInfos.addElement(sis.info(j)); // add each info
}
}
// merge newly added segments in log(n) passes
while (segmentInfos.size() > start+mergeFactor) {
for (int base = start; base < segmentInfos.size(); base++) {
int end = Math.min(segmentInfos.size(), base+mergeFactor);
if (end-base > 1)
mergeSegments(base, end);
}
}
optimize(); // final cleanup
// testInvariants();
}
/** Merges the provided indexes into this index.
* <p>After this completes, the index is optimized. </p>
* <p>The provided IndexReaders are not closed.</p>
*/
public synchronized void addIndexes(IndexReader[] readers)
throws IOException {
optimize(); // start with zero or 1 seg
final String mergedName = newSegmentName();
SegmentMerger merger = new SegmentMerger(this, mergedName);
final Vector segmentsToDelete = new Vector();
IndexReader sReader = null;
if (segmentInfos.size() == 1){ // add existing index, if any
sReader = SegmentReader.get(segmentInfos.info(0));
merger.add(sReader);
segmentsToDelete.addElement(sReader); // queue segment for deletion
}
for (int i = 0; i < readers.length; i++) // add new indexes
merger.add(readers[i]);
int docCount = merger.merge(); // merge 'em
segmentInfos.setSize(0); // pop old infos & add new
segmentInfos.addElement(new SegmentInfo(mergedName, docCount, directory));
if(sReader != null)
sReader.close();
synchronized (directory) { // in- & inter-process sync
new Lock.With(directory.makeLock(COMMIT_LOCK_NAME), commitLockTimeout) {
public Object doBody() throws IOException {
segmentInfos.write(directory); // commit changes
return null;
}
}.run();
}
deleteSegments(segmentsToDelete); // delete now-unused segments
if (useCompoundFile) {
final Vector filesToDelete = merger.createCompoundFile(mergedName + ".tmp");
synchronized (directory) { // in- & inter-process sync
new Lock.With(directory.makeLock(COMMIT_LOCK_NAME), commitLockTimeout) {
public Object doBody() throws IOException {
// make compound file visible for SegmentReaders
directory.renameFile(mergedName + ".tmp", mergedName + ".cfs");
return null;
}
}.run();
}
// delete now unused files of segment
deleteFiles(filesToDelete);
}
// testInvariants();
}
/** Merges all RAM-resident segments. */
private final void flushRamSegments() throws IOException {
int minSegment = segmentInfos.size()-1;
int docCount = 0;
while (minSegment >= 0 &&
(segmentInfos.info(minSegment)).dir == ramDirectory) {
docCount += segmentInfos.info(minSegment).docCount;
minSegment--;
}
if (minSegment < 0 || // add one FS segment?
(docCount + segmentInfos.info(minSegment).docCount) > mergeFactor ||
!(segmentInfos.info(segmentInfos.size()-1).dir == ramDirectory))
minSegment++;
if (minSegment >= segmentInfos.size())
return; // none to merge
mergeSegments(minSegment);
}
/** Incremental segment merger. */
private final void maybeMergeSegments() throws IOException {
long targetMergeDocs = minMergeDocs;
while (targetMergeDocs <= maxMergeDocs) {
// find segments smaller than current target size
int minSegment = segmentInfos.size() - singleDocSegmentsCount; // top 1-doc segments are taken for sure
int mergeDocs = singleDocSegmentsCount;
while (--minSegment >= 0) {
SegmentInfo si = segmentInfos.info(minSegment);
if (si.docCount >= targetMergeDocs)
break;
mergeDocs += si.docCount;
}
if (mergeDocs >= targetMergeDocs) { // found a merge to do
mergeSegments(minSegment+1);
singleDocSegmentsCount = 0;
} else {
break;
}
targetMergeDocs *= mergeFactor; // increase target size
}
}
/** Pops segments off of segmentInfos stack down to minSegment, merges them,
and pushes the merged index onto the top of the segmentInfos stack. */
private final void mergeSegments(int minSegment)
throws IOException {
mergeSegments(minSegment, segmentInfos.size());
}
/** Merges the named range of segments, replacing them in the stack with a
* single segment. */
private final void mergeSegments(int minSegment, int end)
throws IOException {
final String mergedName = newSegmentName();
if (infoStream != null) infoStream.print("merging segments");
SegmentMerger merger = new SegmentMerger(this, mergedName);
final Vector segmentsToDelete = new Vector();
for (int i = minSegment; i < end; i++) {
SegmentInfo si = segmentInfos.info(i);
if (infoStream != null)
infoStream.print(" " + si.name + " (" + si.docCount + " docs)");
IndexReader reader = SegmentReader.get(si);
merger.add(reader);
if ((reader.directory() == this.directory) || // if we own the directory
(reader.directory() == this.ramDirectory))
segmentsToDelete.addElement(reader); // queue segment for deletion
}
int mergedDocCount = merger.merge();
if (infoStream != null) {
infoStream.println(" into "+mergedName+" ("+mergedDocCount+" docs)");
}
for (int i = end-1; i > minSegment; i--) // remove old infos & add new
segmentInfos.remove(i);
segmentInfos.set(minSegment, new SegmentInfo(mergedName, mergedDocCount,
directory));
// close readers before we attempt to delete now-obsolete segments
merger.closeReaders();
synchronized (directory) { // in- & inter-process sync
new Lock.With(directory.makeLock(COMMIT_LOCK_NAME), commitLockTimeout) {
public Object doBody() throws IOException {
segmentInfos.write(directory); // commit before deleting
return null;
}
}.run();
}
deleteSegments(segmentsToDelete); // delete now-unused segments
if (useCompoundFile) {
final Vector filesToDelete = merger.createCompoundFile(mergedName + ".tmp");
synchronized (directory) { // in- & inter-process sync
new Lock.With(directory.makeLock(COMMIT_LOCK_NAME), commitLockTimeout) {
public Object doBody() throws IOException {
// make compound file visible for SegmentReaders
directory.renameFile(mergedName + ".tmp", mergedName + ".cfs");
return null;
}
}.run();
}
// delete now unused files of segment
deleteFiles(filesToDelete);
}
}
/***
private synchronized void testInvariants() {
// index segments should decrease in size
int maxSegLevel = 0;
for (int i=segmentInfos.size()-1; i>=0; i--) {
SegmentInfo si = segmentInfos.info(i);
int segLevel = (si.docCount)/minMergeDocs;
if (segLevel < maxSegLevel) {
throw new RuntimeException("Segment #" + i + " is too small. " + segInfo());
}
maxSegLevel = Math.max(maxSegLevel,segLevel);
}
// check if merges needed
long targetMergeDocs = minMergeDocs;
int minSegment = segmentInfos.size();
while (targetMergeDocs <= maxMergeDocs && minSegment>=0) {
int mergeDocs = 0;
while (--minSegment >= 0) {
SegmentInfo si = segmentInfos.info(minSegment);
if (si.docCount >= targetMergeDocs) break;
mergeDocs += si.docCount;
}
if (mergeDocs >= targetMergeDocs) {
throw new RuntimeException("Merge needed at level "+targetMergeDocs + " :"+segInfo());
}
targetMergeDocs *= mergeFactor; // increase target size
}
}
private String segInfo() {
StringBuffer sb = new StringBuffer("minMergeDocs="+minMergeDocs+" singleDocSegmentsCount="+singleDocSegmentsCount+" segsizes:");
for (int i=0; i<segmentInfos.size(); i++) {
sb.append(segmentInfos.info(i).docCount);
sb.append(",");
}
return sb.toString();
}
***/
/*
* Some operating systems (e.g. Windows) don't permit a file to be deleted
* while it is opened for read (e.g. by another process or thread). So we
* assume that when a delete fails it is because the file is open in another
* process, and queue the file for subsequent deletion.
*/
private final void deleteSegments(Vector segments) throws IOException {
Vector deletable = new Vector();
deleteFiles(readDeleteableFiles(), deletable); // try to delete deleteable
for (int i = 0; i < segments.size(); i++) {
SegmentReader reader = (SegmentReader)segments.elementAt(i);
if (reader.directory() == this.directory)
deleteFiles(reader.files(), deletable); // try to delete our files
else
deleteFiles(reader.files(), reader.directory()); // delete other files
}
writeDeleteableFiles(deletable); // note files we can't delete
}
private final void deleteFiles(Vector files) throws IOException {
Vector deletable = new Vector();
deleteFiles(readDeleteableFiles(), deletable); // try to delete deleteable
deleteFiles(files, deletable); // try to delete our files
writeDeleteableFiles(deletable); // note files we can't delete
}
private final void deleteFiles(Vector files, Directory directory)
throws IOException {
for (int i = 0; i < files.size(); i++)
directory.deleteFile((String)files.elementAt(i));
}
private final void deleteFiles(Vector files, Vector deletable)
throws IOException {
for (int i = 0; i < files.size(); i++) {
String file = (String)files.elementAt(i);
try {
directory.deleteFile(file); // try to delete each file
} catch (IOException e) { // if delete fails
if (directory.fileExists(file)) {
if (infoStream != null)
infoStream.println(e.toString() + "; Will re-try later.");
deletable.addElement(file); // add to deletable
}
}
}
}
private final Vector readDeleteableFiles() throws IOException {
Vector result = new Vector();
if (!directory.fileExists(IndexFileNames.DELETABLE))
return result;
IndexInput input = directory.openInput(IndexFileNames.DELETABLE);
try {
for (int i = input.readInt(); i > 0; i--) // read file names
result.addElement(input.readString());
} finally {
input.close();
}
return result;
}
private final void writeDeleteableFiles(Vector files) throws IOException {
IndexOutput output = directory.createOutput("deleteable.new");
try {
output.writeInt(files.size());
for (int i = 0; i < files.size(); i++)
output.writeString((String)files.elementAt(i));
} finally {
output.close();
}
directory.renameFile("deleteable.new", IndexFileNames.DELETABLE);
}
}

View File

@ -0,0 +1,31 @@
Index: IndexWriter.java
===================================================================
--- IndexWriter.java (revision 432381)
+++ IndexWriter.java (working copy)
@@ -711,12 +711,10 @@
mergeDocs += si.docCount;
}
- if (mergeDocs >= targetMergeDocs) { // found a merge to do
+ if (mergeDocs >= targetMergeDocs) // found a merge to do
mergeSegments(minSegment+1);
- singleDocSegmentsCount = 0;
- } else {
+ else
break;
- }
targetMergeDocs *= mergeFactor; // increase target size
}
@@ -749,6 +747,11 @@
segmentsToDelete.addElement(reader); // queue segment for deletion
}
+ // update 1-doc segments counter accordin to range of merged segments
+ if (singleDocSegmentsCount>0) {
+ singleDocSegmentsCount = Math.min(singleDocSegmentsCount, segmentInfos.size()-end);
+ }
+
int mergedDocCount = merger.merge();
if (infoStream != null) {

View File

@ -29,9 +29,18 @@ import java.io.IOException;
* <li> implementation of an index as a single file;
* </ul>
*
* Directory locking is implemented by an instance of {@link
* LockFactory}, and can be changed for each Directory
* instance using {@link #setLockFactory}.
*
* @author Doug Cutting
*/
public abstract class Directory {
/** Holds the LockFactory instance (implements locking for
* this Directory instance). */
protected LockFactory lockFactory;
/** Returns an array of strings, one for each file in the directory. */
public abstract String[] list()
throws IOException;
@ -75,9 +84,43 @@ public abstract class Directory {
/** Construct a {@link Lock}.
* @param name the name of the lock file
*/
public abstract Lock makeLock(String name);
public Lock makeLock(String name) {
return lockFactory.makeLock(name);
}
/** Closes the store. */
public abstract void close()
throws IOException;
/**
* Set the LockFactory that this Directory instance should
* use for its locking implementation. Each * instance of
* LockFactory should only be used for one directory (ie,
* do not share a single instance across multiple
* Directories).
*
* @param lockFactory instance of {@link LockFactory}.
*/
public void setLockFactory(LockFactory lockFactory) {
this.lockFactory = lockFactory;
lockFactory.setLockPrefix(this.getLockID());
}
/**
* Get the LockFactory that this Directory instance is using for its locking implementation.
*/
public LockFactory getLockFactory() {
return this.lockFactory;
}
/**
* Return a string identifier that uniquely differentiates
* this Directory instance from other Directory instances.
* This ID should be the same if two Directory instances
* (even in different JVMs and/or on different machines)
* are considered "the same index". This is how locking
* "scopes" to the right index.
*/
public String getLockID() {
return this.toString();
}
}

View File

@ -29,6 +29,12 @@ import org.apache.lucene.index.IndexFileNameFilter;
/**
* Straightforward implementation of {@link Directory} as a directory of files.
* Locking implementation is by default the {@link SimpleFSLockFactory}, but
* can be changed either by passing in a {@link LockFactory} instance to
* <code>getDirectory</code>, or specifying the LockFactory class by setting
* <code>org.apache.lucene.store.FSDirectoryLockFactoryClass</code> Java system
* property, or by calling {@link #setLockFactory} after creating
* the Directory.
*
* @see Directory
* @author Doug Cutting
@ -46,6 +52,9 @@ public class FSDirectory extends Directory {
private static boolean disableLocks = false;
// TODO: should this move up to the Directory base class? Also: should we
// make a per-instance (in addition to the static "default") version?
/**
* Set whether Lucene's use of lock files is disabled. By default,
* lock files are enabled. They should only be disabled if the index
@ -63,13 +72,17 @@ public class FSDirectory extends Directory {
return FSDirectory.disableLocks;
}
// TODO: LOCK_DIR really should only appear in the SimpleFSLockFactory
// (and any other file-system based locking implementations). When we
// can next break backwards compatibility we should deprecate it and then
// move it.
/**
* Directory specified by <code>org.apache.lucene.lockDir</code>
* or <code>java.io.tmpdir</code> system property
* or <code>java.io.tmpdir</code> system property. This may be deprecated in the future. Please use
* {@link SimpleFSLockFactory#LOCK_DIR} instead.
*/
public static final String LOCK_DIR =
System.getProperty("org.apache.lucene.lockDir",
System.getProperty("java.io.tmpdir"));
public static final String LOCK_DIR = SimpleFSLockFactory.LOCK_DIR;
/** The default class which implements filesystem-based directories. */
private static Class IMPL;
@ -114,7 +127,25 @@ public class FSDirectory extends Directory {
* @return the FSDirectory for the named file. */
public static FSDirectory getDirectory(String path, boolean create)
throws IOException {
return getDirectory(new File(path), create);
return getDirectory(path, create, null);
}
/** Returns the directory instance for the named location, using the
* provided LockFactory implementation.
*
* <p>Directories are cached, so that, for a given canonical path, the same
* FSDirectory instance will always be returned. This permits
* synchronization on directories.
*
* @param path the path to the directory.
* @param create if true, create, or erase any existing contents.
* @param lockFactory instance of {@link LockFactory} providing the
* locking implementation.
* @return the FSDirectory for the named file. */
public static FSDirectory getDirectory(String path, boolean create,
LockFactory lockFactory)
throws IOException {
return getDirectory(new File(path), create, lockFactory);
}
/** Returns the directory instance for the named location.
@ -128,6 +159,24 @@ public class FSDirectory extends Directory {
* @return the FSDirectory for the named file. */
public static FSDirectory getDirectory(File file, boolean create)
throws IOException {
return getDirectory(file, create, null);
}
/** Returns the directory instance for the named location, using the
* provided LockFactory implementation.
*
* <p>Directories are cached, so that, for a given canonical path, the same
* FSDirectory instance will always be returned. This permits
* synchronization on directories.
*
* @param file the path to the directory.
* @param create if true, create, or erase any existing contents.
* @param lockFactory instance of {@link LockFactory} providing the
* locking implementation.
* @return the FSDirectory for the named file. */
public static FSDirectory getDirectory(File file, boolean create,
LockFactory lockFactory)
throws IOException {
file = new File(file.getCanonicalPath());
FSDirectory dir;
synchronized (DIRECTORIES) {
@ -138,10 +187,19 @@ public class FSDirectory extends Directory {
} catch (Exception e) {
throw new RuntimeException("cannot load FSDirectory class: " + e.toString(), e);
}
dir.init(file, create);
dir.init(file, create, lockFactory);
DIRECTORIES.put(file, dir);
} else if (create) {
dir.create();
} else {
// Catch the case where a Directory is pulled from the cache, but has a
// different LockFactory instance.
if (lockFactory != null && lockFactory != dir.getLockFactory()) {
throw new IOException("Directory was previously created with a different LockFactory instance; please pass null as the lockFactory instance and use setLockFactory to change it");
}
if (create) {
dir.create();
}
}
}
synchronized (dir) {
@ -152,27 +210,12 @@ public class FSDirectory extends Directory {
private File directory = null;
private int refCount;
private File lockDir;
protected FSDirectory() {}; // permit subclassing
private void init(File path, boolean create) throws IOException {
directory = path;
if (LOCK_DIR == null) {
lockDir = directory;
}
else {
lockDir = new File(LOCK_DIR);
}
// Ensure that lockDir exists and is a directory.
if (!lockDir.exists()) {
if (!lockDir.mkdirs())
throw new IOException("Cannot create directory: " + lockDir.getAbsolutePath());
} else if (!lockDir.isDirectory()) {
throw new IOException("Found regular file where directory expected: " +
lockDir.getAbsolutePath());
}
if (create) {
create();
}
@ -181,6 +224,64 @@ public class FSDirectory extends Directory {
throw new IOException(path + " not a directory");
}
private void init(File path, boolean create, LockFactory lockFactory) throws IOException {
// Set up lockFactory with cascaded defaults: if an instance was passed in,
// use that; else if locks are disabled, use NoLockFactory; else if the
// system property org.apache.lucene.lockClass is set, instantiate that;
// else, use SimpleFSLockFactory:
if (lockFactory == null) {
if (disableLocks) {
// Locks are disabled:
lockFactory = NoLockFactory.getNoLockFactory();
} else {
String lockClassName = System.getProperty("org.apache.lucene.store.FSDirectoryLockFactoryClass");
if (lockClassName != null) {
Class c;
try {
c = Class.forName(lockClassName);
} catch (ClassNotFoundException e) {
throw new IOException("unable to find LockClass " + lockClassName);
}
try {
lockFactory = (LockFactory) c.newInstance();
} catch (IllegalAccessException e) {
throw new IOException("IllegalAccessException when instantiating LockClass " + lockClassName);
} catch (InstantiationException e) {
throw new IOException("InstantiationException when instantiating LockClass " + lockClassName);
} catch (ClassCastException e) {
throw new IOException("unable to cast LockClass " + lockClassName + " instance to a LockFactory");
}
} else {
// Our default lock is SimpleFSLockFactory:
File lockDir;
if (LOCK_DIR == null) {
lockDir = directory;
} else {
lockDir = new File(LOCK_DIR);
}
lockFactory = new SimpleFSLockFactory(lockDir);
}
}
}
// Must initialize directory here because setLockFactory uses it
// (when the LockFactory calls getLockID). But we want to create
// the lockFactory before calling init() because init() needs to
// use the lockFactory to clear old locks. So this breaks
// chicken/egg:
directory = path;
setLockFactory(lockFactory);
init(path, create);
}
private synchronized void create() throws IOException {
if (!directory.exists())
if (!directory.mkdirs())
@ -198,17 +299,7 @@ public class FSDirectory extends Directory {
throw new IOException("Cannot delete " + file);
}
String lockPrefix = getLockPrefix().toString(); // clear old locks
files = lockDir.list();
if (files == null)
throw new IOException("Cannot read lock directory " + lockDir.getAbsolutePath());
for (int i = 0; i < files.length; i++) {
if (!files[i].startsWith(lockPrefix))
continue;
File lockFile = new File(lockDir, files[i]);
if (!lockFile.delete())
throw new IOException("Cannot delete " + lockFile);
}
lockFactory.clearAllLocks();
}
/** Returns an array of strings, one for each Lucene index file in the directory. */
@ -338,51 +429,8 @@ public class FSDirectory extends Directory {
private static final char[] HEX_DIGITS =
{'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'};
/** Constructs a {@link Lock} with the specified name. Locks are implemented
* with {@link File#createNewFile()}.
*
* @param name the name of the lock file
* @return an instance of <code>Lock</code> holding the lock
*/
public Lock makeLock(String name) {
StringBuffer buf = getLockPrefix();
buf.append("-");
buf.append(name);
// create a lock file
final File lockFile = new File(lockDir, buf.toString());
return new Lock() {
public boolean obtain() throws IOException {
if (disableLocks)
return true;
if (!lockDir.exists()) {
if (!lockDir.mkdirs()) {
throw new IOException("Cannot create lock directory: " + lockDir);
}
}
return lockFile.createNewFile();
}
public void release() {
if (disableLocks)
return;
lockFile.delete();
}
public boolean isLocked() {
if (disableLocks)
return false;
return lockFile.exists();
}
public String toString() {
return "Lock@" + lockFile;
}
};
}
private StringBuffer getLockPrefix() {
public String getLockID() {
String dirName; // name to be hashed
try {
dirName = directory.getCanonicalPath();
@ -402,7 +450,7 @@ public class FSDirectory extends Directory {
buf.append(HEX_DIGITS[b & 0xf]);
}
return buf;
return buf.toString();
}
/** Closes the store to future operations. */

View File

@ -0,0 +1,56 @@
package org.apache.lucene.store;
/**
* Copyright 2006 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
/**
* Base class for Locking implementation. {@link Directory} uses
* instances of this class to implement locking.
*/
public abstract class LockFactory {
protected String lockPrefix = "";
/**
* Set the prefix in use for all locks created in this LockFactory.
*/
public void setLockPrefix(String lockPrefix) {
this.lockPrefix = lockPrefix;
}
/**
* Get the prefix in use for all locks created in this LockFactory.
*/
public String getLockPrefix() {
return this.lockPrefix;
}
/**
* Return a new Lock instance identified by lockName.
* @param lockName name of the lock to be created.
*/
public abstract Lock makeLock(String lockName);
/**
* Clear any existing locks. Only call this at a time when you
* are certain the lock files are not in use. {@link FSDirectory}
* calls this when creating a new index.
*/
public abstract void clearAllLocks() throws IOException;
}

View File

@ -0,0 +1,62 @@
package org.apache.lucene.store;
/**
* Copyright 2006 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
/**
* Use this {@link LockFactory} to disable locking entirely.
* This LockFactory is used when you call {@link FSDirectory#setDisableLocks}.
* Only one instance of this lock is created. You should call {@link
* #getNoLockFactory()} to get the instance.
*
* @see LockFactory
*/
public class NoLockFactory extends LockFactory {
// Single instance returned whenever makeLock is called.
private static NoLock singletonLock = new NoLock();
private static NoLockFactory singleton = new NoLockFactory();
public static NoLockFactory getNoLockFactory() {
return singleton;
}
public Lock makeLock(String lockName) {
return singletonLock;
}
public void clearAllLocks() {};
};
class NoLock extends Lock {
public boolean obtain() throws IOException {
return true;
}
public void release() {
}
public boolean isLocked() {
return false;
}
public String toString() {
return "NoLock";
}
}

View File

@ -27,7 +27,9 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
/**
* A memory-resident {@link Directory} implementation.
* A memory-resident {@link Directory} implementation. Locking
* implementation is by default the {@link SingleInstanceLockFactory}
* but can be changed with {@link #setLockFactory}.
*
* @version $Id$
*/
@ -39,6 +41,7 @@ public final class RAMDirectory extends Directory implements Serializable {
/** Constructs an empty {@link Directory}. */
public RAMDirectory() {
setLockFactory(new SingleInstanceLockFactory());
}
/**
@ -56,6 +59,7 @@ public final class RAMDirectory extends Directory implements Serializable {
}
private RAMDirectory(Directory dir, boolean closeDir) throws IOException {
this();
final String[] files = dir.list();
byte[] buf = new byte[BufferedIndexOutput.BUFFER_SIZE];
for (int i = 0; i < files.length; i++) {
@ -175,29 +179,6 @@ public final class RAMDirectory extends Directory implements Serializable {
return new RAMInputStream(file);
}
/** Construct a {@link Lock}.
* @param name the name of the lock file
*/
public final Lock makeLock(final String name) {
return new Lock() {
public boolean obtain() throws IOException {
synchronized (files) {
if (!fileExists(name)) {
createOutput(name).close();
return true;
}
return false;
}
}
public void release() {
deleteFile(name);
}
public boolean isLocked() {
return fileExists(name);
}
};
}
/** Closes the store to future operations, releasing associated memory. */
public final void close() {
files = null;

View File

@ -0,0 +1,205 @@
package org.apache.lucene.store;
/**
* Copyright 2004 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.io.File;
import java.io.Serializable;
import java.util.Hashtable;
import java.util.Enumeration;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
/**
* A memory-resident {@link Directory} implementation.
*
* @version $Id: RAMDirectory.java 419723 2006-07-06 22:14:07Z hossman $
*/
public final class RAMDirectory extends Directory implements Serializable {
private static final long serialVersionUID = 1l;
Hashtable files = new Hashtable();
/** Constructs an empty {@link Directory}. */
public RAMDirectory() {
}
/**
* Creates a new <code>RAMDirectory</code> instance from a different
* <code>Directory</code> implementation. This can be used to load
* a disk-based index into memory.
* <P>
* This should be used only with indices that can fit into memory.
*
* @param dir a <code>Directory</code> value
* @exception IOException if an error occurs
*/
public RAMDirectory(Directory dir) throws IOException {
this(dir, false);
}
private RAMDirectory(Directory dir, boolean closeDir) throws IOException {
final String[] files = dir.list();
byte[] buf = new byte[BufferedIndexOutput.BUFFER_SIZE];
for (int i = 0; i < files.length; i++) {
// make place on ram disk
IndexOutput os = createOutput(files[i]);
// read current file
IndexInput is = dir.openInput(files[i]);
// and copy to ram disk
long len = is.length();
long readCount = 0;
while (readCount < len) {
int toRead = readCount + BufferedIndexOutput.BUFFER_SIZE > len ? (int)(len - readCount) : BufferedIndexOutput.BUFFER_SIZE;
is.readBytes(buf, 0, toRead);
os.writeBytes(buf, toRead);
readCount += toRead;
}
// graceful cleanup
is.close();
os.close();
}
if(closeDir)
dir.close();
}
/**
* Creates a new <code>RAMDirectory</code> instance from the {@link FSDirectory}.
*
* @param dir a <code>File</code> specifying the index directory
*/
public RAMDirectory(File dir) throws IOException {
this(FSDirectory.getDirectory(dir, false), true);
}
/**
* Creates a new <code>RAMDirectory</code> instance from the {@link FSDirectory}.
*
* @param dir a <code>String</code> specifying the full index directory path
*/
public RAMDirectory(String dir) throws IOException {
this(FSDirectory.getDirectory(dir, false), true);
}
/** Returns an array of strings, one for each file in the directory. */
public final String[] list() {
String[] result = new String[files.size()];
int i = 0;
Enumeration names = files.keys();
while (names.hasMoreElements())
result[i++] = (String)names.nextElement();
return result;
}
/** Returns true iff the named file exists in this directory. */
public final boolean fileExists(String name) {
RAMFile file = (RAMFile)files.get(name);
return file != null;
}
/** Returns the time the named file was last modified. */
public final long fileModified(String name) {
RAMFile file = (RAMFile)files.get(name);
return file.lastModified;
}
/** Set the modified time of an existing file to now. */
public void touchFile(String name) {
// final boolean MONITOR = false;
RAMFile file = (RAMFile)files.get(name);
long ts2, ts1 = System.currentTimeMillis();
do {
try {
Thread.sleep(0, 1);
} catch (InterruptedException e) {}
ts2 = System.currentTimeMillis();
// if (MONITOR) {
// count++;
// }
} while(ts1 == ts2);
file.lastModified = ts2;
// if (MONITOR)
// System.out.println("SLEEP COUNT: " + count);
}
/** Returns the length in bytes of a file in the directory. */
public final long fileLength(String name) {
RAMFile file = (RAMFile)files.get(name);
return file.length;
}
/** Removes an existing file in the directory. */
public final void deleteFile(String name) {
files.remove(name);
}
/** Removes an existing file in the directory. */
public final void renameFile(String from, String to) {
RAMFile file = (RAMFile)files.get(from);
files.remove(from);
files.put(to, file);
}
/** Creates a new, empty file in the directory with the given name.
Returns a stream writing this file. */
public final IndexOutput createOutput(String name) {
RAMFile file = new RAMFile();
files.put(name, file);
return new RAMOutputStream(file);
}
/** Returns a stream reading an existing file. */
public final IndexInput openInput(String name) {
RAMFile file = (RAMFile)files.get(name);
return new RAMInputStream(file);
}
/** Construct a {@link Lock}.
* @param name the name of the lock file
*/
public final Lock makeLock(final String name) {
return new Lock() {
public boolean obtain() throws IOException {
synchronized (files) {
if (!fileExists(name)) {
createOutput(name).close();
return true;
}
return false;
}
}
public void release() {
deleteFile(name);
}
public boolean isLocked() {
return fileExists(name);
}
};
}
/** Closes the store to future operations, releasing associated memory. */
public final void close() {
files = null;
}
}

View File

@ -0,0 +1,133 @@
package org.apache.lucene.store;
/**
* Copyright 2006 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.File;
import java.io.IOException;
/**
* Implements {@link LockFactory} using {@link File#createNewFile()}. This is
* currently the default LockFactory used for {@link FSDirectory} if no
* LockFactory instance is otherwise provided.
*
* Note that there are known problems with this locking implementation on NFS.
*
* @see LockFactory
*/
public class SimpleFSLockFactory extends LockFactory {
/**
* Directory specified by <code>org.apache.lucene.lockDir</code>
* system property. If that is not set, then <code>java.io.tmpdir</code>
* system property is used.
*/
public static final String LOCK_DIR =
System.getProperty("org.apache.lucene.lockDir",
System.getProperty("java.io.tmpdir"));
private File lockDir;
/**
* Instantiate using default LOCK_DIR: <code>org.apache.lucene.lockDir</code>
* system property, or (if that is null) then <code>java.io.tmpdir</code>.
*/
public SimpleFSLockFactory() throws IOException {
lockDir = new File(LOCK_DIR);
init(lockDir);
}
/**
* Instantiate using the provided directory (as a File instance).
* @param lockDir where lock files should be created.
*/
public SimpleFSLockFactory(File lockDir) throws IOException {
init(lockDir);
}
/**
* Instantiate using the provided directory name (String).
* @param lockDirName where lock files should be created.
*/
public SimpleFSLockFactory(String lockDirName) throws IOException {
lockDir = new File(lockDirName);
init(lockDir);
}
protected void init(File lockDir) throws IOException {
this.lockDir = lockDir;
}
public Lock makeLock(String lockName) {
return new SimpleFSLock(lockDir, lockPrefix + "-" + lockName);
}
public void clearAllLocks() throws IOException {
String[] files = lockDir.list();
if (files == null)
throw new IOException("Cannot read lock directory " +
lockDir.getAbsolutePath());
String prefix = lockPrefix + "-";
for (int i = 0; i < files.length; i++) {
if (!files[i].startsWith(prefix))
continue;
File lockFile = new File(lockDir, files[i]);
if (!lockFile.delete())
throw new IOException("Cannot delete " + lockFile);
}
}
};
class SimpleFSLock extends Lock {
File lockFile;
File lockDir;
public SimpleFSLock(File lockDir, String lockFileName) {
this.lockDir = lockDir;
lockFile = new File(lockDir, lockFileName);
}
public boolean obtain() throws IOException {
// Ensure that lockDir exists and is a directory:
if (!lockDir.exists()) {
if (!lockDir.mkdirs())
throw new IOException("Cannot create directory: " +
lockDir.getAbsolutePath());
} else if (!lockDir.isDirectory()) {
throw new IOException("Found regular file where directory expected: " +
lockDir.getAbsolutePath());
}
return lockFile.createNewFile();
}
public void release() {
lockFile.delete();
}
public boolean isLocked() {
return lockFile.exists();
}
public String toString() {
return "SimpleFSLock@" + lockFile;
}
}

View File

@ -0,0 +1,81 @@
package org.apache.lucene.store;
/**
* Copyright 2006 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.HashSet;
import java.util.Enumeration;
/**
* Implements {@link LockFactory} for a single in-process instance,
* meaning all locking will take place through this one instance.
* Only use this {@link LockFactory} when you are certain all
* IndexReaders and IndexWriters for a given index are running
* against a single shared in-process Directory instance. This is
* currently the default locking for RAMDirectory.
*
* @see LockFactory
*/
public class SingleInstanceLockFactory extends LockFactory {
private HashSet locks = new HashSet();
public Lock makeLock(String lockName) {
// We do not use the LockPrefix at all, because the private
// HashSet instance effectively scopes the locking to this
// single Directory instance.
return new SingleInstanceLock(locks, lockName);
}
public void clearAllLocks() throws IOException {
locks = new HashSet();
}
};
class SingleInstanceLock extends Lock {
String lockName;
private HashSet locks;
public SingleInstanceLock(HashSet locks, String lockName) {
this.locks = locks;
this.lockName = lockName;
}
public boolean obtain() throws IOException {
synchronized(locks) {
return locks.add(lockName);
}
}
public void release() {
synchronized(locks) {
locks.remove(lockName);
}
}
public boolean isLocked() {
synchronized(locks) {
return locks.contains(lockName);
}
}
public String toString() {
return "SingleInstanceLock: " + lockName;
}
}

View File

@ -0,0 +1,443 @@
package org.apache.lucene.store;
/**
* Copyright 2004 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import junit.framework.TestCase;
import java.util.Hashtable;
import java.util.Enumeration;
import java.io.IOException;
import java.io.File;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.Hits;
import org.apache.lucene.analysis.WhitespaceAnalyzer;
public class TestLockFactory extends TestCase {
// Verify: we can provide our own LockFactory implementation, the right
// methods are called at the right time, locks are created, etc.
public void testCustomLockFactory() throws IOException {
Directory dir = new RAMDirectory();
MockLockFactory lf = new MockLockFactory();
dir.setLockFactory(lf);
// Lock prefix should have been set:
assertTrue("lock prefix was not set by the RAMDirectory", lf.lockPrefixSet);
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), true);
// add 100 documents (so that commit lock is used)
for (int i = 0; i < 100; i++) {
addDoc(writer);
}
// Both write lock and commit lock should have been created:
assertEquals("# of unique locks created (after instantiating IndexWriter)",
2, lf.locksCreated.size());
assertTrue("# calls to makeLock <= 2 (after instantiating IndexWriter)",
lf.makeLockCount > 2);
for(Enumeration e = lf.locksCreated.keys(); e.hasMoreElements();) {
String lockName = (String) e.nextElement();
MockLockFactory.MockLock lock = (MockLockFactory.MockLock) lf.locksCreated.get(lockName);
assertTrue("# calls to Lock.obtain is 0 (after instantiating IndexWriter)",
lock.lockAttempts > 0);
}
writer.close();
}
// Verify: we can use the NoLockFactory with RAMDirectory w/ no
// exceptions raised:
// Verify: NoLockFactory allows two IndexWriters
public void testRAMDirectoryNoLocking() throws IOException {
Directory dir = new RAMDirectory();
dir.setLockFactory(NoLockFactory.getNoLockFactory());
assertTrue("RAMDirectory.setLockFactory did not take",
NoLockFactory.class.isInstance(dir.getLockFactory()));
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), true);
// Create a 2nd IndexWriter. This is normally not allowed but it should run through since we're not
// using any locks:
IndexWriter writer2 = null;
try {
writer2 = new IndexWriter(dir, new WhitespaceAnalyzer(), false);
} catch (Exception e) {
fail("Should not have hit an IOException with no locking");
}
writer.close();
if (writer2 != null) {
writer2.close();
}
}
// Verify: SingleInstanceLockFactory is the default lock for RAMDirectory
// Verify: RAMDirectory does basic locking correctly (can't create two IndexWriters)
public void testDefaultRAMDirectory() throws IOException {
Directory dir = new RAMDirectory();
assertTrue("RAMDirectory did not use correct LockFactory: got " + dir.getLockFactory(),
SingleInstanceLockFactory.class.isInstance(dir.getLockFactory()));
IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), true);
// Create a 2nd IndexWriter. This should fail:
IndexWriter writer2 = null;
try {
writer2 = new IndexWriter(dir, new WhitespaceAnalyzer(), false);
fail("Should have hit an IOException with two IndexWriters on default SingleInstanceLockFactory");
} catch (IOException e) {
}
writer.close();
if (writer2 != null) {
writer2.close();
}
}
// Verify: SimpleFSLockFactory is the default for FSDirectory
// Verify: FSDirectory does basic locking correctly
public void testDefaultFSDirectory() throws IOException {
String indexDirName = "index.TestLockFactory1";
IndexWriter writer = new IndexWriter(indexDirName, new WhitespaceAnalyzer(), true);
assertTrue("FSDirectory did not use correct LockFactory: got " + writer.getDirectory().getLockFactory(),
SimpleFSLockFactory.class.isInstance(writer.getDirectory().getLockFactory()));
IndexWriter writer2 = null;
// Create a 2nd IndexWriter. This should fail:
try {
writer2 = new IndexWriter(indexDirName, new WhitespaceAnalyzer(), false);
fail("Should have hit an IOException with two IndexWriters on default SimpleFSLockFactory");
} catch (IOException e) {
}
writer.close();
if (writer2 != null) {
writer2.close();
}
// Cleanup
rmDir(indexDirName);
}
// Verify: FSDirectory's default lockFactory clears all locks correctly
public void testFSDirectoryTwoCreates() throws IOException {
String indexDirName = "index.TestLockFactory2";
IndexWriter writer = new IndexWriter(indexDirName, new WhitespaceAnalyzer(), true);
assertTrue("FSDirectory did not use correct LockFactory: got " + writer.getDirectory().getLockFactory(),
SimpleFSLockFactory.class.isInstance(writer.getDirectory().getLockFactory()));
// Create a 2nd IndexWriter. This should not fail:
IndexWriter writer2 = null;
try {
writer2 = new IndexWriter(indexDirName, new WhitespaceAnalyzer(), true);
} catch (IOException e) {
fail("Should not have hit an IOException with two IndexWriters with create=true, on default SimpleFSLockFactory");
}
writer.close();
if (writer2 != null) {
writer2.close();
}
// Cleanup
rmDir(indexDirName);
}
// Verify: setting custom lock factory class (as system property) works:
// Verify: FSDirectory does basic locking correctly
public void testLockClassProperty() throws IOException {
String indexDirName = "index.TestLockFactory3";
System.setProperty("org.apache.lucene.store.FSDirectoryLockFactoryClass",
"org.apache.lucene.store.NoLockFactory");
IndexWriter writer = new IndexWriter(indexDirName, new WhitespaceAnalyzer(), true);
assertTrue("FSDirectory did not use correct LockFactory: got " + writer.getDirectory().getLockFactory(),
NoLockFactory.class.isInstance(writer.getDirectory().getLockFactory()));
// Put back to the correct default for subsequent tests:
System.setProperty("org.apache.lucene.store.FSDirectoryLockFactoryClass",
"org.apache.lucene.store.SimpleFSLockFactory");
writer.close();
// Cleanup
rmDir(indexDirName);
}
// Verify: setDisableLocks works
public void testDisableLocks() throws IOException {
String indexDirName = "index.TestLockFactory4";
assertTrue("Locks are already disabled", !FSDirectory.getDisableLocks());
FSDirectory.setDisableLocks(true);
IndexWriter writer = new IndexWriter(indexDirName, new WhitespaceAnalyzer(), true);
assertTrue("FSDirectory did not use correct default LockFactory: got " + writer.getDirectory().getLockFactory(),
NoLockFactory.class.isInstance(writer.getDirectory().getLockFactory()));
// Should be no error since locking is disabled:
IndexWriter writer2 = null;
try {
writer2 = new IndexWriter(indexDirName, new WhitespaceAnalyzer(), false);
} catch (IOException e) {
fail("Should not have hit an IOException with locking disabled");
}
// Put back to the correct default for subsequent tests:
System.setProperty("org.apache.lucene.store.FSDirectoryLockFactoryClass",
"org.apache.lucene.store.SimpleFSLockFactory");
FSDirectory.setDisableLocks(false);
writer.close();
if (writer2 != null) {
writer2.close();
}
// Cleanup
rmDir(indexDirName);
}
// Verify: if I try to getDirectory() with two different locking implementations, I get an IOException
public void testFSDirectoryDifferentLockFactory() throws IOException {
String indexDirName = "index.TestLockFactory5";
LockFactory lf = new SingleInstanceLockFactory();
FSDirectory fs1 = FSDirectory.getDirectory(indexDirName, true, lf);
// Different lock factory instance should hit IOException:
try {
FSDirectory fs2 = FSDirectory.getDirectory(indexDirName, true, new SingleInstanceLockFactory());
fail("Should have hit an IOException because LockFactory instances differ");
} catch (IOException e) {
}
FSDirectory fs2 = null;
// Same lock factory instance should not:
try {
fs2 = FSDirectory.getDirectory(indexDirName, true, lf);
} catch (IOException e) {
fail("Should not have hit an IOException because LockFactory instances are the same");
}
fs1.close();
if (fs2 != null) {
fs2.close();
}
// Cleanup
rmDir(indexDirName);
}
// Verify: do stress test, by opening IndexReaders and
// IndexWriters over & over in 2 threads and making sure
// no unexpected exceptions are raised:
public void testStressLocks() throws IOException {
String indexDirName = "index.TestLockFactory6";
FSDirectory fs1 = FSDirectory.getDirectory(indexDirName, true);
// fs1.setLockFactory(NoLockFactory.getNoLockFactory());
// First create a 1 doc index:
IndexWriter w = new IndexWriter(fs1, new WhitespaceAnalyzer(), true);
addDoc(w);
w.close();
WriterThread writer = new WriterThread(100, fs1);
SearcherThread searcher = new SearcherThread(100, fs1);
writer.start();
searcher.start();
while(writer.isAlive() || searcher.isAlive()) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
}
assertTrue("IndexWriter hit unexpected exceptions", !writer.hitException);
assertTrue("IndexSearcher hit unexpected exceptions", !searcher.hitException);
// Cleanup
rmDir(indexDirName);
}
private class WriterThread extends Thread {
private Directory dir;
private int numIteration;
public boolean hitException = false;
public WriterThread(int numIteration, Directory dir) {
this.numIteration = numIteration;
this.dir = dir;
}
public void run() {
WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer();
IndexWriter writer = null;
for(int i=0;i<this.numIteration;i++) {
try {
writer = new IndexWriter(dir, analyzer, false);
} catch (IOException e) {
if (e.toString().indexOf(" timed out:") == -1) {
hitException = true;
} else {
// lock obtain timed out
// NOTE: we should at some point
// consider this a failure? The lock
// obtains, across IndexReader &
// IndexWriters should be "fair" (ie
// FIFO).
}
} catch (Exception e) {
hitException = true;
System.out.println("Stress Test Index Writer: creation hit unexpected exception: " + e.toString());
e.printStackTrace(System.out);
}
if (writer != null) {
try {
addDoc(writer);
} catch (IOException e) {
hitException = true;
System.out.println("Stress Test Index Writer: addDoc hit unexpected exception: " + e.toString());
e.printStackTrace(System.out);
}
try {
writer.close();
} catch (IOException e) {
hitException = true;
System.out.println("Stress Test Index Writer: close hit unexpected exception: " + e.toString());
e.printStackTrace(System.out);
}
writer = null;
}
}
}
}
private class SearcherThread extends Thread {
private Directory dir;
private int numIteration;
public boolean hitException = false;
public SearcherThread(int numIteration, Directory dir) {
this.numIteration = numIteration;
this.dir = dir;
}
public void run() {
IndexSearcher searcher = null;
WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer();
Query query = new TermQuery(new Term("content", "aaa"));
for(int i=0;i<this.numIteration;i++) {
try{
searcher = new IndexSearcher(dir);
} catch (Exception e) {
hitException = true;
System.out.println("Stress Test Index Searcher: create hit unexpected exception: " + e.toString());
e.printStackTrace(System.out);
}
if (searcher != null) {
Hits hits = null;
try {
hits = searcher.search(query);
} catch (IOException e) {
hitException = true;
System.out.println("Stress Test Index Searcher: search hit unexpected exception: " + e.toString());
e.printStackTrace(System.out);
}
// System.out.println(hits.length() + " total results");
try {
searcher.close();
} catch (IOException e) {
hitException = true;
System.out.println("Stress Test Index Searcher: close hit unexpected exception: " + e.toString());
e.printStackTrace(System.out);
}
searcher = null;
}
}
}
}
public class MockLockFactory extends LockFactory {
public boolean lockPrefixSet;
public Hashtable locksCreated = new Hashtable();
public int makeLockCount = 0;
public void setLockPrefix(String lockPrefix) {
super.setLockPrefix(lockPrefix);
lockPrefixSet = true;
}
synchronized public Lock makeLock(String lockName) {
Lock lock = new MockLock();
locksCreated.put(lockName, lock);
makeLockCount++;
return lock;
}
public void clearAllLocks() {}
public class MockLock extends Lock {
public int lockAttempts;
public boolean obtain() {
lockAttempts++;
return true;
}
public void release() {
// do nothing
}
public boolean isLocked() {
return false;
}
}
}
private void addDoc(IndexWriter writer) throws IOException {
Document doc = new Document();
doc.add(new Field("content", "aaa", Field.Store.NO, Field.Index.TOKENIZED));
writer.addDocument(doc);
}
private void rmDir(String dirName) {
File dir = new java.io.File(dirName);
String[] files = dir.list(); // clear old files
for (int i = 0; i < files.length; i++) {
File file = new File(dir, files[i]);
file.delete();
}
dir.delete();
}
}