You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-user@lucene.apache.org by Paul Taylor <pa...@fastmail.fm> on 2011/09/22 12:56:54 UTC
OverlappingFileLockException when optimizing large index
Id made just a few changes to my code to remove deprecations that
ocurred when I upgraded to Lucene 3.1, all tests work fine but when I
tried to build a real index it gives this error during optimization stage
Exception in thread "main" java.nio.channels.OverlappingFileLockException
at
sun.nio.ch.FileChannelImpl$SharedFileLockTable.checkList(FileChannelImpl.java:1166)
at
sun.nio.ch.FileChannelImpl$SharedFileLockTable.add(FileChannelImpl.java:1068)
at sun.nio.ch.FileChannelImpl.tryLock(FileChannelImpl.java:868)
at java.nio.channels.FileChannel.tryLock(FileChannel.java:962)
at
org.apache.lucene.store.NativeFSLock.obtain(NativeFSLockFactory.java:216)
at org.apache.lucene.store.Lock.obtain(Lock.java:72)
at org.apache.lucene.index.IndexWriter.<init>(IndexWriter.java:1097)
at
org.musicbrainz.search.index.ThreadedIndexWriter.<init>(ThreadedIndexWriter.java:66)
at
org.musicbrainz.search.index.IndexBuilder.createIndexWriter(IndexBuilder.java:229)
at
org.musicbrainz.search.index.IndexBuilder.main(IndexBuilder.java:152)
and a few minutes later I also got
xception in thread "Lucene Merge Thread #101"
org.apache.lucene.index.MergePolicy$MergeException: java.io.IOException:
File too large
at
org.apache.lucene.index.ConcurrentMergeScheduler.handleMergeException(ConcurrentMergeScheduler.java:517)
at
org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:482)
Caused by: java.io.IOException: File too large
at java.io.RandomAccessFile.writeBytes(Native Method)
at java.io.RandomAccessFile.write(RandomAccessFile.java:482)
at
org.apache.lucene.store.FSDirectory$FSIndexOutput.flushBuffer(FSDirectory.java:469)
at
org.apache.lucene.store.BufferedIndexOutput.flushBuffer(BufferedIndexOutput.java:99)
at
org.apache.lucene.store.BufferedIndexOutput.flush(BufferedIndexOutput.java:88)
at
org.apache.lucene.store.BufferedIndexOutput.close(BufferedIndexOutput.java:113)
at
org.apache.lucene.store.FSDirectory$FSIndexOutput.close(FSDirectory.java:478)
at org.apache.lucene.util.IOUtils.closeSafely(IOUtils.java:80)
at org.apache.lucene.index.FieldsWriter.close(FieldsWriter.java:111)
at
org.apache.lucene.index.SegmentMerger.mergeFields(SegmentMerger.java:245)
at org.apache.lucene.index.SegmentMerger.merge(SegmentMerger.java:110)
at
org.apache.lucene.index.IndexWriter.mergeMiddle(IndexWriter.java:3938)
at org.apache.lucene.index.IndexWriter.merge(IndexWriter.java:3614)
at
org.apache.lucene.index.ConcurrentMergeScheduler.doMerge(ConcurrentMergeScheduler.java:388)
at
org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:456)
not sure if one is a consequence of another, any ideas ?
The idea behind the ThreadedIndexWriter class is that I can build a
number of indexes one a a time from a database source, but when an index
built I then optimize it whilst the next index is being built.
package org.musicbrainz.search.index;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import java.io.IOException;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
public class ThreadedIndexWriter extends IndexWriter {
private ExecutorService threadPool;
private Analyzer defaultAnalyzer;
private class Job implements Runnable {
Document doc;
Analyzer analyzer;
public Job(Document doc, Analyzer analyzer) {
this.doc = doc;
this.analyzer = analyzer;
}
public void run() {
try {
ThreadedIndexWriter.super.addDocument(doc, analyzer);
} catch (IOException ioe) {
ioe.printStackTrace(System.err);
}
}
}
public ThreadedIndexWriter(Directory dir,
IndexWriterConfig config,
int numThreads,
int maxQueueSize)
throws IOException
{
super(dir, config);
defaultAnalyzer = config.getAnalyzer();
threadPool = new ThreadPoolExecutor(
numThreads, numThreads, 0,
TimeUnit.SECONDS,
new ArrayBlockingQueue<Runnable>(maxQueueSize, false),
new ThreadPoolExecutor.CallerRunsPolicy());
}
public void addDocument(Document doc) {
threadPool.execute(new Job(doc, defaultAnalyzer));
}
public void addDocument(Document doc, Analyzer a) {
threadPool.execute(new Job(doc, a));
}
public void updateDocument(Term term, Document doc) {
throw new UnsupportedOperationException();
}
public void updateDocument(Term term, Document doc, Analyzer a) {
throw new UnsupportedOperationException();
}
public void close() throws IOException {
finish();
super.close();
}
public void close(boolean doWait) throws IOException {
finish();
super.close(doWait);
}
public void rollback() throws IOException {
finish();
super.rollback();
}
private void finish() {
threadPool.shutdown();
while (true) {
try {
if (threadPool.awaitTermination(Long.MAX_VALUE,
TimeUnit.SECONDS)) {
break;
}
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
}
}
}
}
Paul
---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org
Re: OverlappingFileLockException when optimizing large index
Posted by Paul Taylor <pa...@fastmail.fm>.
On 22/09/2011 13:57, Paul Taylor wrote:
> On 22/09/2011 11:56, Paul Taylor wrote:
>> Id made just a few changes to my code to remove deprecations that
>> ocurred when I upgraded to Lucene 3.1, all tests work fine but when I
>> tried to build a real index it gives this error during optimization
>> stage
>>
>> Exception in thread "main"
>> java.nio.channels.OverlappingFileLockException
>> at
>> sun.nio.ch.FileChannelImpl$SharedFileLockTable.checkList(FileChannelImpl.java:1166)
>> at
>> sun.nio.ch.FileChannelImpl$SharedFileLockTable.add(FileChannelImpl.java:1068)
>> at sun.nio.ch.FileChannelImpl.tryLock(FileChannelImpl.java:868)
>> at java.nio.channels.FileChannel.tryLock(FileChannel.java:962)
>> at
>> org.apache.lucene.store.NativeFSLock.obtain(NativeFSLockFactory.java:216)
>> at org.apache.lucene.store.Lock.obtain(Lock.java:72)
>> at org.apache.lucene.index.IndexWriter.<init>(IndexWriter.java:1097)
>> at
>> org.musicbrainz.search.index.ThreadedIndexWriter.<init>(ThreadedIndexWriter.java:66)
>> at
>> org.musicbrainz.search.index.IndexBuilder.createIndexWriter(IndexBuilder.java:229)
>> at
>> org.musicbrainz.search.index.IndexBuilder.main(IndexBuilder.java:152)
>
Using
fsDir = FSDirectory.open(new File(path), NoLockFactory.getNoLockFactory() );
does work but dont see why this is neccessary.
But this might be a speed improvement I could make anyway, although Im
building multiple indexes, any one index is only being built by one
thread, and the indexes are not being accessed by any index reader
whilst being built
Paul
---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org
Re: OverlappingFileLockException when optimizing large index
Posted by Paul Taylor <pa...@fastmail.fm>.
On 22/09/2011 11:56, Paul Taylor wrote:
> Id made just a few changes to my code to remove deprecations that
> ocurred when I upgraded to Lucene 3.1, all tests work fine but when I
> tried to build a real index it gives this error during optimization stage
>
> Exception in thread "main" java.nio.channels.OverlappingFileLockException
> at
> sun.nio.ch.FileChannelImpl$SharedFileLockTable.checkList(FileChannelImpl.java:1166)
> at
> sun.nio.ch.FileChannelImpl$SharedFileLockTable.add(FileChannelImpl.java:1068)
> at sun.nio.ch.FileChannelImpl.tryLock(FileChannelImpl.java:868)
> at java.nio.channels.FileChannel.tryLock(FileChannel.java:962)
> at
> org.apache.lucene.store.NativeFSLock.obtain(NativeFSLockFactory.java:216)
> at org.apache.lucene.store.Lock.obtain(Lock.java:72)
> at org.apache.lucene.index.IndexWriter.<init>(IndexWriter.java:1097)
> at
> org.musicbrainz.search.index.ThreadedIndexWriter.<init>(ThreadedIndexWriter.java:66)
> at
> org.musicbrainz.search.index.IndexBuilder.createIndexWriter(IndexBuilder.java:229)
> at
> org.musicbrainz.search.index.IndexBuilder.main(IndexBuilder.java:152)
Some progress with this Ive realized the problem isn't with my code
changes but where the indexes are being created. I am running on a Mac
with OSX10.7 Lion with a connected external format as MS-DOS FAT-32.
If I build the indexes to my mac hardrive it always works, but if I
write to the external harddrive fails(note this is just an externally
connected drive not a network drive), the problem occurs when trying to
initilize the 2nd IndexWriter, the first IndexWriter has finished
building the index but hasnt closed the writer yet because it has now
started optmizing the index.
I dont understand why they seem to be using the same lock because they
are creating different indexes and indexing them to different subfolders.
So it seems the problem might be because Im running MacOS but using a
Windows filesystem to store the indexes so instead of using
FSDirectory.open() I instantiated an MMAPDirectory, but got the same error.
I then tried specifying a different lock factory
fsDir = FSDirectory.open(new File(path), new SimpleFSLockFactory() );
and now it fails in the same place but with a timeout problem
Exception in thread "main"
org.apache.lucene.store.LockObtainFailedException: Lock obtain timed
out:
SimpleFSLock@/Volumes/DDRIVE/Shared/Musicbrainz/releasegroup_index/write.lock
at org.apache.lucene.store.Lock.obtain(Lock.java:84)
at org.apache.lucene.index.IndexWriter.<init>(IndexWriter.java:1112)
at
org.musicbrainz.search.index.ThreadedIndexWriter.<init>(ThreadedIndexWriter.java:66)
at
org.musicbrainz.search.index.IndexBuilder.createIndexWriter(IndexBuilder.java:230)
at
org.musicbrainz.search.index.IndexBuilder.main(IndexBuilder.java:153)
Paul
---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org
Re: OverlappingFileLockException when optimizing large index
Posted by Paul Taylor <pa...@fastmail.fm>.
On 22/09/2011 13:53, Michael McCandless wrote:
> The OverlappingLockExc sounds like somehow you are trying to open two
> writers at once on the same index. Maybe try to boil down your code
> to a smaller test case?
Hi Mike
Just missed your reply please see my latest reply, I am opening two
writers but NOT on the same index.
> What OS/filesystem? It's odd to get the "File too large" error. I do
> wish Lucene would somehow decorate IOEs with the filename.....
>
Worked out this error is unrelated to the first one, and ocurred whilst
optimizing, Im using
FAT32, would that be a 4GB file limit ?
Paul
---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org
Re: OverlappingFileLockException when optimizing large index
Posted by Michael McCandless <lu...@mikemccandless.com>.
The OverlappingLockExc sounds like somehow you are trying to open two
writers at once on the same index. Maybe try to boil down your code
to a smaller test case?
What OS/filesystem? It's odd to get the "File too large" error. I do
wish Lucene would somehow decorate IOEs with the filename.....
Mike McCandless
http://blog.mikemccandless.com
On Thu, Sep 22, 2011 at 6:56 AM, Paul Taylor <pa...@fastmail.fm> wrote:
> Id made just a few changes to my code to remove deprecations that ocurred
> when I upgraded to Lucene 3.1, all tests work fine but when I tried to build
> a real index it gives this error during optimization stage
>
> Exception in thread "main" java.nio.channels.OverlappingFileLockException
> at
> sun.nio.ch.FileChannelImpl$SharedFileLockTable.checkList(FileChannelImpl.java:1166)
> at
> sun.nio.ch.FileChannelImpl$SharedFileLockTable.add(FileChannelImpl.java:1068)
> at sun.nio.ch.FileChannelImpl.tryLock(FileChannelImpl.java:868)
> at java.nio.channels.FileChannel.tryLock(FileChannel.java:962)
> at
> org.apache.lucene.store.NativeFSLock.obtain(NativeFSLockFactory.java:216)
> at org.apache.lucene.store.Lock.obtain(Lock.java:72)
> at org.apache.lucene.index.IndexWriter.<init>(IndexWriter.java:1097)
> at
> org.musicbrainz.search.index.ThreadedIndexWriter.<init>(ThreadedIndexWriter.java:66)
> at
> org.musicbrainz.search.index.IndexBuilder.createIndexWriter(IndexBuilder.java:229)
> at org.musicbrainz.search.index.IndexBuilder.main(IndexBuilder.java:152)
>
> and a few minutes later I also got
>
> xception in thread "Lucene Merge Thread #101"
> org.apache.lucene.index.MergePolicy$MergeException: java.io.IOException:
> File too large
> at
> org.apache.lucene.index.ConcurrentMergeScheduler.handleMergeException(ConcurrentMergeScheduler.java:517)
> at
> org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:482)
> Caused by: java.io.IOException: File too large
> at java.io.RandomAccessFile.writeBytes(Native Method)
> at java.io.RandomAccessFile.write(RandomAccessFile.java:482)
> at
> org.apache.lucene.store.FSDirectory$FSIndexOutput.flushBuffer(FSDirectory.java:469)
> at
> org.apache.lucene.store.BufferedIndexOutput.flushBuffer(BufferedIndexOutput.java:99)
> at
> org.apache.lucene.store.BufferedIndexOutput.flush(BufferedIndexOutput.java:88)
> at
> org.apache.lucene.store.BufferedIndexOutput.close(BufferedIndexOutput.java:113)
> at
> org.apache.lucene.store.FSDirectory$FSIndexOutput.close(FSDirectory.java:478)
> at org.apache.lucene.util.IOUtils.closeSafely(IOUtils.java:80)
> at org.apache.lucene.index.FieldsWriter.close(FieldsWriter.java:111)
> at
> org.apache.lucene.index.SegmentMerger.mergeFields(SegmentMerger.java:245)
> at org.apache.lucene.index.SegmentMerger.merge(SegmentMerger.java:110)
> at org.apache.lucene.index.IndexWriter.mergeMiddle(IndexWriter.java:3938)
> at org.apache.lucene.index.IndexWriter.merge(IndexWriter.java:3614)
> at
> org.apache.lucene.index.ConcurrentMergeScheduler.doMerge(ConcurrentMergeScheduler.java:388)
> at
> org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run(ConcurrentMergeScheduler.java:456)
>
>
> not sure if one is a consequence of another, any ideas ?
>
> The idea behind the ThreadedIndexWriter class is that I can build a number
> of indexes one a a time from a database source, but when an index built I
> then optimize it whilst the next index is being built.
>
> package org.musicbrainz.search.index;
>
> import org.apache.lucene.analysis.Analyzer;
> import org.apache.lucene.document.Document;
> import org.apache.lucene.index.IndexWriter;
> import org.apache.lucene.index.IndexWriterConfig;
> import org.apache.lucene.index.Term;
> import org.apache.lucene.store.Directory;
>
> import java.io.IOException;
> import java.util.concurrent.ArrayBlockingQueue;
> import java.util.concurrent.ExecutorService;
> import java.util.concurrent.ThreadPoolExecutor;
> import java.util.concurrent.TimeUnit;
>
> public class ThreadedIndexWriter extends IndexWriter {
> private ExecutorService threadPool;
> private Analyzer defaultAnalyzer;
>
> private class Job implements Runnable {
> Document doc;
> Analyzer analyzer;
>
> public Job(Document doc, Analyzer analyzer) {
> this.doc = doc;
> this.analyzer = analyzer;
> }
>
> public void run() {
> try {
>
> ThreadedIndexWriter.super.addDocument(doc, analyzer);
> } catch (IOException ioe) {
> ioe.printStackTrace(System.err);
> }
> }
> }
>
> public ThreadedIndexWriter(Directory dir,
> IndexWriterConfig config,
> int numThreads,
> int maxQueueSize)
>
> throws IOException
>
> {
> super(dir, config);
> defaultAnalyzer = config.getAnalyzer();
> threadPool = new ThreadPoolExecutor(
> numThreads, numThreads, 0,
> TimeUnit.SECONDS,
> new ArrayBlockingQueue<Runnable>(maxQueueSize, false),
> new ThreadPoolExecutor.CallerRunsPolicy());
> }
>
> public void addDocument(Document doc) {
> threadPool.execute(new Job(doc, defaultAnalyzer));
> }
>
> public void addDocument(Document doc, Analyzer a) {
> threadPool.execute(new Job(doc, a));
> }
>
> public void updateDocument(Term term, Document doc) {
> throw new UnsupportedOperationException();
>
> }
>
> public void updateDocument(Term term, Document doc, Analyzer a) {
>
> throw new UnsupportedOperationException();
> }
>
> public void close() throws IOException {
> finish();
> super.close();
> }
>
> public void close(boolean doWait) throws IOException {
> finish();
>
> super.close(doWait);
> }
>
> public void rollback() throws IOException {
> finish();
> super.rollback();
> }
>
> private void finish() {
> threadPool.shutdown();
> while (true) {
> try {
> if (threadPool.awaitTermination(Long.MAX_VALUE,
> TimeUnit.SECONDS)) {
> break;
> }
> } catch (InterruptedException ie) {
> Thread.currentThread().interrupt();
> throw new RuntimeException(ie);
> }
> }
> }
> }
>
> Paul
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-user-help@lucene.apache.org
>
>
---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org