add --shared
[pylucene.git] / lucene-java-3.4.0 / lucene / backwards / src / test / org / apache / lucene / index / TestIndexWriterWithThreads.java
1 package org.apache.lucene.index;
2
3 /**
4  * Licensed to the Apache Software Foundation (ASF) under one or more
5  * contributor license agreements.  See the NOTICE file distributed with
6  * this work for additional information regarding copyright ownership.
7  * The ASF licenses this file to You under the Apache License, Version 2.0
8  * (the "License"); you may not use this file except in compliance with
9  * the License.  You may obtain a copy of the License at
10  *
11  *     http://www.apache.org/licenses/LICENSE-2.0
12  *
13  * Unless required by applicable law or agreed to in writing, software
14  * distributed under the License is distributed on an "AS IS" BASIS,
15  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16  * See the License for the specific language governing permissions and
17  * limitations under the License.
18  */
19
20 import java.io.IOException;
21
22 import org.apache.lucene.analysis.MockAnalyzer;
23 import org.apache.lucene.document.Document;
24 import org.apache.lucene.document.Field;
25 import org.apache.lucene.store.AlreadyClosedException;
26 import org.apache.lucene.store.Directory;
27 import org.apache.lucene.store.MockDirectoryWrapper;
28 import org.apache.lucene.util.LuceneTestCase;
29 import org.apache.lucene.util.ThreadInterruptedException;
30
31 /**
32  * MultiThreaded IndexWriter tests
33  */
34 public class TestIndexWriterWithThreads extends LuceneTestCase {
35
36   // Used by test cases below
37   private class IndexerThread extends Thread {
38
39     boolean diskFull;
40     Throwable error;
41     AlreadyClosedException ace;
42     IndexWriter writer;
43     boolean noErrors;
44     volatile int addCount;
45
46     public IndexerThread(IndexWriter writer, boolean noErrors) {
47       this.writer = writer;
48       this.noErrors = noErrors;
49     }
50
51     @Override
52     public void run() {
53
54       final Document doc = new Document();
55       doc.add(newField("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
56
57       int idUpto = 0;
58       int fullCount = 0;
59       final long stopTime = System.currentTimeMillis() + 200;
60
61       do {
62         try {
63           writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
64           addCount++;
65         } catch (IOException ioe) {
66           if (VERBOSE) {
67             System.out.println("TEST: expected exc:");
68             ioe.printStackTrace(System.out);
69           }
70           //System.out.println(Thread.currentThread().getName() + ": hit exc");
71           //ioe.printStackTrace(System.out);
72           if (ioe.getMessage().startsWith("fake disk full at") ||
73               ioe.getMessage().equals("now failing on purpose")) {
74             diskFull = true;
75             try {
76               Thread.sleep(1);
77             } catch (InterruptedException ie) {
78               throw new ThreadInterruptedException(ie);
79             }
80             if (fullCount++ >= 5)
81               break;
82           } else {
83             if (noErrors) {
84               System.out.println(Thread.currentThread().getName() + ": ERROR: unexpected IOException:");
85               ioe.printStackTrace(System.out);
86               error = ioe;
87             }
88             break;
89           }
90         } catch (Throwable t) {
91           //t.printStackTrace(System.out);
92           if (noErrors) {
93             System.out.println(Thread.currentThread().getName() + ": ERROR: unexpected Throwable:");
94             t.printStackTrace(System.out);
95             error = t;
96           }
97           break;
98         }
99       } while(System.currentTimeMillis() < stopTime);
100     }
101   }
102
103   // LUCENE-1130: make sure immediate disk full on creating
104   // an IndexWriter (hit during DW.ThreadState.init()), with
105   // multiple threads, is OK:
106   public void testImmediateDiskFullWithThreads() throws Exception {
107
108     int NUM_THREADS = 3;
109
110     for(int iter=0;iter<10;iter++) {
111       if (VERBOSE) {
112         System.out.println("\nTEST: iter=" + iter);
113       }
114       MockDirectoryWrapper dir = newDirectory();
115       IndexWriter writer = new IndexWriter(
116           dir,
117           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).
118               setMaxBufferedDocs(2).
119               setMergeScheduler(new ConcurrentMergeScheduler()).
120               setMergePolicy(newLogMergePolicy(4))
121       );
122       ((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).setSuppressExceptions();
123       dir.setMaxSizeInBytes(4*1024+20*iter);
124       writer.setInfoStream(VERBOSE ? System.out : null);
125
126       IndexerThread[] threads = new IndexerThread[NUM_THREADS];
127
128       for(int i=0;i<NUM_THREADS;i++)
129         threads[i] = new IndexerThread(writer, true);
130
131       for(int i=0;i<NUM_THREADS;i++)
132         threads[i].start();
133
134       for(int i=0;i<NUM_THREADS;i++) {
135         // Without fix for LUCENE-1130: one of the
136         // threads will hang
137         threads[i].join();
138         assertTrue("hit unexpected Throwable", threads[i].error == null);
139       }
140
141       // Make sure once disk space is avail again, we can
142       // cleanly close:
143       dir.setMaxSizeInBytes(0);
144       writer.close(false);
145       dir.close();
146     }
147   }
148   
149
150   // LUCENE-1130: make sure we can close() even while
151   // threads are trying to add documents.  Strictly
152   // speaking, this isn't valid us of Lucene's APIs, but we
153   // still want to be robust to this case:
154   public void testCloseWithThreads() throws Exception {
155     int NUM_THREADS = 3;
156
157     for(int iter=0;iter<7;iter++) {
158       Directory dir = newDirectory();
159       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))
160         .setMaxBufferedDocs(10).setMergeScheduler(new ConcurrentMergeScheduler()).setMergePolicy(newLogMergePolicy(4));
161       // We expect AlreadyClosedException
162       ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
163       IndexWriter writer = new IndexWriter(dir, conf);
164
165       IndexerThread[] threads = new IndexerThread[NUM_THREADS];
166
167       for(int i=0;i<NUM_THREADS;i++)
168         threads[i] = new IndexerThread(writer, false);
169
170       for(int i=0;i<NUM_THREADS;i++)
171         threads[i].start();
172
173       boolean done = false;
174       while(!done) {
175         Thread.sleep(100);
176         for(int i=0;i<NUM_THREADS;i++)
177           // only stop when at least one thread has added a doc
178           if (threads[i].addCount > 0) {
179             done = true;
180             break;
181           }
182       }
183
184       writer.close(false);
185
186       // Make sure threads that are adding docs are not hung:
187       for(int i=0;i<NUM_THREADS;i++) {
188         // Without fix for LUCENE-1130: one of the
189         // threads will hang
190         threads[i].join();
191         if (threads[i].isAlive())
192           fail("thread seems to be hung");
193       }
194
195       // Quick test to make sure index is not corrupt:
196       IndexReader reader = IndexReader.open(dir, true);
197       TermDocs tdocs = reader.termDocs(new Term("field", "aaa"));
198       int count = 0;
199       while(tdocs.next()) {
200         count++;
201       }
202       assertTrue(count > 0);
203       reader.close();
204       
205       dir.close();
206     }
207   }
208
209   // Runs test, with multiple threads, using the specific
210   // failure to trigger an IOException
211   public void _testMultipleThreadsFailure(MockDirectoryWrapper.Failure failure) throws Exception {
212
213     int NUM_THREADS = 3;
214
215     for(int iter=0;iter<2;iter++) {
216       if (VERBOSE) {
217         System.out.println("TEST: iter=" + iter);
218       }
219       MockDirectoryWrapper dir = newDirectory();
220       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
221           new MockAnalyzer(random)).setMaxBufferedDocs(2)
222           .setMergeScheduler(new ConcurrentMergeScheduler())
223           .setMergePolicy(newLogMergePolicy(4));
224       // We expect disk full exceptions in the merge threads
225       ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
226       IndexWriter writer = new IndexWriter(dir, conf);
227       writer.setInfoStream(VERBOSE ? System.out : null);
228       
229       IndexerThread[] threads = new IndexerThread[NUM_THREADS];
230
231       for(int i=0;i<NUM_THREADS;i++)
232         threads[i] = new IndexerThread(writer, true);
233
234       for(int i=0;i<NUM_THREADS;i++)
235         threads[i].start();
236
237       Thread.sleep(10);
238
239       dir.failOn(failure);
240       failure.setDoFail();
241
242       for(int i=0;i<NUM_THREADS;i++) {
243         threads[i].join();
244         assertTrue("hit unexpected Throwable", threads[i].error == null);
245       }
246
247       boolean success = false;
248       try {
249         writer.close(false);
250         success = true;
251       } catch (IOException ioe) {
252         failure.clearDoFail();
253         writer.close(false);
254       }
255
256       if (success) {
257         IndexReader reader = IndexReader.open(dir, true);
258         for(int j=0;j<reader.maxDoc();j++) {
259           if (!reader.isDeleted(j)) {
260             reader.document(j);
261             reader.getTermFreqVectors(j);
262           }
263         }
264         reader.close();
265       }
266
267       dir.close();
268     }
269   }
270
271   // Runs test, with one thread, using the specific failure
272   // to trigger an IOException
273   public void _testSingleThreadFailure(MockDirectoryWrapper.Failure failure) throws IOException {
274     MockDirectoryWrapper dir = newDirectory();
275
276     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random))
277       .setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler()));
278     final Document doc = new Document();
279     doc.add(newField("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
280
281     for(int i=0;i<6;i++)
282       writer.addDocument(doc);
283
284     dir.failOn(failure);
285     failure.setDoFail();
286     try {
287       writer.addDocument(doc);
288       writer.addDocument(doc);
289       writer.commit();
290       fail("did not hit exception");
291     } catch (IOException ioe) {
292     }
293     failure.clearDoFail();
294     writer.addDocument(doc);
295     writer.close(false);
296     dir.close();
297   }
298
299   // Throws IOException during FieldsWriter.flushDocument and during DocumentsWriter.abort
300   private static class FailOnlyOnAbortOrFlush extends MockDirectoryWrapper.Failure {
301     private boolean onlyOnce;
302     public FailOnlyOnAbortOrFlush(boolean onlyOnce) {
303       this.onlyOnce = onlyOnce;
304     }
305     @Override
306     public void eval(MockDirectoryWrapper dir)  throws IOException {
307       if (doFail) {
308         StackTraceElement[] trace = new Exception().getStackTrace();
309         boolean sawAbortOrFlushDoc = false;
310         boolean sawClose = false;
311         for (int i = 0; i < trace.length; i++) {
312           if ("abort".equals(trace[i].getMethodName()) ||
313               "flushDocument".equals(trace[i].getMethodName())) {
314             sawAbortOrFlushDoc = true;
315           }
316           if ("close".equals(trace[i].getMethodName())) {
317             sawClose = true;
318           }
319         }
320         if (sawAbortOrFlushDoc && !sawClose) {
321           if (onlyOnce)
322             doFail = false;
323           //System.out.println(Thread.currentThread().getName() + ": now fail");
324           //new Throwable().printStackTrace(System.out);
325           throw new IOException("now failing on purpose");
326         }
327       }
328     }
329   }
330
331
332
333   // LUCENE-1130: make sure initial IOException, and then 2nd
334   // IOException during rollback(), is OK:
335   public void testIOExceptionDuringAbort() throws IOException {
336     _testSingleThreadFailure(new FailOnlyOnAbortOrFlush(false));
337   }
338
339   // LUCENE-1130: make sure initial IOException, and then 2nd
340   // IOException during rollback(), is OK:
341   public void testIOExceptionDuringAbortOnlyOnce() throws IOException {
342     _testSingleThreadFailure(new FailOnlyOnAbortOrFlush(true));
343   }
344
345   // LUCENE-1130: make sure initial IOException, and then 2nd
346   // IOException during rollback(), with multiple threads, is OK:
347   public void testIOExceptionDuringAbortWithThreads() throws Exception {
348     _testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(false));
349   }
350
351   // LUCENE-1130: make sure initial IOException, and then 2nd
352   // IOException during rollback(), with multiple threads, is OK:
353   public void testIOExceptionDuringAbortWithThreadsOnlyOnce() throws Exception {
354     _testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(true));
355   }
356
357   // Throws IOException during DocumentsWriter.writeSegment
358   private static class FailOnlyInWriteSegment extends MockDirectoryWrapper.Failure {
359     private boolean onlyOnce;
360     public FailOnlyInWriteSegment(boolean onlyOnce) {
361       this.onlyOnce = onlyOnce;
362     }
363     @Override
364     public void eval(MockDirectoryWrapper dir)  throws IOException {
365       if (doFail) {
366         StackTraceElement[] trace = new Exception().getStackTrace();
367         for (int i = 0; i < trace.length; i++) {
368           if ("flush".equals(trace[i].getMethodName()) && "org.apache.lucene.index.DocFieldProcessor".equals(trace[i].getClassName())) {
369             if (onlyOnce)
370               doFail = false;
371             throw new IOException("now failing on purpose");
372           }
373         }
374       }
375     }
376   }
377
378   // LUCENE-1130: test IOException in writeSegment
379   public void testIOExceptionDuringWriteSegment() throws IOException {
380     _testSingleThreadFailure(new FailOnlyInWriteSegment(false));
381   }
382
383   // LUCENE-1130: test IOException in writeSegment
384   public void testIOExceptionDuringWriteSegmentOnlyOnce() throws IOException {
385     _testSingleThreadFailure(new FailOnlyInWriteSegment(true));
386   }
387
388   // LUCENE-1130: test IOException in writeSegment, with threads
389   public void testIOExceptionDuringWriteSegmentWithThreads() throws Exception {
390     _testMultipleThreadsFailure(new FailOnlyInWriteSegment(false));
391   }
392
393   // LUCENE-1130: test IOException in writeSegment, with threads
394   public void testIOExceptionDuringWriteSegmentWithThreadsOnlyOnce() throws Exception {
395     _testMultipleThreadsFailure(new FailOnlyInWriteSegment(true));
396   }
397 }