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

svn commit: r1031085 [6/8] - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/backwards/src/ lucene/backwards/src/test/org/apache/lucene/analysis/ lucene/backwards/src/test/org/apache/lucene/document/ lucene/backwards/src/test/org/apache/lucene/ind...

Modified: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1031085&r1=1031084&r2=1031085&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Thu Nov  4 17:15:30 2010
@@ -155,379 +155,7 @@ public class TestIndexWriter extends Luc
         doc.add(newField("content", "aaa " + index, Field.Store.YES, Field.Index.ANALYZED));
         doc.add(newField("id", "" + index, Field.Store.YES, Field.Index.ANALYZED));
         writer.addDocument(doc);
-    }
-
-    /*
-      Test: make sure when we run out of disk space or hit
-      random IOExceptions in any of the addIndexes(*) calls
-      that 1) index is not corrupt (searcher can open/search
-      it) and 2) transactional semantics are followed:
-      either all or none of the incoming documents were in
-      fact added.
-    */
-    public void testAddIndexOnDiskFull() throws IOException
-    {
-      int START_COUNT = 57;
-      int NUM_DIR = 50;
-      int END_COUNT = START_COUNT + NUM_DIR*25;
-
-      // Build up a bunch of dirs that have indexes which we
-      // will then merge together by calling addIndexes(*):
-      Directory[] dirs = new Directory[NUM_DIR];
-      long inputDiskUsage = 0;
-      for(int i=0;i<NUM_DIR;i++) {
-        dirs[i] = newDirectory();
-        IndexWriter writer  = new IndexWriter(dirs[i], newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT)));
-        for(int j=0;j<25;j++) {
-          addDocWithIndex(writer, 25*i+j);
-        }
-        writer.close();
-        String[] files = dirs[i].listAll();
-        for(int j=0;j<files.length;j++) {
-          inputDiskUsage += dirs[i].fileLength(files[j]);
-        }
-      }
-
-      // Now, build a starting index that has START_COUNT docs.  We
-      // will then try to addIndexes into a copy of this:
-      MockDirectoryWrapper startDir = newDirectory();
-      IndexWriter writer = new IndexWriter(startDir, newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT)));
-      for(int j=0;j<START_COUNT;j++) {
-        addDocWithIndex(writer, j);
-      }
-      writer.close();
-
-      // Make sure starting index seems to be working properly:
-      Term searchTerm = new Term("content", "aaa");        
-      IndexReader reader = IndexReader.open(startDir, true);
-      assertEquals("first docFreq", 57, reader.docFreq(searchTerm));
-
-      IndexSearcher searcher = new IndexSearcher(reader);
-      ScoreDoc[] hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
-      assertEquals("first number of hits", 57, hits.length);
-      searcher.close();
-      reader.close();
-
-      // Iterate with larger and larger amounts of free
-      // disk space.  With little free disk space,
-      // addIndexes will certainly run out of space &
-      // fail.  Verify that when this happens, index is
-      // not corrupt and index in fact has added no
-      // documents.  Then, we increase disk space by 2000
-      // bytes each iteration.  At some point there is
-      // enough free disk space and addIndexes should
-      // succeed and index should show all documents were
-      // added.
-
-      // String[] files = startDir.listAll();
-      long diskUsage = startDir.sizeInBytes();
-
-      long startDiskUsage = 0;
-      String[] files = startDir.listAll();
-      for(int i=0;i<files.length;i++) {
-        startDiskUsage += startDir.fileLength(files[i]);
-      }
-
-      for(int iter=0;iter<3;iter++) {
-
-        if (VERBOSE)
-          System.out.println("TEST: iter=" + iter);
-
-        // Start with 100 bytes more than we are currently using:
-        long diskFree = diskUsage+100;
-
-        int method = iter;
-
-        boolean success = false;
-        boolean done = false;
-
-        String methodName;
-        if (0 == method) {
-          methodName = "addIndexes(Directory[]) + optimize()";
-        } else if (1 == method) {
-          methodName = "addIndexes(IndexReader[])";
-        } else {
-          methodName = "addIndexes(Directory[])";
-        }
-
-        while(!done) {
-
-          // Make a new dir that will enforce disk usage:
-          MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory(startDir));
-          writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT)).setOpenMode(OpenMode.APPEND));
-          IOException err = null;
-
-          MergeScheduler ms = writer.getConfig().getMergeScheduler();
-          for(int x=0;x<2;x++) {
-            if (ms instanceof ConcurrentMergeScheduler)
-              // This test intentionally produces exceptions
-              // in the threads that CMS launches; we don't
-              // want to pollute test output with these.
-              if (0 == x)
-                ((ConcurrentMergeScheduler) ms).setSuppressExceptions();
-              else
-                ((ConcurrentMergeScheduler) ms).clearSuppressExceptions();
-
-            // Two loops: first time, limit disk space &
-            // throw random IOExceptions; second time, no
-            // disk space limit:
-
-            double rate = 0.05;
-            double diskRatio = ((double) diskFree)/diskUsage;
-            long thisDiskFree;
-
-            String testName = null;
-
-            if (0 == x) {
-              thisDiskFree = diskFree;
-              if (diskRatio >= 2.0) {
-                rate /= 2;
-              }
-              if (diskRatio >= 4.0) {
-                rate /= 2;
-              }
-              if (diskRatio >= 6.0) {
-                rate = 0.0;
-              }
-              if (VERBOSE)
-                testName = "disk full test " + methodName + " with disk full at " + diskFree + " bytes";
-            } else {
-              thisDiskFree = 0;
-              rate = 0.0;
-              if (VERBOSE)
-                testName = "disk full test " + methodName + " with unlimited disk space";
-            }
-
-            if (VERBOSE)
-              System.out.println("\ncycle: " + testName);
-
-            dir.setTrackDiskUsage(true);
-            dir.setMaxSizeInBytes(thisDiskFree);
-            dir.setRandomIOExceptionRate(rate, diskFree);
-
-            try {
-
-              if (0 == method) {
-                writer.addIndexes(dirs);
-                writer.optimize();
-              } else if (1 == method) {
-                IndexReader readers[] = new IndexReader[dirs.length];
-                for(int i=0;i<dirs.length;i++) {
-                  readers[i] = IndexReader.open(dirs[i], true);
-                }
-                try {
-                  writer.addIndexes(readers);
-                } finally {
-                  for(int i=0;i<dirs.length;i++) {
-                    readers[i].close();
-                  }
-                }
-              } else {
-                writer.addIndexes(dirs);
-              }
-
-              success = true;
-              if (VERBOSE) {
-                System.out.println("  success!");
-              }
-
-              if (0 == x) {
-                done = true;
-              }
-
-            } catch (IOException e) {
-              success = false;
-              err = e;
-              if (VERBOSE) {
-                System.out.println("  hit IOException: " + e);
-                e.printStackTrace(System.out);
-              }
-
-              if (1 == x) {
-                e.printStackTrace(System.out);
-                fail(methodName + " hit IOException after disk space was freed up");
-              }
-            }
-
-            // Make sure all threads from
-            // ConcurrentMergeScheduler are done
-            _TestUtil.syncConcurrentMerges(writer);
-
-            if (VERBOSE) {
-              System.out.println("  now test readers");
-            }
-
-            // Finally, verify index is not corrupt, and, if
-            // we succeeded, we see all docs added, and if we
-            // failed, we see either all docs or no docs added
-            // (transactional semantics):
-            try {
-              reader = IndexReader.open(dir, true);
-            } catch (IOException e) {
-              e.printStackTrace(System.out);
-              fail(testName + ": exception when creating IndexReader: " + e);
-            }
-            int result = reader.docFreq(searchTerm);
-            if (success) {
-              if (result != START_COUNT) {
-                fail(testName + ": method did not throw exception but docFreq('aaa') is " + result + " instead of expected " + START_COUNT);
-              }
-            } else {
-              // On hitting exception we still may have added
-              // all docs:
-              if (result != START_COUNT && result != END_COUNT) {
-                err.printStackTrace(System.out);
-                fail(testName + ": method did throw exception but docFreq('aaa') is " + result + " instead of expected " + START_COUNT + " or " + END_COUNT);
-              }
-            }
-
-            searcher = new IndexSearcher(reader);
-            try {
-              hits = searcher.search(new TermQuery(searchTerm), null, END_COUNT).scoreDocs;
-            } catch (IOException e) {
-              e.printStackTrace(System.out);
-              fail(testName + ": exception when searching: " + e);
-            }
-            int result2 = hits.length;
-            if (success) {
-              if (result2 != result) {
-                fail(testName + ": method did not throw exception but hits.length for search on term 'aaa' is " + result2 + " instead of expected " + result);
-              }
-            } else {
-              // On hitting exception we still may have added
-              // all docs:
-              if (result2 != result) {
-                err.printStackTrace(System.out);
-                fail(testName + ": method did throw exception but hits.length for search on term 'aaa' is " + result2 + " instead of expected " + result);
-              }
-            }
-
-            searcher.close();
-            reader.close();
-            if (VERBOSE) {
-              System.out.println("  count is " + result);
-            }
-
-            if (done || result == END_COUNT) {
-              break;
-            }
-          }
-
-          if (VERBOSE) {
-            System.out.println("  start disk = " + startDiskUsage + "; input disk = " + inputDiskUsage + "; max used = " + dir.getMaxUsedSizeInBytes());
-          }
-
-          if (done) {
-            // Javadocs state that temp free Directory space
-            // required is at most 2X total input size of
-            // indices so let's make sure:
-            assertTrue("max free Directory space required exceeded 1X the total input index sizes during " + methodName +
-                       ": max temp usage = " + (dir.getMaxUsedSizeInBytes()-startDiskUsage) + " bytes; " +
-                       "starting disk usage = " + startDiskUsage + " bytes; " +
-                       "input index disk usage = " + inputDiskUsage + " bytes",
-                       (dir.getMaxUsedSizeInBytes()-startDiskUsage) < 2*(startDiskUsage + inputDiskUsage));
-          }
-
-          // Make sure we don't hit disk full during close below:
-          dir.setMaxSizeInBytes(0);
-          dir.setRandomIOExceptionRate(0.0, 0);
-
-          writer.close();
-
-          // Wait for all BG threads to finish else
-          // dir.close() will throw IOException because
-          // there are still open files
-          _TestUtil.syncConcurrentMerges(ms);
-
-          dir.close();
-
-          // Try again with 5000 more bytes of free space:
-          diskFree += 5000;
-        }
-      }
-
-      startDir.close();
-      for (Directory dir : dirs)
-        dir.close();
-    }
-
-    /*
-     * Make sure IndexWriter cleans up on hitting a disk
-     * full exception in addDocument.
-     * TODO: how to do this on windows with FSDirectory?
-     */
-    public void testAddDocumentOnDiskFull() throws IOException {
-
-      for(int pass=0;pass<2;pass++) {
-        if (VERBOSE)
-          System.out.println("TEST: pass=" + pass);
-        boolean doAbort = pass == 1;
-        long diskFree = 200;
-        while(true) {
-          if (VERBOSE)
-            System.out.println("TEST: cycle: diskFree=" + diskFree);
-          MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory());
-          dir.setMaxSizeInBytes(diskFree);
-          IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT)));
-          MergeScheduler ms = writer.getConfig().getMergeScheduler();
-          if (ms instanceof ConcurrentMergeScheduler)
-            // This test intentionally produces exceptions
-            // in the threads that CMS launches; we don't
-            // want to pollute test output with these.
-            ((ConcurrentMergeScheduler) ms).setSuppressExceptions();
-
-          boolean hitError = false;
-          try {
-            for(int i=0;i<200;i++) {
-              addDoc(writer);
-            }
-            writer.commit();
-          } catch (IOException e) {
-            if (VERBOSE) {
-              System.out.println("TEST: exception on addDoc");
-              e.printStackTrace(System.out);
-            }
-            hitError = true;
-          }
-
-          if (hitError) {
-            if (doAbort) {
-              writer.rollback();
-            } else {
-              try {
-                writer.close();
-              } catch (IOException e) {
-                if (VERBOSE) {
-                  System.out.println("TEST: exception on close");
-                  e.printStackTrace(System.out);
-                }
-                dir.setMaxSizeInBytes(0);
-                writer.close();
-              }
-            }
-
-            //_TestUtil.syncConcurrentMerges(ms);
-
-            if (_TestUtil.anyFilesExceptWriteLock(dir)) {
-              assertNoUnreferencedFiles(dir, "after disk full during addDocument");
-              
-              // Make sure reader can open the index:
-              IndexReader.open(dir, true).close();
-            }
-              
-            dir.close();
-            // Now try again w/ more space:
-
-            diskFree += 500;
-          } else {
-            //_TestUtil.syncConcurrentMerges(writer);
-            writer.close();
-            dir.close();
-            break;
-          }
-        }
-      }
-    }                                               
+    }                            
 
     public static void assertNoUnreferencedFiles(Directory dir, String message) throws IOException {
       String[] startFiles = dir.listAll();
@@ -2242,418 +1870,6 @@ public class TestIndexWriter extends Luc
     directory.close();
   }
 
-  // Used by test cases below
-  private class IndexerThread extends Thread {
-
-    boolean diskFull;
-    Throwable error;
-    AlreadyClosedException ace;
-    IndexWriter writer;
-    boolean noErrors;
-    volatile int addCount;
-
-    public IndexerThread(IndexWriter writer, boolean noErrors) {
-      this.writer = writer;
-      this.noErrors = noErrors;
-    }
-
-    @Override
-    public void run() {
-
-      final Document doc = new Document();
-      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));
-
-      int idUpto = 0;
-      int fullCount = 0;
-      final long stopTime = System.currentTimeMillis() + 200;
-
-      do {
-        try {
-          writer.updateDocument(new Term("id", ""+(idUpto++)), doc);
-          addCount++;
-        } catch (IOException ioe) {
-          //System.out.println(Thread.currentThread().getName() + ": hit exc");
-          //ioe.printStackTrace(System.out);
-          if (ioe.getMessage().startsWith("fake disk full at") ||
-              ioe.getMessage().equals("now failing on purpose")) {
-            diskFull = true;
-            try {
-              Thread.sleep(1);
-            } catch (InterruptedException ie) {
-              throw new ThreadInterruptedException(ie);
-            }
-            if (fullCount++ >= 5)
-              break;
-          } else {
-            if (noErrors) {
-              System.out.println(Thread.currentThread().getName() + ": ERROR: unexpected IOException:");
-              ioe.printStackTrace(System.out);
-              error = ioe;
-            }
-            break;
-          }
-        } catch (Throwable t) {
-          //t.printStackTrace(System.out);
-          if (noErrors) {
-            System.out.println(Thread.currentThread().getName() + ": ERROR: unexpected Throwable:");
-            t.printStackTrace(System.out);
-            error = t;
-          }
-          break;
-        }
-      } while(System.currentTimeMillis() < stopTime);
-    }
-  }
-
-  // LUCENE-1130: make sure we can close() even while
-  // threads are trying to add documents.  Strictly
-  // speaking, this isn't valid us of Lucene's APIs, but we
-  // still want to be robust to this case:
-  public void testCloseWithThreads() throws Exception {
-    int NUM_THREADS = 3;
-
-    for(int iter=0;iter<7;iter++) {
-      Directory dir = newDirectory();
-      IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT))
-        .setMaxBufferedDocs(10).setMergeScheduler(new ConcurrentMergeScheduler());
-      // We expect AlreadyClosedException
-      ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
-      IndexWriter writer = new IndexWriter(dir, conf);
-      ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(4);
-
-      IndexerThread[] threads = new IndexerThread[NUM_THREADS];
-
-      for(int i=0;i<NUM_THREADS;i++)
-        threads[i] = new IndexerThread(writer, false);
-
-      for(int i=0;i<NUM_THREADS;i++)
-        threads[i].start();
-
-      boolean done = false;
-      while(!done) {
-        Thread.sleep(100);
-        for(int i=0;i<NUM_THREADS;i++)
-          // only stop when at least one thread has added a doc
-          if (threads[i].addCount > 0) {
-            done = true;
-            break;
-          }
-      }
-
-      writer.close(false);
-
-      // Make sure threads that are adding docs are not hung:
-      for(int i=0;i<NUM_THREADS;i++) {
-        // Without fix for LUCENE-1130: one of the
-        // threads will hang
-        threads[i].join();
-        if (threads[i].isAlive())
-          fail("thread seems to be hung");
-      }
-
-      // Quick test to make sure index is not corrupt:
-      IndexReader reader = IndexReader.open(dir, true);
-      TermDocs tdocs = reader.termDocs(new Term("field", "aaa"));
-      int count = 0;
-      while(tdocs.next()) {
-        count++;
-      }
-      assertTrue(count > 0);
-      reader.close();
-      
-      dir.close();
-    }
-  }
-
-  // LUCENE-1130: make sure immeidate disk full on creating
-  // an IndexWriter (hit during DW.ThreadState.init()) is
-  // OK:
-  public void testImmediateDiskFull() throws IOException {
-    MockDirectoryWrapper dir = newDirectory();
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT))
-        .setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler()));
-    dir.setMaxSizeInBytes(Math.max(1, dir.getRecomputedActualSizeInBytes()));
-    final Document doc = new Document();
-    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));
-    try {
-      writer.addDocument(doc);
-      fail("did not hit disk full");
-    } catch (IOException ioe) {
-    }
-    // Without fix for LUCENE-1130: this call will hang:
-    try {
-      writer.addDocument(doc);
-      fail("did not hit disk full");
-    } catch (IOException ioe) {
-    }
-    try {
-      writer.close(false);
-      fail("did not hit disk full");
-    } catch (IOException ioe) {
-    }
-
-    // Make sure once disk space is avail again, we can
-    // cleanly close:
-    dir.setMaxSizeInBytes(0);
-    writer.close(false);
-    dir.close();
-  }
-
-  // LUCENE-1130: make sure immediate disk full on creating
-  // an IndexWriter (hit during DW.ThreadState.init()), with
-  // multiple threads, is OK:
-  public void testImmediateDiskFullWithThreads() throws Exception {
-
-    int NUM_THREADS = 3;
-
-    for(int iter=0;iter<10;iter++) {
-      MockDirectoryWrapper dir = newDirectory();
-      IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT))
-         .setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler());
-      // We expect disk full exceptions in the merge threads
-      ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
-      IndexWriter writer = new IndexWriter(dir, conf);
-      ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(4);
-      dir.setMaxSizeInBytes(4*1024+20*iter);
-
-      IndexerThread[] threads = new IndexerThread[NUM_THREADS];
-
-      for(int i=0;i<NUM_THREADS;i++)
-        threads[i] = new IndexerThread(writer, true);
-
-      for(int i=0;i<NUM_THREADS;i++)
-        threads[i].start();
-
-      for(int i=0;i<NUM_THREADS;i++) {
-        // Without fix for LUCENE-1130: one of the
-        // threads will hang
-        threads[i].join();
-        assertTrue("hit unexpected Throwable", threads[i].error == null);
-      }
-
-      // Make sure once disk space is avail again, we can
-      // cleanly close:
-      dir.setMaxSizeInBytes(0);
-      writer.close(false);
-      dir.close();
-    }
-  }
-
-  // Throws IOException during FieldsWriter.flushDocument and during DocumentsWriter.abort
-  private static class FailOnlyOnAbortOrFlush extends MockDirectoryWrapper.Failure {
-    private boolean onlyOnce;
-    public FailOnlyOnAbortOrFlush(boolean onlyOnce) {
-      this.onlyOnce = onlyOnce;
-    }
-    @Override
-    public void eval(MockDirectoryWrapper dir)  throws IOException {
-      if (doFail) {
-        StackTraceElement[] trace = new Exception().getStackTrace();
-        for (int i = 0; i < trace.length; i++) {
-          if ("abort".equals(trace[i].getMethodName()) ||
-              "flushDocument".equals(trace[i].getMethodName())) {
-            if (onlyOnce)
-              doFail = false;
-            //System.out.println(Thread.currentThread().getName() + ": now fail");
-            //new Throwable().printStackTrace(System.out);
-            throw new IOException("now failing on purpose");
-          }
-        }
-      }
-    }
-  }
-
-  // Runs test, with one thread, using the specific failure
-  // to trigger an IOException
-  public void _testSingleThreadFailure(MockDirectoryWrapper.Failure failure) throws IOException {
-    MockDirectoryWrapper dir = newDirectory();
-
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT))
-      .setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler()));
-    final Document doc = new Document();
-    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));
-
-    for(int i=0;i<6;i++)
-      writer.addDocument(doc);
-
-    dir.failOn(failure);
-    failure.setDoFail();
-    try {
-      writer.addDocument(doc);
-      writer.addDocument(doc);
-      writer.commit();
-      fail("did not hit exception");
-    } catch (IOException ioe) {
-    }
-    failure.clearDoFail();
-    writer.addDocument(doc);
-    writer.close(false);
-    dir.close();
-  }
-
-  // Runs test, with multiple threads, using the specific
-  // failure to trigger an IOException
-  public void _testMultipleThreadsFailure(MockDirectoryWrapper.Failure failure) throws Exception {
-
-    int NUM_THREADS = 3;
-
-    for(int iter=0;iter<2;iter++) {
-      MockDirectoryWrapper dir = newDirectory();
-      IndexWriterConfig conf = newIndexWriterConfig( TEST_VERSION_CURRENT,
-          new WhitespaceAnalyzer(TEST_VERSION_CURRENT)).setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler());
-      // We expect disk full exceptions in the merge threads
-      ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
-      IndexWriter writer = new IndexWriter(dir, conf);
-      ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(4);
-
-      IndexerThread[] threads = new IndexerThread[NUM_THREADS];
-
-      for(int i=0;i<NUM_THREADS;i++)
-        threads[i] = new IndexerThread(writer, true);
-
-      for(int i=0;i<NUM_THREADS;i++)
-        threads[i].start();
-
-      Thread.sleep(10);
-
-      dir.failOn(failure);
-      failure.setDoFail();
-
-      for(int i=0;i<NUM_THREADS;i++) {
-        threads[i].join();
-        assertTrue("hit unexpected Throwable", threads[i].error == null);
-      }
-
-      boolean success = false;
-      try {
-        writer.close(false);
-        success = true;
-      } catch (IOException ioe) {
-        failure.clearDoFail();
-        writer.close(false);
-      }
-
-      if (success) {
-        IndexReader reader = IndexReader.open(dir, true);
-        for(int j=0;j<reader.maxDoc();j++) {
-          if (!reader.isDeleted(j)) {
-            reader.document(j);
-            reader.getTermFreqVectors(j);
-          }
-        }
-        reader.close();
-      }
-
-      dir.close();
-    }
-  }
-
-  // LUCENE-1130: make sure initial IOException, and then 2nd
-  // IOException during rollback(), is OK:
-  public void testIOExceptionDuringAbort() throws IOException {
-    _testSingleThreadFailure(new FailOnlyOnAbortOrFlush(false));
-  }
-
-  // LUCENE-1130: make sure initial IOException, and then 2nd
-  // IOException during rollback(), is OK:
-  public void testIOExceptionDuringAbortOnlyOnce() throws IOException {
-    _testSingleThreadFailure(new FailOnlyOnAbortOrFlush(true));
-  }
-
-  // LUCENE-1130: make sure initial IOException, and then 2nd
-  // IOException during rollback(), with multiple threads, is OK:
-  public void testIOExceptionDuringAbortWithThreads() throws Exception {
-    _testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(false));
-  }
-
-  // LUCENE-1130: make sure initial IOException, and then 2nd
-  // IOException during rollback(), with multiple threads, is OK:
-  public void testIOExceptionDuringAbortWithThreadsOnlyOnce() throws Exception {
-    _testMultipleThreadsFailure(new FailOnlyOnAbortOrFlush(true));
-  }
-
-  // Throws IOException during DocumentsWriter.closeDocStore
-  private static class FailOnlyInCloseDocStore extends MockDirectoryWrapper.Failure {
-    private boolean onlyOnce;
-    public FailOnlyInCloseDocStore(boolean onlyOnce) {
-      this.onlyOnce = onlyOnce;
-    }
-    @Override
-    public void eval(MockDirectoryWrapper dir)  throws IOException {
-      if (doFail) {
-        StackTraceElement[] trace = new Exception().getStackTrace();
-        for (int i = 0; i < trace.length; i++) {
-          if ("closeDocStore".equals(trace[i].getMethodName())) {
-            if (onlyOnce)
-              doFail = false;
-            throw new IOException("now failing on purpose");
-          }
-        }
-      }
-    }
-  }
-
-  // LUCENE-1130: test IOException in closeDocStore
-  public void testIOExceptionDuringCloseDocStore() throws IOException {
-    _testSingleThreadFailure(new FailOnlyInCloseDocStore(false));
-  }
-
-  // LUCENE-1130: test IOException in closeDocStore
-  public void testIOExceptionDuringCloseDocStoreOnlyOnce() throws IOException {
-    _testSingleThreadFailure(new FailOnlyInCloseDocStore(true));
-  }
-
-  // LUCENE-1130: test IOException in closeDocStore, with threads
-  public void testIOExceptionDuringCloseDocStoreWithThreads() throws Exception {
-    _testMultipleThreadsFailure(new FailOnlyInCloseDocStore(false));
-  }
-
-  // LUCENE-1130: test IOException in closeDocStore, with threads
-  public void testIOExceptionDuringCloseDocStoreWithThreadsOnlyOnce() throws Exception {
-    _testMultipleThreadsFailure(new FailOnlyInCloseDocStore(true));
-  }
-
-  // Throws IOException during DocumentsWriter.writeSegment
-  private static class FailOnlyInWriteSegment extends MockDirectoryWrapper.Failure {
-    private boolean onlyOnce;
-    public FailOnlyInWriteSegment(boolean onlyOnce) {
-      this.onlyOnce = onlyOnce;
-    }
-    @Override
-    public void eval(MockDirectoryWrapper dir)  throws IOException {
-      if (doFail) {
-        StackTraceElement[] trace = new Exception().getStackTrace();
-        for (int i = 0; i < trace.length; i++) {
-          if ("flush".equals(trace[i].getMethodName()) && "org.apache.lucene.index.DocFieldProcessor".equals(trace[i].getClassName())) {
-            if (onlyOnce)
-              doFail = false;
-            throw new IOException("now failing on purpose");
-          }
-        }
-      }
-    }
-  }
-
-  // LUCENE-1130: test IOException in writeSegment
-  public void testIOExceptionDuringWriteSegment() throws IOException {
-    _testSingleThreadFailure(new FailOnlyInWriteSegment(false));
-  }
-
-  // LUCENE-1130: test IOException in writeSegment
-  public void testIOExceptionDuringWriteSegmentOnlyOnce() throws IOException {
-    _testSingleThreadFailure(new FailOnlyInWriteSegment(true));
-  }
-
-  // LUCENE-1130: test IOException in writeSegment, with threads
-  public void testIOExceptionDuringWriteSegmentWithThreads() throws Exception {
-    _testMultipleThreadsFailure(new FailOnlyInWriteSegment(false));
-  }
-
-  // LUCENE-1130: test IOException in writeSegment, with threads
-  public void testIOExceptionDuringWriteSegmentWithThreadsOnlyOnce() throws Exception {
-    _testMultipleThreadsFailure(new FailOnlyInWriteSegment(true));
-  }
-
   // LUCENE-1084: test unlimited field length
   public void testUnlimitedMaxFieldLength() throws IOException {
     Directory dir = newDirectory();
@@ -5182,65 +4398,4 @@ public class TestIndexWriter extends Luc
     w.close();
     dir.close();
   }
-
-  private static class FailTwiceDuringMerge extends MockDirectoryWrapper.Failure {
-    public boolean didFail1;
-    public boolean didFail2;
-
-    @Override
-    public void eval(MockDirectoryWrapper dir)  throws IOException {
-      if (!doFail) {
-        return;
-      }
-      StackTraceElement[] trace = new Exception().getStackTrace();
-      for (int i = 0; i < trace.length; i++) {
-        if ("org.apache.lucene.index.SegmentMerger".equals(trace[i].getClassName()) && "mergeTerms".equals(trace[i].getMethodName()) && !didFail1) {
-          didFail1 = true;
-          throw new IOException("fake disk full during mergeTerms");
-        }
-        if ("org.apache.lucene.util.BitVector".equals(trace[i].getClassName()) && "write".equals(trace[i].getMethodName()) && !didFail2) {
-          didFail2 = true;
-          throw new IOException("fake disk full while writing BitVector");
-        }
-      }
-    }
-  }
-  
-  // LUCENE-2593
-  public void testCorruptionAfterDiskFullDuringMerge() throws IOException {
-    MockDirectoryWrapper dir = newDirectory();
-    //IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(rand, TEST_VERSION_CURRENT, new MockAnalyzer()).setReaderPooling(true));
-    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT)).setMergeScheduler(new SerialMergeScheduler()).setReaderPooling(true));
-
-    ((LogMergePolicy) w.getMergePolicy()).setMergeFactor(2);
-
-    Document doc = new Document();
-    doc.add(newField("f", "doctor who", Field.Store.YES, Field.Index.ANALYZED));
-    w.addDocument(doc);
-
-    w.commit();
-
-    w.deleteDocuments(new Term("f", "who"));
-    w.addDocument(doc);
-    
-    // disk fills up!
-    FailTwiceDuringMerge ftdm = new FailTwiceDuringMerge();
-    ftdm.setDoFail();
-    dir.failOn(ftdm);
-
-    try {
-      w.commit();
-      fail("fake disk full IOExceptions not hit");
-    } catch (IOException ioe) {
-      // expected
-      assertTrue(ftdm.didFail1);
-    }
-    _TestUtil.checkIndex(dir);
-    ftdm.clearDoFail();
-    w.addDocument(doc);
-    w.close();
-
-    _TestUtil.checkIndex(dir);
-    dir.close();
-  }
 }

Copied: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java (from r1031076, lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java?p2=lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java&p1=lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java&r1=1031076&r2=1031085&rev=1031085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java Thu Nov  4 17:15:30 2010
@@ -20,13 +20,12 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.WhitespaceAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -148,7 +147,7 @@ public class TestIndexWriterWithThreads 
 
     for(int iter=0;iter<7;iter++) {
       Directory dir = newDirectory();
-      IndexWriterConfig conf = newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer())
+      IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT))
         .setMaxBufferedDocs(10).setMergeScheduler(new ConcurrentMergeScheduler());
       // We expect AlreadyClosedException
       ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
@@ -187,12 +186,9 @@ public class TestIndexWriterWithThreads 
 
       // Quick test to make sure index is not corrupt:
       IndexReader reader = IndexReader.open(dir, true);
-      DocsEnum tdocs = MultiFields.getTermDocsEnum(reader,
-                                                  MultiFields.getDeletedDocs(reader),
-                                                  "field",
-                                                  new BytesRef("aaa"));
+      TermDocs tdocs = reader.termDocs(new Term("field", "aaa"));
       int count = 0;
-      while(tdocs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+      while(tdocs.next()) {
         count++;
       }
       assertTrue(count > 0);
@@ -211,7 +207,7 @@ public class TestIndexWriterWithThreads 
     for(int iter=0;iter<2;iter++) {
       MockDirectoryWrapper dir = newDirectory();
       IndexWriterConfig conf = newIndexWriterConfig( TEST_VERSION_CURRENT,
-          new MockAnalyzer()).setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler());
+          new WhitespaceAnalyzer(TEST_VERSION_CURRENT)).setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler());
       // We expect disk full exceptions in the merge threads
       ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
       IndexWriter writer = new IndexWriter(dir, conf);
@@ -246,9 +242,8 @@ public class TestIndexWriterWithThreads 
 
       if (success) {
         IndexReader reader = IndexReader.open(dir, true);
-        final Bits delDocs = MultiFields.getDeletedDocs(reader);
         for(int j=0;j<reader.maxDoc();j++) {
-          if (delDocs == null || !delDocs.get(j)) {
+          if (!reader.isDeleted(j)) {
             reader.document(j);
             reader.getTermFreqVectors(j);
           }
@@ -265,7 +260,7 @@ public class TestIndexWriterWithThreads 
   public void _testSingleThreadFailure(MockDirectoryWrapper.Failure failure) throws IOException {
     MockDirectoryWrapper dir = newDirectory();
 
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer())
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT))
       .setMaxBufferedDocs(2).setMergeScheduler(new ConcurrentMergeScheduler()));
     final Document doc = new Document();
     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));

Propchange: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov  4 17:15:30 2010
@@ -1,4 +1,4 @@
-/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,9637
 20,963781,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984715,984968,985453,985455,985672,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223
 ,999545,999842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,102686
 8,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,9637
 20,963781,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984715,984968,985453,985455,985672,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223
 ,999545,999842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,102686
 8,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754,1031076
 /lucene/dev/trunk/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java:932749
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java:924791,924850,930201
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/search/TestCachingWrapperFilter.java:748824

Propchange: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov  4 17:15:30 2010
@@ -1,4 +1,4 @@
-/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,963720
 ,963781,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984715,984968,985453,985455,985672,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,9
 99545,999842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,1026868,
 1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,963720
 ,963781,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984715,984968,985453,985455,985672,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,9
 99545,999842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,1026868,
 1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754,1031076
 /lucene/dev/trunk/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java:932749
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java:924791,924850,930201
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java:748824

Propchange: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/spans/TestSpans.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov  4 17:15:30 2010
@@ -1,4 +1,4 @@
-/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/spans/TestSpans.java:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,963720,963781
 ,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984715,984968,985453,985455,985672,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,9
 99842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,1026868,1026882
 ,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/spans/TestSpans.java:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,963720,963781
 ,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984715,984968,985453,985455,985672,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,9
 99842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,1026868,1026882
 ,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754,1031076
 /lucene/dev/trunk/src/test/org/apache/lucene/search/spans/TestSpans.java:932749
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/search/spans/TestSpans.java:924791,924850,930201
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/search/spans/TestSpans.java:748824

Propchange: lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov  4 17:15:30 2010
@@ -1,5 +1,5 @@
 /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:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,953628,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,9637
 20,963781,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223
 ,999545,999842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,102686
 8,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940666,940699,940730,940878-940892,940994,941270,941363,942166,942288,942676,942719,943142,943493,943931,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950458,950613,951126,951355,951397,951521,953407,953628,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960371,960374,960719,962555,963372,963654,9637
 20,963781,963873,963906,963909,963920,964019,964054,964430,964459,964720,964753,964832,964856,965103,965110,965222,965230,965299,965327,965330,965585,966354,966878,967080,979453,979809,980369,980428,980436,980501,980909,980911,980917,981265,981550,981598,981650,981661,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983622,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986173,986612,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,989785,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991310,991497,992424,992469,992567,992571,992623,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995376,995772,996268,996357,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223
 ,999545,999842,1000424,1000581,1000675,1001006,1001420,1001661,1001796,1003614,1003631,1003645,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022165,1022191,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026841,102686
 8,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030754,1031076
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/util/TestAttributeSource.java:924791,924850,930201
 /lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/util/TestAttributeSource.java:896850,909334,948516
 /lucene/java/trunk/src/test/org/apache/lucene/util/TestAttributeSource.java:924483-925561

Propchange: lucene/dev/branches/branch_3x/solr/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov  4 17:15:30 2010
@@ -1,5 +1,5 @@
 /lucene/dev/branches/preflexfixes/solr:967125-979432
-/lucene/dev/trunk/solr:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940556,940666,940699,940730,940878-940892,940994,941270,941363,942166,942235,942288,942676,942719,943142,943493,943931,944516,944528,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950125,950207,950458,950613,950723,950835,951126,951355,951397,951521,953628,953886,954336,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960374,960719,961941,962555,962714,963372,
 963654,963720,963781,963873,963888,963906,963909,963920,964019,964054,964267,964430,964459,964720,964753,964832,964856,965103,965222,965230,965327,965330,965585,965938,966354,966878,967080,979049,979453,979807,979809,980369,980428,980436,980501,980555,980592,980909,980911,980917,981265,981550,981598,981650,981661,981690,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986612,986773,987005,987009,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991497,992424,992469,992567,992571,992623,992913,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995253,995264,995286,995376,995557,995772,996268,996357,996416,996511,996611,996623,99
 6647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,999842,1000000,1000424,1000581,1000675,1001006,1001318,1001320,1001420,1001661,1001957,1002032,1003107,1003614,1003631,1003645,1003703,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022111,1022165,1022191,1022357,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,10243
 95,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026823,1026841,1026868,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030098,1030754
+/lucene/dev/trunk/solr:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940556,940666,940699,940730,940878-940892,940994,941270,941363,942166,942235,942288,942676,942719,943142,943493,943931,944516,944528,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950125,950207,950458,950613,950723,950835,951126,951355,951397,951521,953628,953886,954336,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960374,960719,961941,962555,962714,963372,
 963654,963720,963781,963873,963888,963906,963909,963920,964019,964054,964267,964430,964459,964720,964753,964832,964856,965103,965222,965230,965327,965330,965585,965938,966354,966878,967080,979049,979453,979807,979809,980369,980428,980436,980501,980555,980592,980909,980911,980917,981265,981550,981598,981650,981661,981690,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986612,986773,987005,987009,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991497,992424,992469,992567,992571,992623,992913,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995253,995264,995286,995376,995557,995772,996268,996357,996416,996511,996611,996623,99
 6647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,999842,1000000,1000424,1000581,1000675,1001006,1001318,1001320,1001420,1001661,1001957,1002032,1003107,1003614,1003631,1003645,1003703,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022111,1022165,1022191,1022357,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,1024305,10243
 95,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026823,1026841,1026868,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030098,1030754,1031076
 /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

Propchange: lucene/dev/branches/branch_3x/solr/example/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov  4 17:15:30 2010
@@ -1,4 +1,4 @@
-/lucene/dev/trunk/solr/example:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940556,940666,940699,940730,940878-940892,940994,941270,941363,942166,942235,942288,942676,942719,943142,943493,943931,944516,944528,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950125,950207,950458,950613,950723,950835,951126,951355,951397,951521,953628,953886,954336,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960374,961821,961941,962555,962714
 ,963372,963654,963720,963781,963873,963888,963906,963909,963920,964019,964054,964267,964430,964459,964720,964753,964832,964856,965103,965222,965230,965327,965330,965585,965938,966354,966878,967080,979049,979453,979807,979809,980369,980428,980436,980501,980555,980592,980909,980911,980917,981265,981550,981598,981650,981661,981690,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986612,986773,987005,987009,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991497,992424,992469,992567,992571,992623,992913,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995253,995264,995286,995376,995557,995772,996268,996357,996416,996511,996611,9
 96623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,999842,1000000,1000424,1000581,1000675,1001006,1001318,1001320,1001420,1001661,1001957,1002032,1003107,1003614,1003631,1003645,1003703,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022111,1022165,1022191,1022357,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,10243
 05,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026823,1026841,1026868,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030098,1030754
+/lucene/dev/trunk/solr/example:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940556,940666,940699,940730,940878-940892,940994,941270,941363,942166,942235,942288,942676,942719,943142,943493,943931,944516,944528,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950125,950207,950458,950613,950723,950835,951126,951355,951397,951521,953628,953886,954336,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960374,961821,961941,962555,962714
 ,963372,963654,963720,963781,963873,963888,963906,963909,963920,964019,964054,964267,964430,964459,964720,964753,964832,964856,965103,965222,965230,965327,965330,965585,965938,966354,966878,967080,979049,979453,979807,979809,980369,980428,980436,980501,980555,980592,980909,980911,980917,981265,981550,981598,981650,981661,981690,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986612,986773,987005,987009,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991497,992424,992469,992567,992571,992623,992913,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995253,995264,995286,995376,995557,995772,996268,996357,996416,996511,996611,9
 96623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,999842,1000000,1000424,1000581,1000675,1001006,1001318,1001320,1001420,1001661,1001957,1002032,1003107,1003614,1003631,1003645,1003703,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022111,1022165,1022191,1022357,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,1024256,1024292,10243
 05,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026823,1026841,1026868,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030098,1030754,1031076
 /lucene/java/branches/lucene_2_4/solr/example:748824
 /lucene/java/branches/lucene_2_9/solr/example:817269-818600,825998,829134,829881,831036,896850,909334
 /lucene/java/branches/lucene_2_9_back_compat_tests/solr/example:818601-821336

Propchange: lucene/dev/branches/branch_3x/solr/lib/commons-httpclient-3.1.jar
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Nov  4 17:15:30 2010
@@ -1,3 +1,3 @@
 /lucene/dev/branches/preflexfixes/solr/lib/commons-httpclient-3.1.jar:967125-979432
-/lucene/dev/trunk/solr/lib/commons-httpclient-3.1.jar:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940556,940666,940699,940730,940878-940892,940994,941270,941363,942166,942235,942288,942676,942719,943142,943493,943931,944516,944528,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950125,950207,950458,950613,950723,950835,951126,951355,951397,951521,953628,953886,954336,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960374,9607
 19,961941,962555,962714,963372,963654,963720,963781,963873,963888,963906,963909,963920,964019,964054,964267,964430,964459,964720,964753,964832,964856,965103,965222,965230,965327,965330,965585,965938,966354,966878,967080,979049,979453,979807,979809,980369,980428,980436,980501,980555,980592,980909,980911,980917,981265,981550,981598,981650,981661,981690,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986612,986773,987005,987009,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991497,992424,992469,992567,992571,992623,992913,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995253,995264,995286,995376,995557,995772,996268,996357
 ,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,999842,1000000,1000424,1000581,1000675,1001006,1001318,1001320,1001420,1001661,1001957,1003107,1003614,1003631,1003645,1003703,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022111,1022165,1022191,1022357,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,102425
 6,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026823,1026841,1026868,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030098,1030754
+/lucene/dev/trunk/solr/lib/commons-httpclient-3.1.jar:931298,931337,931502,932129-932131,932163,932304,932369,932374,932398,932417,932541,932576,932587,932698,932731-932749,932752,932773,932795,932828,932856-932857,932862,932864,932878,932963,932998-932999,933541-933575,933598,933613,933679,933879,934339,934954,935014-935048,935065,935186-935513,935521-935522,935553-935962,936522,936544,936605,936657-936726,937039,937360,938582-938646,938989,939111,939611,939649,940433,940447,940451-940452,940556,940666,940699,940730,940878-940892,940994,941270,941363,942166,942235,942288,942676,942719,943142,943493,943931,944516,944528,945057,945090,945130,945245,945343,945420,946139,946330,946338,946599,948011,948082,948429,949288,949311,949445,949976,949997,950008,950042,950125,950207,950458,950613,950723,950835,951126,951355,951397,951521,953628,953886,954336,955547,955613,955615,955796-955797,955809-955996,956097,956125,956173,956316,956715,957465,957520,957634,957707,960367,960374,9607
 19,961941,962555,962714,963372,963654,963720,963781,963873,963888,963906,963909,963920,964019,964054,964267,964430,964459,964720,964753,964832,964856,965103,965222,965230,965327,965330,965585,965938,966354,966878,967080,979049,979453,979807,979809,980369,980428,980436,980501,980555,980592,980909,980911,980917,981265,981550,981598,981650,981661,981690,981857,981936,982073,982084,982201,982725,982824,983100,983212,983216,983313,983495,983500,983530,983632,983778,984187,984202,984232,984510,984968,985453,985455,985672,985875,986158,986612,986773,987005,987009,987122,988087,988206,988216,988259,988346,988478,988527,988543,988592,988613,988688,988710,988736,988739,989004,989010,989013,989030,989035,989315,989321,989334,990160-990161,990180,990189,990281,990301,990451,990459,990766,990781,990854,991053,991191,991497,992424,992469,992567,992571,992623,992913,993106,993194,993199,993287,993408,994935,994976,994979,995247,995250,995253,995264,995286,995376,995557,995772,996268,996357
 ,996416,996511,996611,996623,996647-996653,996720,996942,996978,997180,997230,998055,998505,998684,999016,999223,999545,999842,1000000,1000424,1000581,1000675,1001006,1001318,1001320,1001420,1001661,1001957,1003107,1003614,1003631,1003645,1003703,1003841-1003852,1003873,1003877,1003906,1003938,1003954,1003978,1003990,1004038,1004082,1004179,1004200,1004215,1004241,1004335,1005310,1005356,1005363,1006146,1006290,1006324,1021340,1021357,1021360,1021439,1021449,1021969-1021971,1022111,1022165,1022191,1022357,1022632,1022708-1022710,1022730-1022735,1022748-1022755,1022762-1022793,1022798-1022802,1022805,1022826,1022927,1022939,1022956,1022989,1022998,1023006,1023009,1023022,1023040,1023106,1023235-1023246,1023250,1023264-1023265,1023312,1023329-1023330,1023346-1023347,1023355,1023493,1023509-1023511,1023518,1023520,1023535-1023536,1023562,1023579-1023588,1023594-1023595,1023600-1023602,1023606,1023621,1023635,1023637,1023711,1023845,1023870,1024196,1024219,1024233,1024238,102425
 6,1024292,1024305,1024395,1024402,1024408,1024476,1024486,1025597,1025669,1025929,1026044,1026167,1026336,1026431,1026460,1026592,1026606,1026610,1026738,1026823,1026841,1026868,1026882,1027998,1028039,1028386,1029096,1029325,1029333,1029345,1030012,1030019,1030073,1030078,1030098,1030754,1031076
 /lucene/solr/trunk/lib/commons-httpclient-3.1.jar:922950-923910,923912-925091