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 2015/04/12 23:39:34 UTC

svn commit: r1673066 [2/2] - in /lucene/dev/trunk/lucene: core/src/test/org/apache/lucene/search/payloads/ core/src/test/org/apache/lucene/search/spans/ test-framework/src/java/org/apache/lucene/analysis/ test-framework/src/java/org/apache/lucene/searc...

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/spans/TestSpansEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/spans/TestSpansEnum.java?rev=1673066&r1=1673065&r2=1673066&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/spans/TestSpansEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/spans/TestSpansEnum.java Sun Apr 12 21:39:34 2015
@@ -18,27 +18,23 @@ package org.apache.lucene.search.spans;
  */
 
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 
-import org.apache.lucene.analysis.*;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.search.CheckHits;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 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.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Test;
+
+import static org.apache.lucene.search.spans.SpanTestUtil.*;
 
 /**
  * Tests Spans (v2)
@@ -49,50 +45,11 @@ public class TestSpansEnum extends Lucen
   private static IndexReader reader;
   private static Directory directory;
 
-  static final class SimplePayloadFilter extends TokenFilter {
-    int pos;
-    final PayloadAttribute payloadAttr;
-    final CharTermAttribute termAttr;
-
-    public SimplePayloadFilter(TokenStream input) {
-      super(input);
-      pos = 0;
-      payloadAttr = input.addAttribute(PayloadAttribute.class);
-      termAttr = input.addAttribute(CharTermAttribute.class);
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (input.incrementToken()) {
-        payloadAttr.setPayload(new BytesRef(("pos: " + pos).getBytes(StandardCharsets.UTF_8)));
-        pos++;
-        return true;
-      } else {
-        return false;
-      }
-    }
-
-    @Override
-    public void reset() throws IOException {
-      super.reset();
-      pos = 0;
-    }
-  }
-
-  static Analyzer simplePayloadAnalyzer;
   @BeforeClass
   public static void beforeClass() throws Exception {
-    simplePayloadAnalyzer = new Analyzer() {
-        @Override
-        public TokenStreamComponents createComponents(String fieldName) {
-          Tokenizer tokenizer = new MockTokenizer(MockTokenizer.SIMPLE, true);
-          return new TokenStreamComponents(tokenizer, new SimplePayloadFilter(tokenizer));
-        }
-    };
-
     directory = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
-        newIndexWriterConfig(simplePayloadAnalyzer)
+        newIndexWriterConfig(new MockAnalyzer(random()))
             .setMaxBufferedDocs(TestUtil.nextInt(random(), 100, 1000)).setMergePolicy(newLogMergePolicy()));
     //writer.infoStream = System.out;
     for (int i = 0; i < 10; i++) {
@@ -117,71 +74,48 @@ public class TestSpansEnum extends Lucen
     searcher = null;
     reader = null;
     directory = null;
-    simplePayloadAnalyzer = null;
   }
 
   private void checkHits(Query query, int[] results) throws IOException {
     CheckHits.checkHits(random(), query, "field", searcher, results);
   }
 
-  SpanTermQuery spanTQ(String term) {
-    return new SpanTermQuery(new Term("field", term));
-  }
-
-  @Test
   public void testSpansEnumOr1() throws Exception {
-    SpanTermQuery t1 = spanTQ("one");
-    SpanTermQuery t2 = spanTQ("two");
-    SpanOrQuery soq = new SpanOrQuery(t1, t2);
-    checkHits(soq, new int[] {1, 2, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19});
+    checkHits(spanOrQuery("field", "one", "two"), 
+              new int[] {1, 2, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19});
   }
 
-  @Test
   public void testSpansEnumOr2() throws Exception {
-    SpanTermQuery t1 = spanTQ("one");
-    SpanTermQuery t11 = spanTQ("eleven");
-    SpanOrQuery soq = new SpanOrQuery(t1, t11);
-    checkHits(soq, new int[] {1, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19});
+    checkHits(spanOrQuery("field", "one", "eleven"), 
+              new int[] {1, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19});
   }
 
-  @Test
   public void testSpansEnumOr3() throws Exception {
-    SpanTermQuery t12 = spanTQ("twelve");
-    SpanTermQuery t11 = spanTQ("eleven");
-    SpanOrQuery soq = new SpanOrQuery(t12, t11);
-    checkHits(soq, new int[] {});
+    checkHits(spanOrQuery("field", "twelve", "eleven"), 
+              new int[] {});
+  }
+  
+  public SpanQuery spanTQ(String s) {
+    return spanTermQuery("field", s);
   }
 
-  @Test
   public void testSpansEnumOrNot1() throws Exception {
-    SpanTermQuery t1 = spanTQ("one");
-    SpanTermQuery t2 = spanTQ("two");
-    SpanOrQuery soq = new SpanOrQuery(t1, t2);
-    SpanNotQuery snq = new SpanNotQuery(soq, t1);
-    checkHits(snq, new int[] {2,12});
+    checkHits(spanNotQuery(spanOrQuery("field", "one", "two"), spanTermQuery("field", "one")),
+              new int[] {2,12});
   }
 
-  @Test
   public void testSpansEnumNotBeforeAfter1() throws Exception {
-    SpanTermQuery t1 = spanTQ("one");
-    SpanTermQuery t100 = spanTQ("hundred");
-    SpanNotQuery snq = new SpanNotQuery(t100, t1, 0, 0);
-    checkHits(snq, new int[] {10, 11, 12, 13, 14, 15, 16, 17, 18, 19}); // include all "one hundred ..."
+    checkHits(spanNotQuery(spanTermQuery("field", "hundred"), spanTermQuery("field", "one")), 
+              new int[] {10, 11, 12, 13, 14, 15, 16, 17, 18, 19}); // include all "one hundred ..."
   }
 
-  @Test
   public void testSpansEnumNotBeforeAfter2() throws Exception {
-    SpanTermQuery t1 = spanTQ("one");
-    SpanTermQuery t100 = spanTQ("hundred");
-    SpanNotQuery snq = new SpanNotQuery(t100, t1, 1, 0);
-    checkHits(snq, new int[] {}); // exclude all "one hundred ..."
+    checkHits(spanNotQuery(spanTermQuery("field", "hundred"), spanTermQuery("field", "one"), 1, 0),
+              new int[] {}); // exclude all "one hundred ..."
   }
 
-  @Test
   public void testSpansEnumNotBeforeAfter3() throws Exception {
-    SpanTermQuery t1 = spanTQ("one");
-    SpanTermQuery t100 = spanTQ("hundred");
-    SpanNotQuery snq = new SpanNotQuery(t100, t1, 0, 1);
-    checkHits(snq, new int[] {10, 12, 13, 14, 15, 16, 17, 18, 19}); // exclude "one hundred one"
+    checkHits(spanNotQuery(spanTermQuery("field", "hundred"), spanTermQuery("field", "one"), 0, 1),
+              new int[] {10, 12, 13, 14, 15, 16, 17, 18, 19}); // exclude "one hundred one"
   }
 }

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/SimplePayloadFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/SimplePayloadFilter.java?rev=1673066&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/SimplePayloadFilter.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/analysis/SimplePayloadFilter.java Sun Apr 12 21:39:34 2015
@@ -0,0 +1,56 @@
+package org.apache.lucene.analysis;
+
+/*
+ * 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 java.nio.charset.StandardCharsets;
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.util.BytesRef;
+
+/** Simple payload filter that sets the payload as pos: XXXX */
+public final class SimplePayloadFilter extends TokenFilter {
+  int pos;
+  final PayloadAttribute payloadAttr;
+  final CharTermAttribute termAttr;
+
+  public SimplePayloadFilter(TokenStream input) {
+    super(input);
+    pos = 0;
+    payloadAttr = input.addAttribute(PayloadAttribute.class);
+    termAttr = input.addAttribute(CharTermAttribute.class);
+  }
+
+  @Override
+  public boolean incrementToken() throws IOException {
+    if (input.incrementToken()) {
+      payloadAttr.setPayload(new BytesRef(("pos: " + pos).getBytes(StandardCharsets.UTF_8)));
+      pos++;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    pos = 0;
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java?rev=1673066&r1=1673065&r2=1673066&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/BaseExplanationTestCase.java Sun Apr 12 21:39:34 2015
@@ -25,18 +25,15 @@ import org.apache.lucene.document.Sorted
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.spans.SpanFirstQuery;
-import org.apache.lucene.search.spans.SpanNearQuery;
-import org.apache.lucene.search.spans.SpanNotQuery;
-import org.apache.lucene.search.spans.SpanOrQuery;
 import org.apache.lucene.search.spans.SpanQuery;
-import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
+import static org.apache.lucene.search.spans.SpanTestUtil.*;
+
 /**
  * Tests primitive queries (ie: that rewrite to themselves) to
  * insure they match the expected set of docs, and that the score of each
@@ -140,58 +137,66 @@ public abstract class BaseExplanationTes
   }
 
   /** MACRO for SpanTermQuery */
-  public SpanTermQuery st(String s) {
-    return new SpanTermQuery(new Term(FIELD,s));
+  public SpanQuery st(String s) {
+    return spanTermQuery(FIELD, s);
   }
   
   /** MACRO for SpanNotQuery */
-  public SpanNotQuery snot(SpanQuery i, SpanQuery e) {
-    return new SpanNotQuery(i,e);
+  public SpanQuery snot(SpanQuery i, SpanQuery e) {
+    return spanNotQuery(i, e);
   }
 
   /** MACRO for SpanOrQuery containing two SpanTerm queries */
-  public SpanOrQuery sor(String s, String e) {
-    return sor(st(s), st(e));
+  public SpanQuery sor(String s, String e) {
+    return spanOrQuery(FIELD, s, e);
   }
+  
   /** MACRO for SpanOrQuery containing two SpanQueries */
-  public SpanOrQuery sor(SpanQuery s, SpanQuery e) {
-    return new SpanOrQuery(s, e);
+  public SpanQuery sor(SpanQuery s, SpanQuery e) {
+    return spanOrQuery(s, e);
   }
   
   /** MACRO for SpanOrQuery containing three SpanTerm queries */
-  public SpanOrQuery sor(String s, String m, String e) {
-    return sor(st(s), st(m), st(e));
+  public SpanQuery sor(String s, String m, String e) {
+    return spanOrQuery(FIELD, s, m, e);
   }
   /** MACRO for SpanOrQuery containing two SpanQueries */
-  public SpanOrQuery sor(SpanQuery s, SpanQuery m, SpanQuery e) {
-    return new SpanOrQuery(s, m, e);
+  public SpanQuery sor(SpanQuery s, SpanQuery m, SpanQuery e) {
+    return spanOrQuery(s, m, e);
   }
   
   /** MACRO for SpanNearQuery containing two SpanTerm queries */
-  public SpanNearQuery snear(String s, String e, int slop, boolean inOrder) {
+  public SpanQuery snear(String s, String e, int slop, boolean inOrder) {
     return snear(st(s), st(e), slop, inOrder);
   }
+  
   /** MACRO for SpanNearQuery containing two SpanQueries */
-  public SpanNearQuery snear(SpanQuery s, SpanQuery e,
-                             int slop, boolean inOrder) {
-    return new SpanNearQuery(new SpanQuery[] { s, e }, slop, inOrder);
+  public SpanQuery snear(SpanQuery s, SpanQuery e, int slop, boolean inOrder) {
+    if (inOrder) {
+      return spanNearOrderedQuery(slop, s, e);
+    } else {
+      return spanNearUnorderedQuery(slop, s, e);
+    }
   }
   
   
   /** MACRO for SpanNearQuery containing three SpanTerm queries */
-  public SpanNearQuery snear(String s, String m, String e,
+  public SpanQuery snear(String s, String m, String e,
                              int slop, boolean inOrder) {
     return snear(st(s), st(m), st(e), slop, inOrder);
   }
   /** MACRO for SpanNearQuery containing three SpanQueries */
-  public SpanNearQuery snear(SpanQuery s, SpanQuery m, SpanQuery e,
-                             int slop, boolean inOrder) {
-    return new SpanNearQuery(new SpanQuery[] { s, m, e }, slop, inOrder);
+  public SpanQuery snear(SpanQuery s, SpanQuery m, SpanQuery e, int slop, boolean inOrder) {
+    if (inOrder) {
+      return spanNearOrderedQuery(slop, s, m, e);
+    } else {
+      return spanNearUnorderedQuery(slop, s, m, e);
+    }
   }
   
   /** MACRO for SpanFirst(SpanTermQuery) */
-  public SpanFirstQuery sf(String s, int b) {
-    return new SpanFirstQuery(st(s), b);
+  public SpanQuery sf(String s, int b) {
+    return spanFirstQuery(st(s), b);
   }
 
   /**

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java?rev=1673066&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/spans/SpanTestUtil.java Sun Apr 12 21:39:34 2015
@@ -0,0 +1,161 @@
+package org.apache.lucene.search.spans;
+
+/*
+ * 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.index.Term;
+import org.apache.lucene.search.QueryUtils;
+
+import static org.junit.Assert.*;
+
+/** Some utility methods used for testing span queries */
+public class SpanTestUtil {
+  
+  /** 
+   * Adds additional asserts to a spanquery. Highly recommended 
+   * if you want tests to actually be debuggable.
+   */
+  public static SpanQuery spanQuery(SpanQuery query) {
+    QueryUtils.check(query);
+    return new AssertingSpanQuery(query);
+  }
+  
+  /**
+   * Makes a new SpanTermQuery (with additional asserts).
+   */
+  public static SpanQuery spanTermQuery(String field, String term) {
+    return spanQuery(new SpanTermQuery(new Term(field, term)));
+  }
+  
+  /**
+   * Makes a new SpanOrQuery (with additional asserts) from the provided {@code terms}.
+   */
+  public static SpanQuery spanOrQuery(String field, String... terms) {
+    SpanQuery[] subqueries = new SpanQuery[terms.length];
+    for (int i = 0; i < terms.length; i++) {
+      subqueries[i] = spanTermQuery(field, terms[i]);
+    }
+    return spanOrQuery(subqueries);
+  }
+  
+  /**
+   * Makes a new SpanOrQuery (with additional asserts).
+   */
+  public static SpanQuery spanOrQuery(SpanQuery... subqueries) {
+    return spanQuery(new SpanOrQuery(subqueries));
+  }
+  
+  /**
+   * Makes a new SpanNotQuery (with additional asserts).
+   */
+  public static SpanQuery spanNotQuery(SpanQuery include, SpanQuery exclude) {
+    return spanQuery(new SpanNotQuery(include, exclude));
+  }
+  
+  /**
+   * Makes a new SpanNotQuery (with additional asserts).
+   */
+  public static SpanQuery spanNotQuery(SpanQuery include, SpanQuery exclude, int pre, int post) {
+    return spanQuery(new SpanNotQuery(include, exclude, pre, post));
+  }
+  
+  /**
+   * Makes a new SpanFirstQuery (with additional asserts).
+   */
+  public static SpanQuery spanFirstQuery(SpanQuery query, int end) {
+    return spanQuery(new SpanFirstQuery(query, end));
+  }
+  
+  /**
+   * Makes a new SpanPositionRangeQuery (with additional asserts).
+   */
+  public static SpanQuery spanPositionRangeQuery(SpanQuery query, int start, int end) {
+    return spanQuery(new SpanPositionRangeQuery(query, start, end));
+  }
+  
+  /**
+   * Makes a new ordered SpanNearQuery (with additional asserts) from the provided {@code terms}
+   */
+  public static SpanQuery spanNearOrderedQuery(String field, int slop, String... terms) {
+    SpanQuery[] subqueries = new SpanQuery[terms.length];
+    for (int i = 0; i < terms.length; i++) {
+      subqueries[i] = spanTermQuery(field, terms[i]);
+    }
+    return spanNearOrderedQuery(slop, subqueries);
+  }
+  
+  /**
+   * Makes a new ordered SpanNearQuery (with additional asserts)
+   */
+  public static SpanQuery spanNearOrderedQuery(int slop, SpanQuery... subqueries) {
+    return spanQuery(new SpanNearQuery(subqueries, slop, true));
+  }
+  
+  /**
+   * Makes a new unordered SpanNearQuery (with additional asserts) from the provided {@code terms}
+   */
+  public static SpanQuery spanNearUnorderedQuery(String field, int slop, String... terms) {
+    SpanQuery[] subqueries = new SpanQuery[terms.length];
+    for (int i = 0; i < terms.length; i++) {
+      subqueries[i] = spanTermQuery(field, terms[i]);
+    }
+    return spanNearUnorderedQuery(slop, subqueries);
+  }
+  
+  /**
+   * Makes a new unordered SpanNearQuery (with additional asserts)
+   */
+  public static SpanQuery spanNearUnorderedQuery(int slop, SpanQuery... subqueries) {
+    return spanQuery(new SpanNearQuery(subqueries, slop, false));
+  }
+  
+  /** 
+   * Assert the next iteration from {@code spans} is a match
+   * from {@code start} to {@code end} in {@code doc}.
+   */
+  public static void assertNext(Spans spans, int doc, int start, int end) throws IOException {
+    if (spans.docID() >= doc) {
+      assertEquals("docId", doc, spans.docID());
+    } else { // nextDoc needed before testing start/end
+      if (spans.docID() >= 0) {
+        assertEquals("nextStartPosition of previous doc", Spans.NO_MORE_POSITIONS, spans.nextStartPosition());
+        assertEquals("endPosition of previous doc", Spans.NO_MORE_POSITIONS, spans.endPosition());
+      }
+      assertEquals("nextDoc", doc, spans.nextDoc());
+      if (doc != Spans.NO_MORE_DOCS) {
+        assertEquals("first startPosition", -1, spans.startPosition());
+        assertEquals("first endPosition", -1, spans.endPosition());
+      }
+    }
+    if (doc != Spans.NO_MORE_DOCS) {
+      assertEquals("nextStartPosition", start, spans.nextStartPosition());
+      assertEquals("startPosition", start, spans.startPosition());
+      assertEquals("endPosition", end, spans.endPosition());
+    }
+  }
+  
+  /** 
+   * Assert that {@code spans} is exhausted.
+   */
+  public static void assertFinished(Spans spans) throws Exception {
+    if (spans != null) { // null Spans is empty
+      assertNext(spans, Spans.NO_MORE_DOCS, -2, -2); // start and end positions will be ignored
+    }
+  }
+}