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/09/21 19:22:27 UTC

svn commit: r1388574 [31/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/eclipse/dot.settings/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/anal...

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/build.xml?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/build.xml (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/build.xml Fri Sep 21 17:21:34 2012
@@ -26,6 +26,7 @@
 
   <path id="classpath">
     <pathelement location="${common.dir}/build/core/classes/java"/>
+    <pathelement location="${common.dir}/build/codecs/classes/java"/>
     <path refid="junit-path"/>
     <path refid="ant-path"/>
   </path>
@@ -35,7 +36,7 @@
       and *not* to depend on clover; clover already includes the
       test-framework sources in each module's test instrumentation.
    -->
-  <target name="compile-core" depends="init,compile-lucene-core"
+  <target name="compile-core" depends="init,compile-lucene-core,compile-codecs"
           description="Compiles test-framework classes">
     <compile srcdir="${src.dir}" destdir="${build.dir}/classes/java">
       <classpath refid="classpath"/>
@@ -46,7 +47,7 @@
   </target>
 
   <target name="javadocs-core" depends="javadocs"/>
-  <target name="javadocs" depends="init,javadocs-lucene-core">
+  <target name="javadocs" depends="init,javadocs-lucene-core,javadocs-lucene-codecs">
     <sequential>
       <mkdir dir="${javadoc.dir}/test-framework"/>
       <invoke-javadoc overview="${src.dir}/overview.html"
@@ -57,6 +58,7 @@
           <link offline="true" href="${javadoc.link.junit}"
                 packagelistLoc="${javadoc.packagelist.dir}/junit"/>
           <link href="../core/"/>
+          <link href="../codecs/"/>
           <link href=""/>
         </sources>
       </invoke-javadoc>

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/ivy.xml?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/ivy.xml (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/ivy.xml Fri Sep 21 17:21:34 2012
@@ -30,11 +30,10 @@
 
     <dependencies defaultconf="default">
       <dependency org="org.apache.ant" name="ant" rev="1.8.2" transitive="false" />
-      <dependency org="org.apache.ant" name="ant-junit" rev="1.8.2" transitive="false" />
 
       <dependency org="junit" name="junit" rev="4.10" transitive="false" conf="default->*;junit4-stdalone->*" />
-      <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="2.0.0.rc5" transitive="false" conf="default->*;junit4-stdalone->*" />
-      <dependency org="com.carrotsearch.randomizedtesting" name="randomizedtesting-runner" rev="2.0.0.rc5" transitive="false" conf="default->*;junit4-stdalone->*" />
+      <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="2.0.1" transitive="false" conf="default->*;junit4-stdalone->*" />
+      <dependency org="com.carrotsearch.randomizedtesting" name="randomizedtesting-runner" rev="2.0.1" transitive="false" conf="default->*;junit4-stdalone->*" />
 
       <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
     </dependencies>

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java Fri Sep 21 17:21:34 2012
@@ -34,6 +34,8 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -238,8 +240,8 @@ public abstract class CollationTestBase 
     StringBuilder buff = new StringBuilder(10);
     int n = result.length;
     for (int i = 0 ; i < n ; ++i) {
-      Document doc = searcher.doc(result[i].doc);
-      IndexableField[] v = doc.getFields("tracer");
+      StoredDocument doc = searcher.doc(result[i].doc);
+      StorableField[] v = doc.getFields("tracer");
       for (int j = 0 ; j < v.length ; ++j) {
         buff.append(v[j].stringValue());
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockCharFilter.java Fri Sep 21 17:21:34 2012
@@ -42,11 +42,6 @@ public class MockCharFilter extends Char
   public MockCharFilter(Reader in) {
     this(in, 0);
   }
-
-  @Override
-  public void close() throws IOException {
-    in.close();
-  }
   
   int currentOffset = -1;
   int delta = 0;
@@ -66,7 +61,7 @@ public class MockCharFilter extends Char
     }
     
     // otherwise actually read one    
-    int ch = in.read();
+    int ch = input.read();
     if (ch < 0)
       return ch;
     

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockHoleInjectingTokenFilter.java Fri Sep 21 17:21:34 2012
@@ -29,7 +29,9 @@ import org.apache.lucene.util._TestUtil;
 // a MockRemovesTokensTF, ideally subclassing FilteringTF
 // (in modules/analysis)
 
-// Randomly injects holes:
+/** 
+ * Randomly injects holes (similar to what a stopfilter would do)
+ */
 public final class MockHoleInjectingTokenFilter extends TokenFilter {
 
   private final long randomSeed;

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockPayloadAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockPayloadAnalyzer.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockPayloadAnalyzer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockPayloadAnalyzer.java Fri Sep 21 17:21:34 2012
@@ -27,8 +27,9 @@ import java.io.Reader;
 
 
 /**
- *
- *
+ * Wraps a whitespace tokenizer with a filter that sets
+ * the first token, and odd tokens to posinc=1, and all others
+ * to 0, encoding the position as pos: XXX in the payload.
  **/
 public final class MockPayloadAnalyzer extends Analyzer {
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenizer.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenizer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockTokenizer.java Fri Sep 21 17:21:34 2012
@@ -19,12 +19,16 @@ package org.apache.lucene.analysis;
 
 import java.io.IOException;
 import java.io.Reader;
+import java.nio.CharBuffer;
+import java.util.Random;
 
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 
+import com.carrotsearch.randomizedtesting.RandomizedContext;
+
 /**
  * Tokenizer for testing.
  * <p>
@@ -77,6 +81,9 @@ public class MockTokenizer extends Token
   private int lastOffset = 0; // only for asserting
   private boolean enableChecks = true;
   
+  // evil: but we don't change the behavior with this random, we only switch up how we read
+  private final Random random = new Random(RandomizedContext.current().getRandom().nextLong());
+  
   public MockTokenizer(AttributeFactory factory, Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase, int maxTokenLength) {
     super(factory, input);
     this.runAutomaton = runAutomaton;
@@ -139,25 +146,52 @@ public class MockTokenizer extends Token
   }
 
   protected int readCodePoint() throws IOException {
-    int ch = input.read();
+    int ch = readChar();
     if (ch < 0) {
       return ch;
     } else {
       assert !Character.isLowSurrogate((char) ch) : "unpaired low surrogate: " + Integer.toHexString(ch);
       off++;
       if (Character.isHighSurrogate((char) ch)) {
-        int ch2 = input.read();
+        int ch2 = readChar();
         if (ch2 >= 0) {
           off++;
           assert Character.isLowSurrogate((char) ch2) : "unpaired high surrogate: " + Integer.toHexString(ch) + ", followed by: " + Integer.toHexString(ch2);
           return Character.toCodePoint((char) ch, (char) ch2);
         } else {
           assert false : "stream ends with unpaired high surrogate: " + Integer.toHexString(ch);
-	}
+        }
       }
       return ch;
     }
   }
+  
+  protected int readChar() throws IOException {
+    switch(random.nextInt(10)) {
+      case 0: {
+        // read(char[])
+        char c[] = new char[1];
+        int ret = input.read(c);
+        return ret < 0 ? ret : c[0];
+      }
+      case 1: {
+        // read(char[], int, int)
+        char c[] = new char[2];
+        int ret = input.read(c, 1, 1);
+        return ret < 0 ? ret : c[1];
+      }
+      case 2: {
+        // read(CharBuffer)
+        char c[] = new char[1];
+        CharBuffer cb = CharBuffer.wrap(c);
+        int ret = input.read(cb);
+        return ret < 0 ? ret : c[0];
+      }
+      default: 
+        // read()
+        return input.read();
+    }
+  }
 
   protected boolean isTokenChar(int c) {
     state = runAutomaton.step(state, c);
@@ -193,10 +227,10 @@ public class MockTokenizer extends Token
   }
 
   @Override
-  public void setReader(Reader input) throws IOException {
-    super.setReader(input);
+  boolean setReaderTestPoint() {
     assert !enableChecks || streamState == State.CLOSE : "setReader() called in wrong state: " + streamState;
     streamState = State.SETREADER;
+    return true;
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java Fri Sep 21 17:21:34 2012
@@ -18,53 +18,31 @@ package org.apache.lucene.codecs.asserti
  */
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.SegmentInfoFormat;
-import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40Codec; // javadocs @link
-import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 
 /**
  * Acts like {@link Lucene40Codec} but with additional asserts.
  */
-public class AssertingCodec extends Codec {
+public final class AssertingCodec extends FilterCodec {
 
   private final PostingsFormat postings = new AssertingPostingsFormat();
-  private final SegmentInfoFormat infos = new Lucene40SegmentInfoFormat();
-  private final StoredFieldsFormat fields = new Lucene40StoredFieldsFormat();
-  private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat();
   private final TermVectorsFormat vectors = new AssertingTermVectorsFormat();
-  private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
-  private final NormsFormat norms = new Lucene40NormsFormat();
-  private final LiveDocsFormat liveDocs = new Lucene40LiveDocsFormat();
-  
+
   public AssertingCodec() {
     super("Asserting");
   }
 
   @Override
-  public PostingsFormat postingsFormat() {
-    return postings;
-  }
-
-  @Override
-  public DocValuesFormat docValuesFormat() {
-    return docValues;
+  protected Codec delegate() {
+    return Codec.forName("Lucene40");
   }
 
   @Override
-  public StoredFieldsFormat storedFieldsFormat() {
-    return fields;
+  public PostingsFormat postingsFormat() {
+    return postings;
   }
 
   @Override
@@ -72,23 +50,4 @@ public class AssertingCodec extends Code
     return vectors;
   }
 
-  @Override
-  public FieldInfosFormat fieldInfosFormat() {
-    return fieldInfos;
-  }
-
-  @Override
-  public SegmentInfoFormat segmentInfoFormat() {
-    return infos;
-  }
-
-  @Override
-  public NormsFormat normsFormat() {
-    return norms;
-  }
-
-  @Override
-  public LiveDocsFormat liveDocsFormat() {
-    return liveDocs;
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -40,7 +40,7 @@ import org.apache.lucene.util.OpenBitSet
 /**
  * Just like {@link Lucene40PostingsFormat} but with additional asserts.
  */
-public class AssertingPostingsFormat extends PostingsFormat {
+public final class AssertingPostingsFormat extends PostingsFormat {
   private final PostingsFormat in = new Lucene40PostingsFormat();
   
   public AssertingPostingsFormat() {
@@ -83,14 +83,9 @@ public class AssertingPostingsFormat ext
     }
 
     @Override
-    public int size() throws IOException {
+    public int size() {
       return in.size();
     }
-
-    @Override
-    public long getUniqueTermCount() throws IOException {
-      return in.getUniqueTermCount();
-    }
   }
   
   static class AssertingFieldsConsumer extends FieldsConsumer {

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene40Postings.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene40Postings.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene40Postings.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/TestBloomFilteredLucene40Postings.java Fri Sep 21 17:21:34 2012
@@ -1,77 +1,74 @@
-package org.apache.lucene.codecs.bloom;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.util.FuzzySet;
-import org.apache.lucene.util.hash.MurmurHash2;
-
-/**
- * A class used for testing {@link BloomFilteringPostingsFormat} with a concrete
- * delegate (Lucene40). Creates a Bloom filter on ALL fields and with tiny
- * amounts of memory reserved for the filter. DO NOT USE IN A PRODUCTION
- * APPLICATION This is not a realistic application of Bloom Filters as they
- * ordinarily are larger and operate on only primary key type fields.
- */
-public class TestBloomFilteredLucene40Postings extends PostingsFormat {
-  
-  private BloomFilteringPostingsFormat delegate;
-  
-  // Special class used to avoid OOM exceptions where Junit tests create many
-  // fields.
-  static class LowMemoryBloomFactory extends BloomFilterFactory {
-    @Override
-    public FuzzySet getSetForField(SegmentWriteState state,FieldInfo info) {
-      return FuzzySet.createSetBasedOnMaxMemory(1024, new MurmurHash2());
-    }
-    
-    @Override
-    public boolean isSaturated(FuzzySet bloomFilter, FieldInfo fieldInfo) {
-      // For test purposes always maintain the BloomFilter - even past the point
-      // of usefulness when all bits are set
-      return false;
-    }
-  }
-  
-  public TestBloomFilteredLucene40Postings() {
-    super("TestBloomFilteredLucene40Postings");
-    delegate = new BloomFilteringPostingsFormat(new Lucene40PostingsFormat(),
-        new LowMemoryBloomFactory());
-  }
-  
-  @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
-      throws IOException {
-    return delegate.fieldsConsumer(state);
-  }
-  
-  @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state)
-      throws IOException {
-    return delegate.fieldsProducer(state);
-  }
-}
+package org.apache.lucene.codecs.bloom;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+/**
+ * A class used for testing {@link BloomFilteringPostingsFormat} with a concrete
+ * delegate (Lucene40). Creates a Bloom filter on ALL fields and with tiny
+ * amounts of memory reserved for the filter. DO NOT USE IN A PRODUCTION
+ * APPLICATION This is not a realistic application of Bloom Filters as they
+ * ordinarily are larger and operate on only primary key type fields.
+ */
+public final class TestBloomFilteredLucene40Postings extends PostingsFormat {
+  
+  private BloomFilteringPostingsFormat delegate;
+  
+  // Special class used to avoid OOM exceptions where Junit tests create many
+  // fields.
+  static class LowMemoryBloomFactory extends BloomFilterFactory {
+    @Override
+    public FuzzySet getSetForField(SegmentWriteState state,FieldInfo info) {
+      return FuzzySet.createSetBasedOnMaxMemory(1024);
+    }
+    
+    @Override
+    public boolean isSaturated(FuzzySet bloomFilter, FieldInfo fieldInfo) {
+      // For test purposes always maintain the BloomFilter - even past the point
+      // of usefulness when all bits are set
+      return false;
+    }
+  }
+  
+  public TestBloomFilteredLucene40Postings() {
+    super("TestBloomFilteredLucene40Postings");
+    delegate = new BloomFilteringPostingsFormat(new Lucene40PostingsFormat(),
+        new LowMemoryBloomFactory());
+  }
+  
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
+      throws IOException {
+    return delegate.fieldsConsumer(state);
+  }
+  
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state)
+      throws IOException {
+    return delegate.fieldsProducer(state);
+  }
+}

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/package.html?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/package.html (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/bloom/package.html Fri Sep 21 17:21:34 2012
@@ -1,25 +1,25 @@
-<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<html>
-<head>
-   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
-</head>
-<body>
-Support for generating test indexes using the BloomFilteringPostingsFormat
-</body>
-</html>
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Support for generating test indexes using the BloomFilteringPostingsFormat
+</body>
+</html>

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40ords/Lucene40WithOrds.java Fri Sep 21 17:21:34 2012
@@ -19,17 +19,18 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.BlockTermsReader;
-import org.apache.lucene.codecs.BlockTermsWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.FixedGapTermsIndexReader;
-import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.TermsIndexReaderBase;
-import org.apache.lucene.codecs.TermsIndexWriterBase;
+import org.apache.lucene.codecs.blockterms.BlockTermsReader;
+import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
+import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
+import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
+import org.apache.lucene.codecs.lucene40.Lucene40Codec; // javadocs
 import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
 import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
 import org.apache.lucene.index.SegmentReadState;
@@ -39,7 +40,11 @@ import org.apache.lucene.util.BytesRef;
 // TODO: we could make separate base class that can wrapp
 // any PostingsBaseFormat and make it ord-able...
 
-public class Lucene40WithOrds extends PostingsFormat {
+/**
+ * Customized version of {@link Lucene40Codec} that uses
+ * {@link FixedGapTermsIndexWriter}.
+ */
+public final class Lucene40WithOrds extends PostingsFormat {
     
   public Lucene40WithOrds() {
     super("Lucene40WithOrds");
@@ -109,7 +114,7 @@ public class Lucene40WithOrds extends Po
       FieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
-                                                state.segmentInfo.name,
+                                                state.segmentInfo,
                                                 postings,
                                                 state.context,
                                                 TERMS_CACHE_SIZE,

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -19,17 +19,17 @@ package org.apache.lucene.codecs.mockint
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.BlockTermsReader;
-import org.apache.lucene.codecs.BlockTermsWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.FixedGapTermsIndexReader;
-import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.TermsIndexReaderBase;
-import org.apache.lucene.codecs.TermsIndexWriterBase;
+import org.apache.lucene.codecs.blockterms.BlockTermsReader;
+import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
+import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
+import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
 import org.apache.lucene.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.codecs.intblock.FixedIntBlockIndexOutput;
 import org.apache.lucene.codecs.sep.IntIndexInput;
@@ -49,7 +49,7 @@ import org.apache.lucene.util.IOUtils;
  * used here just writes each block as a series of vInt.
  */
 
-public class MockFixedIntBlockPostingsFormat extends PostingsFormat {
+public final class MockFixedIntBlockPostingsFormat extends PostingsFormat {
 
   private final int blockSize;
 
@@ -72,6 +72,9 @@ public class MockFixedIntBlockPostingsFo
     return new MockIntFactory(blockSize);
   }
 
+  /**
+   * Encodes blocks as vInts of a fixed block size.
+   */
   public static class MockIntFactory extends IntStreamFactory {
     private final int blockSize;
 
@@ -180,7 +183,7 @@ public class MockFixedIntBlockPostingsFo
       FieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
-                                                state.segmentInfo.name,
+                                                state.segmentInfo,
                                                 postingsReader,
                                                 state.context,
                                                 1024,

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -19,17 +19,17 @@ package org.apache.lucene.codecs.mockint
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.BlockTermsReader;
-import org.apache.lucene.codecs.BlockTermsWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.FixedGapTermsIndexReader;
-import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.TermsIndexReaderBase;
-import org.apache.lucene.codecs.TermsIndexWriterBase;
+import org.apache.lucene.codecs.blockterms.BlockTermsReader;
+import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
+import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
+import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
 import org.apache.lucene.codecs.intblock.VariableIntBlockIndexInput;
 import org.apache.lucene.codecs.intblock.VariableIntBlockIndexOutput;
 import org.apache.lucene.codecs.sep.IntIndexInput;
@@ -53,7 +53,7 @@ import org.apache.lucene.util.IOUtils;
  * int is <= 3, else 2*baseBlockSize.
  */
 
-public class MockVariableIntBlockPostingsFormat extends PostingsFormat {
+public final class MockVariableIntBlockPostingsFormat extends PostingsFormat {
   private final int baseBlockSize;
   
   public MockVariableIntBlockPostingsFormat() {
@@ -70,6 +70,10 @@ public class MockVariableIntBlockPosting
     return getName() + "(baseBlockSize="+ baseBlockSize + ")";
   }
 
+  /**
+   * If the first value is <= 3, writes baseBlockSize vInts at once,
+   * otherwise writes 2*baseBlockSize vInts.
+   */
   public static class MockIntFactory extends IntStreamFactory {
 
     private final int baseBlockSize;
@@ -203,7 +207,7 @@ public class MockVariableIntBlockPosting
       FieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
-                                                state.segmentInfo.name,
+                                                state.segmentInfo,
                                                 postingsReader,
                                                 state.context,
                                                 1024,

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -22,22 +22,22 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.lucene.codecs.BlockTermsReader;
-import org.apache.lucene.codecs.BlockTermsWriter;
 import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.BlockTreeTermsWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.FixedGapTermsIndexReader;
-import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
 import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsIndexReaderBase;
-import org.apache.lucene.codecs.TermsIndexWriterBase;
-import org.apache.lucene.codecs.VariableGapTermsIndexReader;
-import org.apache.lucene.codecs.VariableGapTermsIndexWriter;
+import org.apache.lucene.codecs.blockterms.BlockTermsReader;
+import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
+import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
+import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
+import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
+import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
 import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
 import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
 import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
@@ -66,7 +66,7 @@ import org.apache.lucene.util._TestUtil;
  * Randomly combines terms index impl w/ postings impls.
  */
 
-public class MockRandomPostingsFormat extends PostingsFormat {
+public final class MockRandomPostingsFormat extends PostingsFormat {
   private final Random seedRandom;
   private final String SEED_EXT = "sd";
   
@@ -328,7 +328,7 @@ public class MockRandomPostingsFormat ex
       try {
         fields = new BlockTreeTermsReader(state.dir,
                                           state.fieldInfos,
-                                          state.segmentInfo.name,
+                                          state.segmentInfo,
                                           postingsReader,
                                           state.context,
                                           state.segmentSuffix,
@@ -398,7 +398,7 @@ public class MockRandomPostingsFormat ex
         fields = new BlockTermsReader(indexReader,
                                       state.dir,
                                       state.fieldInfos,
-                                      state.segmentInfo.name,
+                                      state.segmentInfo,
                                       postingsReader,
                                       state.context,
                                       termsCacheSize,

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -19,17 +19,17 @@ package org.apache.lucene.codecs.mocksep
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.BlockTermsReader;
-import org.apache.lucene.codecs.BlockTermsWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.FixedGapTermsIndexReader;
-import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.TermsIndexReaderBase;
-import org.apache.lucene.codecs.TermsIndexWriterBase;
+import org.apache.lucene.codecs.blockterms.BlockTermsReader;
+import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
+import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
+import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
 import org.apache.lucene.codecs.sep.SepPostingsReader;
 import org.apache.lucene.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.SegmentWriteState;
@@ -42,7 +42,7 @@ import org.apache.lucene.util.BytesRef;
  * This is here just to test the core sep codec
  * classes.
  */
-public class MockSepPostingsFormat extends PostingsFormat {
+public final class MockSepPostingsFormat extends PostingsFormat {
 
   public MockSepPostingsFormat() {
     super("MockSep");
@@ -107,7 +107,7 @@ public class MockSepPostingsFormat exten
       FieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir,
                                                 state.fieldInfos,
-                                                state.segmentInfo.name,
+                                                state.segmentInfo,
                                                 postingsReader,
                                                 state.context,
                                                 1024,

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntFactory.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntFactory.java Fri Sep 21 17:21:34 2012
@@ -25,7 +25,10 @@ import org.apache.lucene.codecs.sep.IntS
 
 import java.io.IOException;
 
-/** @lucene.experimental */
+/** 
+ * Encodes ints directly as vInts with {@link MockSingleIntIndexOutput}
+ * @lucene.experimental 
+ */
 public class MockSingleIntFactory extends IntStreamFactory {
   @Override
   public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java Fri Sep 21 17:21:34 2012
@@ -28,7 +28,7 @@ import org.apache.lucene.store.IndexInpu
 
 /** Reads IndexInputs written with {@link
  *  MockSingleIntIndexOutput}.  NOTE: this class is just for
- *  demonstration puprposes (it is a very slow way to read a
+ *  demonstration purposes (it is a very slow way to read a
  *  block of ints).
  *
  * @lucene.experimental
@@ -46,7 +46,7 @@ public class MockSingleIntIndexInput ext
 
   @Override
   public Reader reader() throws IOException {
-    return new Reader((IndexInput) in.clone());
+    return new Reader(in.clone());
   }
 
   @Override
@@ -54,6 +54,9 @@ public class MockSingleIntIndexInput ext
     in.close();
   }
 
+  /**
+   * Just reads a vInt directly from the file.
+   */
   public static class Reader extends IntIndexInput.Reader {
     // clone:
     private final IndexInput in;
@@ -84,7 +87,7 @@ public class MockSingleIntIndexInput ext
     }
 
     @Override
-    public void set(IntIndexInput.Index other) {
+    public void copyFrom(IntIndexInput.Index other) {
       fp = ((MockSingleIntIndexInputIndex) other).fp;
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/nestedpulsing/NestedPulsingPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -41,7 +41,7 @@ import org.apache.lucene.util.IOUtils;
 // TODO: if we create PulsingPostingsBaseFormat then we
 // can simplify this? note: I don't like the *BaseFormat
 // hierarchy, maybe we can clean that up...
-public class NestedPulsingPostingsFormat extends PostingsFormat {
+public final class NestedPulsingPostingsFormat extends PostingsFormat {
   public NestedPulsingPostingsFormat() {
     super("NestedPulsing");
   }
@@ -81,7 +81,7 @@ public class NestedPulsingPostingsFormat
       pulsingReaderInner = new PulsingPostingsReader(docsReader);
       pulsingReader = new PulsingPostingsReader(pulsingReaderInner);
       FieldsProducer ret = new BlockTreeTermsReader(
-                                                    state.dir, state.fieldInfos, state.segmentInfo.name,
+                                                    state.dir, state.fieldInfos, state.segmentInfo,
                                                     pulsingReader,
                                                     state.context,
                                                     state.segmentSuffix,

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.ramonly
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -49,7 +50,6 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.UnmodifiableIterator;
 
 /** Stores all postings data in RAM, but writes a small
  *  token (header + single int) to identify which "slot" the
@@ -57,7 +57,7 @@ import org.apache.lucene.util.Unmodifiab
  *
  *  NOTE: this codec sorts terms by reverse-unicode-order! */
 
-public class RAMOnlyPostingsFormat extends PostingsFormat {
+public final class RAMOnlyPostingsFormat extends PostingsFormat {
 
   // For fun, test that we can override how terms are
   // sorted, and basic things still work -- this comparator
@@ -113,7 +113,7 @@ public class RAMOnlyPostingsFormat exten
 
     @Override
     public Iterator<String> iterator() {
-      return new UnmodifiableIterator<String>(fieldToTerms.keySet().iterator());
+      return Collections.unmodifiableSet(fieldToTerms.keySet()).iterator();
     }
 
     @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AlcoholicMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AlcoholicMergePolicy.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AlcoholicMergePolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/AlcoholicMergePolicy.java Fri Sep 21 17:21:34 2012
@@ -68,7 +68,7 @@ public class AlcoholicMergePolicy extend
     return info.sizeInBytes();
   }
   
-  public static enum Drink {
+  private static enum Drink {
     
     Beer(15), Wine(17), Champagne(21), WhiteRussian(22), SingleMalt(30);
     
@@ -77,11 +77,6 @@ public class AlcoholicMergePolicy extend
     Drink(long drunkFactor) {
       this.drunkFactor = drunkFactor;
     }
-    
-    public long drunk() {
-      return drunkFactor;
-    }
-    
   }
   
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java Fri Sep 21 17:21:34 2012
@@ -290,6 +290,10 @@ class DocHelper {
   public static int numFields(Document doc) {
     return doc.getFields().size();
   }
+
+  public static int numFields(StoredDocument doc) {
+    return doc.getFields().size();
+  }
   
   public static Document createDocument(int n, String indexName, int numFields) {
     StringBuilder sb = new StringBuilder();

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Fri Sep 21 17:21:34 2012
@@ -20,9 +20,14 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.NoSuchElementException;
 import java.util.Set;
 
+import org.apache.lucene.util.FilterIterator;
+
+/**
+ * A {@link FilterAtomicReader} that exposes only a subset
+ * of fields from the underlying wrapped reader.
+ */
 public final class FieldFilterAtomicReader extends FilterAtomicReader {
   
   private final Set<String> fields;
@@ -58,8 +63,9 @@ public final class FieldFilterAtomicRead
       return null;
     }
     f = new FieldFilterFields(f);
-    // we need to check for emptyness, so we can return null:
-    return (f.iterator().next() == null) ? null : f;
+    // we need to check for emptyness, so we can return
+    // null:
+    return f.iterator().hasNext() ? f : null;
   }
 
   @Override
@@ -134,55 +140,16 @@ public final class FieldFilterAtomicRead
 
     @Override
     public int size() {
-      // TODO: add faster implementation!
-      int c = 0;
-      final Iterator<String> it = iterator();
-      while (it.next() != null) {
-        c++;
-      }
-      return c;
+      // this information is not cheap, return -1 like MultiFields does:
+      return -1;
     }
 
     @Override
     public Iterator<String> iterator() {
-      final Iterator<String> in = super.iterator();
-      return new Iterator<String>() {
-        String cached = null;
-        
-        @Override
-        public String next() {
-          if (cached != null) {
-            String next = cached;
-            cached = null;
-            return next;
-          } else {
-            String next = doNext();
-            if (next == null) {
-              throw new NoSuchElementException();
-            } else {
-              return next;
-            }
-          }
-        }
-
-        @Override
-        public boolean hasNext() {
-          return cached != null || (cached = doNext()) != null;
-        }
-        
-        private String doNext() {
-          while (in.hasNext()) {
-            String field = in.next();
-            if (hasField(field)) {
-              return field;
-            }
-          }
-          return null;
-        }
-
+      return new FilterIterator<String>(super.iterator()) {
         @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
+        protected boolean predicateFunction(String field) {
+          return hasField(field);
         }
       };
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Fri Sep 21 17:21:34 2012
@@ -29,6 +29,7 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
+import org.apache.lucene.codecs.block.BlockPostingsFormat;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
@@ -79,7 +80,7 @@ public class RandomCodec extends Lucene4
       }
       previousMappings.put(name, codec);
       // Safety:
-      assert previousMappings.size() < 10000;
+      assert previousMappings.size() < 10000: "test went insane";
     }
     return codec;
   }
@@ -94,6 +95,7 @@ public class RandomCodec extends Lucene4
 
     add(avoidCodecs,
         new Lucene40PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
+        new BlockPostingsFormat(minItemsPerBlock, maxItemsPerBlock),
         new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
                                  LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : lowFreqCutoff)),
         new Pulsing40PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
@@ -117,7 +119,7 @@ public class RandomCodec extends Lucene4
     Collections.shuffle(formats, random);
 
     // Avoid too many open files:
-    formats.subList(4, formats.size()).clear();
+    formats = formats.subList(0, 4);
   }
 
   public RandomCodec(Random random) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java Fri Sep 21 17:21:34 2012
@@ -106,7 +106,7 @@ public class RandomIndexWriter implement
     flushAt = _TestUtil.nextInt(r, 10, 1000);
     codec = w.getConfig().getCodec();
     if (LuceneTestCase.VERBOSE) {
-      System.out.println("RIW config=" + w.getConfig());
+      System.out.println("RIW dir=" + dir + " config=" + w.getConfig());
       System.out.println("codec default=" + codec.getName());
     }
     /* TODO: find some way to make this random...
@@ -139,13 +139,13 @@ public class RandomIndexWriter implement
   
   /**
    * Adds a Document.
-   * @see IndexWriter#addDocument(Iterable)
+   * @see IndexWriter#addDocument(org.apache.lucene.index.IndexDocument)
    */
-  public <T extends IndexableField> void addDocument(final Iterable<T> doc) throws IOException {
+  public <T extends IndexableField> void addDocument(final IndexDocument doc) throws IOException {
     addDocument(doc, w.getAnalyzer());
   }
 
-  public <T extends IndexableField> void addDocument(final Iterable<T> doc, Analyzer a) throws IOException {
+  public <T extends IndexableField> void addDocument(final IndexDocument doc, Analyzer a) throws IOException {
     if (doDocValues && doc instanceof Document) {
       randomPerDocFieldValues((Document) doc);
     }
@@ -154,11 +154,11 @@ public class RandomIndexWriter implement
       // (but we need to clone them), and only when
       // getReader, commit, etc. are called, we do an
       // addDocuments?  Would be better testing.
-      w.addDocuments(new Iterable<Iterable<T>>() {
+      w.addDocuments(new Iterable<IndexDocument>() {
 
         @Override
-        public Iterator<Iterable<T>> iterator() {
-          return new Iterator<Iterable<T>>() {
+        public Iterator<IndexDocument> iterator() {
+          return new Iterator<IndexDocument>() {
             boolean done;
             
             @Override
@@ -172,7 +172,7 @@ public class RandomIndexWriter implement
             }
 
             @Override
-            public Iterable<T> next() {
+            public IndexDocument next() {
               if (done) {
                 throw new IllegalStateException();
               }
@@ -273,30 +273,30 @@ public class RandomIndexWriter implement
     }
   }
   
-  public void addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
+  public void addDocuments(Iterable<? extends IndexDocument> docs) throws IOException {
     w.addDocuments(docs);
     maybeCommit();
   }
 
-  public void updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
+  public void updateDocuments(Term delTerm, Iterable<? extends IndexDocument> docs) throws IOException {
     w.updateDocuments(delTerm, docs);
     maybeCommit();
   }
 
   /**
    * Updates a document.
-   * @see IndexWriter#updateDocument(Term, Iterable)
+   * @see IndexWriter#updateDocument(Term, org.apache.lucene.index.IndexDocument)
    */
-  public <T extends IndexableField> void updateDocument(Term t, final Iterable<T> doc) throws IOException {
+  public <T extends IndexableField> void updateDocument(Term t, final IndexDocument doc) throws IOException {
     if (doDocValues) {
       randomPerDocFieldValues((Document) doc);
     }
     if (r.nextInt(5) == 3) {
-      w.updateDocuments(t, new Iterable<Iterable<T>>() {
+      w.updateDocuments(t, new Iterable<IndexDocument>() {
 
         @Override
-        public Iterator<Iterable<T>> iterator() {
-          return new Iterator<Iterable<T>>() {
+        public Iterator<IndexDocument> iterator() {
+          return new Iterator<IndexDocument>() {
             boolean done;
             
             @Override
@@ -310,7 +310,7 @@ public class RandomIndexWriter implement
             }
 
             @Override
-            public Iterable<T> next() {
+            public IndexDocument next() {
               if (done) {
                 throw new IllegalStateException();
               }

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -89,19 +89,19 @@ public abstract class ThreadedIndexingAn
     return in;
   }
 
-  protected void updateDocuments(Term id, List<? extends Iterable<? extends IndexableField>> docs) throws Exception {
+  protected void updateDocuments(Term id, List<? extends IndexDocument> docs) throws Exception {
     writer.updateDocuments(id, docs);
   }
 
-  protected void addDocuments(Term id, List<? extends Iterable<? extends IndexableField>> docs) throws Exception {
+  protected void addDocuments(Term id, List<? extends IndexDocument> docs) throws Exception {
     writer.addDocuments(docs);
   }
 
-  protected void addDocument(Term id, Iterable<? extends IndexableField> doc) throws Exception {
+  protected void addDocument(Term id, IndexDocument doc) throws Exception {
     writer.addDocument(doc);
   }
 
-  protected void updateDocument(Term term, Iterable<? extends IndexableField> doc) throws Exception {
+  protected void updateDocument(Term term, IndexDocument doc) throws Exception {
     writer.updateDocument(term, doc);
   }
 
@@ -334,15 +334,15 @@ public abstract class ThreadedIndexingAn
                   // Verify 1) IW is correctly setting
                   // diagnostics, and 2) segment warming for
                   // merged segments is actually happening:
-                  for(AtomicReader sub : ((DirectoryReader) s.getIndexReader()).getSequentialSubReaders()) {
-                    SegmentReader segReader = (SegmentReader) sub;
+                  for(final AtomicReaderContext sub : s.getIndexReader().leaves()) {
+                    SegmentReader segReader = (SegmentReader) sub.reader();
                     Map<String,String> diagnostics = segReader.getSegmentInfo().info.getDiagnostics();
                     assertNotNull(diagnostics);
                     String source = diagnostics.get("source");
                     assertNotNull(source);
                     if (source.equals("merge")) {
                       assertTrue("sub reader " + sub + " wasn't warmed: warmed=" + warmed + " diagnostics=" + diagnostics + " si=" + segReader.getSegmentInfo(),
-                                 !assertMergedSegmentsWarmed || warmed.containsKey(((SegmentReader) sub).core));
+                                 !assertMergedSegmentsWarmed || warmed.containsKey(segReader.core));
                     }
                   }
                   if (s.getIndexReader().numDocs() > 0) {
@@ -464,7 +464,7 @@ public abstract class ThreadedIndexingAn
         final int inc = Math.max(1, maxDoc/50);
         for(int docID=0;docID<maxDoc;docID += inc) {
           if (liveDocs == null || liveDocs.get(docID)) {
-            final Document doc = reader.document(docID);
+            final StoredDocument doc = reader.document(docID);
             sum += doc.getFields().size();
           }
         }
@@ -564,7 +564,7 @@ public abstract class ThreadedIndexingAn
               startDocID = docID;
             }
             lastDocID = docID;
-            final Document doc = s.doc(docID);
+            final StoredDocument doc = s.doc(docID);
             assertEquals(subDocs.packID, doc.get("packID"));
           }
 

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.index.AtomicRea
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util._TestUtil;
 
 /** 
  * Helper class that adds some extra checks to ensure correct
@@ -101,16 +102,11 @@ public class AssertingIndexSearcher exte
       }
     };
   }
-
+  
   @Override
   protected Query wrapFilter(Query query, Filter filter) {
     if (random.nextBoolean())
       return super.wrapFilter(query, filter);
-    return (filter == null) ? query : new FilteredQuery(query, filter) {
-      @Override
-      protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
-        return random.nextBoolean();
-      }
-    };
+    return (filter == null) ? query : new FilteredQuery(query, filter, _TestUtil.randomFilterStrategy(random));
   }
 }

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -354,7 +354,7 @@ public class QueryUtils {
           IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
           indexSearcher.setSimilarity(s.getSimilarity());
           Weight w = indexSearcher.createNormalizedWeight(q);
-          AtomicReaderContext ctx = previousReader.getTopReaderContext();
+          AtomicReaderContext ctx = previousReader.getContext();
           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;

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java Fri Sep 21 17:21:34 2012
@@ -173,20 +173,8 @@ public abstract class SearchEquivalenceT
   protected void assertSubsetOf(Query q1, Query q2, Filter filter) throws Exception {
     // TRUNK ONLY: test both filter code paths
     if (filter != null && random().nextBoolean()) {
-      final boolean q1RandomAccess = random().nextBoolean();
-      final boolean q2RandomAccess = random().nextBoolean();
-      q1 = new FilteredQuery(q1, filter) {
-        @Override
-        protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
-          return q1RandomAccess;
-        }
-      };
-      q2 = new FilteredQuery(q2, filter) {
-        @Override
-        protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
-          return q2RandomAccess;
-        }
-      };
+      q1 = new FilteredQuery(q1, filter, _TestUtil.randomFilterStrategy(random()));
+      q2 = new FilteredQuery(q2, filter,  _TestUtil.randomFilterStrategy(random()));
       filter = null;
     }
     

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -42,9 +42,15 @@ import org.apache.lucene.util._TestUtil;
 //   - doc blocks?  so we can test joins/grouping...
 //   - controlled consistency (NRTMgr)
 
+/**
+ * Base test class for simulating distributed search across multiple shards.
+ */
 public abstract class ShardSearchingTestBase extends LuceneTestCase {
 
   // TODO: maybe SLM should throw this instead of returning null...
+  /**
+   * Thrown when the lease for a searcher has expired.
+   */
   public static class SearcherExpiredException extends RuntimeException {
     public SearcherExpiredException(String message) {
       super(message);
@@ -179,7 +185,7 @@ public abstract class ShardSearchingTest
     }
     try {
       for(Term term : terms) {
-        final TermContext termContext = TermContext.build(s.getIndexReader().getTopReaderContext(), term, false);
+        final TermContext termContext = TermContext.build(s.getIndexReader().getContext(), term, false);
         stats.put(term, s.termStatistics(term, termContext));
       }
     } finally {
@@ -462,15 +468,17 @@ public abstract class ShardSearchingTest
 
     // Get the current (fresh) searcher for this node
     public ShardIndexSearcher acquire() {
-      final ShardIndexSearcher s = currentShardSearcher;
-      // TODO: this isn't thread safe.... in theory the
-      // reader could get decRef'd to 0 before we have a
-      // chance to incRef, ie if a reopen happens right
-      // after the above line, this thread gets stalled, and
-      // the old IR is closed.  But because we use SLM in
-      // this test, this will be exceptionally rare:
-      s.getIndexReader().incRef();
-      return s;
+      while(true) {
+        final ShardIndexSearcher s = currentShardSearcher;
+        // In theory the reader could get decRef'd to 0
+        // before we have a chance to incRef, ie if a reopen
+        // happens right after the above line, this thread
+        // gets stalled, and the old IR is closed.  So we
+        // must try/retry until incRef succeeds:
+        if (s.getIndexReader().tryIncRef()) {
+          return s;
+        }
+      }
     }
 
     public void release(ShardIndexSearcher s) throws IOException {
@@ -604,6 +612,9 @@ public abstract class ShardSearchingTest
     }
   }
 
+  /**
+   * An IndexSearcher and associated version (lease)
+   */
   protected static class SearcherAndVersion {
     public final IndexSearcher searcher;
     public final long version;

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=1388574&r1=1388573&r2=1388574&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 Fri Sep 21 17:21:34 2012
@@ -67,6 +67,7 @@ public class MockDirectoryWrapper extend
   boolean noDeleteOpenFile = true;
   boolean preventDoubleWrite = true;
   boolean trackDiskUsage = false;
+  boolean wrapLockFactory = true;
   private Set<String> unSyncedFiles;
   private Set<String> createdFiles;
   private Set<String> openFilesForWrite = new HashSet<String>();
@@ -114,11 +115,7 @@ public class MockDirectoryWrapper extend
     this.throttledOutput = new ThrottledIndexOutput(ThrottledIndexOutput
         .mBitsToBytes(40 + randomState.nextInt(10)), 5 + randomState.nextInt(5), null);
     // force wrapping of lockfactory
-    try {
-      setLockFactory(new MockLockFactoryWrapper(this, delegate.getLockFactory()));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+    this.lockFactory = new MockLockFactoryWrapper(this, delegate.getLockFactory());
 
     // 2% of the time use rate limiter
     if (randomState.nextInt(50) == 17) {
@@ -149,6 +146,12 @@ public class MockDirectoryWrapper extend
     preventDoubleWrite = value;
   }
   
+  /**
+   * Enum for controlling hard disk throttling.
+   * Set via {@link MockDirectoryWrapper #setThrottling(Throttling)}
+   * <p>
+   * WARNING: can make tests very slow.
+   */
   public static enum Throttling {
     /** always emulate a slow hard disk. could be very slow! */
     ALWAYS,
@@ -440,7 +443,7 @@ public class MockDirectoryWrapper extend
     }
     
     //System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
-    IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, LuceneTestCase.newIOContext(randomState)), name);
+    IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, LuceneTestCase.newIOContext(randomState, context)), name);
     addFileHandle(io, name, Handle.Output);
     openFilesForWrite.add(name);
     
@@ -494,7 +497,7 @@ public class MockDirectoryWrapper extend
       throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
     }
 
-    IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name, LuceneTestCase.newIOContext(randomState)));
+    IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name, LuceneTestCase.newIOContext(randomState, context)));
     addFileHandle(ii, name, Handle.Input);
     return ii;
   }
@@ -530,6 +533,19 @@ public class MockDirectoryWrapper extend
   public void setAssertNoUnrefencedFilesOnClose(boolean v) {
     assertNoUnreferencedFilesOnClose = v;
   }
+  
+  /**
+   * Set to false if you want to return the pure lockfactory
+   * and not wrap it with MockLockFactoryWrapper.
+   * <p>
+   * Be careful if you turn this off: MockDirectoryWrapper might
+   * no longer be able to detect if you forget to close an IndexWriter,
+   * and spit out horribly scary confusing exceptions instead of
+   * simply telling you that.
+   */
+  public void setWrapLockFactory(boolean v) {
+    this.wrapLockFactory = v;
+  }
 
   @Override
   public synchronized void close() throws IOException {
@@ -699,25 +715,33 @@ public class MockDirectoryWrapper extend
   @Override
   public synchronized Lock makeLock(String name) {
     maybeYield();
-    return delegate.makeLock(name);
+    return getLockFactory().makeLock(name);
   }
 
   @Override
   public synchronized void clearLock(String name) throws IOException {
     maybeYield();
-    delegate.clearLock(name);
+    getLockFactory().clearLock(name);
   }
 
   @Override
   public synchronized void setLockFactory(LockFactory lockFactory) throws IOException {
     maybeYield();
+    // sneaky: we must pass the original this way to the dir, because
+    // some impls (e.g. FSDir) do instanceof here.
     delegate.setLockFactory(lockFactory);
+    // now set our wrapped factory here
+    this.lockFactory = new MockLockFactoryWrapper(this, lockFactory);
   }
 
   @Override
   public synchronized LockFactory getLockFactory() {
     maybeYield();
-    return delegate.getLockFactory();
+    if (wrapLockFactory) {
+      return lockFactory;
+    } else {
+      return delegate.getLockFactory();
+    }
   }
 
   @Override
@@ -766,14 +790,6 @@ public class MockDirectoryWrapper extend
         addFileHandle(ii, name, Handle.Input);
         return ii;
       }
-
-      @Override
-      public IndexInput openFullSlice() throws IOException {
-        maybeYield();
-        IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openFullSlice());
-        addFileHandle(ii, name, Handle.Input);
-        return ii;
-      }
       
     };
     addFileHandle(handle, name, Handle.Slice);

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java Fri Sep 21 17:21:34 2012
@@ -68,7 +68,7 @@ public class MockIndexInputWrapper exten
   public MockIndexInputWrapper clone() {
     ensureOpen();
     dir.inputCloneCount.incrementAndGet();
-    IndexInput iiclone = (IndexInput) delegate.clone();
+    IndexInput iiclone = delegate.clone();
     MockIndexInputWrapper clone = new MockIndexInputWrapper(dir, name, iiclone);
     clone.isClone = true;
     // Pending resolution on LUCENE-686 we may want to
@@ -119,12 +119,6 @@ public class MockIndexInputWrapper exten
   }
 
   @Override
-  public void copyBytes(IndexOutput out, long numBytes) throws IOException {
-    ensureOpen();
-    delegate.copyBytes(out, numBytes);
-  }
-
-  @Override
   public void readBytes(byte[] b, int offset, int len, boolean useBuffer)
       throws IOException {
     ensureOpen();

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableFile.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableFile.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/CloseableFile.java Fri Sep 21 17:21:34 2012
@@ -24,25 +24,30 @@ import java.io.*;
  */
 final class CloseableFile implements Closeable {
   private final File file;
+  private final TestRuleMarkFailure failureMarker;
 
-  public CloseableFile(File file) {
+  public CloseableFile(File file, TestRuleMarkFailure failureMarker) {
     this.file = file;
+    this.failureMarker = failureMarker;
   }
 
   @Override
   public void close() throws IOException {
-    if (file.exists()) {
-      try {
-        _TestUtil.rmDir(file);
-      } catch (IOException e) {
-        // Ignore the exception from rmDir.
-      }
-
-      // Re-check.
+    // only if there were no other test failures.
+    if (failureMarker.wasSuccessful()) {
       if (file.exists()) {
-        throw new IOException(
+        try {
+          _TestUtil.rmDir(file);
+        } catch (IOException e) {
+          // Ignore the exception from rmDir.
+        }
+
+        // Re-check.
+        if (file.exists()) {
+          throw new IOException(
             "Could not remove: " + file.getAbsolutePath());
-      }
+        }
+    }
     }
   }
 }
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/English.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/English.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/English.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/test-framework/src/java/org/apache/lucene/util/English.java Fri Sep 21 17:21:34 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.util;
  */
 
 /**
+ * Converts numbers to english strings for testing.
  * @lucene.internal
  */ 
 public final class English {