You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/01/25 21:32:53 UTC

svn commit: r1235919 [2/12] - in /lucene/dev/branches/lucene3661: ./ dev-tools/eclipse/ dev-tools/idea/lucene/contrib/ dev-tools/maven/ dev-tools/maven/solr/core/ dev-tools/maven/solr/solrj/ lucene/ lucene/contrib/ lucene/contrib/sandbox/src/test/org/a...

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/ReaderUtil.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/ReaderUtil.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/ReaderUtil.java Wed Jan 25 20:32:44 2012
@@ -120,42 +120,6 @@ public final class ReaderUtil {
 
     protected abstract void add(int base, IndexReader r) throws IOException;
   }
-
-  /**
-   * Returns sub IndexReader that contains the given document id.
-   *    
-   * @param doc id of document
-   * @param reader parent reader
-   * @return sub reader of parent which contains the specified doc id
-   */
-  public static IndexReader subReader(int doc, IndexReader reader) {
-    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
-    ReaderUtil.gatherSubReaders(subReadersList, reader);
-    IndexReader[] subReaders = subReadersList
-        .toArray(new IndexReader[subReadersList.size()]);
-    int[] docStarts = new int[subReaders.length];
-    int maxDoc = 0;
-    for (int i = 0; i < subReaders.length; i++) {
-      docStarts[i] = maxDoc;
-      maxDoc += subReaders[i].maxDoc();
-    }
-    return subReaders[subIndex(doc, docStarts)];
-  }
-  
-  /**
-   * Returns sub-reader subIndex from reader.
-   * 
-   * @param reader parent reader
-   * @param subIndex index of desired sub reader
-   * @return the subreader at subIndex
-   */
-  public static IndexReader subReader(IndexReader reader, int subIndex) {
-    List<IndexReader> subReadersList = new ArrayList<IndexReader>();
-    ReaderUtil.gatherSubReaders(subReadersList, reader);
-    IndexReader[] subReaders = subReadersList
-        .toArray(new IndexReader[subReadersList.size()]);
-    return subReaders[subIndex];
-  }
   
   public static ReaderContext buildReaderContext(IndexReader reader) {
     return new ReaderContextBuilder(reader).build();

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/SentinelIntSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/SentinelIntSet.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/SentinelIntSet.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/SentinelIntSet.java Wed Jan 25 20:32:44 2012
@@ -96,13 +96,13 @@ public class SentinelIntSet {
   public int put(int key) {
     int s = find(key);
     if (s < 0) {
+      count++;
       if (count >= rehashCount) {
         rehash();
         s = getSlot(key);
       } else {
         s = -s-1;
       }
-      count++;
       keys[s] = key;
     }
     return s;

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java Wed Jan 25 20:32:44 2012
@@ -32,12 +32,13 @@ import org.apache.lucene.util.BytesRef;
 public final class ByteSequenceOutputs extends Outputs<BytesRef> {
 
   private final static BytesRef NO_OUTPUT = new BytesRef();
+  private final static ByteSequenceOutputs singleton = new ByteSequenceOutputs();
 
   private ByteSequenceOutputs() {
   }
 
   public static ByteSequenceOutputs getSingleton() {
-    return new ByteSequenceOutputs();
+    return singleton;
   }
 
   @Override

Modified: lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/fst/IntSequenceOutputs.java Wed Jan 25 20:32:44 2012
@@ -32,12 +32,13 @@ import org.apache.lucene.util.IntsRef;
 public final class IntSequenceOutputs extends Outputs<IntsRef> {
 
   private final static IntsRef NO_OUTPUT = new IntsRef();
+  private final static IntSequenceOutputs singleton = new IntSequenceOutputs();
 
   private IntSequenceOutputs() {
   }
 
   public static IntSequenceOutputs getSingleton() {
-    return new IntSequenceOutputs();
+    return singleton;
   }
 
   @Override

Modified: lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Wed Jan 25 20:32:44 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.analysis;
  * limitations under the License.
  */
 
+import java.io.Reader;
 import java.io.StringReader;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -289,8 +290,12 @@ public abstract class BaseTokenStreamTes
       }
     }
   };
-
+  
   public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength) throws IOException {
+    checkRandomData(random, a, iterations, maxWordLength, random.nextBoolean());
+  }
+
+  public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean useCharFilter) throws IOException {
     for (int i = 0; i < iterations; i++) {
       String text;
       switch(_TestUtil.nextInt(random, 0, 4)) {
@@ -311,7 +316,9 @@ public abstract class BaseTokenStreamTes
         System.out.println("NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
       }
 
-      TokenStream ts = a.tokenStream("dummy", new StringReader(text));
+      int remainder = random.nextInt(10);
+      Reader reader = new StringReader(text);
+      TokenStream ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
       assertTrue("has no CharTermAttribute", ts.hasAttribute(CharTermAttribute.class));
       CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
       OffsetAttribute offsetAtt = ts.hasAttribute(OffsetAttribute.class) ? ts.getAttribute(OffsetAttribute.class) : null;
@@ -339,30 +346,38 @@ public abstract class BaseTokenStreamTes
         if (VERBOSE) {
           System.out.println("NOTE: BaseTokenStreamTestCase: re-run analysis");
         }
+        reader = new StringReader(text);
+        ts = a.tokenStream("dummy", useCharFilter ? new MockCharFilter(reader, remainder) : reader);
         if (typeAtt != null && posIncAtt != null && offsetAtt != null) {
           // offset + pos + type
-          assertAnalyzesToReuse(a, text, 
+          assertTokenStreamContents(ts, 
             tokens.toArray(new String[tokens.size()]),
             toIntArray(startOffsets),
             toIntArray(endOffsets),
             types.toArray(new String[types.size()]),
-            toIntArray(positions));
+            toIntArray(positions),
+            text.length());
         } else if (posIncAtt != null && offsetAtt != null) {
           // offset + pos
-          assertAnalyzesToReuse(a, text, 
+          assertTokenStreamContents(ts, 
               tokens.toArray(new String[tokens.size()]),
               toIntArray(startOffsets),
               toIntArray(endOffsets),
-              toIntArray(positions));
+              null,
+              toIntArray(positions),
+              text.length());
         } else if (offsetAtt != null) {
           // offset
-          assertAnalyzesToReuse(a, text, 
+          assertTokenStreamContents(ts, 
               tokens.toArray(new String[tokens.size()]),
               toIntArray(startOffsets),
-              toIntArray(endOffsets));
+              toIntArray(endOffsets),
+              null,
+              null,
+              text.length());
         } else {
           // terms only
-          assertAnalyzesToReuse(a, text, 
+          assertTokenStreamContents(ts, 
               tokens.toArray(new String[tokens.size()]));
         }
       }

Modified: lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWFieldsWriter.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWFieldsWriter.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWFieldsWriter.java Wed Jan 25 20:32:44 2012
@@ -137,7 +137,7 @@ class PreFlexRWFieldsWriter extends Fiel
         }
 
         if ((++df % termsOut.skipInterval) == 0) {
-          skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
+          skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength, false, 0);
           skipListWriter.bufferSkip(df);
         }
 

Modified: lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/search/ShardSearchingTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/search/ShardSearchingTestBase.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/search/ShardSearchingTestBase.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/search/ShardSearchingTestBase.java Wed Jan 25 20:32:44 2012
@@ -268,8 +268,19 @@ public abstract class ShardSearchingTest
             assert subStats != null;
           }
         
-          docFreq += subStats.docFreq();
-          totalTermFreq += subStats.totalTermFreq();
+          int nodeDocFreq = subStats.docFreq();
+          if (docFreq >= 0 && nodeDocFreq >= 0) {
+            docFreq += nodeDocFreq;
+          } else {
+            docFreq = -1;
+          }
+          
+          long nodeTotalTermFreq = subStats.totalTermFreq();
+          if (totalTermFreq >= 0 && nodeTotalTermFreq >= 0) {
+            totalTermFreq += nodeTotalTermFreq;
+          } else {
+            totalTermFreq = -1;
+          }
         }
 
         return new TermStatistics(term.bytes(), docFreq, totalTermFreq);
@@ -299,9 +310,29 @@ public abstract class ShardSearchingTest
           // Collection stats are pre-shared on reopen, so,
           // we better not have a cache miss:
           assert nodeStats != null: "myNodeID=" + myNodeID + " nodeID=" + nodeID + " version=" + nodeVersions[nodeID] + " field=" + field;
-          docCount += nodeStats.docCount();
-          sumTotalTermFreq += nodeStats.sumTotalTermFreq();
-          sumDocFreq += nodeStats.sumDocFreq();
+          
+          int nodeDocCount = nodeStats.docCount();
+          if (docCount >= 0 && nodeDocCount >= 0) {
+            docCount += nodeDocCount;
+          } else {
+            docCount = -1;
+          }
+          
+          long nodeSumTotalTermFreq = nodeStats.sumTotalTermFreq();
+          if (sumTotalTermFreq >= 0 && nodeSumTotalTermFreq >= 0) {
+            sumTotalTermFreq += nodeSumTotalTermFreq;
+          } else {
+            sumTotalTermFreq = -1;
+          }
+          
+          long nodeSumDocFreq = nodeStats.sumDocFreq();
+          if (sumDocFreq >= 0 && nodeSumDocFreq >= 0) {
+            sumDocFreq += nodeSumDocFreq;
+          } else {
+            sumDocFreq = -1;
+          }
+          
+          assert nodeStats.maxDoc() >= 0;
           maxDoc += nodeStats.maxDoc();
         }
 

Modified: lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/LuceneTestCase.java Wed Jan 25 20:32:44 2012
@@ -283,7 +283,8 @@ public abstract class LuceneTestCase ext
     int randomVal = random.nextInt(10);
     
     if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal < 2)) { // preflex-only setup
-      codec = new PreFlexRWCodec();
+      codec = Codec.forName("Lucene3x");
+      assert (codec instanceof PreFlexRWCodec) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
       PREFLEX_IMPERSONATION_IS_ACTIVE = true;
     } else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9)) {
       codec = new SimpleTextCodec();

Modified: lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/_TestUtil.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test-framework/java/org/apache/lucene/util/_TestUtil.java Wed Jan 25 20:32:44 2012
@@ -249,7 +249,42 @@ public class _TestUtil {
     }
   }
   
-  // TODO: make this more evil
+  private static final String[] HTML_CHAR_ENTITIES = {
+      "AElig", "Aacute", "Acirc", "Agrave", "Alpha", "AMP", "Aring", "Atilde",
+      "Auml", "Beta", "COPY", "Ccedil", "Chi", "Dagger", "Delta", "ETH",
+      "Eacute", "Ecirc", "Egrave", "Epsilon", "Eta", "Euml", "Gamma", "GT",
+      "Iacute", "Icirc", "Igrave", "Iota", "Iuml", "Kappa", "Lambda", "LT",
+      "Mu", "Ntilde", "Nu", "OElig", "Oacute", "Ocirc", "Ograve", "Omega",
+      "Omicron", "Oslash", "Otilde", "Ouml", "Phi", "Pi", "Prime", "Psi",
+      "QUOT", "REG", "Rho", "Scaron", "Sigma", "THORN", "Tau", "Theta",
+      "Uacute", "Ucirc", "Ugrave", "Upsilon", "Uuml", "Xi", "Yacute", "Yuml",
+      "Zeta", "aacute", "acirc", "acute", "aelig", "agrave", "alefsym",
+      "alpha", "amp", "and", "ang", "apos", "aring", "asymp", "atilde",
+      "auml", "bdquo", "beta", "brvbar", "bull", "cap", "ccedil", "cedil",
+      "cent", "chi", "circ", "clubs", "cong", "copy", "crarr", "cup",
+      "curren", "dArr", "dagger", "darr", "deg", "delta", "diams", "divide",
+      "eacute", "ecirc", "egrave", "empty", "emsp", "ensp", "epsilon",
+      "equiv", "eta", "eth", "euml", "euro", "exist", "fnof", "forall",
+      "frac12", "frac14", "frac34", "frasl", "gamma", "ge", "gt", "hArr",
+      "harr", "hearts", "hellip", "iacute", "icirc", "iexcl", "igrave",
+      "image", "infin", "int", "iota", "iquest", "isin", "iuml", "kappa",
+      "lArr", "lambda", "lang", "laquo", "larr", "lceil", "ldquo", "le",
+      "lfloor", "lowast", "loz", "lrm", "lsaquo", "lsquo", "lt", "macr",
+      "mdash", "micro", "middot", "minus", "mu", "nabla", "nbsp", "ndash",
+      "ne", "ni", "not", "notin", "nsub", "ntilde", "nu", "oacute", "ocirc",
+      "oelig", "ograve", "oline", "omega", "omicron", "oplus", "or", "ordf",
+      "ordm", "oslash", "otilde", "otimes", "ouml", "para", "part", "permil",
+      "perp", "phi", "pi", "piv", "plusmn", "pound", "prime", "prod", "prop",
+      "psi", "quot", "rArr", "radic", "rang", "raquo", "rarr", "rceil",
+      "rdquo", "real", "reg", "rfloor", "rho", "rlm", "rsaquo", "rsquo",
+      "sbquo", "scaron", "sdot", "sect", "shy", "sigma", "sigmaf", "sim",
+      "spades", "sub", "sube", "sum", "sup", "sup1", "sup2", "sup3", "supe",
+      "szlig", "tau", "there4", "theta", "thetasym", "thinsp", "thorn",
+      "tilde", "times", "trade", "uArr", "uacute", "uarr", "ucirc", "ugrave",
+      "uml", "upsih", "upsilon", "uuml", "weierp", "xi", "yacute", "yen",
+      "yuml", "zeta", "zwj", "zwnj"
+  };
+  
   public static String randomHtmlishString(Random random, int numElements) {
     final int end = random.nextInt(numElements);
     if (end == 0) {
@@ -258,17 +293,80 @@ public class _TestUtil {
     }
     StringBuilder sb = new StringBuilder();
     for (int i = 0; i < end; i++) {
-      int val = random.nextInt(10);
+      int val = random.nextInt(25);
       switch(val) {
         case 0: sb.append("<p>"); break;
-        case 1: sb.append("</p>"); break;
-        case 2: sb.append("<!--"); break;
-        case 3: sb.append("-->"); break;
-        case 4: sb.append("&#"); break;
-        case 5: sb.append(";"); break;
-        case 6: sb.append((char)_TestUtil.nextInt(random, '0', '9')); break;
-        default:
-          sb.append((char)_TestUtil.nextInt(random, 'a', 'z'));
+        case 1: {
+          sb.append("<");
+          sb.append("    ".substring(nextInt(random, 0, 4)));
+          sb.append(randomSimpleString(random));
+          for (int j = 0 ; j < nextInt(random, 0, 10) ; ++j) {
+            sb.append(' ');
+            sb.append(randomSimpleString(random));
+            sb.append(" ".substring(nextInt(random, 0, 1)));
+            sb.append('=');
+            sb.append(" ".substring(nextInt(random, 0, 1)));
+            sb.append("\"".substring(nextInt(random, 0, 1)));
+            sb.append(randomSimpleString(random));
+            sb.append("\"".substring(nextInt(random, 0, 1)));
+          }
+          sb.append("    ".substring(nextInt(random, 0, 4)));
+          sb.append("/".substring(nextInt(random, 0, 1)));
+          sb.append(">".substring(nextInt(random, 0, 1)));
+          break;
+        }
+        case 2: {
+          sb.append("</");
+          sb.append("    ".substring(nextInt(random, 0, 4)));
+          sb.append(randomSimpleString(random));
+          sb.append("    ".substring(nextInt(random, 0, 4)));
+          sb.append(">".substring(nextInt(random, 0, 1)));
+          break;
+        }
+        case 3: sb.append(">"); break;
+        case 4: sb.append("</p>"); break;
+        case 5: sb.append("<!--"); break;
+        case 6: sb.append("<!--#"); break;
+        case 7: sb.append("<script><!-- f('"); break;
+        case 8: sb.append("</script>"); break;
+        case 9: sb.append("<?"); break;
+        case 10: sb.append("?>"); break;
+        case 11: sb.append("\""); break;
+        case 12: sb.append("\\\""); break;
+        case 13: sb.append("'"); break;
+        case 14: sb.append("\\'"); break;
+        case 15: sb.append("-->"); break;
+        case 16: {
+          sb.append("&");
+          switch(nextInt(random, 0, 2)) {
+            case 0: sb.append(randomSimpleString(random)); break;
+            case 1: sb.append(HTML_CHAR_ENTITIES[random.nextInt(HTML_CHAR_ENTITIES.length)]); break;
+          }
+          sb.append(";".substring(nextInt(random, 0, 1)));
+          break;
+        }
+        case 17: {
+          sb.append("&#");
+          if (0 == nextInt(random, 0, 1)) {
+            sb.append(nextInt(random, 0, Integer.MAX_VALUE - 1));
+            sb.append(";".substring(nextInt(random, 0, 1)));
+          }
+          break;
+        } 
+        case 18: {
+          sb.append("&#x");
+          if (0 == nextInt(random, 0, 1)) {
+            sb.append(Integer.toString(nextInt(random, 0, Integer.MAX_VALUE - 1), 16));
+            sb.append(";".substring(nextInt(random, 0, 1)));
+          }
+          break;
+        }
+          
+        case 19: sb.append(";"); break;
+        case 20: sb.append(nextInt(random, 0, Integer.MAX_VALUE - 1)); break;
+        case 21: sb.append("\n");
+        case 22: sb.append("          ".substring(nextInt(random, 0, 10)));
+        default: sb.append(randomSimpleString(random));
       }
     }
     return sb.toString();

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestPostingsOffsets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestPostingsOffsets.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestPostingsOffsets.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestPostingsOffsets.java Wed Jan 25 20:32:44 2012
@@ -22,29 +22,46 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CannedAnalyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockPayloadAnalyzer;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.NumericField;
+import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.English;
 import org.apache.lucene.util.LuceneTestCase;
-import org.junit.Assume;
+import org.apache.lucene.util._TestUtil;
 
 public class TestPostingsOffsets extends LuceneTestCase {
+  IndexWriterConfig iwc;
+  
+  public void setUp() throws Exception {
+    super.setUp();
+    // Currently only SimpleText and Lucene40 can index offsets into postings:
+    assumeTrue("codec does not support offsets", Codec.getDefault().getName().equals("SimpleText") || Codec.getDefault().getName().equals("Lucene40"));
+    iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
+    
+    if (Codec.getDefault().getName().equals("Lucene40")) {
+      // pulsing etc are not implemented
+      iwc.setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat()));
+    }
+  }
 
   public void testBasic() throws Exception {
-
-    // Currently only SimpleText can index offsets into postings:
-    Assume.assumeTrue(Codec.getDefault().getName().equals("SimpleText"));
-
     Directory dir = newDirectory();
-    RandomIndexWriter w = new RandomIndexWriter(random, dir);
+    
+    RandomIndexWriter w = new RandomIndexWriter(random, dir, iwc);
     Document doc = new Document();
 
     FieldType ft = new FieldType(TextField.TYPE_UNSTORED);
@@ -94,16 +111,117 @@ public class TestPostingsOffsets extends
     r.close();
     dir.close();
   }
+  
+  public void testSkipping() throws Exception {
+    doTestNumbers(false);
+  }
+  
+  public void testPayloads() throws Exception {
+    doTestNumbers(true);
+  }
+  
+  public void doTestNumbers(boolean withPayloads) throws Exception {
+    Directory dir = newDirectory();
+    Analyzer analyzer = withPayloads ? new MockPayloadAnalyzer() : new MockAnalyzer(random);
+    iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    if (Codec.getDefault().getName().equals("Lucene40")) {
+      // pulsing etc are not implemented
+      iwc.setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat()));
+    }
+    iwc.setMergePolicy(newLogMergePolicy()); // will rely on docids a bit for skipping
+    RandomIndexWriter w = new RandomIndexWriter(random, dir, iwc);
+    
+    FieldType ft = new FieldType(TextField.TYPE_STORED);
+    ft.setIndexOptions(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    if (random.nextBoolean()) {
+      ft.setStoreTermVectors(true);
+      ft.setStoreTermVectorOffsets(random.nextBoolean());
+      ft.setStoreTermVectorPositions(random.nextBoolean());
+    }
+    
+    int numDocs = atLeast(500);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      doc.add(new Field("numbers", English.intToEnglish(i), ft));
+      doc.add(new Field("oddeven", (i % 2) == 0 ? "even" : "odd", ft));
+      doc.add(new StringField("id", "" + i));
+      w.addDocument(doc);
+    }
+    
+    IndexReader reader = w.getReader();
+    w.close();
+    
+    String terms[] = { "one", "two", "three", "four", "five", "six", "seven", "eight", "nine", "ten", "hundred" };
+    
+    for (String term : terms) {
+      DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef(term), true);
+      int doc;
+      while((doc = dp.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+        String storedNumbers = reader.document(doc).get("numbers");
+        int freq = dp.freq();
+        for (int i = 0; i < freq; i++) {
+          dp.nextPosition();
+          int start = dp.startOffset();
+          assert start >= 0;
+          int end = dp.endOffset();
+          assert end >= 0 && end >= start;
+          // check that the offsets correspond to the term in the src text
+          assertTrue(storedNumbers.substring(start, end).equals(term));
+          if (withPayloads) {
+            // check that we have a payload and it starts with "pos"
+            assertTrue(dp.hasPayload());
+            BytesRef payload = dp.getPayload();
+            assertTrue(payload.utf8ToString().startsWith("pos:"));
+          } // note: withPayloads=false doesnt necessarily mean we dont have them from MockAnalyzer!
+        }
+      }
+    }
+    
+    // check we can skip correctly
+    int numSkippingTests = atLeast(50);
+    
+    for (int j = 0; j < numSkippingTests; j++) {
+      int num = _TestUtil.nextInt(random, 100, Math.min(numDocs-1, 999));
+      DocsAndPositionsEnum dp = MultiFields.getTermPositionsEnum(reader, null, "numbers", new BytesRef("hundred"), true);
+      int doc = dp.advance(num);
+      assertEquals(num, doc);
+      int freq = dp.freq();
+      for (int i = 0; i < freq; i++) {
+        String storedNumbers = reader.document(doc).get("numbers");
+        dp.nextPosition();
+        int start = dp.startOffset();
+        assert start >= 0;
+        int end = dp.endOffset();
+        assert end >= 0 && end >= start;
+        // check that the offsets correspond to the term in the src text
+        assertTrue(storedNumbers.substring(start, end).equals("hundred"));
+        if (withPayloads) {
+          // check that we have a payload and it starts with "pos"
+          assertTrue(dp.hasPayload());
+          BytesRef payload = dp.getPayload();
+          assertTrue(payload.utf8ToString().startsWith("pos:"));
+        } // note: withPayloads=false doesnt necessarily mean we dont have them from MockAnalyzer!
+      }
+    }
+    
+    // check that other fields (without offsets) work correctly
+    
+    for (int i = 0; i < numDocs; i++) {
+      DocsEnum dp = MultiFields.getTermDocsEnum(reader, null, "id", new BytesRef("" + i), false);
+      assertEquals(i, dp.nextDoc());
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, dp.nextDoc());
+    }
+    
+    reader.close();
+    dir.close();
+  }
 
   public void testRandom() throws Exception {
-    // Currently only SimpleText can index offsets into postings:
-    Assume.assumeTrue(Codec.getDefault().getName().equals("SimpleText"));
-
     // token -> docID -> tokens
     final Map<String,Map<Integer,List<Token>>> actualTokens = new HashMap<String,Map<Integer,List<Token>>>();
 
     Directory dir = newDirectory();
-    RandomIndexWriter w = new RandomIndexWriter(random, dir);
+    RandomIndexWriter w = new RandomIndexWriter(random, dir, iwc);
 
     final int numDocs = atLeast(20);
     //final int numDocs = atLeast(5);

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java Wed Jan 25 20:32:44 2012
@@ -31,6 +31,7 @@ import org.apache.lucene.search.CheckHit
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryUtils;
+import org.apache.lucene.search.similarities.TFIDFSimilarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.AfterClass;
@@ -240,6 +241,8 @@ public class TestFieldMaskingSpanQuery e
   }
   
   public void testSimple2() throws Exception {
+    assumeTrue("Broken scoring: LUCENE-3723", 
+        searcher.getSimilarityProvider().get("id") instanceof TFIDFSimilarity);
     SpanQuery q1 = new SpanTermQuery(new Term("gender", "female"));
     SpanQuery q2 = new SpanTermQuery(new Term("last", "smith"));
     SpanQuery q = new SpanNearQuery(new SpanQuery[]
@@ -310,6 +313,8 @@ public class TestFieldMaskingSpanQuery e
   }
   
   public void testSpans2() throws Exception {
+    assumeTrue("Broken scoring: LUCENE-3723", 
+        searcher.getSimilarityProvider().get("id") instanceof TFIDFSimilarity);
     SpanQuery qA1 = new SpanTermQuery(new Term("gender", "female"));
     SpanQuery qA2 = new SpanTermQuery(new Term("first",  "james"));
     SpanQuery qA  = new SpanOrQuery(qA1, new FieldMaskingSpanQuery(qA2, "gender"));

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/TestSentinelIntSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/TestSentinelIntSet.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/TestSentinelIntSet.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/TestSentinelIntSet.java Wed Jan 25 20:32:44 2012
@@ -20,6 +20,8 @@ package org.apache.lucene.util;
 
 import org.junit.Test;
 
+import java.util.HashSet;
+
 /**
  *
  *
@@ -45,4 +47,32 @@ public class TestSentinelIntSet extends 
     assertEquals(20, set.size());
     assertEquals(24, set.rehashCount);
   }
+  
+
+  @Test
+  public void testRandom() throws Exception {
+    for (int i=0; i<10000; i++) {
+      int initSz = random.nextInt(20);
+      int num = random.nextInt(30);
+      int maxVal = (random.nextBoolean() ? random.nextInt(50) : random.nextInt(Integer.MAX_VALUE)) + 1;
+
+      HashSet<Integer> a = new HashSet<Integer>(initSz);
+      SentinelIntSet b = new SentinelIntSet(initSz, -1);
+      
+      for (int j=0; j<num; j++) {
+        int val = random.nextInt(maxVal);
+        boolean exists = !a.add(val);
+        boolean existsB = b.exists(val);
+        assertEquals(exists, existsB);
+        int slot = b.find(val);
+        assertEquals(exists, slot>=0);
+        b.put(val);
+        
+        assertEquals(a.size(), b.size());
+      }
+      
+    }
+
+  }
+  
 }

Modified: lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/util/fst/TestFSTs.java Wed Jan 25 20:32:44 2012
@@ -1055,6 +1055,50 @@ public class TestFSTs extends LuceneTest
     }
   }
 
+  // NOTE: this test shows a case where our current builder
+  // fails to produce minimal FST:
+  /*
+  public void test3() throws Exception {
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
+    IntsRef scratchIntsRef = new IntsRef();
+    builder.add(Util.toIntsRef(new BytesRef("aa$"), scratchIntsRef), outputs.get(0));
+    builder.add(Util.toIntsRef(new BytesRef("aab$"), scratchIntsRef), 1L);
+    builder.add(Util.toIntsRef(new BytesRef("bbb$"), scratchIntsRef), 2L);
+    final FST<Long> fst = builder.finish();
+    //System.out.println("NODES " + fst.getNodeCount() + " ARCS " + fst.getArcCount());
+    // NOTE: we produce 7 nodes today
+    assertEquals(6, fst.getNodeCount());
+    // NOTE: we produce 8 arcs today
+    assertEquals(7, fst.getNodeCount());
+    //Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
+    //Util.toDot(fst, w, false, false);
+    //w.close();
+  }
+  */
+
+  // NOTE: this test shows a case where our current builder
+  // fails to produce minimal FST:
+  /*
+  public void test4() throws Exception {
+    final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+    Builder<BytesRef> builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, outputs);
+    IntsRef scratchIntsRef = new IntsRef();
+    builder.add(Util.toIntsRef(new BytesRef("aa$"), scratchIntsRef), outputs.getNoOutput());
+    builder.add(Util.toIntsRef(new BytesRef("aab$"), scratchIntsRef), new BytesRef("1"));
+    builder.add(Util.toIntsRef(new BytesRef("bbb$"), scratchIntsRef), new BytesRef("11"));
+    final FST<BytesRef> fst = builder.finish();
+    //System.out.println("NODES " + fst.getNodeCount() + " ARCS " + fst.getArcCount());
+    // NOTE: we produce 7 nodes today
+    assertEquals(6, fst.getNodeCount());
+    // NOTE: we produce 8 arcs today
+    assertEquals(7, fst.getNodeCount());
+    //Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
+    //Util.toDot(fst, w, false, false);
+    //w.close();
+  }
+  */
+
   // Build FST for all unique terms in the test line docs
   // file, up until a time limit
   public void testRealTerms() throws Exception {

Modified: lucene/dev/branches/lucene3661/modules/analysis/common/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/modules/analysis/common/build.xml?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/modules/analysis/common/build.xml (original)
+++ lucene/dev/branches/lucene3661/modules/analysis/common/build.xml Wed Jan 25 20:32:44 2012
@@ -31,14 +31,38 @@
   <target name="compile-core" depends="jflex-notice, common.compile-core"/>
 
   <target name="jflex" depends="jflex-check,clean-jflex,gen-uax29-supp-macros,
-                                jflex-StandardAnalyzer,jflex-UAX29URLEmailTokenizer,jflex-wiki-tokenizer"/>
+                                jflex-StandardAnalyzer,jflex-UAX29URLEmailTokenizer,
+                                jflex-wiki-tokenizer,jflex-HTMLStripCharFilter"/>
 
   <target name="gen-uax29-supp-macros">
     <subant target="gen-uax29-supp-macros">
        <fileset dir="../icu" includes="build.xml"/>
     </subant>
   </target>
-  
+
+  <target name="jflex-HTMLStripCharFilter"
+          depends="init,jflex-check,generate-jflex-html-char-entities"
+          if="jflex.present">
+    <taskdef classname="jflex.anttask.JFlexTask" name="jflex">
+      <classpath refid="jflex.classpath"/>
+    </taskdef>
+    <jflex file="src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex"
+           outdir="src/java/org/apache/lucene/analysis/charfilter"
+           nobak="on"/>
+    <!-- Remove the inappropriate JFlex-generated constructors -->
+    <replaceregexp file="src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.java"
+                   match="/\*\*\s*\*\s*Creates a new scanner.*this\(new java\.io\.InputStreamReader\(in\)\);\s*\}"
+                   replace="" flags="sg"/>
+  </target>
+
+  <target name="generate-jflex-html-char-entities">
+    <exec dir="src/java/org/apache/lucene/analysis/charfilter"
+          output="src/java/org/apache/lucene/analysis/charfilter/HTMLCharacterEntities.jflex"
+          executable="${python.exe}" failonerror="true" logerror="true">
+      <arg value="htmlentity.py"/>
+    </exec>
+  </target>
+
   <target name="jflex-wiki-tokenizer" depends="init,jflex-check" if="jflex.present">
     <taskdef classname="jflex.anttask.JFlexTask" name="jflex">
       <classpath refid="jflex.classpath"/>

Modified: lucene/dev/branches/lucene3661/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/BaseCharFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/BaseCharFilter.java?rev=1235919&r1=1235918&r2=1235919&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/BaseCharFilter.java (original)
+++ lucene/dev/branches/lucene3661/modules/analysis/common/src/java/org/apache/lucene/analysis/charfilter/BaseCharFilter.java Wed Jan 25 20:32:44 2012
@@ -20,6 +20,8 @@ package org.apache.lucene.analysis.charf
 import org.apache.lucene.analysis.CharStream;
 import org.apache.lucene.util.ArrayUtil;
 
+import java.util.Arrays;
+
 /**
  * Base utility class for implementing a {@link CharFilter}.
  * You subclass this, and then record mappings by calling
@@ -71,6 +73,19 @@ public abstract class BaseCharFilter ext
       0 : diffs[size-1];
   }
 
+  /**
+   * <p>
+   *   Adds an offset correction mapping at the given output stream offset.
+   * </p>
+   * <p>
+   *   Assumption: the offset given with each successive call to this method
+   *   will not be smaller than the offset given at the previous invocation.
+   * </p>
+   *
+   * @param off The output stream offset at which to apply the correction
+   * @param cumulativeDiff The input offset is given by adding this
+   *                       to the output offset
+   */
   protected void addOffCorrectMap(int off, int cumulativeDiff) {
     if (offsets == null) {
       offsets = new int[64];
@@ -80,7 +95,15 @@ public abstract class BaseCharFilter ext
       diffs = ArrayUtil.grow(diffs);
     }
     
-    offsets[size] = off;
-    diffs[size++] = cumulativeDiff; 
+    assert (size == 0 || off >= offsets[size])
+        : "Offset #" + size + "(" + off + ") is less than the last recorded offset "
+          + offsets[size] + "\n" + Arrays.toString(offsets) + "\n" + Arrays.toString(diffs);
+    
+    if (size == 0 || off != offsets[size - 1]) {
+      offsets[size] = off;
+      diffs[size++] = cumulativeDiff;
+    } else { // Overwrite the diff at the last recorded offset
+      diffs[size - 1] = cumulativeDiff;
+    }
   }
 }