Return-Path: X-Original-To: apmail-lucene-commits-archive@www.apache.org Delivered-To: apmail-lucene-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3A1DFD3AA for ; Tue, 17 Jul 2012 23:38:57 +0000 (UTC) Received: (qmail 73148 invoked by uid 500); 17 Jul 2012 23:38:57 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 73138 invoked by uid 99); 17 Jul 2012 23:38:57 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 17 Jul 2012 23:38:57 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 17 Jul 2012 23:38:54 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 25E1823888CD; Tue, 17 Jul 2012 23:38:35 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1362724 - in /lucene/dev/branches/LUCENE-2878/lucene: core/src/java/org/apache/lucene/search/ core/src/java/org/apache/lucene/search/positions/ core/src/test/org/apache/lucene/search/positions/ highlighter/src/test/org/apache/lucene/search... Date: Tue, 17 Jul 2012 23:38:34 -0000 To: commits@lucene.apache.org From: simonw@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20120717233835.25E1823888CD@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: simonw Date: Tue Jul 17 23:38:33 2012 New Revision: 1362724 URL: http://svn.apache.org/viewvc?rev=1362724&view=rev Log: LUCENE-2878: handle collision in sloppy pharse and factor out sloppyness from conjunction pos iterator Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionPositionIterator.java lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueue.java lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/MaxLengthPositionIntervalIterator.java lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestPositionOffsets.java lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestSimplePositions.java lucene/dev/branches/LUCENE-2878/lucene/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java Tue Jul 17 23:38:33 2012 @@ -84,7 +84,7 @@ final class PhrasePositions { /** for debug purposes */ @Override public String toString() { - String s = "d:"+doc+" offset:"+offset+" position:"+position+" c:"+count + " actualPos: " + (position + offset); + String s = "d:"+doc+" offset:"+offset+" position:"+position+" c:"+count; if (rptGroup >=0 ) { s += " rpt:"+rptGroup+",i"+rptInd; } Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java Tue Jul 17 23:38:33 2012 @@ -46,7 +46,6 @@ abstract class PhraseScorer extends Scor this.postings = postings; reset(false); } - protected PhrasePositions[] _THEPOS; void reset(boolean needsOffsets) throws IOException { // convert tps to a list of phrase positions. // note: phrase-position differs from term-position in that its position @@ -54,14 +53,11 @@ abstract class PhraseScorer extends Scor // this allows to easily identify a matching (exact) phrase // when all PhrasePositions have exactly the same position. if (postings.length > 0) { - _THEPOS = new PhrasePositions[postings.length]; min = new PhrasePositions(postings[0].postings, postings[0].position, 0, postings[0].terms); - _THEPOS[0] = min; max = min; max.doc = -1; for (int i = 1; i < postings.length; i++) { PhrasePositions pp = new PhrasePositions(postings[i].postings, postings[i].position, i, postings[i].terms); - _THEPOS[i] = pp; max.next = pp; max = pp; max.doc = -1; Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Tue Jul 17 23:38:33 2012 @@ -20,10 +20,12 @@ package org.apache.lucene.search; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import org.apache.lucene.index.DocsAndPositionsEnum; @@ -35,7 +37,7 @@ import org.apache.lucene.search.similari import org.apache.lucene.util.OpenBitSet; final class SloppyPhraseScorer extends PhraseScorer { - +// private MaxLengthPositionIntervalIterator iter; private final int slop; private final int numPostings; private final PhraseQueue pq; // for advancing min position @@ -54,15 +56,17 @@ final class SloppyPhraseScorer extends P this.slop = slop; this.numPostings = postings==null ? 0 : postings.length; pq = new PhraseQueue(postings.length); +// iter = (MaxLengthPositionIntervalIterator) positions(false, false, false); } - String current() { - StringBuilder b = new StringBuilder(); - for (PhrasePositions phrasePositions : _THEPOS) { - b.append(phrasePositions).append("\n"); - } - return b.toString(); - } +// String current() { +// StringBuilder b = new StringBuilder(); +// int i = 0; +// for (PhrasePositions phrasePositions : postings) { +// b.append(i++).append(phrasePositions).append("\n"); +// } +// return b.toString(); +// } /** * Score a candidate doc for all slop-valid position-combinations (matches) @@ -84,53 +88,74 @@ final class SloppyPhraseScorer extends P */ @Override protected float phraseFreq() throws IOException { +// float freq = 0.0f; +// +// if (1 ==1) { +// if (iter.docID() != this.docID()) { +// iter.advanceTo(this.docID()); +// } +// while(iter.next() != null) { +// freq += docScorer.computeSlopFactor(iter.matchLength()); // score match +// } +// +// return freq; +// } +// freq = 0.0f; + if (!initPhrasePositions()) { return 0.0f; } + float freq = 0.0f; PhrasePositions pp = pq.pop(); int matchLength = end - pp.position; int next = pq.top().position; - int _lPos = pp.position; - int _lend = end; - String _s = current(); - Term[] _lTerms = pp.terms; +// int _lPos = pp.position; +// int _lend = end; +// String _s = current(); +// Term[] _lTerms = pp.terms; while (advancePP(pp)) { if (hasRpts && !advanceRpts(pp)) { break; // pps exhausted } if (pp.position > next) { // done minimizing current match-length if (matchLength <= slop) { - System.out.println(_s); - System.out.println("match: " + _lPos + " " + _lend + " " + Arrays.toString(_lTerms)); +// System.out.println("match: " + _lPos + " " + _lend + " " + Arrays.toString(_lTerms) + " " + matchLength); +// System.out.println(_s); +// System.out.println( docScorer.computeSlopFactor(matchLength)); freq += docScorer.computeSlopFactor(matchLength); // score match } pq.add(pp); pp = pq.pop(); next = pq.top().position; matchLength = end - pp.position; - _lPos = pp.position; - _lend = end; - _lTerms = pp.terms; - _s = current(); +// _lPos = pp.position; +// _lend = end; +// _lTerms = pp.terms; +// _s = current(); } else { int matchLength2 = end - pp.position; if (matchLength2 < matchLength) { - _lPos = pp.position; - _lend = end; - _lTerms = pp.terms; - _s = current(); +// _lPos = pp.position; +// _lend = end; +// _lTerms = pp.terms; +// _s = current(); matchLength = matchLength2; } } } if (matchLength <= slop) { - System.out.println(_s); - System.out.println("match: " + _lPos + " " + _lend + " " + Arrays.toString(_lTerms)); +// System.out.println("match: " + _lPos + " " + _lend + " " + Arrays.toString(_lTerms) + " " + matchLength); +// System.out.println(_s); +// System.out.println( docScorer.computeSlopFactor(matchLength)); + freq += docScorer.computeSlopFactor(matchLength); // score match + } +// System.out.println("res: " + freq + " doc: " + this.docID()); + return freq; } @@ -518,11 +543,12 @@ final class SloppyPhraseScorer extends P public PositionIntervalIterator positions(boolean needsPayloads, boolean needsOffsets, boolean collectPositions) throws IOException { // nocommit - payloads? - PositionIntervalIterator[] termIters = new PositionIntervalIterator[postings.length]; - Map map = new HashMap(); + Map map = new HashMap(); + for (int i = 0; i < postings.length; i++) { - MinPosition minPositions; Term term = postings[i].terms[0]; +// System.out.println(Arrays.toString(postings[i].terms)); + IterAndOffsets iterAndOffset; /* * NOCOMMIT This currently only works if there is only one term per position. * For multiple terms we need to extend the MaxLengthPI. and specialize @@ -531,68 +557,43 @@ final class SloppyPhraseScorer extends P * ords to them internally everything else should just work as before */ if (!map.containsKey(term)) { - minPositions = new MinPosition(); - map.put(term, minPositions); + DocsAndPositionsEnum docsAndPosEnum = postings[i].factory + .docsAndPositionsEnum(needsOffsets); + iterAndOffset = new IterAndOffsets(new TermScorer.TermPositions(this, docsAndPosEnum, needsPayloads, + collectPositions)); + map.put(term, iterAndOffset); } else { - minPositions = map.get(term); + iterAndOffset = map.get(term); } - DocsAndPositionsEnum docsAndPosEnum = postings[i].factory - .docsAndPositionsEnum(needsOffsets); - termIters[i] = new GapEnforcingPositionIterator(this, collectPositions, minPositions, - new TermScorer.TermPositions(this, docsAndPosEnum, needsPayloads, - collectPositions), 0); + iterAndOffset.offsets.add(postings[i].position); +// System.out.println("POS: " + postings[i].position + " " + term); } - ConjunctionPositionIterator iter = new ConjunctionPositionIterator(this, - collectPositions, termIters); - return new MaxLengthPositionIntervalIterator(this, slop, iter); - } - - private static final class MinPosition { - int position = -1; + Collection values = map.values(); + PositionIntervalIterator[] iters = new PositionIntervalIterator[values.size()]; + int i = 0; + for (IterAndOffsets iterAndOffsets : values) { + iters[i++] = MaxLengthPositionIntervalIterator.create(this, collectPositions, iterAndOffsets.iter, iterAndOffsets.toIntArray()); + } + return new MaxLengthPositionIntervalIterator(this, slop, collectPositions, new ConjunctionPositionIterator(this, collectPositions, iters)); } - private static class GapEnforcingPositionIterator extends PositionIntervalIterator { - - private final MinPosition minPosition; - private final PositionIntervalIterator other; - private final int delta; - - public GapEnforcingPositionIterator(Scorer scorer, boolean collectPositions, MinPosition minPosition, PositionIntervalIterator other, int delta) { - super(scorer, collectPositions); - this.other = other; - this.minPosition = minPosition; - this.delta = delta; + private static class IterAndOffsets { + final List offsets = new ArrayList(); + final PositionIntervalIterator iter; + IterAndOffsets(PositionIntervalIterator iter) { + this.iter = iter; } - - @Override - public int advanceTo(int docId) throws IOException { - return other.advanceTo(docId); - } - - @Override - public PositionInterval next() throws IOException { - PositionInterval i = null; - while((i = other.next()) != null) { - assert i.end == i.begin; - if (i.begin > minPosition.position - delta) { - minPosition.position = i.begin; - break; - } + + public int[] toIntArray() { + int[] array = new int[offsets.size()]; + for (int i = 0; i < array.length; i++) { + array[i] = offsets.get(i).intValue(); } - - return i; - } - - @Override - public void collect(PositionCollector collector) { - assert collectPositions; - other.collect(collector); - } - - @Override - public PositionIntervalIterator[] subs(boolean inOrder) { - return other.subs(inOrder); + return array; } + } + + } Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java Tue Jul 17 23:38:33 2012 @@ -38,9 +38,7 @@ public final class ConjunctionPositionIt private final IntervalQueueAnd queue; private final int nrMustMatch; private SnapshotPositionCollector snapshot; - private int matchDistance; // nocommit specialize for this or better move that - // out of this class entirely - + public ConjunctionPositionIterator(Scorer scorer, boolean collectPositions, PositionIntervalIterator... iterators) throws IOException { this(scorer, collectPositions, iterators.length, iterators); @@ -51,16 +49,16 @@ public final class ConjunctionPositionIt throws IOException { super(scorer, iterators, new IntervalQueueAnd(iterators.length), collectPositions); - queue = (IntervalQueueAnd) super.queue; // avoid lots of casts? + this.queue = (IntervalQueueAnd) super.queue; // avoid lots of casts? this.nrMustMatch = minimuNumShouldMatch; } void advance() throws IOException { final IntervalRef top = queue.top(); PositionInterval interval = null; - if ((interval = iterators[top.ord].next()) != null) { + if ((interval = iterators[top.index].next()) != null) { top.interval = interval; - queue.updateRightExtreme(top); + queue.updateRightExtreme(interval); queue.updateTop(); } else { queue.pop(); @@ -79,20 +77,16 @@ public final class ConjunctionPositionIt } do { queue.updateCurrentCandidate(); - PositionInterval top = updateMatchDistance(queue.top()); // nocommit this - // should be in a - // specialized - // class - used - // for scoring in - // sloppy phrase - if (queue.currentCandidate.begin == top.begin - && queue.currentCandidate.end == top.end) { - return queue.currentCandidate; - } + PositionInterval top = queue.top().interval; if (collectPositions) { snapShotSubPositions(); // this looks odd? -> see SnapShotCollector below for // details! } + if (queue.currentCandidate.begin == top.begin + && queue.currentCandidate.end == top.end) { + return queue.currentCandidate; + } + advance(); if (queue.size() < nrMustMatch) { break; @@ -101,12 +95,6 @@ public final class ConjunctionPositionIt return queue.currentCandidate; // TODO support payloads } - private final PositionInterval updateMatchDistance(IntervalRef top) { - final int end = queue.rightExtreme - queue.rightExtremeOrd; - final int head = (top.interval.begin - top.ord); - matchDistance = end - head; - return top.interval; - } @Override public int advanceTo(int docId) throws IOException { @@ -118,14 +106,15 @@ public final class ConjunctionPositionIt final PositionInterval interval = iterators[i].next(); if (interval != null) { - IntervalRef intervalRef = new IntervalRef(interval, i); - queue.updateRightExtreme(intervalRef); + IntervalRef intervalRef = new IntervalRef(interval, i); // TODO maybe reuse? + queue.updateRightExtreme(intervalRef.interval); queue.add(intervalRef); } } return currentDoc; } + private void snapShotSubPositions() { if (snapshot == null) { snapshot = new SnapshotPositionCollector(queue.size()); @@ -154,10 +143,6 @@ public final class ConjunctionPositionIt } } - int matchDistance() { // nocommit move out! - return matchDistance; - } - /* * Due to the laziness of this position iterator and the minimizing algorithm * we advance the underlying iterators before the consumer can call collect on Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionPositionIterator.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionPositionIterator.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionPositionIterator.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionPositionIterator.java Tue Jul 17 23:38:33 2012 @@ -43,7 +43,7 @@ public final class DisjunctionPositionIt void advance() throws IOException { final IntervalRef top = queue.top(); PositionInterval interval = null; - if ((interval = iterators[top.ord].next()) != null) { + if ((interval = iterators[top.index].next()) != null) { top.interval = interval; queue.updateTop(); } else { @@ -72,7 +72,7 @@ public final class DisjunctionPositionIt public void collect(PositionCollector collector) { assert collectPositions; collector.collectComposite(scorer, queue.currentCandidate, currentDoc); - iterators[queue.top().ord].collect(collector); + iterators[queue.top().index].collect(collector); } @Override Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueue.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueue.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueue.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueue.java Tue Jul 17 23:38:33 2012 @@ -43,12 +43,13 @@ abstract class IntervalQueue extends Pri final static class IntervalRef { PositionInterval interval; - int ord; // the ordinal of this ref in the ordered case + int ord; + int index; IntervalRef(PositionInterval interval, int index) { super(); this.interval = interval; - this.ord = index; + this.index = index; } } Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java Tue Jul 17 23:38:33 2012 @@ -25,7 +25,6 @@ final class IntervalQueueAnd extends Int int rightExtreme = Integer.MIN_VALUE; int rightExtremeOffset = Integer.MIN_VALUE; - int rightExtremeOrd = Integer.MIN_VALUE; // the ord of the queues right extreme - ordered case! public IntervalQueueAnd(int size) { super(size); @@ -37,22 +36,17 @@ final class IntervalQueueAnd extends Int currentCandidate.end = Integer.MIN_VALUE; rightExtreme = Integer.MIN_VALUE; rightExtremeOffset = Integer.MIN_VALUE; - rightExtremeOrd = Integer.MIN_VALUE; } - public void updateRightExtreme(IntervalRef ref) { - if (rightExtreme < ref.interval.end) { - rightExtreme = ref.interval.end; - rightExtremeOrd = ref.ord; - } - - rightExtremeOffset = Math.max(rightExtremeOffset, ref.interval.offsetEnd); + public void updateRightExtreme(PositionInterval interval) { + rightExtreme = Math.max(rightExtreme, interval.end); + rightExtremeOffset = Math.max(rightExtremeOffset, interval.offsetEnd); } public boolean topContainsQueueInterval() { PositionInterval interval = top().interval; - return interval.begin <= currentCandidate.begin - && currentCandidate.end <= rightExtreme; + return interval.begin < currentCandidate.begin + && currentCandidate.end < rightExtreme; } public void updateCurrentCandidate() { @@ -67,6 +61,6 @@ final class IntervalQueueAnd extends Int protected boolean lessThan(IntervalRef left, IntervalRef right) { final PositionInterval a = left.interval; final PositionInterval b = right.interval; - return a.begin < b.begin || (a.begin == b.begin && a.end >= b.end); + return a.begin < b.begin || (a.begin == b.begin && a.end > b.end) || a.offsetBegin < b.offsetBegin; } } Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/MaxLengthPositionIntervalIterator.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/MaxLengthPositionIntervalIterator.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/MaxLengthPositionIntervalIterator.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/MaxLengthPositionIntervalIterator.java Tue Jul 17 23:38:33 2012 @@ -1,4 +1,5 @@ package org.apache.lucene.search.positions; + /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -18,8 +19,7 @@ package org.apache.lucene.search.positio import java.io.IOException; import org.apache.lucene.search.Scorer; - - +import org.apache.lucene.search.positions.IntervalQueue.IntervalRef; /** * An interval iterator that has the semantics of sloppy phrase query. @@ -27,46 +27,192 @@ import org.apache.lucene.search.Scorer; public class MaxLengthPositionIntervalIterator extends PositionIntervalIterator { private final int maxLen; - private ConjunctionPositionIterator iter; - + private int matchDistance; + private final PositionIntervalIterator iterator; + public MaxLengthPositionIntervalIterator(Scorer scorer, int maxLength, - ConjunctionPositionIterator iter) throws IOException { - super(scorer, iter.collectPositions); + boolean collectPositions, PositionIntervalIterator iterator) + throws IOException { + super(scorer, collectPositions); this.maxLen = maxLength; - this.iter = iter; - } - - @Override - public int advanceTo(int docId) throws IOException { - return iter.advanceTo(docId); + this.iterator = iterator; } @Override public PositionInterval next() throws IOException { PositionInterval current; do { - current = iter.next(); - if (current == null) { - break; - } - //NOCOMMIT this is an impl detail of ConjuIter that shoudl reside somewhere else - // maybe specialize for this? - if (iter.matchDistance() <= maxLen) { + if ((current = iterator.next()) != null) { + matchDistance = current.end - current.begin; + if (matchDistance <= maxLen) { +// System.out.println(matchDistance); + break; + } + } else { break; } - } while(true); + } while (true); return current; } @Override + public int advanceTo(int docId) throws IOException { + return currentDoc = iterator.advanceTo(docId); + } + + public int matchLength() { + return matchDistance; + } + + public static PositionIntervalIterator create(Scorer scorer, boolean collectPositions, + PositionIntervalIterator iterator, int... offsets) { + if (offsets.length == 1) { + return new SingleSlopplyIntervalIterator(scorer, collectPositions, iterator, offsets[0]); + } else { + return new SloppyGroupIntervalIterator(scorer, collectPositions, iterator, offsets); + } + + } + + private final static class SingleSlopplyIntervalIterator extends + PositionIntervalIterator { + private PositionInterval realInterval; + private final PositionInterval sloppyInterval = new PositionInterval(); + private final PositionIntervalIterator iterator; + private int offset; + + public SingleSlopplyIntervalIterator(Scorer scorer, + boolean collectPositions, PositionIntervalIterator iterator, int offset) { + super(scorer, collectPositions); + this.iterator = iterator; + this.offset = offset; + } + + @Override + public int advanceTo(int docId) throws IOException { + return currentDoc = iterator.advanceTo(docId); + } + + @Override + public PositionInterval next() throws IOException { + if ((realInterval = iterator.next()) != null) { + sloppyInterval.begin = sloppyInterval.end = realInterval.begin - offset; + sloppyInterval.offsetBegin = realInterval.offsetBegin; + sloppyInterval.offsetEnd = realInterval.offsetEnd; + return sloppyInterval; + } + return null; + } + + @Override + public void collect(PositionCollector collector) { + collector.collectLeafPosition(scorer, realInterval, currentDoc); + + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return null; + } + + } + + private final static class SloppyGroupIntervalIterator extends + PositionIntervalIterator { + + private final PositionInterval sloppyGroupInterval = new PositionInterval(); + private final int[] offsets; + private final PositionInterval[] intervalPositions; + private final PositionIntervalIterator groupIterator; + private int currentIndex; + private boolean initialized; + + public SloppyGroupIntervalIterator(Scorer scorer, boolean collectPositions, + PositionIntervalIterator groupIterator, int... offsets) { + super(scorer, collectPositions); + this.offsets = offsets; + this.groupIterator = groupIterator; + this.intervalPositions = new PositionInterval[offsets.length]; + for (int i = 0; i < intervalPositions.length; i++) { + intervalPositions[i] = new PositionInterval(); + } + } + + @Override + public int advanceTo(int docId) throws IOException { + initialized = false; + return currentDoc = groupIterator.advanceTo(docId); + } + + @Override + public PositionInterval next() throws IOException { + sloppyGroupInterval.begin = Integer.MAX_VALUE; + sloppyGroupInterval.end = Integer.MIN_VALUE; + if (!initialized) { + initialized = true; + + currentIndex = 0; + for (int i = 0; i < offsets.length; i++) { + PositionInterval current; + if ((current = groupIterator.next()) != null) { + intervalPositions[i].copy(current); + + int p = current.begin - offsets[i]; + sloppyGroupInterval.begin = Math.min(sloppyGroupInterval.begin, p); + sloppyGroupInterval.end = Math.max(sloppyGroupInterval.end, p); + } else { + return null; + } + } + sloppyGroupInterval.offsetBegin = intervalPositions[0].offsetBegin; + sloppyGroupInterval.offsetEnd = intervalPositions[intervalPositions.length-1].offsetEnd; + return sloppyGroupInterval; + } + PositionInterval current; + if ((current = groupIterator.next()) != null) { + final int currentFirst = currentIndex++ % intervalPositions.length; + intervalPositions[currentFirst].copy(current); + int currentIdx = currentIndex; + for (int i = 0; i < intervalPositions.length; i++) { // find min / max + int idx = currentIdx++ % intervalPositions.length; + int p = intervalPositions[idx].begin - offsets[i]; + sloppyGroupInterval.begin = Math.min(sloppyGroupInterval.begin, p); + sloppyGroupInterval.end = Math.max(sloppyGroupInterval.end, p); + } + sloppyGroupInterval.offsetBegin = intervalPositions[currentIndex % intervalPositions.length].offsetBegin; + sloppyGroupInterval.offsetEnd = intervalPositions[currentFirst].offsetEnd; + return sloppyGroupInterval; + } + return null; + } + + @Override + public void collect(PositionCollector collector) { + int currentIdx = currentIndex+1; + for (int i = 0; i < intervalPositions.length; i++) { // find min / max + int idx = currentIdx++ % intervalPositions.length; + collector.collectLeafPosition(scorer, intervalPositions[idx], + currentDoc); + } + + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return new PositionIntervalIterator[] {groupIterator}; + } + + } + + @Override public void collect(PositionCollector collector) { assert collectPositions; - iter.collect(collector); + this.iterator.collect(collector); + } @Override public PositionIntervalIterator[] subs(boolean inOrder) { - return iter.subs(inOrder); + return null; } - } Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestPositionOffsets.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestPositionOffsets.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestPositionOffsets.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestPositionOffsets.java Tue Jul 17 23:38:33 2012 @@ -34,8 +34,6 @@ import org.apache.lucene.store.Directory import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util._TestUtil; -import java.io.IOException; - public class TestPositionOffsets extends LuceneTestCase { // What am I testing here? @@ -112,8 +110,8 @@ public class TestPositionOffsets extends assertEquals(0, positions.advanceTo(nextDoc)); for (int i = 0; i < startOffsets.length; i++) { PositionIntervalIterator.PositionInterval interval = positions.next(); - assertEquals(startOffsets[i], interval.offsetBegin); - assertEquals(endOffsets[i], interval.offsetEnd); + assertEquals("i: " + i, startOffsets[i], interval.offsetBegin); + assertEquals("i: " + i, endOffsets[i], interval.offsetEnd); } assertNull(positions.next()); Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestSimplePositions.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestSimplePositions.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestSimplePositions.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/core/src/test/org/apache/lucene/search/positions/TestSimplePositions.java Tue Jul 17 23:38:33 2012 @@ -84,7 +84,7 @@ public class TestSimplePositions extends "u2 xx u1 u2",//8 "u1 u2 xx u2",//9 "u2 u1 xx u2",//10 - "t1 t2 t1 t3 t2 t3"};//10 + "t1 t2 t1 t3 t2 t3"};//11 public TermQuery makeTermQuery(String text) { return new TermQuery(new Term(field, text)); @@ -253,7 +253,7 @@ public class TestSimplePositions extends PositionInterval interval = positions.next(); assertNotNull("Does not have next and it should", interval); assertEquals("doc", 4, positions.docID()); - assertEquals("start", 1, interval.begin); + assertEquals("start " + interval, 1, interval.begin); assertEquals("end", 2, interval.end); positions.advanceTo(5); @@ -353,6 +353,13 @@ public class TestSimplePositions extends assertEquals("doc", 9, iterator.docID()); assertEquals("start", 0, interval.begin); assertEquals("end", 1, interval.end); + + // this on is tricky since the two terms are overlapping - need to find a solution for this ie. dedicated query? +// interval = iterator.next(); +// assertNotNull("Does not have next and it should", interval); +// assertEquals("doc", 9, iterator.docID()); +// assertEquals("start", 0, interval.begin); +// assertEquals("end", 3, interval.end); interval = iterator.next(); assertNull("Has next and it shouldn't", interval); Modified: lucene/dev/branches/LUCENE-2878/lucene/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java?rev=1362724&r1=1362723&r2=1362724&view=diff ============================================================================== --- lucene/dev/branches/LUCENE-2878/lucene/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java (original) +++ lucene/dev/branches/LUCENE-2878/lucene/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java Tue Jul 17 23:38:33 2012 @@ -47,6 +47,7 @@ import org.apache.lucene.search.position import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util._TestUtil; +import org.junit.Ignore; import java.io.IOException; import java.io.StringReader; @@ -380,17 +381,65 @@ public class PosHighlighterTest extends frags[0]); close(); } + + public void testMultiPhraseQuery() throws Exception { + MultiPhraseQuery query = new MultiPhraseQuery(); + insertDocs(analyzer, "pease porridge hot but not too hot or otherwise pease porridge cold"); + + query.add(terms(F, "pease"), 0); + query.add(terms(F, "porridge"), 1); + query.add(terms(F, "hot", "cold"), 2); + query.setSlop(1); + + String[] frags = doSearch(query, Integer.MAX_VALUE); + assertEquals("pease porridge hot but not too hot or otherwise pease porridge cold", frags[0]); + + close(); + } + + @Ignore("not implemented yet") + public void testMultiPhraseQueryCollisions() throws Exception { + MultiPhraseQuery query = new MultiPhraseQuery(); + insertDocs(analyzer, "pease porridge hot not too hot or otherwise pease porridge porridge"); + + query.add(terms(F, "pease"), 0); + query.add(terms(F, "porridge"), 1); + query.add(terms(F, "coldasice", "porridge" ), 2); + query.setSlop(1); + + String[] frags = doSearch(query, Integer.MAX_VALUE); + assertEquals("pease porridge hot but not too hot or otherwise pease porridge porridge", frags[0]); + + close(); + } + + private Term[] terms(String field, String...tokens) { + Term[] terms = new Term[tokens.length]; + for (int i = 0; i < tokens.length; i++) { + terms[i] = new Term(field, tokens[i]); + } + return terms; + } public void testSloppyPhraseQuery() throws Exception { assertSloppyPhrase( "a b c d a b c d e f", "a b c d a b c d e f", 2, "c", "a"); assertSloppyPhrase( "a c e b d e f a b","a c e b d e f a b", 2, "a", "b"); - assertSloppyPhrase( "X A X B A","X A X B A", 2, "X", "A", "A"); - assertSloppyPhrase( "A A X A X B A X B B A A X B A A","A A X A X B A X B B A A X B A A", 2, "X", "A", "A"); - assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A A X A X B A X B B A A X B A A", 2, "X", "A", "A"); - assertSloppyPhrase( "A A X A X B A", "A A X A X B A", 2, "X", "A", "A"); + assertSloppyPhrase( "Y A X B A", "Y A X B A", 2, "X", "A", "A"); + + assertSloppyPhrase( "X A X B A","X A X B A", 2, "X", "A", "A"); // non overlapping minmal!! + assertSloppyPhrase( "A A A X",null, 2, "X", "A", "A"); + assertSloppyPhrase( "A A X A", "A A X A", 2, "X", "A", "A"); + assertSloppyPhrase( "A A X A Y B A", "A A X A Y B A", 2, "X", "A", "A"); + assertSloppyPhrase( "A A X", null, 2, "X", "A", "A"); + assertSloppyPhrase( "A X A", null, 1, "X", "A", "A"); + + assertSloppyPhrase( "A X B A", "A X B A", 2, "X", "A", "A"); + assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A A X A X B A X B B A A X B A A", 2, "X", "A", "A"); + assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A A X A X B A X B B A A X B A A", 2, "X", "A", "A"); + + assertSloppyPhrase( "A A X A X B A", "A A X A X B A", 2, "X", "A", "A"); assertSloppyPhrase( "A A Y A X B A", "A A Y A X B A", 2, "X", "A", "A"); assertSloppyPhrase( "A A Y A X B A A", "A A Y A X B A A", 2, "X", "A", "A"); - assertSloppyPhrase( "A A X A Y B A", "A A X A Y B A", 2, "X", "A", "A"); assertSloppyPhrase( "A A X A Y B A", null , 1, "X", "A", "A"); close(); } @@ -403,9 +452,10 @@ public class PosHighlighterTest extends } pq.setSlop(slop); +// System.out.println(doc); String[] frags = doSearch(pq, 50); if (expected == null) { - assertNull(frags); + assertNull(frags != null ? frags[0] : "", frags); } else { assertEquals(expected, frags[0]); }