You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2010/11/12 10:15:38 UTC

svn commit: r1034304 [5/7] - in /lucene/dev/branches/docvalues: ./ lucene/ lucene/contrib/ lucene/contrib/benchmark/conf/ lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/contrib/benchmark/src/test/org/apache/lucene/be...

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Fri Nov 12 09:15:30 2010
@@ -17,16 +17,32 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Random;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Reader;
 
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 
 public class TestIndexWriterExceptions extends LuceneTestCase {
 
@@ -214,4 +230,861 @@ public class TestIndexWriterExceptions e
     _TestUtil.checkIndex(dir);
     dir.close();
   }
+  
+  // LUCENE-1198
+  private static final class MockIndexWriter2 extends IndexWriter {
+
+    public MockIndexWriter2(Directory dir, IndexWriterConfig conf) throws IOException {
+      super(dir, conf);
+    }
+
+    boolean doFail;
+
+    @Override
+    boolean testPoint(String name) {
+      if (doFail && name.equals("DocumentsWriter.ThreadState.init start"))
+        throw new RuntimeException("intentionally failing");
+      return true;
+    }
+  }
+  
+  private class CrashingFilter extends TokenFilter {
+    String fieldName;
+    int count;
+
+    public CrashingFilter(String fieldName, TokenStream input) {
+      super(input);
+      this.fieldName = fieldName;
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (this.fieldName.equals("crash") && count++ >= 4)
+        throw new IOException("I'm experiencing problems");
+      return input.incrementToken();
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      count = 0;
+    }
+  }
+
+  public void testExceptionDocumentsWriterInit() throws IOException {
+    Directory dir = newDirectory();
+    MockIndexWriter2 w = new MockIndexWriter2(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+    Document doc = new Document();
+    doc.add(newField("field", "a field", Field.Store.YES,
+                      Field.Index.ANALYZED));
+    w.addDocument(doc);
+    w.doFail = true;
+    try {
+      w.addDocument(doc);
+      fail("did not hit exception");
+    } catch (RuntimeException re) {
+      // expected
+    }
+    w.close();
+    _TestUtil.checkIndex(dir);
+    dir.close();
+  }
+
+  // LUCENE-1208
+  public void testExceptionJustBeforeFlush() throws IOException {
+    Directory dir = newDirectory();
+    MockIndexWriter w = new MockIndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(2));
+    Document doc = new Document();
+    doc.add(newField("field", "a field", Field.Store.YES,
+                      Field.Index.ANALYZED));
+    w.addDocument(doc);
+
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      public TokenStream tokenStream(String fieldName, Reader reader) {
+        return new CrashingFilter(fieldName, new MockTokenizer(reader, MockTokenizer.WHITESPACE, false));
+      }
+    };
+
+    Document crashDoc = new Document();
+    crashDoc.add(newField("crash", "do it on token 4", Field.Store.YES,
+                           Field.Index.ANALYZED));
+    try {
+      w.addDocument(crashDoc, analyzer);
+      fail("did not hit expected exception");
+    } catch (IOException ioe) {
+      // expected
+    }
+    w.addDocument(doc);
+    w.close();
+    dir.close();
+  }    
+
+  private static final class MockIndexWriter3 extends IndexWriter {
+
+    public MockIndexWriter3(Directory dir, IndexWriterConfig conf) throws IOException {
+      super(dir, conf);
+    }
+
+    boolean doFail;
+    boolean failed;
+
+    @Override
+    boolean testPoint(String name) {
+      if (doFail && name.equals("startMergeInit")) {
+        failed = true;
+        throw new RuntimeException("intentionally failing");
+      }
+      return true;
+    }
+  }
+  
+
+  // LUCENE-1210
+  public void testExceptionOnMergeInit() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer())
+      .setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler());
+    ((LogMergePolicy) conf.getMergePolicy()).setMergeFactor(2);
+    MockIndexWriter3 w = new MockIndexWriter3(dir, conf);
+    w.doFail = true;
+    Document doc = new Document();
+    doc.add(newField("field", "a field", Field.Store.YES,
+                      Field.Index.ANALYZED));
+    for(int i=0;i<10;i++)
+      try {
+        w.addDocument(doc);
+      } catch (RuntimeException re) {
+        break;
+      }
+
+    ((ConcurrentMergeScheduler) w.getConfig().getMergeScheduler()).sync();
+    assertTrue(w.failed);
+    w.close();
+    dir.close();
+  }
+  
+  // LUCENE-1072
+  public void testExceptionFromTokenStream() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig( TEST_VERSION_CURRENT, new Analyzer() {
+
+      @Override
+      public TokenStream tokenStream(String fieldName, Reader reader) {
+        return new TokenFilter(new MockTokenizer(reader, MockTokenizer.SIMPLE, true)) {
+          private int count = 0;
+
+          @Override
+          public boolean incrementToken() throws IOException {
+            if (count++ == 5) {
+              throw new IOException();
+            }
+            return input.incrementToken();
+          }
+        };
+      }
+
+    });
+    IndexWriter writer = new IndexWriter(dir, conf);
+
+    Document doc = new Document();
+    String contents = "aa bb cc dd ee ff gg hh ii jj kk";
+    doc.add(newField("content", contents, Field.Store.NO,
+        Field.Index.ANALYZED));
+    try {
+      writer.addDocument(doc);
+      fail("did not hit expected exception");
+    } catch (Exception e) {
+    }
+
+    // Make sure we can add another normal document
+    doc = new Document();
+    doc.add(newField("content", "aa bb cc dd", Field.Store.NO,
+        Field.Index.ANALYZED));
+    writer.addDocument(doc);
+
+    // Make sure we can add another normal document
+    doc = new Document();
+    doc.add(newField("content", "aa bb cc dd", Field.Store.NO,
+        Field.Index.ANALYZED));
+    writer.addDocument(doc);
+
+    writer.close();
+    IndexReader reader = IndexReader.open(dir, true);
+    final Term t = new Term("content", "aa");
+    assertEquals(reader.docFreq(t), 3);
+
+    // Make sure the doc that hit the exception was marked
+    // as deleted:
+    DocsEnum tdocs = MultiFields.getTermDocsEnum(reader,
+                                              MultiFields.getDeletedDocs(reader),
+                                              t.field(),
+                                              new BytesRef(t.text()));
+
+    int count = 0;
+    while(tdocs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+      count++;
+    }
+    assertEquals(2, count);
+
+    assertEquals(reader.docFreq(new Term("content", "gg")), 0);
+    reader.close();
+    dir.close();
+  }
+
+  private static class FailOnlyOnFlush extends MockDirectoryWrapper.Failure {
+    boolean doFail = false;
+    int count;
+
+    @Override
+    public void setDoFail() {
+      this.doFail = true;
+    }
+    @Override
+    public void clearDoFail() {
+      this.doFail = false;
+    }
+
+    @Override
+    public void eval(MockDirectoryWrapper dir)  throws IOException {
+      if (doFail) {
+        StackTraceElement[] trace = new Exception().getStackTrace();
+        boolean sawAppend = false;
+        boolean sawFlush = false;
+        for (int i = 0; i < trace.length; i++) {
+          if ("org.apache.lucene.index.FreqProxTermsWriter".equals(trace[i].getClassName()) && "appendPostings".equals(trace[i].getMethodName()))
+            sawAppend = true;
+          if ("doFlush".equals(trace[i].getMethodName()))
+            sawFlush = true;
+        }
+
+        if (sawAppend && sawFlush && count++ >= 30) {
+          doFail = false;
+          throw new IOException("now failing during flush");
+        }
+      }
+    }
+  }
+
+  // LUCENE-1072: make sure an errant exception on flushing
+  // one segment only takes out those docs in that one flush
+  public void testDocumentsWriterAbort() throws IOException {
+    MockDirectoryWrapper dir = newDirectory();
+    FailOnlyOnFlush failure = new FailOnlyOnFlush();
+    failure.setDoFail();
+    dir.failOn(failure);
+
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(2));
+    Document doc = new Document();
+    String contents = "aa bb cc dd ee ff gg hh ii jj kk";
+    doc.add(newField("content", contents, Field.Store.NO,
+        Field.Index.ANALYZED));
+    boolean hitError = false;
+    for(int i=0;i<200;i++) {
+      try {
+        writer.addDocument(doc);
+      } catch (IOException ioe) {
+        // only one flush should fail:
+        assertFalse(hitError);
+        hitError = true;
+      }
+    }
+    assertTrue(hitError);
+    writer.close();
+    IndexReader reader = IndexReader.open(dir, true);
+    assertEquals(198, reader.docFreq(new Term("content", "aa")));
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocumentsWriterExceptions() throws IOException {
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      public TokenStream tokenStream(String fieldName, Reader reader) {
+        return new CrashingFilter(fieldName, new MockTokenizer(reader, MockTokenizer.WHITESPACE, false));
+      }
+    };
+
+    for(int i=0;i<2;i++) {
+      MockDirectoryWrapper dir = newDirectory();
+      IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer));
+      //writer.setInfoStream(System.out);
+      Document doc = new Document();
+      doc.add(newField("contents", "here are some contents", Field.Store.YES,
+                        Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+      writer.addDocument(doc);
+      writer.addDocument(doc);
+      doc.add(newField("crash", "this should crash after 4 terms", Field.Store.YES,
+                        Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+      doc.add(newField("other", "this will not get indexed", Field.Store.YES,
+                        Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+      try {
+        writer.addDocument(doc);
+        fail("did not hit expected exception");
+      } catch (IOException ioe) {
+      }
+
+      if (0 == i) {
+        doc = new Document();
+        doc.add(newField("contents", "here are some contents", Field.Store.YES,
+                          Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+        writer.addDocument(doc);
+        writer.addDocument(doc);
+      }
+      writer.close();
+
+      IndexReader reader = IndexReader.open(dir, true);
+      int expected = 3+(1-i)*2;
+      assertEquals(expected, reader.docFreq(new Term("contents", "here")));
+      assertEquals(expected, reader.maxDoc());
+      int numDel = 0;
+      final Bits delDocs = MultiFields.getDeletedDocs(reader);
+      assertNotNull(delDocs);
+      for(int j=0;j<reader.maxDoc();j++) {
+        if (delDocs.get(j))
+          numDel++;
+        else {
+          reader.document(j);
+          reader.getTermFreqVectors(j);
+        }
+      }
+      reader.close();
+
+      assertEquals(1, numDel);
+
+      writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT,
+          analyzer).setMaxBufferedDocs(10));
+      doc = new Document();
+      doc.add(newField("contents", "here are some contents", Field.Store.YES,
+                        Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+      for(int j=0;j<17;j++)
+        writer.addDocument(doc);
+      writer.optimize();
+      writer.close();
+
+      reader = IndexReader.open(dir, true);
+      expected = 19+(1-i)*2;
+      assertEquals(expected, reader.docFreq(new Term("contents", "here")));
+      assertEquals(expected, reader.maxDoc());
+      numDel = 0;
+      assertNull(MultiFields.getDeletedDocs(reader));
+      for(int j=0;j<reader.maxDoc();j++) {
+        reader.document(j);
+        reader.getTermFreqVectors(j);
+      }
+      reader.close();
+      assertEquals(0, numDel);
+
+      dir.close();
+    }
+  }
+
+  public void testDocumentsWriterExceptionThreads() throws Exception {
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      public TokenStream tokenStream(String fieldName, Reader reader) {
+        return new CrashingFilter(fieldName, new MockTokenizer(reader, MockTokenizer.WHITESPACE, false));
+      }
+    };
+
+    final int NUM_THREAD = 3;
+    final int NUM_ITER = 100;
+
+    for(int i=0;i<2;i++) {
+      MockDirectoryWrapper dir = newDirectory();
+
+      {
+        final IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer).setMaxBufferedDocs(-1));
+        ((LogMergePolicy) writer.getMergePolicy()).setMergeFactor(10);
+        final int finalI = i;
+
+        Thread[] threads = new Thread[NUM_THREAD];
+        for(int t=0;t<NUM_THREAD;t++) {
+          threads[t] = new Thread() {
+              @Override
+              public void run() {
+                try {
+                  for(int iter=0;iter<NUM_ITER;iter++) {
+                    Document doc = new Document();
+                    doc.add(newField("contents", "here are some contents", Field.Store.YES,
+                                      Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+                    writer.addDocument(doc);
+                    writer.addDocument(doc);
+                    doc.add(newField("crash", "this should crash after 4 terms", Field.Store.YES,
+                                      Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+                    doc.add(newField("other", "this will not get indexed", Field.Store.YES,
+                                      Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+                    try {
+                      writer.addDocument(doc);
+                      fail("did not hit expected exception");
+                    } catch (IOException ioe) {
+                    }
+
+                    if (0 == finalI) {
+                      doc = new Document();
+                      doc.add(newField("contents", "here are some contents", Field.Store.YES,
+                                        Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+                      writer.addDocument(doc);
+                      writer.addDocument(doc);
+                    }
+                  }
+                } catch (Throwable t) {
+                  synchronized(this) {
+                    System.out.println(Thread.currentThread().getName() + ": ERROR: hit unexpected exception");
+                    t.printStackTrace(System.out);
+                  }
+                  fail();
+                }
+              }
+            };
+          threads[t].start();
+        }
+
+        for(int t=0;t<NUM_THREAD;t++)
+          threads[t].join();
+            
+        writer.close();
+      }
+
+      IndexReader reader = IndexReader.open(dir, true);
+      int expected = (3+(1-i)*2)*NUM_THREAD*NUM_ITER;
+      assertEquals("i=" + i, expected, reader.docFreq(new Term("contents", "here")));
+      assertEquals(expected, reader.maxDoc());
+      int numDel = 0;
+      final Bits delDocs = MultiFields.getDeletedDocs(reader);
+      assertNotNull(delDocs);
+      for(int j=0;j<reader.maxDoc();j++) {
+        if (delDocs.get(j))
+          numDel++;
+        else {
+          reader.document(j);
+          reader.getTermFreqVectors(j);
+        }
+      }
+      reader.close();
+
+      assertEquals(NUM_THREAD*NUM_ITER, numDel);
+
+      IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
+          TEST_VERSION_CURRENT, analyzer).setMaxBufferedDocs(10));
+      Document doc = new Document();
+      doc.add(newField("contents", "here are some contents", Field.Store.YES,
+                        Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+      for(int j=0;j<17;j++)
+        writer.addDocument(doc);
+      writer.optimize();
+      writer.close();
+
+      reader = IndexReader.open(dir, true);
+      expected += 17-NUM_THREAD*NUM_ITER;
+      assertEquals(expected, reader.docFreq(new Term("contents", "here")));
+      assertEquals(expected, reader.maxDoc());
+      assertNull(MultiFields.getDeletedDocs(reader));
+      for(int j=0;j<reader.maxDoc();j++) {
+        reader.document(j);
+        reader.getTermFreqVectors(j);
+      }
+      reader.close();
+
+      dir.close();
+    }
+  }
+  
+  // Throws IOException during MockDirectoryWrapper.sync
+  private static class FailOnlyInSync extends MockDirectoryWrapper.Failure {
+    boolean didFail;
+    @Override
+    public void eval(MockDirectoryWrapper dir)  throws IOException {
+      if (doFail) {
+        StackTraceElement[] trace = new Exception().getStackTrace();
+        for (int i = 0; i < trace.length; i++) {
+          if (doFail && "org.apache.lucene.store.MockDirectoryWrapper".equals(trace[i].getClassName()) && "sync".equals(trace[i].getMethodName())) {
+            didFail = true;
+            throw new IOException("now failing on purpose during sync");
+          }
+        }
+      }
+    }
+  }
+  
+  // TODO: these are also in TestIndexWriter... add a simple doc-writing method
+  // like this to LuceneTestCase?
+  private void addDoc(IndexWriter writer) throws IOException
+  {
+      Document doc = new Document();
+      doc.add(newField("content", "aaa", Field.Store.NO, Field.Index.ANALYZED));
+      writer.addDocument(doc);
+  }
+  
+  // LUCENE-1044: test exception during sync
+  public void testExceptionDuringSync() throws IOException {
+    MockDirectoryWrapper dir = newDirectory();
+    FailOnlyInSync failure = new FailOnlyInSync();
+    dir.failOn(failure);
+
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer())
+        .setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler()));
+    failure.setDoFail();
+    ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(5);
+
+    for (int i = 0; i < 23; i++) {
+      addDoc(writer);
+      if ((i-1)%2 == 0) {
+        try {
+          writer.commit();
+        } catch (IOException ioe) {
+          // expected
+        }
+      }
+    }
+
+    ((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
+    assertTrue(failure.didFail);
+    failure.clearDoFail();
+    writer.close();
+
+    IndexReader reader = IndexReader.open(dir, true);
+    assertEquals(23, reader.numDocs());
+    reader.close();
+    dir.close();
+  }
+  
+  private static class FailOnlyInCommit extends MockDirectoryWrapper.Failure {
+
+    boolean fail1, fail2;
+
+    @Override
+    public void eval(MockDirectoryWrapper dir)  throws IOException {
+      StackTraceElement[] trace = new Exception().getStackTrace();
+      boolean isCommit = false;
+      boolean isDelete = false;
+      for (int i = 0; i < trace.length; i++) {
+        if ("org.apache.lucene.index.SegmentInfos".equals(trace[i].getClassName()) && "prepareCommit".equals(trace[i].getMethodName()))
+          isCommit = true;
+        if ("org.apache.lucene.store.MockDirectoryWrapper".equals(trace[i].getClassName()) && "deleteFile".equals(trace[i].getMethodName()))
+          isDelete = true;
+      }
+
+      if (isCommit) {
+        if (!isDelete) {
+          fail1 = true;
+          throw new RuntimeException("now fail first");
+        } else {
+          fail2 = true;
+          throw new IOException("now fail during delete");
+        }
+      }
+    }
+  }
+  
+  // LUCENE-1214
+  public void testExceptionsDuringCommit() throws Throwable {
+    MockDirectoryWrapper dir = newDirectory();
+    FailOnlyInCommit failure = new FailOnlyInCommit();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+    Document doc = new Document();
+    doc.add(newField("field", "a field", Field.Store.YES,
+                      Field.Index.ANALYZED));
+    w.addDocument(doc);
+    dir.failOn(failure);
+    try {
+      w.close();
+      fail();
+    } catch (IOException ioe) {
+      fail("expected only RuntimeException");
+    } catch (RuntimeException re) {
+      // Expected
+    }
+    assertTrue(failure.fail1 && failure.fail2);
+    w.rollback();
+    dir.close();
+  }
+  
+  public void testOptimizeExceptions() throws IOException {
+    Directory startDir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxBufferedDocs(2);
+    ((LogMergePolicy) conf.getMergePolicy()).setMergeFactor(100);
+    IndexWriter w = new IndexWriter(startDir, conf);
+    for(int i=0;i<27;i++)
+      addDoc(w);
+    w.close();
+
+    for(int i=0;i<200;i++) {
+      MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory(startDir));
+      conf = newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMergeScheduler(new ConcurrentMergeScheduler());
+      ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
+      w = new IndexWriter(dir, conf);
+      dir.setRandomIOExceptionRate(0.5, 100);
+      try {
+        w.optimize();
+      } catch (IOException ioe) {
+        if (ioe.getCause() == null)
+          fail("optimize threw IOException without root cause");
+      }
+      dir.setRandomIOExceptionRate(0, 0);
+      w.close();
+      dir.close();
+    }
+    startDir.close();
+  }
+  
+  // LUCENE-1429
+  public void testOutOfMemoryErrorCausesCloseToFail() throws Exception {
+
+    final List<Throwable> thrown = new ArrayList<Throwable>();
+    final Directory dir = newDirectory();
+    final IndexWriter writer = new IndexWriter(dir,
+        newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer())) {
+        @Override
+        public void message(final String message) {
+          if (message.startsWith("now flush at close") && 0 == thrown.size()) {
+            thrown.add(null);
+            throw new OutOfMemoryError("fake OOME at " + message);
+          }
+        }
+      };
+
+    // need to set an info stream so message is called
+    writer.setInfoStream(new PrintStream(new ByteArrayOutputStream()));
+    try {
+      writer.close();
+      fail("OutOfMemoryError expected");
+    }
+    catch (final OutOfMemoryError expected) {}
+
+    // throws IllegalStateEx w/o bug fix
+    writer.close();
+    dir.close();
+  }
+  
+  // LUCENE-1347
+  private static final class MockIndexWriter4 extends IndexWriter {
+
+    public MockIndexWriter4(Directory dir, IndexWriterConfig conf) throws IOException {
+      super(dir, conf);
+    }
+
+    boolean doFail;
+
+    @Override
+    boolean testPoint(String name) {
+      if (doFail && name.equals("rollback before checkpoint"))
+        throw new RuntimeException("intentionally failing");
+      return true;
+    }
+  }
+  
+  // LUCENE-1347
+  public void testRollbackExceptionHang() throws Throwable {
+    Directory dir = newDirectory();
+    MockIndexWriter4 w = new MockIndexWriter4(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+
+    addDoc(w);
+    w.doFail = true;
+    try {
+      w.rollback();
+      fail("did not hit intentional RuntimeException");
+    } catch (RuntimeException re) {
+      // expected
+    }
+    
+    w.doFail = false;
+    w.rollback();
+    dir.close();
+  }
+  
+  // LUCENE-1044: Simulate checksum error in segments_N
+  public void testSegmentsChecksumError() throws IOException {
+    Directory dir = newDirectory();
+
+    IndexWriter writer = null;
+
+    writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+
+    // add 100 documents
+    for (int i = 0; i < 100; i++) {
+      addDoc(writer);
+    }
+
+    // close
+    writer.close();
+
+    long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
+    assertTrue("segment generation should be > 0 but got " + gen, gen > 0);
+
+    final String segmentsFileName = SegmentInfos.getCurrentSegmentFileName(dir);
+    IndexInput in = dir.openInput(segmentsFileName);
+    IndexOutput out = dir.createOutput(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", 1+gen));
+    out.copyBytes(in, in.length()-1);
+    byte b = in.readByte();
+    out.writeByte((byte) (1+b));
+    out.close();
+    in.close();
+
+    IndexReader reader = null;
+    try {
+      reader = IndexReader.open(dir, true);
+    } catch (IOException e) {
+      e.printStackTrace(System.out);
+      fail("segmentInfos failed to retry fallback to correct segments_N file");
+    }
+    reader.close();
+    dir.close();
+  }
+  
+  // Simulate a corrupt index by removing last byte of
+  // latest segments file and make sure we get an
+  // IOException trying to open the index:
+  public void testSimulatedCorruptIndex1() throws IOException {
+      Directory dir = newDirectory();
+
+      IndexWriter writer = null;
+
+      writer  = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+
+      // add 100 documents
+      for (int i = 0; i < 100; i++) {
+          addDoc(writer);
+      }
+
+      // close
+      writer.close();
+
+      long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
+      assertTrue("segment generation should be > 0 but got " + gen, gen > 0);
+
+      String fileNameIn = SegmentInfos.getCurrentSegmentFileName(dir);
+      String fileNameOut = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS,
+                                                                 "",
+                                                                 1+gen);
+      IndexInput in = dir.openInput(fileNameIn);
+      IndexOutput out = dir.createOutput(fileNameOut);
+      long length = in.length();
+      for(int i=0;i<length-1;i++) {
+        out.writeByte(in.readByte());
+      }
+      in.close();
+      out.close();
+      dir.deleteFile(fileNameIn);
+
+      IndexReader reader = null;
+      try {
+        reader = IndexReader.open(dir, true);
+        fail("reader did not hit IOException on opening a corrupt index");
+      } catch (Exception e) {
+      }
+      if (reader != null) {
+        reader.close();
+      }
+      dir.close();
+  }
+  
+  // Simulate a corrupt index by removing one of the cfs
+  // files and make sure we get an IOException trying to
+  // open the index:
+  public void testSimulatedCorruptIndex2() throws IOException {
+      Directory dir = newDirectory();
+
+      IndexWriter writer = null;
+
+      writer  = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+      ((LogMergePolicy) writer.getMergePolicy()).setUseCompoundFile(true);
+
+      // add 100 documents
+      for (int i = 0; i < 100; i++) {
+          addDoc(writer);
+      }
+
+      // close
+      writer.close();
+
+      long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
+      assertTrue("segment generation should be > 0 but got " + gen, gen > 0);
+
+      String[] files = dir.listAll();
+      boolean corrupted = false;
+      for(int i=0;i<files.length;i++) {
+        if (files[i].endsWith(".cfs")) {
+          dir.deleteFile(files[i]);
+          corrupted = true;
+          break;
+        }
+      }
+      assertTrue("failed to find cfs file to remove", corrupted);
+
+      IndexReader reader = null;
+      try {
+        reader = IndexReader.open(dir, true);
+        fail("reader did not hit IOException on opening a corrupt index");
+      } catch (Exception e) {
+      }
+      if (reader != null) {
+        reader.close();
+      }
+      dir.close();
+  }
+  
+  // Simulate a writer that crashed while writing segments
+  // file: make sure we can still open the index (ie,
+  // gracefully fallback to the previous segments file),
+  // and that we can add to the index:
+  public void testSimulatedCrashedWriter() throws IOException {
+      MockDirectoryWrapper dir = newDirectory();
+      dir.setPreventDoubleWrite(false);
+
+      IndexWriter writer = null;
+
+      writer  = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+
+      // add 100 documents
+      for (int i = 0; i < 100; i++) {
+          addDoc(writer);
+      }
+
+      // close
+      writer.close();
+
+      long gen = SegmentInfos.getCurrentSegmentGeneration(dir);
+      assertTrue("segment generation should be > 0 but got " + gen, gen > 0);
+
+      // Make the next segments file, with last byte
+      // missing, to simulate a writer that crashed while
+      // writing segments file:
+      String fileNameIn = SegmentInfos.getCurrentSegmentFileName(dir);
+      String fileNameOut = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS,
+                                                                 "",
+                                                                 1+gen);
+      IndexInput in = dir.openInput(fileNameIn);
+      IndexOutput out = dir.createOutput(fileNameOut);
+      long length = in.length();
+      for(int i=0;i<length-1;i++) {
+        out.writeByte(in.readByte());
+      }
+      in.close();
+      out.close();
+
+      IndexReader reader = null;
+      try {
+        reader = IndexReader.open(dir, true);
+      } catch (Exception e) {
+        fail("reader failed to open on a crashed index");
+      }
+      reader.close();
+
+      try {
+        writer  = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.CREATE));
+      } catch (Exception e) {
+        e.printStackTrace(System.out);
+        fail("writer failed to open on a crashed index");
+      }
+
+      // add 100 documents
+      for (int i = 0; i < 100; i++) {
+          addDoc(writer);
+      }
+
+      // close
+      writer.close();
+      dir.close();
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java Fri Nov 12 09:15:30 2010
@@ -164,7 +164,7 @@ public class TestIndexWriterMergePolicy 
       addDoc(writer);
     }
     writer.commit();
-    ((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
+    writer.waitForMerges();
     writer.commit();
     checkInvariants(writer);
 
@@ -203,7 +203,7 @@ public class TestIndexWriterMergePolicy 
       addDoc(writer);
     }
     writer.commit();
-    ((ConcurrentMergeScheduler) writer.getConfig().getMergeScheduler()).sync();
+    writer.waitForMerges();
     writer.commit();
     checkInvariants(writer);
     assertEquals(10, writer.maxDoc());

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java Fri Nov 12 09:15:30 2010
@@ -81,7 +81,7 @@ public class TestSegmentMerger extends L
     assertTrue(docsMerged == 2);
     //Should be able to open a new SegmentReader against the new directory
     SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, -1,
-        null, false, merger.hasProx(), merger.getCodec()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, null);
+        null, false, merger.hasProx(), merger.getSegmentCodecs()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
 
     assertTrue(mergedReader != null);
     assertTrue(mergedReader.numDocs() == 2);

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java Fri Nov 12 09:15:30 2010
@@ -81,6 +81,16 @@ public class MockSingleIntIndexInput ext
     }
 
     @Override
+    public void read(IntIndexInput.Reader indexIn, boolean absolute)
+      throws IOException {
+      if (absolute) {
+        fp = indexIn.readVLong();
+      } else {
+        fp += indexIn.readVLong();
+      }
+    }
+
+    @Override
     public void set(IntIndexInput.Index other) {
       fp = ((Index) other).fp;
     }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java Fri Nov 12 09:15:30 2010
@@ -76,6 +76,18 @@ public class MockSingleIntIndexOutput ex
       }
       lastFP = fp;
     }
+
+    @Override
+    public void write(IntIndexOutput indexOut, boolean absolute) 
+      throws IOException {
+      if (absolute) {
+        indexOut.writeVLong(fp);
+      } else {
+        indexOut.writeVLong(fp - lastFP);
+      }
+      lastFP = fp;
+    }
+      
     @Override
     public String toString() {
       return Long.toString(fp);

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java Fri Nov 12 09:15:30 2010
@@ -63,21 +63,19 @@ public class TestIndexValues extends Luc
 
   // TODO test addIndexes
   private static DocValuesCodec docValuesCodec;
-
+  private static CodecProvider provider;
   @BeforeClass
   public static void beforeClassLuceneTestCaseJ4() {
     LuceneTestCase.beforeClassLuceneTestCaseJ4();
-    final CodecProvider cp = CodecProvider.getDefault();
-    docValuesCodec = new DocValuesCodec(cp.lookup(CodecProvider
+    provider = new CodecProvider();
+    docValuesCodec = new DocValuesCodec(CodecProvider.getDefault().lookup(CodecProvider
         .getDefaultCodec()));
-    cp.register(docValuesCodec);
-    CodecProvider.setDefaultCodec(docValuesCodec.name);
+    provider.register(docValuesCodec);
+    provider.setDefaultFieldCodec(docValuesCodec.name);
   }
 
   @AfterClass
   public static void afterClassLuceneTestCaseJ4() {
-    final CodecProvider cp = CodecProvider.getDefault();
-    cp.unregister(docValuesCodec);
     LuceneTestCase.afterClassLuceneTestCaseJ4();
   }
 
@@ -412,6 +410,7 @@ public class TestIndexValues extends Luc
       policy.setUseCompoundFile(useCompoundFile);
       cfg.setMergePolicy(policy);
     }
+    cfg.setCodecProvider(provider);
     return cfg;
   }
 
@@ -434,9 +433,9 @@ public class TestIndexValues extends Luc
       switch (val) {
       case PACKED_INTS:
       case PACKED_INTS_FIXED: {
-        if(val == Values.PACKED_INTS_FIXED)
-          getDocValues(r, val.name());
         DocValues intsReader = getDocValues(r, val.name());
+        assertNotNull(intsReader);
+
         Source ints = getSource(intsReader);
         
         ValuesEnum intsEnum = intsReader.getEnum();

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/queryParser/TestQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/queryParser/TestQueryParser.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/queryParser/TestQueryParser.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/queryParser/TestQueryParser.java Fri Nov 12 09:15:30 2010
@@ -770,11 +770,11 @@ public class TestQueryParser extends Luc
 
     assertQueryEquals("a:b\\\\c*", a, "a:b\\c*");
 
-    assertQueryEquals("a:b\\-?c", a, "a:b-?c");
-    assertQueryEquals("a:b\\+?c", a, "a:b+?c");
-    assertQueryEquals("a:b\\:?c", a, "a:b:?c");
+    assertQueryEquals("a:b\\-?c", a, "a:b\\-?c");
+    assertQueryEquals("a:b\\+?c", a, "a:b\\+?c");
+    assertQueryEquals("a:b\\:?c", a, "a:b\\:?c");
 
-    assertQueryEquals("a:b\\\\?c", a, "a:b\\?c");
+    assertQueryEquals("a:b\\\\?c", a, "a:b\\\\?c");
 
     assertQueryEquals("a:b\\-c~", a, "a:b-c~2.0");
     assertQueryEquals("a:b\\+c~", a, "a:b+c~2.0");
@@ -1062,6 +1062,12 @@ public class TestQueryParser extends Luc
 
   }
 
+  public void testEscapedWildcard() throws Exception {
+    QueryParser qp = new QueryParser(TEST_VERSION_CURRENT, "field", new MockAnalyzer(MockTokenizer.WHITESPACE, false));
+    WildcardQuery q = new WildcardQuery(new Term("field", "foo\\?ba?r"));
+    assertEquals(q, qp.parse("foo\\?ba?r"));
+  }
+  
   public void testRegexps() throws Exception {
     QueryParser qp = new QueryParser(TEST_VERSION_CURRENT, "field", new MockAnalyzer(MockTokenizer.WHITESPACE, false));
     RegexpQuery q = new RegexpQuery(new Term("field", "[a-z][123]"));

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestPhraseQuery.java Fri Nov 12 09:15:30 2010
@@ -596,8 +596,6 @@ public class TestPhraseQuery extends Luc
   }
 
   public void testRandomPhrases() throws Exception {
-    assumeFalse("test runs extremely slow (minutes) with SimpleText", 
-        CodecProvider.getDefaultCodec().equals("SimpleText"));
     Directory dir = newDirectory();
     Analyzer analyzer = new MockAnalyzer();
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcard.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcard.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcard.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/search/TestWildcard.java Fri Nov 12 09:15:30 2010
@@ -205,6 +205,38 @@ public class TestWildcard
     indexStore.close();
   }
 
+  /**
+   * Tests if wildcard escaping works
+   */
+  public void testEscapes() throws Exception {
+    Directory indexStore = getIndexStore("field", 
+        new String[]{"foo*bar", "foo??bar", "fooCDbar", "fooSOMETHINGbar", "foo\\"});
+    IndexSearcher searcher = new IndexSearcher(indexStore, true);
+
+    // without escape: matches foo??bar, fooCDbar, foo*bar, and fooSOMETHINGbar
+    WildcardQuery unescaped = new WildcardQuery(new Term("field", "foo*bar"));
+    assertMatches(searcher, unescaped, 4);
+    
+    // with escape: only matches foo*bar
+    WildcardQuery escaped = new WildcardQuery(new Term("field", "foo\\*bar"));
+    assertMatches(searcher, escaped, 1);
+    
+    // without escape: matches foo??bar and fooCDbar
+    unescaped = new WildcardQuery(new Term("field", "foo??bar"));
+    assertMatches(searcher, unescaped, 2);
+    
+    // with escape: matches foo??bar only
+    escaped = new WildcardQuery(new Term("field", "foo\\?\\?bar"));
+    assertMatches(searcher, escaped, 1);
+    
+    // check escaping at end: lenient parse yields "foo\"
+    WildcardQuery atEnd = new WildcardQuery(new Term("field", "foo\\"));
+    assertMatches(searcher, atEnd, 1);
+    
+    searcher.close();
+    indexStore.close();
+  }
+  
   private Directory getIndexStore(String field, String[] contents)
       throws IOException {
     Directory indexStore = newDirectory();

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/LuceneTestCase.java Fri Nov 12 09:15:30 2010
@@ -230,6 +230,7 @@ public abstract class LuceneTestCase ext
     }
 
     CodecProvider.setDefaultCodec(codec);
+    cp.setDefaultFieldCodec(codec);
 
     if (codec.equals("PreFlex")) {
       // If we're running w/ PreFlex codec we must swap in the
@@ -262,6 +263,8 @@ public abstract class LuceneTestCase ext
     cp.unregister(cp.lookup("MockVariableIntBlock"));
     swapCodec(new PulsingCodec(1));
     CodecProvider.setDefaultCodec(savedDefaultCodec);
+    cp.setDefaultFieldCodec(savedDefaultCodec);
+
   }
 
   // randomly picks from core and test codecs

Propchange: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
-/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java:943137,949730,957490,960490,961612,979161,980654,982195,987811,988512,1025544,1026614
+/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java:943137,949730,957490,960490,961612,979161,980654,982195,987811,988512,1025544,1026614,1034080
 /lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java:967125-979432
-/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java:1021634-1029001
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java:1021634-1034284
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/TestAttributeSource.java:824912-931101
 /lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/util/TestAttributeSource.java:909334,948516
 /lucene/java/trunk/src/test/org/apache/lucene/util/TestAttributeSource.java:924483-924731,924781,925176-925462

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java Fri Nov 12 09:15:30 2010
@@ -69,11 +69,17 @@ public class _TestUtil {
    *  issues are hit, a RuntimeException is thrown; else,
    *  true is returned. */
   public static CheckIndex.Status checkIndex(Directory dir) throws IOException {
+    return checkIndex(dir, CodecProvider.getDefault());
+  }
+  
+  /** This runs the CheckIndex tool on the index in.  If any
+   *  issues are hit, a RuntimeException is thrown; else,
+   *  true is returned. */
+  public static CheckIndex.Status checkIndex(Directory dir, CodecProvider codecs) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-
     CheckIndex checker = new CheckIndex(dir);
     checker.setInfoStream(new PrintStream(bos));
-    CheckIndex.Status indexStatus = checker.checkIndex();
+    CheckIndex.Status indexStatus = checker.checkIndex(null, codecs);
     if (indexStatus == null || indexStatus.clean == false) {
       System.out.println("CheckIndex failed");
       System.out.println(bos.toString());
@@ -217,12 +223,7 @@ public class _TestUtil {
   }
 
   public static CodecProvider alwaysCodec(final Codec c) {
-    return new CodecProvider() {
-      @Override
-      public Codec getWriter(SegmentWriteState state) {
-        return c;
-      }
-
+    CodecProvider p = new CodecProvider() {
       @Override
       public Codec lookup(String name) {
         // can't do this until we fix PreFlexRW to not
@@ -234,6 +235,8 @@ public class _TestUtil {
         }
       }
     };
+    p.setDefaultFieldCodec(c.name);
+    return p;
   }
 
   /** Return a CodecProvider that can read any of the

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/automaton/AutomatonTestUtil.java Fri Nov 12 09:15:30 2010
@@ -92,7 +92,7 @@ public class AutomatonTestUtil {
     } else if (t.min >= UnicodeUtil.UNI_SUR_HIGH_START) {
       if (t.max > UnicodeUtil.UNI_SUR_LOW_END) {
         // after surrogates
-        code = 1+UnicodeUtil.UNI_SUR_LOW_END+r.nextInt(t.max-UnicodeUtil.UNI_SUR_LOW_END+1);
+        code = 1+UnicodeUtil.UNI_SUR_LOW_END+r.nextInt(t.max-UnicodeUtil.UNI_SUR_LOW_END);
       } else {
         throw new IllegalArgumentException("transition accepts only surrogates: " + t);
       }

Modified: lucene/dev/branches/docvalues/modules/analysis/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/CHANGES.txt?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/CHANGES.txt (original)
+++ lucene/dev/branches/docvalues/modules/analysis/CHANGES.txt Fri Nov 12 09:15:30 2010
@@ -1,7 +1,7 @@
 Analysis Module Change Log
 
 ======================= Trunk (not yet released) =======================
-
+   
 API Changes
 
  * LUCENE-2413: Deprecated PatternAnalyzer in common/miscellaneous, in favor 

Modified: lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java (original)
+++ lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/CompoundWordTokenFilterBase.java Fri Nov 12 09:15:30 2010
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedList;
+import java.util.Locale;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Token;
@@ -224,7 +225,7 @@ public abstract class CompoundWordTokenF
   protected static final void addAllLowerCase(CharArraySet target, Collection<?> col) {
     for (Object obj : col) {
       String string = (String) obj;
-      target.add(string.toLowerCase());
+      target.add(string.toLowerCase(Locale.ENGLISH));
     }
   }
   

Modified: lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java (original)
+++ lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/HyphenationCompoundWordTokenFilter.java Fri Nov 12 09:15:30 2010
@@ -19,7 +19,6 @@ package org.apache.lucene.analysis.compo
 
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.InputStreamReader;
 import java.io.Reader;
 import java.util.Set;
 
@@ -267,7 +266,7 @@ public class HyphenationCompoundWordToke
    */
   public static HyphenationTree getHyphenationTree(String hyphenationFilename)
       throws Exception {
-    return getHyphenationTree(new File(hyphenationFilename));
+    return getHyphenationTree(new InputSource(hyphenationFilename));
   }
 
   /**
@@ -279,8 +278,7 @@ public class HyphenationCompoundWordToke
    */
   public static HyphenationTree getHyphenationTree(File hyphenationFile)
       throws Exception {
-    return getHyphenationTree(new InputStreamReader(new FileInputStream(
-        hyphenationFile), "ISO-8859-1"));
+    return getHyphenationTree(new InputSource(hyphenationFile.toURL().toExternalForm()));
   }
 
   /**
@@ -289,13 +287,32 @@ public class HyphenationCompoundWordToke
    * @param hyphenationReader the reader of the XML grammar to load from
    * @return An object representing the hyphenation patterns
    * @throws Exception
+   * @deprecated Don't use Readers with fixed charset to load XML files, unless programatically created.
+   * Use {@link #getHyphenationTree(InputSource)} instead, where you can supply default charset and input
+   * stream, if you like.
    */
+  @Deprecated
   public static HyphenationTree getHyphenationTree(Reader hyphenationReader)
       throws Exception {
-    HyphenationTree tree = new HyphenationTree();
-
-    tree.loadPatterns(new InputSource(hyphenationReader));
+    final InputSource is = new InputSource(hyphenationReader);
+    // we need this to load the DTD in very old parsers (like the one in JDK 1.4).
+    // The DTD itsself is provided via EntityResolver, so it should always load, but
+    // some parsers still want to have a base URL (Crimson).
+    is.setSystemId("urn:java:" + HyphenationTree.class.getName());
+    return getHyphenationTree(is);
+  }
 
+  /**
+   * Create a hyphenator tree
+   * 
+   * @param hyphenationSource the InputSource pointing to the XML grammar
+   * @return An object representing the hyphenation patterns
+   * @throws Exception
+   */
+  public static HyphenationTree getHyphenationTree(InputSource hyphenationSource)
+      throws Exception {
+    HyphenationTree tree = new HyphenationTree();
+    tree.loadPatterns(hyphenationSource);
     return tree;
   }
 

Modified: lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java (original)
+++ lucene/dev/branches/docvalues/modules/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/PatternParser.java Fri Nov 12 09:15:30 2010
@@ -91,7 +91,7 @@ public class PatternParser extends Defau
    * @throws HyphenationException In case of an exception while parsing
    */
   public void parse(String filename) throws HyphenationException {
-    parse(new File(filename));
+    parse(new InputSource(filename));
   }
 
   /**
@@ -266,7 +266,15 @@ public class PatternParser extends Defau
   //
   @Override
   public InputSource resolveEntity(String publicId, String systemId) {
-    return HyphenationDTDGenerator.generateDTD();
+    // supply the internal hyphenation.dtd if possible
+    if (
+      (systemId != null && systemId.matches("(?i).*\\bhyphenation.dtd\\b.*")) ||
+      ("hyphenation-info".equals(publicId))
+    ) {
+      // System.out.println(this.getClass().getResource("hyphenation.dtd").toExternalForm());
+      return new InputSource(this.getClass().getResource("hyphenation.dtd").toExternalForm());
+    }
+    return null;
   }
 
   //
@@ -373,35 +381,6 @@ public class PatternParser extends Defau
 
   }
 
-  //
-  // ErrorHandler methods
-  //
-
-  /**
-   * @see org.xml.sax.ErrorHandler#warning(org.xml.sax.SAXParseException)
-   */
-  @Override
-  public void warning(SAXParseException ex) {
-    errMsg = "[Warning] " + getLocationString(ex) + ": " + ex.getMessage();
-  }
-
-  /**
-   * @see org.xml.sax.ErrorHandler#error(org.xml.sax.SAXParseException)
-   */
-  @Override
-  public void error(SAXParseException ex) {
-    errMsg = "[Error] " + getLocationString(ex) + ": " + ex.getMessage();
-  }
-
-  /**
-   * @see org.xml.sax.ErrorHandler#fatalError(org.xml.sax.SAXParseException)
-   */
-  @Override
-  public void fatalError(SAXParseException ex) throws SAXException {
-    errMsg = "[Fatal Error] " + getLocationString(ex) + ": " + ex.getMessage();
-    throw ex;
-  }
-
   /**
    * Returns a string of the location.
    */
@@ -446,79 +425,3 @@ public class PatternParser extends Defau
     }
   }
 }
-
-class HyphenationDTDGenerator {
-  public static final String DTD_STRING=
-    "<?xml version=\"1.0\" encoding=\"US-ASCII\"?>\n"+
-    "<!--\n"+
-    "  Copyright 1999-2004 The Apache Software Foundation\n"+
-    "\n"+
-    "  Licensed under the Apache License, Version 2.0 (the \"License\");\n"+
-    "  you may not use this file except in compliance with the License.\n"+
-    "  You may obtain a copy of the License at\n"+
-    "\n"+
-    "       http://www.apache.org/licenses/LICENSE-2.0\n"+
-    "\n"+
-    "  Unless required by applicable law or agreed to in writing, software\n"+
-    "  distributed under the License is distributed on an \"AS IS\" BASIS,\n"+
-    "  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n"+
-    "  See the License for the specific language governing permissions and\n"+
-    "  limitations under the License.\n"+
-    "-->\n"+
-    "<!-- $Id: hyphenation.dtd,v 1.3 2004/02/27 18:34:59 jeremias Exp $ -->\n"+
-    "\n"+
-    "<!ELEMENT hyphenation-info (hyphen-char?, hyphen-min?,\n"+
-    "                           classes, exceptions?, patterns)>\n"+
-    "\n"+
-    "<!-- Hyphen character to be used in the exception list as shortcut for\n"+
-    "     <hyphen pre-break=\"-\"/>. Defaults to '-'\n"+
-    "-->\n"+
-    "<!ELEMENT hyphen-char EMPTY>\n"+
-    "<!ATTLIST hyphen-char value CDATA #REQUIRED>\n"+
-    "\n"+
-    "<!-- Default minimun length in characters of hyphenated word fragments\n"+
-    "     before and after the line break. For some languages this is not\n"+
-    "     only for aesthetic purposes, wrong hyphens may be generated if this\n"+
-    "     is not accounted for.\n"+
-    "-->\n"+
-    "<!ELEMENT hyphen-min EMPTY>\n"+
-    "<!ATTLIST hyphen-min before CDATA #REQUIRED>\n"+
-    "<!ATTLIST hyphen-min after CDATA #REQUIRED>\n"+
-    "\n"+
-    "<!-- Character equivalent classes: space separated list of character groups, all\n"+
-    "     characters in a group are to be treated equivalent as far as\n"+
-    "     the hyphenation algorithm is concerned. The first character in a group\n"+
-    "     is the group's equivalent character. Patterns should only contain\n"+
-    "     first characters. It also defines word characters, i.e. a word that\n"+
-    "     contains characters not present in any of the classes is not hyphenated.\n"+
-    "-->\n"+
-    "<!ELEMENT classes (#PCDATA)>\n"+
-    "\n"+
-    "<!-- Hyphenation exceptions: space separated list of hyphenated words.\n"+
-    "     A hyphen is indicated by the hyphen tag, but you can use the\n"+
-    "     hyphen-char defined previously as shortcut. This is in cases\n"+
-    "     when the algorithm procedure finds wrong hyphens or you want\n"+
-    "     to provide your own hyphenation for some words.\n"+
-    "-->\n"+
-    "<!ELEMENT exceptions (#PCDATA|hyphen)* >\n"+
-    "\n"+
-    "<!-- The hyphenation patterns, space separated. A pattern is made of 'equivalent'\n"+
-    "     characters as described before, between any two word characters a digit\n"+
-    "     in the range 0 to 9 may be specified. The absence of a digit is equivalent\n"+
-    "     to zero. The '.' character is reserved to indicate begining or ending\n"+
-    "     of words. -->\n"+
-    "<!ELEMENT patterns (#PCDATA)>\n"+
-    "\n"+
-    "<!-- A \"full hyphen\" equivalent to TeX's \\discretionary\n"+
-    "     with pre-break, post-break and no-break attributes.\n"+
-    "     To be used in the exceptions list, the hyphen character is not\n"+
-    "     automatically added -->\n"+
-    "<!ELEMENT hyphen EMPTY>\n"+
-    "<!ATTLIST hyphen pre CDATA #IMPLIED>\n"+
-    "<!ATTLIST hyphen no CDATA #IMPLIED>\n"+
-    "<!ATTLIST hyphen post CDATA #IMPLIED>\n";
-  
- public static InputSource generateDTD() {
-    return new InputSource(new StringReader(DTD_STRING));
-  }
-}

Modified: lucene/dev/branches/docvalues/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java (original)
+++ lucene/dev/branches/docvalues/modules/analysis/common/src/test/org/apache/lucene/analysis/compound/TestCompoundWordTokenFilter.java Fri Nov 12 09:15:30 2010
@@ -17,9 +17,9 @@ package org.apache.lucene.analysis.compo
  * limitations under the License.
  */
 
-import java.io.InputStreamReader;
 import java.io.Reader;
 import java.io.StringReader;
+import org.xml.sax.InputSource;
 
 import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.Tokenizer;
@@ -31,10 +31,9 @@ public class TestCompoundWordTokenFilter
   public void testHyphenationCompoundWordsDA() throws Exception {
     String[] dict = { "læse", "hest" };
 
-    Reader reader = getHyphenationReader();
-
+    InputSource is = new InputSource(getClass().getResource("da_UTF8.xml").toExternalForm());
     HyphenationTree hyphenator = HyphenationCompoundWordTokenFilter
-        .getHyphenationTree(reader);
+        .getHyphenationTree(is);
 
     HyphenationCompoundWordTokenFilter tf = new HyphenationCompoundWordTokenFilter(TEST_VERSION_CURRENT, 
         new WhitespaceTokenizer(TEST_VERSION_CURRENT, new StringReader(
@@ -50,10 +49,10 @@ public class TestCompoundWordTokenFilter
 
   public void testHyphenationCompoundWordsDELongestMatch() throws Exception {
     String[] dict = { "basketball", "basket", "ball", "kurv" };
-    Reader reader = getHyphenationReader();
 
+    InputSource is = new InputSource(getClass().getResource("da_UTF8.xml").toExternalForm());
     HyphenationTree hyphenator = HyphenationCompoundWordTokenFilter
-        .getHyphenationTree(reader);
+        .getHyphenationTree(is);
 
     // the word basket will not be added due to the longest match option
     HyphenationCompoundWordTokenFilter tf = new HyphenationCompoundWordTokenFilter(TEST_VERSION_CURRENT, 
@@ -73,9 +72,9 @@ public class TestCompoundWordTokenFilter
    * This can be controlled with the min/max subword size.
    */
   public void testHyphenationOnly() throws Exception {
-    Reader reader = getHyphenationReader();
+    InputSource is = new InputSource(getClass().getResource("da_UTF8.xml").toExternalForm());
     HyphenationTree hyphenator = HyphenationCompoundWordTokenFilter
-      .getHyphenationTree(reader);
+        .getHyphenationTree(is);
     
     HyphenationCompoundWordTokenFilter tf = new HyphenationCompoundWordTokenFilter(
         TEST_VERSION_CURRENT,
@@ -185,7 +184,4 @@ public class TestCompoundWordTokenFilter
     assertEquals("Rindfleischüberwachungsgesetz", termAtt.toString());
   }
 
-  private Reader getHyphenationReader() throws Exception {
-    return new InputStreamReader(getClass().getResourceAsStream("da_UTF8.xml"), "UTF-8");
-  }
 }

Propchange: lucene/dev/branches/docvalues/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestISOLatin1AccentFilter.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
-/lucene/dev/branches/branch_3x/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestISOLatin1AccentFilter.java:980654,982195,987811,988512,1025544,1026614
+/lucene/dev/branches/branch_3x/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestISOLatin1AccentFilter.java:980654,982195,987811,988512,1025544,1026614,1034080
 /lucene/dev/branches/preflexfixes/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestISOLatin1AccentFilter.java:967125-979432
-/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestISOLatin1AccentFilter.java:1021634-1029001
+/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestISOLatin1AccentFilter.java:1021634-1034284
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:824912-931101
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:748824
 /lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:829134,829881,831036,896850,909334

Propchange: lucene/dev/branches/docvalues/solr/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
-/lucene/dev/branches/branch_3x/solr:949730,957490*,961612,979161,980654,982195,987811,988512,1025544,1026614
+/lucene/dev/branches/branch_3x/solr:949730,957490*,961612,979161,980654,982195,987811,988512,1025544,1026614,1034080
 /lucene/dev/branches/preflexfixes/solr:967125-979432
-/lucene/dev/trunk/solr:1021634-1029001
+/lucene/dev/trunk/solr:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr:748824
 /lucene/java/branches/lucene_2_9/solr:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr:818601-821336

Modified: lucene/dev/branches/docvalues/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/CHANGES.txt?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/CHANGES.txt (original)
+++ lucene/dev/branches/docvalues/solr/CHANGES.txt Fri Nov 12 09:15:30 2010
@@ -297,6 +297,14 @@ New Features
   built-in load balancing, and infrastructure for future SolrCloud work. 
   (yonik, Mark Miller)
 
+* SOLR-2210: Add icu-based tokenizer and filters to contrib/analysis-extras (rmuir)
+
+* SOLR-1336: Add SmartChinese (word segmentation for Simplified Chinese) 
+  tokenizer and filters to contrib/analysis-extras (rmuir)
+
+* SOLR-2211: Added UAX29TokenizerFactory, which implements UAX#29, a unicode algorithm 
+  with good results for most languages.  (Tom Burton-West via rmuir)
+
 Optimizations
 ----------------------
 
@@ -337,7 +345,8 @@ Optimizations
   improvement is 5%, but can be much greater (up to 10x faster) when facet.offset
   is very large (deep paging). (yonik)
 
-
+* SOLR-2200: Improve the performance of DataImportHandler for large delta-import
+  updates. (Mark Waddle via rmuir)
 
 Bug Fixes
 ----------------------
@@ -534,6 +543,15 @@ Bug Fixes
 * SOLR-2190: change xpath from RSS 0.9 to 1.0 in slashdot sample. (koji)
 
 * SOLR-1962: SolrCore#initIndex should not use a mix of indexPath and newIndexPath (Mark Miller)
+
+* SOLR-2057: DataImportHandler never calls UpdateRequestProcessor.finish()
+  (Drew Farris via koji)
+
+* SOLR-1973: Empty fields in XML update messages confuse DataImportHandler. (koji)
+
+* SOLR-2221: Use StrUtils.parseBool() to get values of boolean options in DIH.
+  true/on/yes (for TRUE) and false/off/no (for FALSE) can be used for sub-options
+  (debug, verbose, synchronous, commit, clean, optimize) for full/delta-import commands. (koji)
   
 Other Changes
 ----------------------
@@ -636,6 +654,9 @@ Other Changes
 
 * SOLR-2013: Add mapping-FoldToASCII.txt to example conf directory.
   (Steven Rowe via koji)
+  
+* SOLR-2213: Upgrade to jQuery 1.4.3 (Erick Erickson via ryan)
+
 
 Build
 ----------------------

Propchange: lucene/dev/branches/docvalues/solr/CHANGES.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/CHANGES.txt:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/CHANGES.txt:967125-979432
-/lucene/dev/trunk/solr/CHANGES.txt:1021634-1029001
+/lucene/dev/trunk/solr/CHANGES.txt:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/CHANGES.txt:748824
 /lucene/java/branches/lucene_2_9/solr/CHANGES.txt:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/CHANGES.txt:818601-821336

Propchange: lucene/dev/branches/docvalues/solr/KEYS
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/KEYS:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/KEYS:967125-979432
-/lucene/dev/trunk/solr/KEYS:1021634-1029001
+/lucene/dev/trunk/solr/KEYS:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/KEYS:748824
 /lucene/java/branches/lucene_2_9/solr/KEYS:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/KEYS:818601-821336

Modified: lucene/dev/branches/docvalues/solr/LICENSE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/LICENSE.txt?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/LICENSE.txt (original)
+++ lucene/dev/branches/docvalues/solr/LICENSE.txt Fri Nov 12 09:15:30 2010
@@ -560,7 +560,7 @@ SOFTWARE. 
 ==========================================================================
 The following license applies to the JQuery JavaScript library
 --------------------------------------------------------------------------
-Copyright (c) 2008 John Resig, http://jquery.com/
+Copyright (c) 2010 John Resig, http://jquery.com/
 
 Permission is hereby granted, free of charge, to any person obtaining
 a copy of this software and associated documentation files (the

Propchange: lucene/dev/branches/docvalues/solr/LICENSE.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/LICENSE.txt:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/LICENSE.txt:967125-979432
-/lucene/dev/trunk/solr/LICENSE.txt:1021634-1029001
+/lucene/dev/trunk/solr/LICENSE.txt:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/LICENSE.txt:748824
 /lucene/java/branches/lucene_2_9/solr/LICENSE.txt:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/LICENSE.txt:818601-821336

Modified: lucene/dev/branches/docvalues/solr/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/NOTICE.txt?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/NOTICE.txt (original)
+++ lucene/dev/branches/docvalues/solr/NOTICE.txt Fri Nov 12 09:15:30 2010
@@ -18,7 +18,7 @@ This product includes tests written with
 Tammo Freese (http://www.easymock.org/)
 
 This product includes the JQuery JavaScript library created by John Resig.
-Copyright (c) 2008 John Resig, http://jquery.com/
+Copyright (c) 2010 John Resig, http://jquery.com/
 
 This product includes the stax-utils jar: https://stax-utils.dev.java.net/
 Copyright (c) 2004, Christian Niles, unit12.net

Propchange: lucene/dev/branches/docvalues/solr/NOTICE.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/NOTICE.txt:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/NOTICE.txt:967125-979432
-/lucene/dev/trunk/solr/NOTICE.txt:1021634-1029001
+/lucene/dev/trunk/solr/NOTICE.txt:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/NOTICE.txt:748824
 /lucene/java/branches/lucene_2_9/solr/NOTICE.txt:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/NOTICE.txt:818601-821336

Propchange: lucene/dev/branches/docvalues/solr/README.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/README.txt:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/README.txt:967125-979432
-/lucene/dev/trunk/solr/README.txt:1021634-1029001
+/lucene/dev/trunk/solr/README.txt:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/README.txt:748824
 /lucene/java/branches/lucene_2_9/solr/README.txt:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/README.txt:818601-821336

Modified: lucene/dev/branches/docvalues/solr/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/build.xml?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/build.xml (original)
+++ lucene/dev/branches/docvalues/solr/build.xml Fri Nov 12 09:15:30 2010
@@ -34,9 +34,6 @@
   <property name="clover.db.dir" location="${dest}/tests/clover/db"/>
   <property name="clover.report.dir" location="${dest}/tests/clover/reports"/>
   
-  <!-- change this together with the default and test's solrconfig.xml after starting a new development branch: -->
-  <property name="tests.luceneMatchVersion" value="4.0"/>
-
     <available
             property="clover.present"
             classname="com.cenqua.clover.tasks.CloverReportTask"
@@ -221,6 +218,7 @@
           <packageset dir="contrib/dataimporthandler/src/main/java" />
           <!--<packageset dir="contrib/clustering/src/main/java" />-->
           <packageset dir="contrib/extraction/src/main/java" />
+          <packageset dir="contrib/analysis-extras/src/java" />
           <group title="Core" packages="org.apache.*" />
           <group title="Common" packages="org.apache.solr.common.*" />
           <group title="SolrJ" packages="org.apache.solr.client.solrj*" />
@@ -509,6 +507,7 @@
       <fileset dir="contrib/dataimporthandler/src/main/java" />
       <fileset dir="contrib/clustering/src/main/java" />
       <fileset dir="contrib/extraction/src/main/java" />
+      <fileset dir="contrib/analysis-extras/src/java" />
     </clover-setup>
   </target>
 
@@ -609,6 +608,8 @@
               basedir="contrib/extraction/src" />
     <!--<solr-jar destfile="${dist}/apache-solr-clustering-src-${version}.jar"
               basedir="contrib/clustering/src" />-->
+    <solr-jar destfile="${dist}/apache-solr-analysis-extras-src-${version}.jar"
+              basedir="contrib/analysis-extras/src" />
   </target>
 
   <target name="dist-javadoc" description="Creates the Solr javadoc distribution files"
@@ -625,6 +626,8 @@
               basedir="${build.javadoc}/contrib-solr-clustering" />-->
     <solr-jar destfile="${dist}/apache-solr-cell-docs-${version}.jar"
               basedir="${build.javadoc}/contrib-solr-cell" />
+    <solr-jar destfile="${dist}/apache-solr-analysis-extras-docs-${version}.jar"
+              basedir="${build.javadoc}/contrib-solr-analysis-extras" />
   </target>
 
   <!-- Creates the solr jar. -->
@@ -721,7 +724,7 @@
       <tarfileset dir="."
         prefix="${fullnamever}"
         includes="LICENSE.txt NOTICE.txt *.txt *.xml lucene-libs/** lib/** src/** example/** client/** contrib/"
-        excludes="lib/README.committers.txt **/data/ **/logs/* **/classes/ **/*.sh **/bin/ src/scripts/ src/site/build/ **/target/ client/ruby/flare/ client/python contrib/**/build/ **/*.iml **/*.ipr **/*.iws contrib/clustering/example/lib/** contrib/clustering/lib/downloads/**" />
+        excludes="lib/README.committers.txt **/data/ **/logs/* **/classes/ **/*.sh **/bin/ src/scripts/ src/site/build/ **/target/ client/ruby/flare/ client/python contrib/**/build/ **/*.iml **/*.ipr **/*.iws contrib/clustering/example/lib/** contrib/clustering/lib/downloads/** contrib/analysis-extras/lib/**" />
       <tarfileset dir="."
         prefix="${fullnamever}"
         includes="src/test/test-files/solr/lib/classes/empty-file-main-lib.txt" />
@@ -952,6 +955,8 @@
       <fileset dir="contrib/clustering/src/test/java"/>
       <fileset dir="contrib/extraction/src/main/java"/>
       <fileset dir="contrib/extraction/src/test/java"/>
+      <fileset dir="contrib/analysis-extras/src/test"/>
+      <fileset dir="contrib/analysis-extras/src/test"/>
     </rat:report>
   </target>
 

Propchange: lucene/dev/branches/docvalues/solr/build.xml
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/build.xml:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/build.xml:967125-979432
-/lucene/dev/trunk/solr/build.xml:1021634-1029001
+/lucene/dev/trunk/solr/build.xml:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/build.xml:748824
 /lucene/java/branches/lucene_2_9/solr/build.xml:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/build.xml:818601-821336

Propchange: lucene/dev/branches/docvalues/solr/client/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/client:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/client:967125-979432
-/lucene/dev/trunk/solr/client:1021634-1029001
+/lucene/dev/trunk/solr/client:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/client:748824
 /lucene/java/branches/lucene_2_9/solr/client:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/client:818601-821336

Modified: lucene/dev/branches/docvalues/solr/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/common-build.xml?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/common-build.xml (original)
+++ lucene/dev/branches/docvalues/solr/common-build.xml Fri Nov 12 09:15:30 2010
@@ -23,6 +23,9 @@
 
   <dirname file="${ant.file.common-solr}" property="common-solr.dir"/>
   
+  <!-- change this together with the default and test's solrconfig.xml after starting a new development branch: -->
+  <property name="tests.luceneMatchVersion" value="4.0"/>
+
   <!-- Initialize property values: allow easy customization via build.properties -->
   <property file="build.properties" />
 

Propchange: lucene/dev/branches/docvalues/solr/common-build.xml
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/common-build.xml:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/common-build.xml:967125-979432
-/lucene/dev/trunk/solr/common-build.xml:1021634-1029001
+/lucene/dev/trunk/solr/common-build.xml:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/common-build.xml:748824
 /lucene/java/branches/lucene_2_9/solr/common-build.xml:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/common-build.xml:818601-821336

Propchange: lucene/dev/branches/docvalues/solr/contrib/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Nov 12 09:15:30 2010
@@ -1,6 +1,6 @@
 /lucene/dev/branches/branch_3x/solr/contrib:949730,961612,979161,980654,982195,987811,988512
 /lucene/dev/branches/preflexfixes/solr/contrib:967125-979432
-/lucene/dev/trunk/solr/contrib:1021634-1029001
+/lucene/dev/trunk/solr/contrib:1021634-1034284
 /lucene/java/branches/lucene_2_4/solr/contrib:748824
 /lucene/java/branches/lucene_2_9/solr/contrib:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/contrib:818601-821336

Propchange: lucene/dev/branches/docvalues/solr/contrib/analysis-extras/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Fri Nov 12 09:15:30 2010
@@ -0,0 +1,2 @@
+build
+lucene-libs

Modified: lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandler.java Fri Nov 12 09:15:30 2010
@@ -26,6 +26,7 @@ import org.apache.solr.common.params.Upd
 import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -101,9 +102,7 @@ public class DataImportHandler extends R
           myName = myName.replaceAll("/","_") ;
         }
       }
-      String debug = (String) initArgs.get(ENABLE_DEBUG);
-      if (debug != null && "no".equals(debug))
-        debugEnabled = false;
+      debugEnabled = StrUtils.parseBool((String)initArgs.get(ENABLE_DEBUG), true);
       NamedList defaults = (NamedList) initArgs.get("defaults");
       if (defaults != null) {
         String configLoc = (String) defaults.get("config");

Modified: lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java Fri Nov 12 09:15:30 2010
@@ -22,6 +22,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.StrUtils;
 
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
@@ -498,29 +499,28 @@ public class DataImporter {
       if (requestParams.containsKey("command"))
         command = (String) requestParams.get("command");
 
-      if ("on".equals(requestParams.get("debug"))) {
+      if (StrUtils.parseBool((String)requestParams.get("debug"),false)) {
         debug = true;
         rows = 10;
         // Set default values suitable for debug mode
         commit = false;
         clean = false;
-        verbose = "true".equals(requestParams.get("verbose"))
-                || "on".equals(requestParams.get("verbose"));
+        verbose = StrUtils.parseBool((String)requestParams.get("verbose"),false);
       }
-      syncMode = "true".equals(requestParams.get("synchronous"));
+      syncMode = StrUtils.parseBool((String)requestParams.get("synchronous"),false);
       if (DELTA_IMPORT_CMD.equals(command) || IMPORT_CMD.equals(command)) {
         clean = false;
       }
       if (requestParams.containsKey("commit"))
-        commit = Boolean.parseBoolean((String) requestParams.get("commit"));
+        commit = StrUtils.parseBool((String) requestParams.get("commit"),true);
       if (requestParams.containsKey("start"))
         start = Integer.parseInt((String) requestParams.get("start"));
       if (requestParams.containsKey("rows"))
         rows = Integer.parseInt((String) requestParams.get("rows"));
       if (requestParams.containsKey("clean"))
-        clean = Boolean.parseBoolean((String) requestParams.get("clean"));
+        clean = StrUtils.parseBool((String) requestParams.get("clean"),true);
       if (requestParams.containsKey("optimize")) {
-        optimize = Boolean.parseBoolean((String) requestParams.get("optimize"));
+        optimize = StrUtils.parseBool((String) requestParams.get("optimize"),true);
         if (optimize)
           commit = true;
       }

Modified: lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java?rev=1034304&r1=1034303&r2=1034304&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java (original)
+++ lucene/dev/branches/docvalues/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java Fri Nov 12 09:15:30 2010
@@ -205,6 +205,11 @@ public class DocBuilder {
         // Finished operation normally, commit now
         finish(lastIndexTimeProps);
       }
+      
+      if (writer != null) {
+        writer.finish();
+      }
+      
       if (document.onImportEnd != null) {
         invokeEventListener(document.onImportEnd);
       }
@@ -841,7 +846,7 @@ public class DocBuilder {
     }
     // identifying the modified rows for this entity
 
-    Set<Map<String, Object>> deltaSet = new HashSet<Map<String, Object>>();
+    Map<String, Map<String, Object>> deltaSet = new HashMap<String, Map<String, Object>>();
     LOG.info("Running ModifiedRowKey() for Entity: " + entity.name);
     //get the modified rows in this entity
     while (true) {
@@ -850,7 +855,7 @@ public class DocBuilder {
       if (row == null)
         break;
 
-      deltaSet.add(row);
+      deltaSet.put(row.get(entity.getPk()).toString(), row);
       importStatistics.rowsCount.incrementAndGet();
       // check for abort
       if (stop.get())
@@ -858,33 +863,29 @@ public class DocBuilder {
     }
     //get the deleted rows for this entity
     Set<Map<String, Object>> deletedSet = new HashSet<Map<String, Object>>();
-    Set<Map<String, Object>> deltaRemoveSet = new HashSet<Map<String, Object>>();
     while (true) {
       Map<String, Object> row = entityProcessor.nextDeletedRowKey();
       if (row == null)
         break;
 
-      //Check to see if this delete is in the current delta set
-      for (Map<String, Object> modifiedRow : deltaSet) {
-        if (modifiedRow.get(entity.getPk()).equals(row.get(entity.getPk()))) {
-          deltaRemoveSet.add(modifiedRow);
-        }
+      deletedSet.add(row);
+      
+      // Remove deleted rows from the delta rows
+      String deletedRowPk = row.get(entity.getPk()).toString();
+      if (deltaSet.containsKey(deletedRowPk)) {
+        deltaSet.remove(deletedRowPk);
       }
 
-      deletedSet.add(row);
       importStatistics.rowsCount.incrementAndGet();
       // check for abort
       if (stop.get())
         return new HashSet();
     }
 
-    //asymmetric Set difference
-    deltaSet.removeAll(deltaRemoveSet);
-
     LOG.info("Completed ModifiedRowKey for Entity: " + entity.name + " rows obtained : " + deltaSet.size());
     LOG.info("Completed DeletedRowKey for Entity: " + entity.name + " rows obtained : " + deletedSet.size());
 
-    myModifiedPks.addAll(deltaSet);
+    myModifiedPks.addAll(deltaSet.values());
     Set<Map<String, Object>> parentKeyList = new HashSet<Map<String, Object>>();
     //all that we have captured is useless (in a sub-entity) if no rows in the parent is modified because of these
     //propogate up the changes in the chain