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 2012/08/13 15:53:27 UTC

svn commit: r1372423 [38/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/common/ dev-tools/maven/lucene/analysis/icu/ d...

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Mon Aug 13 13:52:46 2012
@@ -60,7 +60,7 @@ public abstract class ThreadedIndexingAn
   protected final AtomicInteger delCount = new AtomicInteger();
   protected final AtomicInteger packCount = new AtomicInteger();
 
-  protected Directory dir;
+  protected MockDirectoryWrapper dir;
   protected IndexWriter writer;
 
   private static class SubDocs {
@@ -432,8 +432,8 @@ public abstract class ThreadedIndexingAn
     Random random = new Random(random().nextLong());
     final LineFileDocs docs = new LineFileDocs(random, true);
     final File tempDir = _TestUtil.getTempDir(testName);
-    dir = newFSDirectory(tempDir);
-    ((MockDirectoryWrapper) dir).setCheckIndexOnClose(false); // don't double-checkIndex, we do it ourselves.
+    dir = newMockFSDirectory(tempDir); // some subclasses rely on this being MDW
+    dir.setCheckIndexOnClose(false); // don't double-checkIndex, we do it ourselves.
     final IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, 
         new MockAnalyzer(random())).setInfoStream(new FailOnNonBulkMergesInfoStream());
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java Mon Aug 13 13:52:46 2012
@@ -75,7 +75,7 @@ public class AssertingIndexSearcher exte
 
       @Override
       public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-          boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+          boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
         Scorer scorer = w.scorer(context, scoreDocsInOrder, topScorer, flags, acceptDocs);
         if (scorer != null) {
           // check that scorer obeys disi contract for docID() before next()/advance

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java Mon Aug 13 13:52:46 2012
@@ -33,7 +33,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
-import org.apache.lucene.search.Weight.FeatureFlags;
+import org.apache.lucene.search.Weight.PostingFeatures;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
@@ -281,7 +281,7 @@ public class QueryUtils {
               if (scorer == null) {
                 Weight w = s.createNormalizedWeight(q);
                 AtomicReaderContext context = readerContextArray.get(leafPtr);
-                scorer = w.scorer(context, true, false, FeatureFlags.DOCS, context.reader().getLiveDocs());
+                scorer = w.scorer(context, true, false, PostingFeatures.DOCS_AND_FREQS, context.reader().getLiveDocs());
               }
               
               int op = order[(opidx[0]++) % order.length];
@@ -328,7 +328,7 @@ public class QueryUtils {
               indexSearcher.setSimilarity(s.getSimilarity());
               Weight w = indexSearcher.createNormalizedWeight(q);
               AtomicReaderContext ctx = (AtomicReaderContext)indexSearcher.getTopReaderContext();
-              Scorer scorer = w.scorer(ctx, true, false, FeatureFlags.DOCS, ctx.reader().getLiveDocs());
+              Scorer scorer = w.scorer(ctx, true, false, PostingFeatures.DOCS_AND_FREQS, ctx.reader().getLiveDocs());
               if (scorer != null) {
                 boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
                 Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
@@ -355,7 +355,7 @@ public class QueryUtils {
           indexSearcher.setSimilarity(s.getSimilarity());
           Weight w = indexSearcher.createNormalizedWeight(q);
           AtomicReaderContext ctx = previousReader.getTopReaderContext();
-          Scorer scorer = w.scorer(ctx, true, false, FeatureFlags.DOCS, ctx.reader().getLiveDocs());
+          Scorer scorer = w.scorer(ctx, true, false, PostingFeatures.DOCS_AND_FREQS, ctx.reader().getLiveDocs());
           if (scorer != null) {
             boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
             Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
@@ -386,7 +386,7 @@ public class QueryUtils {
           long startMS = System.currentTimeMillis();
           for (int i=lastDoc[0]+1; i<=doc; i++) {
             Weight w = s.createNormalizedWeight(q);
-            Scorer scorer = w.scorer(context.get(leafPtr), true, false, FeatureFlags.DOCS, liveDocs);
+            Scorer scorer = w.scorer(context.get(leafPtr), true, false, PostingFeatures.DOCS_AND_FREQS, liveDocs);
             Assert.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS);
             Assert.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.docID(),doc,scorer.docID());
             float skipToScore = scorer.score();
@@ -414,7 +414,7 @@ public class QueryUtils {
           IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
           indexSearcher.setSimilarity(s.getSimilarity());
           Weight w = indexSearcher.createNormalizedWeight(q);
-          Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, FeatureFlags.DOCS, previousReader.getLiveDocs());
+          Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, PostingFeatures.DOCS_AND_FREQS, previousReader.getLiveDocs());
           if (scorer != null) {
             boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
             Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
@@ -439,7 +439,7 @@ public class QueryUtils {
       IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
       indexSearcher.setSimilarity(s.getSimilarity());
       Weight w = indexSearcher.createNormalizedWeight(q);
-      Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, FeatureFlags.DOCS, previousReader.getLiveDocs());
+      Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, PostingFeatures.DOCS_AND_FREQS, previousReader.getLiveDocs());
       if (scorer != null) {
         boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
         Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/RandomSimilarityProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/RandomSimilarityProvider.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/RandomSimilarityProvider.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/RandomSimilarityProvider.java Mon Aug 13 13:52:46 2012
@@ -67,12 +67,12 @@ public class RandomSimilarityProvider ex
   final List<Similarity> knownSims;
   Map<String,Similarity> previousMappings = new HashMap<String,Similarity>();
   final int perFieldSeed;
-  final boolean shouldCoord;
+  final int coordType; // 0 = no coord, 1 = coord, 2 = crazy coord
   final boolean shouldQueryNorm;
   
   public RandomSimilarityProvider(Random random) {
     perFieldSeed = random.nextInt();
-    shouldCoord = random.nextBoolean();
+    coordType = random.nextInt(3);
     shouldQueryNorm = random.nextBoolean();
     knownSims = new ArrayList<Similarity>(allSims);
     Collections.shuffle(knownSims, random);
@@ -80,10 +80,12 @@ public class RandomSimilarityProvider ex
   
   @Override
   public float coord(int overlap, int maxOverlap) {
-    if (shouldCoord) {
+    if (coordType == 0) {
+      return 1.0f;
+    } else if (coordType == 1) {
       return defaultSim.coord(overlap, maxOverlap);
     } else {
-      return 1.0f;
+      return overlap / ((float)maxOverlap + 1);
     }
   }
   
@@ -161,6 +163,14 @@ public class RandomSimilarityProvider ex
   
   @Override
   public synchronized String toString() {
-    return "RandomSimilarityProvider(queryNorm=" + shouldQueryNorm + ",coord=" + shouldCoord + "): " + previousMappings.toString();
+    final String coordMethod;
+    if (coordType == 0) {
+      coordMethod = "no";
+    } else if (coordType == 1) {
+      coordMethod = "yes";
+    } else {
+      coordMethod = "crazy";
+    }
+    return "RandomSimilarityProvider(queryNorm=" + shouldQueryNorm + ",coord=" + coordMethod + "): " + previousMappings.toString();
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/ShardSearchingTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/ShardSearchingTestBase.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/ShardSearchingTestBase.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/ShardSearchingTestBase.java Mon Aug 13 13:52:46 2012
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.Closeable;
-import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -27,15 +26,17 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
-import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.PrintStreamInfoStream;
+import org.apache.lucene.util._TestUtil;
 
 // TODO
 //   - doc blocks?  so we can test joins/grouping...
@@ -423,11 +424,16 @@ public abstract class ShardSearchingTest
 
     private volatile ShardIndexSearcher currentShardSearcher;
 
-    public NodeState(Random random, String baseDir, int nodeID, int numNodes) throws IOException {
+    public NodeState(Random random, int nodeID, int numNodes) throws IOException {
       myNodeID = nodeID;
-      dir = newFSDirectory(new File(baseDir + "." + myNodeID));
+      dir = newFSDirectory(_TestUtil.getTempDir("ShardSearchingTestBase"));
       // TODO: set warmer
-      writer = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+      IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
+      iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
+      if (VERBOSE) {
+        iwc.setInfoStream(new PrintStreamInfoStream(System.out));
+      }
+      writer = new IndexWriter(dir, iwc);
       mgr = new SearcherManager(writer, true, null);
       searchers = new SearcherLifetimeManager();
 
@@ -556,14 +562,14 @@ public abstract class ShardSearchingTest
   long endTimeNanos;
   private Thread changeIndicesThread;
 
-  protected void start(String baseDirName, int numNodes, double runTimeSec, int maxSearcherAgeSeconds) throws IOException {
+  protected void start(int numNodes, double runTimeSec, int maxSearcherAgeSeconds) throws IOException {
 
     endTimeNanos = System.nanoTime() + (long) (runTimeSec*1000000000);
     this.maxSearcherAgeSeconds = maxSearcherAgeSeconds;
 
     nodes = new NodeState[numNodes];
     for(int nodeID=0;nodeID<numNodes;nodeID++) {
-      nodes[nodeID] = new NodeState(random(), baseDirName, nodeID, numNodes);
+      nodes[nodeID] = new NodeState(random(), nodeID, numNodes);
     }
 
     long[] nodeVersions = new long[nodes.length];

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Mon Aug 13 13:52:46 2012
@@ -57,8 +57,7 @@ import org.apache.lucene.util._TestUtil;
  * </ul>
  */
 
-public class MockDirectoryWrapper extends Directory {
-  final Directory delegate;
+public class MockDirectoryWrapper extends BaseDirectoryWrapper {
   long maxSize;
 
   // Max actual bytes used. This is set by MockRAMOutputStream:
@@ -67,8 +66,6 @@ public class MockDirectoryWrapper extend
   Random randomState;
   boolean noDeleteOpenFile = true;
   boolean preventDoubleWrite = true;
-  boolean checkIndexOnClose = true;
-  boolean crossCheckTermVectorsOnClose = true;
   boolean trackDiskUsage = false;
   private Set<String> unSyncedFiles;
   private Set<String> createdFiles;
@@ -109,7 +106,7 @@ public class MockDirectoryWrapper extend
   }
 
   public MockDirectoryWrapper(Random random, Directory delegate) {
-    this.delegate = delegate;
+    super(delegate);
     // must make a private random since our methods are
     // called from different threads; else test failures may
     // not be reproducible from the original seed
@@ -251,19 +248,19 @@ public class MockDirectoryWrapper extend
           }
         }
         final IndexOutput tempOut = delegate.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
-        IndexInput in = delegate.openInput(name, LuceneTestCase.newIOContext(randomState));
-        tempOut.copyBytes(in, in.length()/2);
+        IndexInput ii = delegate.openInput(name, LuceneTestCase.newIOContext(randomState));
+        tempOut.copyBytes(ii, ii.length()/2);
         tempOut.close();
-        in.close();
+        ii.close();
 
         // Delete original and copy bytes back:
         deleteFile(name, true);
         
         final IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState));
-        in = delegate.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
-        out.copyBytes(in, in.length());
+        ii = delegate.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
+        out.copyBytes(ii, ii.length());
         out.close();
-        in.close();
+        ii.close();
         deleteFile(tempFileName, true);
       } else if (damage == 3) {
         // The file survived intact:
@@ -317,26 +314,6 @@ public class MockDirectoryWrapper extend
   }
 
   /**
-   * Set whether or not checkindex should be run
-   * on close
-   */
-  public void setCheckIndexOnClose(boolean value) {
-    this.checkIndexOnClose = value;
-  }
-  
-  public boolean getCheckIndexOnClose() {
-    return checkIndexOnClose;
-  }
-
-  public void setCrossCheckTermVectorsOnClose(boolean value) {
-    this.crossCheckTermVectorsOnClose = value;
-  }
-
-  public boolean getCrossCheckTermVectorsOnClose() {
-    return crossCheckTermVectorsOnClose;
-  }
-
-  /**
    * If 0.0, no exceptions will be thrown.  Else this should
    * be a double 0.0 - 1.0.  We will randomly throw an
    * IOException on the first write to an OutputStream based
@@ -574,9 +551,9 @@ public class MockDirectoryWrapper extend
     if (noDeleteOpenFile && openLocks.size() > 0) {
       throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks);
     }
-    open = false;
-    if (checkIndexOnClose) {
-      if (indexPossiblyExists(this)) {
+    isOpen = false;
+    if (getCheckIndexOnClose()) {
+      if (indexPossiblyExists()) {
         if (LuceneTestCase.VERBOSE) {
           System.out.println("\nNOTE: MockDirectoryWrapper: now crash");
         }
@@ -584,7 +561,7 @@ public class MockDirectoryWrapper extend
         if (LuceneTestCase.VERBOSE) {
           System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
         } 
-        _TestUtil.checkIndex(this, crossCheckTermVectorsOnClose);
+        _TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose());
 
         if (assertNoUnreferencedFilesOnClose) {
           // now look for unreferenced files:
@@ -612,26 +589,6 @@ public class MockDirectoryWrapper extend
     }
     delegate.close();
   }
-  
-  /** don't rely upon DirectoryReader.fileExists to determine if we should
-   *  checkIndex() or not. It might mask real problems, where we silently
-   *  don't checkindex at all. instead we look for a segments file.
-   */
-  private boolean indexPossiblyExists(Directory d) {
-    String files[];
-    try {
-      files = d.listAll();
-    } catch (IOException ex) {
-      // this means directory doesn't exist, which is ok. return false
-      return false;
-    }
-    for (String f : files) {
-      if (f.startsWith("segments_")) {
-        return true;
-      }
-    }
-    return false;
-  }
 
   synchronized void removeOpenFile(Closeable c, String name) {
     Integer v = openFiles.get(name);
@@ -657,12 +614,6 @@ public class MockDirectoryWrapper extend
   public synchronized void removeIndexInput(IndexInput in, String name) {
     removeOpenFile(in, name);
   }
-
-  boolean open = true;
-  
-  public synchronized boolean isOpen() {
-    return open;
-  }
   
   /**
    * Objects that represent fail-able conditions. Objects of a derived

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableDirectory.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableDirectory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableDirectory.java Mon Aug 13 13:52:46 2012
@@ -2,6 +2,7 @@ package org.apache.lucene.util;
 
 import java.io.Closeable;
 
+import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.junit.Assert;
 
@@ -23,15 +24,15 @@ import org.junit.Assert;
  */
 
 /**
- * Attempts to close a {@link MockDirectoryWrapper}.
+ * Attempts to close a {@link BaseDirectoryWrapper}.
  * 
  * @see LuceneTestCase#newDirectory(java.util.Random)
  */
 final class CloseableDirectory implements Closeable {
-  private final MockDirectoryWrapper dir;
+  private final BaseDirectoryWrapper dir;
   private final TestRuleMarkFailure failureMarker;
   
-  public CloseableDirectory(MockDirectoryWrapper dir,
+  public CloseableDirectory(BaseDirectoryWrapper dir,
       TestRuleMarkFailure failureMarker) {
     this.dir = dir;
     this.failureMarker = failureMarker;

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneJUnit3MethodProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneJUnit3MethodProvider.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneJUnit3MethodProvider.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneJUnit3MethodProvider.java Mon Aug 13 13:52:46 2012
@@ -17,13 +17,14 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import static com.carrotsearch.randomizedtesting.MethodCollector.flatten;
-import static com.carrotsearch.randomizedtesting.MethodCollector.mutableCopy1;
-
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
 
+import com.carrotsearch.randomizedtesting.ClassModel;
+import com.carrotsearch.randomizedtesting.ClassModel.MethodModel;
 import com.carrotsearch.randomizedtesting.TestMethodProvider;
 
 /**
@@ -31,20 +32,21 @@ import com.carrotsearch.randomizedtestin
  */
 public final class LuceneJUnit3MethodProvider implements TestMethodProvider {
   @Override
-  public Collection<Method> getTestMethods(Class<?> suiteClass, List<List<Method>> methods) {
-    // We will return all methods starting with test* and rely on further validation to weed
-    // out static or otherwise invalid test methods.
-    List<Method> copy = mutableCopy1(flatten(methods));
-    Iterator<Method> i =copy.iterator();
-    while (i.hasNext()) {
-      Method m= i.next();
-      if (!m.getName().startsWith("test") ||
-          !Modifier.isPublic(m.getModifiers()) ||
-           Modifier.isStatic(m.getModifiers()) ||
-           m.getParameterTypes().length != 0) {
-        i.remove();
+  public Collection<Method> getTestMethods(Class<?> suiteClass, ClassModel classModel) {
+    Map<Method,MethodModel> methods = classModel.getMethods();
+    ArrayList<Method> result = new ArrayList<Method>();
+    for (MethodModel mm : methods.values()) {
+      // Skip any methods that have overrieds/ shadows.
+      if (mm.getDown() != null) continue;
+
+      Method m = mm.element;
+      if (m.getName().startsWith("test") &&
+          Modifier.isPublic(m.getModifiers()) &&
+          !Modifier.isStatic(m.getModifiers()) &&
+          m.getParameterTypes().length == 0) {
+        result.add(m);
       }
     }
-    return copy;
+    return result;
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Mon Aug 13 13:52:46 2012
@@ -45,7 +45,13 @@ import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
 import com.carrotsearch.randomizedtesting.*;
 import com.carrotsearch.randomizedtesting.annotations.*;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakGroup.Group;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
+import com.carrotsearch.randomizedtesting.rules.NoInstanceHooksOverridesRule;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsBoolean;
@@ -113,7 +119,15 @@ import static com.carrotsearch.randomize
   RunListenerPrintReproduceInfo.class
 })
 @SeedDecorators({MixWithSuiteName.class}) // See LUCENE-3995 for rationale.
-@ThreadLeaks(failTestIfLeaking = false)
+@ThreadLeakScope(Scope.SUITE)
+@ThreadLeakGroup(Group.MAIN)
+@ThreadLeakAction({Action.WARN, Action.INTERRUPT})
+@ThreadLeakLingering(linger = 20000) // Wait long for leaked threads to complete before failure. zk needs this.
+@ThreadLeakZombies(Consequence.IGNORE_REMAINING_TESTS)
+@TimeoutSuite(millis = 2 * TimeUnits.HOUR)
+@ThreadLeakFilters(defaultFilters = true, filters = {
+    QuickPatchThreadsFilter.class
+})
 public abstract class LuceneTestCase extends Assert {
 
   // --------------------------------------------------------------------
@@ -124,6 +138,7 @@ public abstract class LuceneTestCase ext
   public static final String SYSPROP_WEEKLY = "tests.weekly";
   public static final String SYSPROP_AWAITSFIX = "tests.awaitsfix";
   public static final String SYSPROP_SLOW = "tests.slow";
+  public static final String SYSPROP_BADAPPLES = "tests.badapples";
 
   /** @see #ignoreAfterMaxFailures*/
   private static final String SYSPROP_MAXFAILURES = "tests.maxfailures";
@@ -170,7 +185,21 @@ public abstract class LuceneTestCase ext
   @Retention(RetentionPolicy.RUNTIME)
   @TestGroup(enabled = true, sysProperty = SYSPROP_SLOW)
   public @interface Slow {}
-  
+
+  /**
+   * Annotation for tests that fail frequently. You can disable them
+   * if you want to run a long build and not stop on something that
+   * is a known problem.
+   * <pre>
+   * -Dtests.badapples=false
+   * </pre>
+   */
+  @Documented
+  @Inherited
+  @Retention(RetentionPolicy.RUNTIME)
+  @TestGroup(enabled = true, sysProperty = SYSPROP_BADAPPLES)
+  public @interface BadApple {}
+
   /**
    * Annotation for test classes that should avoid certain codec types
    * (because they are expensive, for example).
@@ -182,7 +211,6 @@ public abstract class LuceneTestCase ext
   public @interface SuppressCodecs {
     String[] value();
   }
-
   
   // -----------------------------------------------------------------
   // Truly immutable fields and constants, initialized once and valid 
@@ -277,6 +305,15 @@ public abstract class LuceneTestCase ext
     CORE_DIRECTORIES.add("RAMDirectory");
   };
   
+  protected static final Set<String> doesntSupportOffsets = new HashSet<String>(Arrays.asList( 
+    "Lucene3x",
+    "MockFixedIntBlock",
+    "MockVariableIntBlock",
+    "MockSep",
+    "MockRandom",
+    "For",
+    "PFor"
+  ));
   
   // -----------------------------------------------------------------
   // Fields initialized in class or instance rules.
@@ -340,12 +377,16 @@ public abstract class LuceneTestCase ext
     .around(ignoreAfterMaxFailures)
     .around(suiteFailureMarker)
     .around(new TestRuleAssertionsRequired())
-    .around(new TestRuleNoStaticHooksShadowing())
-    .around(new TestRuleNoInstanceHooksOverrides())
+    .around(new NoClassHooksShadowingRule())
+    .around(new NoInstanceHooksOverridesRule() {
+      @Override
+      protected boolean verify(Method key) {
+        String name = key.getName();
+        return !(name.equals("setUp") || name.equals("tearDown"));
+      }
+    })
     .around(new SystemPropertiesInvariantRule(IGNORED_INVARIANT_PROPERTIES))
-    .around(new TestRuleIcuHack())
     .around(classNameRule = new TestRuleStoreClassName())
-    .around(new TestRuleReportUncaughtExceptions())
     .around(classEnvRule = new TestRuleSetupAndRestoreClassEnv());
 
 
@@ -372,7 +413,6 @@ public abstract class LuceneTestCase ext
     .outerRule(testFailureMarker)
     .around(ignoreAfterMaxFailures)
     .around(threadAndTestNameRule)
-    .around(new TestRuleReportUncaughtExceptions())
     .around(new SystemPropertiesInvariantRule(IGNORED_INVARIANT_PROPERTIES))
     .around(new TestRuleSetupAndRestoreInstanceEnv())
     .around(new TestRuleFieldCacheSanity())
@@ -780,48 +820,56 @@ public abstract class LuceneTestCase ext
    * Returns a new Directory instance. Use this when the test does not
    * care about the specific Directory implementation (most tests).
    * <p>
-   * The Directory is wrapped with {@link MockDirectoryWrapper}.
-   * By default this means it will be picky, such as ensuring that you
+   * The Directory is wrapped with {@link BaseDirectoryWrapper}.
+   * this means usually it will be picky, such as ensuring that you
    * properly close it and all open files in your test. It will emulate
    * some features of Windows, such as not allowing open files to be
    * overwritten.
    */
-  public static MockDirectoryWrapper newDirectory() {
+  public static BaseDirectoryWrapper newDirectory() {
     return newDirectory(random());
   }
-
+  
   /**
    * Returns a new Directory instance, using the specified random.
    * See {@link #newDirectory()} for more information.
    */
-  public static MockDirectoryWrapper newDirectory(Random r) {
-    Directory impl = newDirectoryImpl(r, TEST_DIRECTORY);
-    MockDirectoryWrapper dir = new MockDirectoryWrapper(r, maybeNRTWrap(r, impl));
-    closeAfterSuite(new CloseableDirectory(dir, suiteFailureMarker));
+  public static BaseDirectoryWrapper newDirectory(Random r) {
+    return wrapDirectory(r, newDirectoryImpl(r, TEST_DIRECTORY), rarely(r));
+  }
 
-    dir.setThrottling(TEST_THROTTLING);
-    if (VERBOSE) {
-      System.out.println("NOTE: LuceneTestCase.newDirectory: returning " + dir);
-    }
-    return dir;
-   }
+  public static MockDirectoryWrapper newMockDirectory() {
+    return newMockDirectory(random());
+  }
+
+  public static MockDirectoryWrapper newMockDirectory(Random r) {
+    return (MockDirectoryWrapper) wrapDirectory(r, newDirectoryImpl(r, TEST_DIRECTORY), false);
+  }
+
+  public static MockDirectoryWrapper newMockFSDirectory(File f) {
+    return (MockDirectoryWrapper) newFSDirectory(f, null, false);
+  }
 
   /**
    * Returns a new Directory instance, with contents copied from the
    * provided directory. See {@link #newDirectory()} for more
    * information.
    */
-  public static MockDirectoryWrapper newDirectory(Directory d) throws IOException {
+  public static BaseDirectoryWrapper newDirectory(Directory d) throws IOException {
     return newDirectory(random(), d);
   }
 
   /** Returns a new FSDirectory instance over the given file, which must be a folder. */
-  public static MockDirectoryWrapper newFSDirectory(File f) {
+  public static BaseDirectoryWrapper newFSDirectory(File f) {
     return newFSDirectory(f, null);
   }
 
   /** Returns a new FSDirectory instance over the given file, which must be a folder. */
-  public static MockDirectoryWrapper newFSDirectory(File f, LockFactory lf) {
+  public static BaseDirectoryWrapper newFSDirectory(File f, LockFactory lf) {
+    return newFSDirectory(f, lf, rarely());
+  }
+
+  private static BaseDirectoryWrapper newFSDirectory(File f, LockFactory lf, boolean bare) {
     String fsdirClass = TEST_DIRECTORY;
     if (fsdirClass.equals("random")) {
       fsdirClass = RandomPicks.randomFrom(random(), FS_DIRECTORIES); 
@@ -838,14 +886,11 @@ public abstract class LuceneTestCase ext
       }
 
       Directory fsdir = newFSDirectoryImpl(clazz, f);
-      MockDirectoryWrapper dir = new MockDirectoryWrapper(
-          random(), maybeNRTWrap(random(), fsdir));
+      BaseDirectoryWrapper wrapped = wrapDirectory(random(), fsdir, bare);
       if (lf != null) {
-        dir.setLockFactory(lf);
+        wrapped.setLockFactory(lf);
       }
-      closeAfterSuite(new CloseableDirectory(dir, suiteFailureMarker));
-      dir.setThrottling(TEST_THROTTLING);
-      return dir;
+      return wrapped;
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -856,22 +901,27 @@ public abstract class LuceneTestCase ext
    * with contents copied from the provided directory. See 
    * {@link #newDirectory()} for more information.
    */
-  public static MockDirectoryWrapper newDirectory(Random r, Directory d) throws IOException {
+  public static BaseDirectoryWrapper newDirectory(Random r, Directory d) throws IOException {
     Directory impl = newDirectoryImpl(r, TEST_DIRECTORY);
     for (String file : d.listAll()) {
      d.copy(impl, file, file, newIOContext(r));
     }
-    MockDirectoryWrapper dir = new MockDirectoryWrapper(r, maybeNRTWrap(r, impl));
-    closeAfterSuite(new CloseableDirectory(dir, suiteFailureMarker));
-    dir.setThrottling(TEST_THROTTLING);
-    return dir;
+    return wrapDirectory(r, impl, rarely(r));
   }
   
-  private static Directory maybeNRTWrap(Random random, Directory directory) {
+  private static BaseDirectoryWrapper wrapDirectory(Random random, Directory directory, boolean bare) {
     if (rarely(random)) {
-      return new NRTCachingDirectory(directory, random.nextDouble(), random.nextDouble());
+      directory = new NRTCachingDirectory(directory, random.nextDouble(), random.nextDouble());
+    }
+    if (bare) {
+      BaseDirectoryWrapper base = new BaseDirectoryWrapper(directory);
+      closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker));
+      return base;
     } else {
-      return directory;
+      MockDirectoryWrapper mock = new MockDirectoryWrapper(random, directory);
+      mock.setThrottling(TEST_THROTTLING);
+      closeAfterSuite(new CloseableDirectory(mock, suiteFailureMarker));
+      return mock;
     }
   }
   
@@ -918,6 +968,10 @@ public abstract class LuceneTestCase ext
       }
       if (!newType.storeTermVectorPositions()) {
         newType.setStoreTermVectorPositions(random.nextBoolean());
+        
+        if (newType.storeTermVectorPositions() && !newType.storeTermVectorPayloads()) {
+          newType.setStoreTermVectorPayloads(random.nextBoolean());
+        }
       }
     }
 
@@ -1046,9 +1100,9 @@ public abstract class LuceneTestCase ext
             // QueryUtils' reader with a fake cache key, so insanity checker cannot walk
             // along our reader:
             if (r instanceof AtomicReader) {
-              r = new FCInvisibleMultiReader(new AssertingAtomicReader((AtomicReader)r));
+              r = new AssertingAtomicReader((AtomicReader)r);
             } else if (r instanceof DirectoryReader) {
-              r = new FCInvisibleMultiReader((DirectoryReader)r);
+              r = new AssertingDirectoryReader((DirectoryReader)r);
             }
             break;
           default:

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAssertionsRequired.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAssertionsRequired.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAssertionsRequired.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAssertionsRequired.java Mon Aug 13 13:52:46 2012
@@ -21,30 +21,24 @@ import org.junit.rules.TestRule;
 import org.junit.runner.Description;
 import org.junit.runners.model.Statement;
 
-import com.carrotsearch.randomizedtesting.ClassValidator;
-
 /**
  * Require assertions for Lucene/Solr packages.
  */
-public class TestRuleAssertionsRequired implements TestRule, ClassValidator {
+public class TestRuleAssertionsRequired implements TestRule {
   @Override
   public Statement apply(final Statement base, final Description description) {
     return new Statement() {
       @Override
       public void evaluate() throws Throwable {
-        validate(description.getTestClass());
+        try {
+          assert false;
+          throw new Exception("Test class requires assertions, enable assertions globally (-ea) or for Solr/Lucene subpackages only.");
+        } catch (AssertionError e) {
+          // Ok, enabled.
+        }
+
         base.evaluate();
       }
     };
   }
-
-  @Override
-  public void validate(Class<?> clazz) throws Throwable {
-    try {
-      assert false;
-      throw new Exception("Test class requires assertions, enable assertions globally (-ea) or for Solr/Lucene subpackages only.");
-    } catch (AssertionError e) {
-      // Ok, enabled.
-    }    
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Mon Aug 13 13:52:46 2012
@@ -87,33 +87,6 @@ final class TestRuleSetupAndRestoreClass
     restoreProperties.put("solr.solr.home", System.getProperty("solr.solr.home"));
     restoreProperties.put("solr.data.dir", System.getProperty("solr.data.dir"));
 
-    // enable the Lucene 3.x PreflexRW codec explicitly, to work around bugs in IBM J9 / Harmony ServiceLoader:
-    try {
-      final java.lang.reflect.Field spiLoaderField = Codec.class.getDeclaredField("loader");
-      spiLoaderField.setAccessible(true);
-      final Object spiLoader = spiLoaderField.get(null);
-      final java.lang.reflect.Field modifiableServicesField = NamedSPILoader.class.getDeclaredField("modifiableServices");
-      modifiableServicesField.setAccessible(true);
-      /* note: re-enable this if we make a Lucene4x impersonator 
-      @SuppressWarnings({"unchecked","rawtypes"}) final Map<String,Codec> serviceMap =
-        (Map) modifiableServicesField.get(spiLoader);
-      if (!(Codec.forName("Lucene3x") instanceof PreFlexRWCodec)) {
-        if (Constants.JAVA_VENDOR.startsWith("IBM")) {
-          // definitely a buggy version
-          System.err.println("ERROR: Your VM's java.util.ServiceLoader implementation is buggy"+
-            " and does not respect classpath order, please report this to the vendor.");
-        } else {
-          // could just be a classpath issue
-          System.err.println("ERROR: fix your classpath to have tests-framework.jar before lucene-core.jar!"+
-              " If you have already done this, then your VM's java.util.ServiceLoader implementation is buggy"+
-              " and does not respect classpath order, please report this to the vendor.");
-        }
-        serviceMap.put("Lucene3x", new PreFlexRWCodec());
-      } */
-    } catch (Exception e) {
-      throw new RuntimeException("Cannot access internals of Codec and NamedSPILoader classes", e);
-    }
-    
     // if verbose: print some debugging stuff about which codecs are loaded.
     if (VERBOSE) {
       Set<String> codecs = Codec.availableCodecs();

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java Mon Aug 13 13:52:46 2012
@@ -857,7 +857,7 @@ public class _TestUtil {
   // Returns a DocsEnum, but randomly sometimes uses a
   // DocsAndFreqsEnum, DocsAndPositionsEnum.  Returns null
   // if field/term doesn't exist:
-  public static DocsEnum docs(Random random, IndexReader r, String field, BytesRef term, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+  public static DocsEnum docs(Random random, IndexReader r, String field, BytesRef term, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
     final Terms terms = MultiFields.getTerms(r, field);
     if (terms == null) {
       return null;
@@ -866,45 +866,30 @@ public class _TestUtil {
     if (!termsEnum.seekExact(term, random.nextBoolean())) {
       return null;
     }
-    if (random.nextBoolean()) {
-      if (random.nextBoolean()) {
-        // TODO: cast re-use to D&PE if we can...?
-        DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, true);
-        if (docsAndPositions == null) {
-          docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, false);
-        }
-        if (docsAndPositions != null) {
-          return docsAndPositions;
-        }
-      }
-      final DocsEnum docsAndFreqs = termsEnum.docs(liveDocs, reuse, true);
-      if (docsAndFreqs != null) {
-        return docsAndFreqs;
-      }
-    }
-    return termsEnum.docs(liveDocs, reuse, needsFreqs);
+    return docs(random, termsEnum, liveDocs, reuse, flags);
   }
 
   // Returns a DocsEnum from a positioned TermsEnum, but
   // randomly sometimes uses a DocsAndFreqsEnum, DocsAndPositionsEnum.
-  public static DocsEnum docs(Random random, TermsEnum termsEnum, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+  public static DocsEnum docs(Random random, TermsEnum termsEnum, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
     if (random.nextBoolean()) {
       if (random.nextBoolean()) {
-        // TODO: cast re-use to D&PE if we can...?
-        DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, true);
-        if (docsAndPositions == null) {
-          docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, false);
+        final int posFlags;
+        switch (random.nextInt(4)) {
+          case 0: posFlags = 0; break;
+          case 1: posFlags = DocsAndPositionsEnum.FLAG_OFFSETS; break;
+          case 2: posFlags = DocsAndPositionsEnum.FLAG_PAYLOADS; break;
+          default: posFlags = DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS; break;
         }
+        // TODO: cast to DocsAndPositionsEnum?
+        DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(liveDocs, null, posFlags);
         if (docsAndPositions != null) {
           return docsAndPositions;
         }
       }
-      final DocsEnum docsAndFreqs = termsEnum.docs(liveDocs, null, true);
-      if (docsAndFreqs != null) {
-        return docsAndFreqs;
-      }
+      flags |= DocsEnum.FLAG_FREQS;
     }
-    return termsEnum.docs(liveDocs, null, needsFreqs);
+    return termsEnum.docs(liveDocs, reuse, flags);
   }
   
   public static CharSequence stringToCharSequence(String string, Random random) {
@@ -971,9 +956,18 @@ public class _TestUtil {
     while (true) {
       try {
         Pattern p = Pattern.compile(_TestUtil.randomRegexpishString(random));
+        String replacement = null;
+        // ignore bugs in Sun's regex impl
+        try {
+          replacement = p.matcher(nonBmpString).replaceAll("_");
+        } catch (StringIndexOutOfBoundsException jdkBug) {
+          System.out.println("WARNING: your jdk is buggy!");
+          System.out.println("Pattern.compile(\"" + p.pattern() + 
+              "\").matcher(\"AB\\uD840\\uDC00C\").replaceAll(\"_\"); should not throw IndexOutOfBounds!");
+        }
         // Make sure the result of applying the pattern to a string with extended
         // unicode characters is a valid utf16 string. See LUCENE-4078 for discussion.
-        if (UnicodeUtil.validUTF16String(p.matcher(nonBmpString).replaceAll("_"))) {
+        if (replacement != null && UnicodeUtil.validUTF16String(replacement)) {
           return p;
         }
       } catch (PatternSyntaxException ignored) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Mon Aug 13 13:52:46 2012
@@ -20,5 +20,6 @@ org.apache.lucene.codecs.mocksep.MockSep
 org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat
 org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
 org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds
+org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings
 org.apache.lucene.codecs.asserting.AssertingPostingsFormat
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/tools/custom-tasks.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/tools/custom-tasks.xml?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/tools/custom-tasks.xml (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/tools/custom-tasks.xml Mon Aug 13 13:52:46 2012
@@ -18,12 +18,13 @@
 
   <macrodef name="license-check-macro">
     <attribute name="dir" />
+    <attribute name="licensedir" />
     <element name="additional-excludes" optional="true" />
     <element name="additional-filters"  optional="true" />
     <sequential>
       <!-- LICENSE and NOTICE verification macro. -->
       <echo>License check under: @{dir}</echo>
-      <licenses>
+      <licenses licenseDirectory="@{licensedir}">
         <fileset dir="@{dir}">
           <include name="**/*.jar" />
           <!-- Speed up scanning a bit. -->
@@ -50,8 +51,9 @@
             <additional-filters />
 
             <!-- Typical version patterns. -->
+            <replaceregex pattern="\.rc[0-9]+" replace="" flags="gi" />
             <replaceregex pattern="\-(r)?([0-9\-\_\.])+(b(eta)?([0-9\-\.])*)?$" replace="" flags="gi" />
-    
+
             <!-- git hashcode pattern: its always 40 chars right? -->
             <replaceregex pattern="\-[a-z0-9]{40,40}$" replace="" flags="gi" />
           </filtermapper>

Modified: lucene/dev/branches/LUCENE-2878/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/tools/src/java/org/apache/lucene/validation/LicenseCheckTask.java Mon Aug 13 13:52:46 2012
@@ -58,6 +58,11 @@ public class LicenseCheckTask extends Ta
    * All JAR files to check.
    */
   private Resources jarResources = new Resources();
+  
+  /**
+   * Directory containing licenses
+   */
+  private File licenseDirectory;
 
   /**
    * License file mapper.
@@ -94,6 +99,10 @@ public class LicenseCheckTask extends Ta
   public void setVerbose(boolean verbose) {
     verboseLevel = (verbose ? Project.MSG_INFO : Project.MSG_VERBOSE);
   }
+  
+  public void setLicenseDirectory(File file) {
+    licenseDirectory = file;
+  }
 
   /**
    * Execute the task.
@@ -153,7 +162,7 @@ public class LicenseCheckTask extends Ta
     log("Scanning: " + jarFile.getPath(), verboseLevel);
 
     // validate the jar matches against our expected hash
-    final File checksumFile = new File(jarFile.getParent(), 
+    final File checksumFile = new File(licenseDirectory, 
                                        jarFile.getName() + "." + CHECKSUM_TYPE);
     if (! (checksumFile.exists() && checksumFile.canRead()) ) {
       log("MISSING " +CHECKSUM_TYPE+ " checksum file for: " + jarFile.getPath(), Project.MSG_ERR);
@@ -200,9 +209,9 @@ public class LicenseCheckTask extends Ta
     Map<File, LicenseType> foundLicenses = new LinkedHashMap<File, LicenseType>();
     List<File> expectedLocations = new ArrayList<File>();
 outer:
-    for (String mappedPath : licenseMapper.mapFileName(jarFile.getPath())) {
+    for (String mappedPath : licenseMapper.mapFileName(jarFile.getName())) {
       for (LicenseType licenseType : LicenseType.values()) {
-        File licensePath = new File(mappedPath + licenseType.licenseFileSuffix());
+        File licensePath = new File(licenseDirectory, mappedPath + licenseType.licenseFileSuffix());
         if (licensePath.exists()) {
           foundLicenses.put(licensePath, licenseType);
           log(" FOUND " + licenseType.name() + " license at " + licensePath.getPath(), 
@@ -218,10 +227,10 @@ outer:
     // Check for NOTICE files.
     for (Map.Entry<File, LicenseType> e : foundLicenses.entrySet()) {
       LicenseType license = e.getValue();
-      String licensePath = e.getKey().getAbsolutePath();
+      String licensePath = e.getKey().getName();
       String baseName = licensePath.substring(
           0, licensePath.length() - license.licenseFileSuffix().length());
-      File noticeFile = new File(baseName + license.noticeFileSuffix());
+      File noticeFile = new File(licenseDirectory, baseName + license.noticeFileSuffix());
 
       if (noticeFile.exists()) {
         log(" FOUND NOTICE file at " + noticeFile.getAbsolutePath(), verboseLevel);