Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosCollector.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosCollector.java (revision 1144713) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosCollector.java (working copy) @@ -1,17 +1,37 @@ package org.apache.lucene.search.poshighlight; +/** + * 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.IndexReader.AtomicReaderContext; import org.apache.lucene.search.Collector; -import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Scorer.ScorerVisitor; +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; /** - * for testing only - collect the first maxDocs docs and throw the rest away + * Collects the first maxDocs docs and their positions matching the query + * + * @lucene.experimental */ -public class PosCollector extends Collector { + +public class PosCollector extends Collector implements PositionCollector { int count; ScorePosDoc docs[]; @@ -21,21 +41,45 @@ } protected Scorer scorer; - + private PositionIntervalIterator positions; + + @Override public void collect(int doc) throws IOException { if (count >= docs.length) return; - assert (scorer != null); - ScorePosDoc spdoc = new ScorePosDoc (doc, scorer.score(), scorer.positions(), 32, false); - docs[count++] = spdoc; + addDoc (doc); + // consume any remaining positions the scorer didn't report + docs[count-1].score=scorer.score(); + positions.advanceTo(doc); + while(positions.next() != null) { + positions.collect(); + } } + + private boolean addDoc (int doc) { + if (count <= 0 || docs[count-1].doc != doc) { + ScorePosDoc spdoc = new ScorePosDoc (doc); + docs[count++] = spdoc; + return true; + } + return false; + } + + @Override + public void collect(Scorer scorer, PositionInterval positions, int docID) { + addDoc(docID); + docs[count - 1].storePosition(positions); + } public boolean acceptsDocsOutOfOrder() { return false; } - public void setScorer(Scorer scorer) { + public void setScorer(Scorer scorer) throws IOException { this.scorer = scorer; + positions = scorer.positions(); + positions.setPositionCollector(this); + // If we want to visit the other scorers, we can, here... } public Scorer getScorer () { @@ -54,21 +98,4 @@ @Override public boolean needsPositions() { return true; } - /** - * For testing/investigation - * @author sokolov - * - */ - class SpanScorerVisitor extends ScorerVisitor { - - @Override - public void visitRequired (Query parent, Query child, Scorer scorer) { - System.out.println ("parent=" + parent + ", child=" + child); - } - - @Override - public void visitOptional (Query parent, Query child, Scorer scorer) { - System.out.println ("parent=" + parent + ", child=" + child); - } - } } Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosHighlighter.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosHighlighter.java (revision 1144713) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosHighlighter.java (working copy) @@ -1,166 +0,0 @@ -package org.apache.lucene.search.poshighlight; - -import java.io.IOException; - -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.TermVectorMapper; -import org.apache.lucene.index.TermVectorOffsetInfo; -import org.apache.lucene.search.highlight.DefaultEncoder; -import org.apache.lucene.search.highlight.Encoder; -import org.apache.lucene.search.highlight.Formatter; -import org.apache.lucene.search.highlight.SimpleHTMLFormatter; -import org.apache.lucene.search.highlight.TextFragment; -import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BytesRef; - -public class PosHighlighter { - private Formatter formatter; - private Encoder encoder; - - - public PosHighlighter() { - this(new SimpleHTMLFormatter()); - } - - public PosHighlighter(Formatter formatter) { - this(formatter,new DefaultEncoder()); - } - - public PosHighlighter(Formatter formatter, Encoder encoder) { - this.formatter = formatter; - this.encoder = encoder; - } - - /** - * - * @param scorer a Scorer positioned at the docID for which highlighting - * fragments are to be retrieved. - * @param mergeContiguousFragments - * @param maxNumFragments the number of fragments to return - * @param fragSize, the requested size of fragments, in characters. Fragments may - * be smaller if there is insufficient text. There is an qttempt to put the first match - * in the center of the fragment. - * @return the first maxNumFragments TextFragments, ordered by (descending) score. - * Each fragment corresponds to a Span, and its score is the Span's score. - * @throws IOException - */ - public String[] getFirstFragments( - ScorePosDoc doc, - IndexReader reader, - String fieldName, - boolean mergeContiguousFragments, - int maxNumFragments, - int fragSize) throws IOException - { - PositionOffsetMapper pom = new PositionOffsetMapper (); - // FIXME: test error cases: for non-stored fields, and fields w/no term vectors - reader.getTermFreqVector(doc.doc, fieldName, pom); - String text = reader.document(doc.doc).getFieldable(fieldName).stringValue(); - String[] frags = new String[maxNumFragments]; - int ifrag = 0; - int ipos = 0; - PositionInterval pos = doc.positions[ipos++]; - while (ifrag < maxNumFragments && pos != null) { - StringBuilder buf = new StringBuilder(); - int matchStart = pom.getStartOffset(pos.begin); - int matchEnd = pom.getEndOffset(pos.end); - int fragStart = Math.max(0, matchStart - (fragSize - (matchEnd-matchStart)) / 2); - int fragEnd = Math.min(fragStart+fragSize, text.length()); - - for (;;) { - // Build up a single fragment, possibly including multiple positions - if (matchStart > fragStart) - buf.append (text, fragStart, matchStart); - buf.append (""); // TODO - parameterize - buf.append (text, matchStart, matchEnd); - buf.append (""); - if (fragEnd <= matchEnd) { - break; - } - boolean done = false; - if (ipos < doc.posCount) { - pos = doc.positions[ipos++]; - matchStart = pom.getStartOffset(pos.begin); - done = (matchStart >= fragEnd); - } - else { - pos = null; // terminate the outer loop - done = true; - } - if (done) { - // Either there are no more matches or the next match comes after the end of this fragment - // In either case, grab some more text to fill out the fragment - buf.append(text, matchEnd, fragEnd); - break; - } - // include the next match in this fragment - fragStart = matchEnd; - matchEnd = pom.getEndOffset(pos.end); - } - // emit a completed fragment - frags[ifrag++] = buf.toString(); - } - return frags; - } - - /** - * @param scorer - * @param mergeContiguousFragments - * @param maxNumFragments number of fragments to retrieve - * @return The first maxNumFragments TextFragments, in document order: - * sorted by their (start ing, then ending) span position - */ - public TextFragment[] getBestFragments( - ScorePosDoc doc, - IndexReader reader, - boolean mergeContiguousFragments, - int maxNumFragments) - { - // TODO - get maxNumFragments top fragments by score - return null; - } - - class PositionOffsetMapper extends TermVectorMapper { - private int maxPos = 0; - private static final int BUF_SIZE = 128; - int startOffset[] = new int[BUF_SIZE], endOffset[] = new int[BUF_SIZE]; - - public void setExpectations(String field, int numTerms, - boolean storeOffsets, boolean storePositions) { - } - - public void map(BytesRef term, int frequency, - TermVectorOffsetInfo[] offsets, int[] positions) - { - for (int i = 0; i < positions.length; i++) { - int pos = positions[i]; - if (pos >= startOffset.length) { - grow (pos + BUF_SIZE); - maxPos = pos; - } else if (pos > maxPos) { - maxPos = pos; - } - startOffset[pos] = offsets[i].getStartOffset(); - endOffset[pos] = offsets[i].getEndOffset(); - } - } - - private void grow (int size) { - startOffset = ArrayUtil.grow (startOffset, size); - endOffset = ArrayUtil.grow (endOffset, size); - } - - public int getStartOffset(int pos) { - return startOffset[pos]; - } - - public int getEndOffset(int pos) { - return endOffset[pos]; - } - - public int getMaxPosition() { - return maxPos; - } - } -} Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosTokenStream.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosTokenStream.java (revision 0) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PosTokenStream.java (revision 0) @@ -0,0 +1,75 @@ +package org.apache.lucene.search.poshighlight; + +/** + * 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.analysis.TokenStream; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; +import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; + +/** + * A TokenStream constructed from a stream of positions and their offsets. + * The document is segmented into tokens at the start and end offset of each interval. The intervals + * are assumed to be non-overlapping. + * + * TODO: abstract the dependency on the current PositionOffsetMapper impl; + * allow for implementations of position->offset maps that don't rely on term vectors. + * + * @lucene.experimental + */ +public class PosTokenStream extends TokenStream { + + //this tokenizer generates four attributes: + // term, offset, positionIncrement? and type? + private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); + private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class); + private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class); + //private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class); + private final String text; + private final PositionIntervalIterator positions; + + // the index of the current position interval + private PositionInterval pos = null; + private final PositionOffsetMapper pom; + + public PosTokenStream (String text, PositionIntervalIterator positions, PositionOffsetMapper pom) { + this.text = text; + this.positions = positions; + this.pom = pom; + } + + @Override + public final boolean incrementToken() throws IOException { + pos = positions.next(); + if (pos == null){ + return false; + } + int b, e; + b = pom.getStartOffset(pos.begin); + e = pom.getEndOffset(pos.end); + termAtt.append(text, b, e); + offsetAtt.setOffset(b, e); + posIncrAtt.setPositionIncrement(1); + return true; + } + +} Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionIntervalArrayIterator.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionIntervalArrayIterator.java (revision 0) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionIntervalArrayIterator.java (revision 0) @@ -0,0 +1,61 @@ +package org.apache.lucene.search.poshighlight; + +/** + * 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.search.positions.PositionIntervalIterator; + +/** + * Present an array of PositionIntervals as an Iterator. + * @lucene.experimental + */ +public class PositionIntervalArrayIterator extends PositionIntervalIterator { + + private int next = 0; + private int count; + private PositionInterval[] positions; + + public PositionIntervalArrayIterator (PositionInterval[] positions, int count) { + super(null); + this.positions = positions; + this.count = count; + } + + @Override + public PositionInterval next() { + if (next >= count) + return null; + return positions[next++]; + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return EMPTY; + } + + @Override + public void collect() { + } + + @Override + public int advanceTo(int docId) throws IOException { + return 0; + } + +} \ No newline at end of file Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionOffsetMapper.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionOffsetMapper.java (revision 0) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionOffsetMapper.java (revision 0) @@ -0,0 +1,73 @@ +package org.apache.lucene.search.poshighlight; + +/** + * 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 org.apache.lucene.index.TermVectorMapper; +import org.apache.lucene.index.TermVectorOffsetInfo; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; + +/** + * Create a map of position->offsets using term vectors. TODO: In highlighting, we don't really need the + * entire map; make a sparse map including only required positions. + * + * @lucene.experimental + */ + +public class PositionOffsetMapper extends TermVectorMapper { + private int maxPos = 0; + private static final int BUF_SIZE = 128; + int startOffset[] = new int[BUF_SIZE], endOffset[] = new int[BUF_SIZE]; + + public void setExpectations(String field, int numTerms, + boolean storeOffsets, boolean storePositions) { + } + + public void map(BytesRef term, int frequency, + TermVectorOffsetInfo[] offsets, int[] positions) + { + for (int i = 0; i < positions.length; i++) { + int pos = positions[i]; + if (pos >= startOffset.length) { + grow (pos + BUF_SIZE); + maxPos = pos; + } else if (pos > maxPos) { + maxPos = pos; + } + startOffset[pos] = offsets[i].getStartOffset(); + endOffset[pos] = offsets[i].getEndOffset(); + } + } + + private void grow (int size) { + startOffset = ArrayUtil.grow (startOffset, size); + endOffset = ArrayUtil.grow (endOffset, size); + } + + public int getStartOffset(int pos) { + return startOffset[pos]; + } + + public int getEndOffset(int pos) { + return endOffset[pos]; + } + + public int getMaxPosition() { + return maxPos; + } +} \ No newline at end of file Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionTreeIterator.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionTreeIterator.java (revision 1144713) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/PositionTreeIterator.java (working copy) @@ -1,52 +0,0 @@ -package org.apache.lucene.search.poshighlight; - -import java.io.IOException; - -import org.apache.lucene.search.positions.PositionIntervalIterator; -import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; - -// retrieves the positions from the leaves of a tree of PositionIntervalIterators -public class PositionTreeIterator { - - static class Frame { - Frame (PositionIntervalIterator positions) { - this.positions = positions; - subs = positions.subs(true); - if (subs.length == 0) - subs = null; - isub = (subs != null) ? 0 : -1; - } - PositionIntervalIterator positions; - PositionIntervalIterator subs[]; - int isub; - }; - - Frame stack[] = new Frame[32]; - int curframe = 0; - - public PositionTreeIterator (PositionIntervalIterator root) { - stack[0] = new Frame(root); - } - - public PositionInterval next() throws IOException { - PositionInterval pos; - if (curframe < 0) - return null; - Frame f = stack[curframe]; - if (f.subs == null) { - pos = stack[curframe].positions.next(); - if (pos != null) - return pos; - } - else if (f.isub < f.subs.length) { - if (curframe >= stack.length) { - throw new ArrayIndexOutOfBoundsException ("PositionTreeIterator stack depth > 32"); - } - stack[++curframe] = new Frame (f.subs[f.isub++]); - return next(); - } - // pop - --curframe; - return next(); - } - } Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/ScorePosDoc.java =================================================================== --- lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/ScorePosDoc.java (revision 1144713) +++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/poshighlight/ScorePosDoc.java (working copy) @@ -1,37 +1,55 @@ package org.apache.lucene.search.poshighlight; -import java.io.IOException; +/** + * 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.util.Comparator; import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.positions.PositionIntervalIterator; import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; import org.apache.lucene.util.ArrayUtil; -/** Used to accumulate span positions while scoring */ +/** Used to accumulate position intervals while scoring + * @lucene.experimental + */ public class ScorePosDoc extends ScoreDoc { public int posCount = 0; public PositionInterval[] positions; - public ScorePosDoc(int doc, float score, PositionIntervalIterator posIter, int maxPositions, boolean orderByScore) throws IOException - { - super(doc, score); - assert doc == posIter.docID(); + public ScorePosDoc(int doc) { + super(doc, 0); positions = new PositionInterval[32]; - storePositions (new PositionTreeIterator (posIter), maxPositions, orderByScore); } - private void storePositions(PositionTreeIterator ptree, - int maxPositions, boolean orderByScore) throws IOException { - - for (PositionInterval pos = ptree.next(); pos != null; pos = ptree.next()) { - if (posCount >= positions.length) { - PositionInterval temp[] = new PositionInterval[positions.length * 2]; - System.arraycopy(positions, 0, temp, 0, positions.length); - } - positions[posCount++] = (PositionInterval) pos.clone(); + public void storePosition (PositionInterval pos) { + ensureStorage(); + positions[posCount++] = (PositionInterval) pos.clone(); + } + + private void ensureStorage () { + if (posCount >= positions.length) { + PositionInterval temp[] = new PositionInterval[positions.length * 2]; + System.arraycopy(positions, 0, temp, 0, positions.length); + positions = temp; } + } + + public PositionInterval[] sortedPositions() { ArrayUtil.mergeSort(positions, 0, posCount, new Comparator() { public int compare(PositionInterval o1, PositionInterval o2) { return @@ -43,6 +61,6 @@ } }); + return positions; } - } Index: lucene/contrib/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java =================================================================== --- lucene/contrib/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java (revision 1144713) +++ lucene/contrib/highlighter/src/test/org/apache/lucene/search/poshighlight/PosHighlighterTest.java (working copy) @@ -1,14 +1,11 @@ package org.apache.lucene.search.poshighlight; -import java.io.BufferedReader; -import java.io.FileInputStream; import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockTokenizer; +import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.Field.Index; @@ -21,6 +18,7 @@ import org.apache.lucene.index.codecs.CoreCodecProvider; import org.apache.lucene.queryParser.QueryParser; import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MultiTermQuery; @@ -28,28 +26,21 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.WildcardQuery; -import org.apache.lucene.search.BooleanClause.Occur; -import org.apache.lucene.search.poshighlight.PosCollector; -import org.apache.lucene.search.poshighlight.PosHighlighter; -import org.apache.lucene.search.positions.OrderedConjunctionPositionIterator; -import org.apache.lucene.search.positions.PositionIntervalIterator; -import org.apache.lucene.search.positions.WithinPositionIterator; -import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; -import org.apache.lucene.search.spans.MockSpanQuery; +import org.apache.lucene.search.highlight.Highlighter; +import org.apache.lucene.search.highlight.InvalidTokenOffsetsException; +import org.apache.lucene.search.highlight.SimpleFragmenter; +import org.apache.lucene.search.highlight.TextFragment; +import org.apache.lucene.search.positions.PositionFilterQuery; +import org.apache.lucene.search.positions.TestBlockPositionsIterator.BlockPositionIteratorFilter; import org.apache.lucene.store.Directory; import org.apache.lucene.store.SimpleFSDirectory; import org.apache.lucene.util.LuceneTestCase; -import org.junit.Ignore; +import org.apache.mahout.math.Arrays; + /** - * Notes: to fully implement, we need: - * 1) ability to walk the individual terms that matched, possibly in a hierarchical way - * if we want to implement really clever highlighting? - * 2) some Collector api like the one I made up, and support in Searcher - * 3) All (or more) queries implemented - * - * For hl perf testing we could test term queries only using the current impl - * @author sokolov - * + * TODO: + * Phrase and Span Queries + * positions callback API */ public class PosHighlighterTest extends LuceneTestCase { @@ -57,8 +48,12 @@ protected Analyzer analyzer; protected QueryParser parser; protected Directory dir; - protected IndexSearcher searcher; + protected IndexSearcher searcher; + private static final String PORRIDGE_VERSE = + "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some" + + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!"; + @Override public void setUp() throws Exception { super.setUp(); @@ -99,28 +94,72 @@ searcher = new IndexSearcher( dir, true ); } - private String[] doSearch(Query q) throws IOException { + private String[] doSearch(Query q) throws IOException, InvalidTokenOffsetsException { return doSearch(q, 100); } - private String[] doSearch(Query q, int maxFragSize) throws IOException { - PosHighlighter ph = new PosHighlighter(); - PosCollector collector = new PosCollector (10); + private class ConstantScorer implements org.apache.lucene.search.highlight.Scorer { + + @Override + public TokenStream init(TokenStream tokenStream) throws IOException { + return tokenStream; + } + + @Override + public void startFragment(TextFragment newFragment) { + } + + @Override + public float getTokenScore() { + return 1; + } + + @Override + public float getFragmentScore() { + return 1; + } + } + + private String[] doSearch(Query q, int maxFragSize) throws IOException, InvalidTokenOffsetsException { + return doSearch (q, maxFragSize, 0); + } + private String[] doSearch(Query q, int maxFragSize, int docIndex) throws IOException, InvalidTokenOffsetsException { + // ConstantScorer is a fragment Scorer, not a search result (document) Scorer + Highlighter highlighter = new Highlighter (new ConstantScorer()); + highlighter.setTextFragmenter(new SimpleFragmenter(maxFragSize)); + PosCollector collector = new PosCollector(10); + if (q instanceof MultiTermQuery) { + ((MultiTermQuery)q).setRewriteMethod (MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE); + } searcher.search(q, collector); - return ph.getFirstFragments(collector.docs[0], searcher.getIndexReader(), F, true, 10, maxFragSize); + ScorePosDoc doc = collector.docs[docIndex]; + if (doc == null) + return null; + String text = searcher.getIndexReader().document(doc.doc).getFieldable(F).stringValue(); + PositionOffsetMapper pom = new PositionOffsetMapper (); + // FIXME: test error cases: for non-stored fields, and fields w/no term vectors + searcher.getIndexReader().getTermFreqVector(doc.doc, F, pom); + + TextFragment[] fragTexts = highlighter.getBestTextFragments(new PosTokenStream + (text, new PositionIntervalArrayIterator(doc.sortedPositions(), doc.posCount), pom), + text, false, 10); + String[] frags = new String[fragTexts.length]; + for (int i = 0; i < frags.length; i++) + frags[i] = fragTexts[i].toString(); + return frags; } public void testTerm () throws Exception { - insertDocs(analyzer, "This is a test"); + insertDocs(analyzer, "This is a test test"); String frags[] = doSearch (new TermQuery(new Term(F, "test"))); - assertEquals ("This is a test", frags[0]); + assertEquals ("This is a test test", frags[0]); } public void testSeveralSnippets () throws Exception { String input = "this is some long text. It has the word long in many places. In fact, it has long on some different fragments. " + "Let us see what happens to long in this case."; - String gold = "this is some long text. It has the word long in many places. In fact, it has long on some different fragments. " + - "Let us see what happens to long in this case."; + String gold = "this is some long text. It has the word long in many places. In fact, it has long on some different fragments. " + + "Let us see what happens to long in this case."; insertDocs(analyzer, input); String frags[] = doSearch (new TermQuery(new Term(F, "long")), input.length()); assertEquals (gold, frags[0]); @@ -132,7 +171,7 @@ bq.add(new BooleanClause (new TermQuery(new Term(F, "This")), Occur.MUST)); bq.add(new BooleanClause (new TermQuery(new Term(F, "test")), Occur.MUST)); String frags[] = doSearch (bq); - assertEquals ("This is a test", frags[0]); + assertEquals ("This is a test", frags[0]); } public void testBooleanAndOtherOrder () throws Exception { @@ -141,41 +180,65 @@ bq.add(new BooleanClause (new TermQuery(new Term(F, "test")), Occur.MUST)); bq.add(new BooleanClause (new TermQuery(new Term(F, "This")), Occur.MUST)); String frags[] = doSearch (bq); - assertEquals ("This is a test", frags[0]); + assertEquals ("This is a test", frags[0]); } - + public void testBooleanOr () throws Exception { - // OR queries not implemented yet... insertDocs(analyzer, "This is a test"); BooleanQuery bq = new BooleanQuery(); bq.add(new BooleanClause (new TermQuery(new Term(F, "test")), Occur.SHOULD)); bq.add(new BooleanClause (new TermQuery(new Term(F, "This")), Occur.SHOULD)); String frags[] = doSearch (bq); - assertEquals ("This is a test", frags[0]); + assertEquals ("This is a test", frags[0]); } - @Ignore("not supproted yet") - public void testPhrase() throws Exception { + public void testSingleMatchScorer () throws Exception { insertDocs(analyzer, "This is a test"); BooleanQuery bq = new BooleanQuery(); + bq.add(new BooleanClause (new TermQuery(new Term(F, "test")), Occur.SHOULD)); + bq.add(new BooleanClause (new TermQuery(new Term(F, "notoccurringterm")), Occur.SHOULD)); + String frags[] = doSearch (bq); + assertEquals ("This is a test", frags[0]); + } + + public void testBooleanNrShouldMatch () throws Exception { + insertDocs(analyzer, "a b c d e f g h i"); + BooleanQuery bq = new BooleanQuery(); + bq.add(new BooleanClause (new TermQuery(new Term(F, "a")), Occur.SHOULD)); + bq.add(new BooleanClause (new TermQuery(new Term(F, "b")), Occur.SHOULD)); + bq.add(new BooleanClause (new TermQuery(new Term(F, "no")), Occur.SHOULD)); + + // This generates a ConjunctionSumScorer + bq.setMinimumNumberShouldMatch(2); + String frags[] = doSearch (bq); + assertEquals ("a b c d e f g h i", frags[0]); + + // This generates no scorer + bq.setMinimumNumberShouldMatch(3); + frags = doSearch (bq); + assertNull (frags); + + // This generates a DisjunctionSumScorer + bq.setMinimumNumberShouldMatch(2); + bq.add(new BooleanClause (new TermQuery(new Term(F, "c")), Occur.SHOULD)); + frags = doSearch (bq); + assertEquals ("a b c d e f g h i", frags[0]); + } + + public void testPhrase() throws Exception { + insertDocs(analyzer, "is it that this is a test, is it"); + BooleanQuery bq = new BooleanQuery(); bq.add(new BooleanClause (new TermQuery(new Term(F, "is")), Occur.MUST)); bq.add(new BooleanClause (new TermQuery(new Term(F, "a")), Occur.MUST)); - MockSpanQuery msq = new MockSpanQuery(bq, false, F, new Filter(1)); - String frags[] = doSearch (msq); - assertEquals ("This is a test", frags[0]); + PositionFilterQuery pfq = new PositionFilterQuery(bq, new BlockPositionIteratorFilter()); + String frags[] = doSearch (pfq); + // make sure we highlight the phrase, and not the terms outside the phrase + assertEquals ("is it that this is a test, is it", frags[0]); } - public static class Filter implements PositionIntervalFilter { - private int slop; - public Filter(int slop) { - this.slop = slop; - } - @Override - public PositionIntervalIterator filter(PositionIntervalIterator iter) { - return new WithinPositionIterator(slop, new OrderedConjunctionPositionIterator(iter)); - } - } - @Ignore("not supproted yet") + /* + * Failing ... PhraseQuery scorer needs positions()? + */ public void testPhraseOriginal() throws Exception { insertDocs(analyzer, "This is a test"); PhraseQuery pq = new PhraseQuery(); @@ -183,44 +246,58 @@ pq.add(new Term(F, "test")); String frags[] = doSearch (pq); //searcher.search(new MockSpanQuery(pq, collector.needsPayloads(), F, null), collector); - assertEquals ("This is a test", frags[0]); + assertEquals ("This is a test", frags[0]); } + public void testNestedBoolean () throws Exception { + insertDocs(analyzer, "This is a test"); + BooleanQuery bq = new BooleanQuery(); + bq.add(new BooleanClause (new TermQuery(new Term(F, "test")), Occur.SHOULD)); + BooleanQuery bq2 = new BooleanQuery(); + bq2.add(new BooleanClause (new TermQuery(new Term(F, "This")), Occur.SHOULD)); + bq2.add(new BooleanClause (new TermQuery(new Term(F, "is")), Occur.SHOULD)); + bq.add(new BooleanClause(bq2, Occur.SHOULD)); + String frags[] = doSearch (bq); + assertEquals ("This is a test", frags[0]); + } + public void testWildcard () throws Exception { insertDocs(analyzer, "This is a test"); - WildcardQuery wildcardQuery = new WildcardQuery(new Term(F, "t*t")); - // TODO enable positions in constant scorer - wildcardQuery.setRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE); - String frags[] = doSearch(wildcardQuery); - assertEquals ("This is a test", frags[0]); + String frags[] = doSearch (new WildcardQuery(new Term(F, "t*t"))); + assertEquals ("This is a test", frags[0]); } -// -// @Ignore("file epistolary-novel.xml does not exist") -// public void testLargerDocument() throws Exception { -// InputStream in = new FileInputStream ("epistolary-novel.xml"); -// insertDocs(analyzer, IOUtils.toString(in)); -// in.close(); -// BooleanQuery bq = new BooleanQuery(); -// bq.add(new BooleanClause (new TermQuery(new Term(F, "unknown")), Occur.MUST)); -// bq.add(new BooleanClause (new TermQuery(new Term(F, "artist")), Occur.MUST)); -// String frags[] = doSearch (bq, 50); -// assertEquals ("is a narration by an unknown observer.\n*[[Jean Web", frags[0]); -// assertEquals ("fin and Sabine]]'' by artist [[Nick Bantock]] is a", frags[1]); -// } -// @Ignore("file epistolary-novel.xml does not exist") -// public void testMultipleDocuments() throws Exception { -// InputStream in = new FileInputStream ("epistolary-novel.xml"); -// insertDocs(analyzer, -// "This document has no matches", -// IOUtils.toString(in), -// "This document has an unknown artist match"); -// BooleanQuery bq = new BooleanQuery(); -// bq.add(new BooleanClause (new TermQuery(new Term(F, "unknown")), Occur.MUST)); -// bq.add(new BooleanClause (new TermQuery(new Term(F, "artist")), Occur.MUST)); -// String frags[] = doSearch (bq, 50); -// assertEquals ("is a narration by an unknown observer.\n*[[Jean Web", frags[0]); -// assertEquals ("fin and Sabine]]'' by artist [[Nick Bantock]] is a", frags[1]); -// } + public void testMultipleDocumentsAnd() throws Exception { + insertDocs(analyzer, + "This document has no matches", + PORRIDGE_VERSE, + "This document has some Pease porridge in it"); + BooleanQuery bq = new BooleanQuery(); + bq.add(new BooleanClause (new TermQuery(new Term(F, "Pease")), Occur.MUST)); + bq.add(new BooleanClause (new TermQuery(new Term(F, "porridge")), Occur.MUST)); + String frags[] = doSearch (bq, 50, 0); + assertEquals ("Pease porridge hot! Pease porridge cold! Pease", frags[0]); + frags = doSearch (bq, 50, 1); + assertEquals ("This document has some Pease porridge in it", frags[0]); + } + + /* + * Failing: need positions callback API since DisjunctionSumScorer consumes all of a doc's + * positions before passing the doc to the collector. + */ + public void testMultipleDocumentsOr() throws Exception { + insertDocs(analyzer, + "This document has no matches", + PORRIDGE_VERSE, + "This document has some Pease porridge in it"); + BooleanQuery bq = new BooleanQuery(); + bq.add(new BooleanClause (new TermQuery(new Term(F, "Pease")), Occur.SHOULD)); + bq.add(new BooleanClause (new TermQuery(new Term(F, "porridge")), Occur.SHOULD)); + String frags[] = doSearch (bq, 50, 0); + System.out.println(Arrays.toString(frags)); + assertEquals ("Pease porridge hot! Pease porridge cold! Pease", frags[0]); + frags = doSearch (bq, 50, 1); + assertEquals ("This document has some Pease porridge in it", frags[0]); + } -} \ No newline at end of file +} Index: lucene/src/java/org/apache/lucene/search/BooleanScorer2.java =================================================================== --- lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (working copy) @@ -145,6 +145,11 @@ public int advance(int target) throws IOException { return scorer.advance(target); } + + @Override + public PositionIntervalIterator positions() throws IOException { + return scorer.positions(); + } } private Scorer countingDisjunctionSumScorer(final List scorers, @@ -276,7 +281,7 @@ */ @Override public void score(Collector collector) throws IOException { - collector.setScorer(this); + collector.setScorer(this); while ((doc = countingSumScorer.nextDoc()) != NO_MORE_DOCS) { collector.collect(doc); } @@ -320,8 +325,6 @@ return doc = countingSumScorer.advance(target); } - - @Override public PositionIntervalIterator positions() throws IOException { return countingSumScorer.positions(); Index: lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java (working copy) @@ -32,6 +32,7 @@ private final Scorer[] scorers; private final float coord; private int lastDoc = -1; + private PositionIntervalIterator positions; public ConjunctionScorer(Weight weight, float coord, boolean needsPositions, Collection scorers) throws IOException { this(weight, coord, needsPositions, scorers.toArray(new Scorer[scorers.size()])); @@ -146,13 +147,16 @@ } return sum * coord; } - + @Override public PositionIntervalIterator positions() throws IOException { - if (scorersOrdered == null) - throw new IllegalStateException("no positions requested for this scorer"); - // only created if needed for this scorer - no penalty for non-positional queries - return new ConjunctionPositionIterator(this, scorersOrdered); + if (positions == null) { + if (scorersOrdered == null) + throw new IllegalStateException("no positions requested for this scorer"); + // only created if needed for this scorer - no penalty for non-positional queries + positions = new ConjunctionPositionIterator(this, scorersOrdered); + } + return positions; } } Index: lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (working copy) @@ -20,6 +20,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader.AtomicReaderContext; import org.apache.lucene.index.Term; +import org.apache.lucene.search.positions.PositionIntervalIterator; import org.apache.lucene.util.ToStringUtils; import java.io.IOException; @@ -202,6 +203,7 @@ return docIdSetIterator.advance(target); } + private Collector wrapCollector(final Collector collector) { return new Collector() { @Override @@ -224,6 +226,16 @@ public boolean acceptsDocsOutOfOrder() { return collector.acceptsDocsOutOfOrder(); } + + @Override + public boolean needsPositions() { + return collector.needsPositions(); + } + + @Override + public boolean needsPayloads() { + return collector.needsPayloads(); + } }; } @@ -246,6 +258,15 @@ return super.score(collector, max, firstDocID); } } + + @Override + public PositionIntervalIterator positions() throws IOException { + if (docIdSetIterator instanceof Scorer) { + return ((Scorer) docIdSetIterator).positions(); + } else { + return super.positions(); + } + } } @Override Index: lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (working copy) @@ -20,6 +20,7 @@ import java.util.List; import java.io.IOException; +import org.apache.lucene.search.positions.ConjunctionPositionIterator; import org.apache.lucene.search.positions.DisjunctionPositionIterator; import org.apache.lucene.search.positions.PositionIntervalIterator; import org.apache.lucene.util.ScorerDocQueue; @@ -49,7 +50,7 @@ * nrMatchers is the number of matching scorers, * and all scorers are after the matching doc, or are exhausted. */ - private ScorerDocQueue scorerDocQueue; + private final ScorerDocQueue scorerDocQueue; /** The document number of the current match. */ private int currentDoc = -1; @@ -59,6 +60,8 @@ private float currentScore = Float.NaN; + private PositionIntervalIterator positions = null; + /** Construct a DisjunctionScorer. * @param weight The weight to be used. * @param needsPositions @@ -86,7 +89,7 @@ this.minimumNrMatchers = minimumNrMatchers; this.subScorers = subScorers; - initScorerDocQueue(); + scorerDocQueue = initScorerDocQueue(); } /** Construct a DisjunctionScorer, using one as the minimum number @@ -98,14 +101,16 @@ /** Called the first time nextDoc() or advance() is called to * initialize scorerDocQueue. + * @return */ - private void initScorerDocQueue() throws IOException { - scorerDocQueue = new ScorerDocQueue(nrScorers); + private ScorerDocQueue initScorerDocQueue() throws IOException { + ScorerDocQueue queue = new ScorerDocQueue(nrScorers); for (Scorer se : subScorers) { if (se.nextDoc() != NO_MORE_DOCS) { - scorerDocQueue.insert(se); + queue.insert(se); } } + return queue; } /** Scores and collects all matching documents. @@ -118,7 +123,7 @@ collector.collect(currentDoc); } } - + /** Expert: Collects matching documents in a range. Hook for optimization. * Note that {@link #nextDoc()} must be called once before this method is called * for the first time. @@ -240,9 +245,12 @@ @Override public PositionIntervalIterator positions() throws IOException { - if (minimumNrMatchers > 1) { - throw new IllegalStateException("positions not implemented for minimum matches > 1"); + if (positions == null) { + if (minimumNrMatchers > 1) { + positions = new ConjunctionPositionIterator(this, subScorers.toArray(new Scorer[0]), minimumNrMatchers); + } + positions = new DisjunctionPositionIterator(this, subScorers.toArray(new Scorer[0])); } - return new DisjunctionPositionIterator(this, subScorers.toArray(new Scorer[0])); + return positions; } } Index: lucene/src/java/org/apache/lucene/search/PositionTermScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/PositionTermScorer.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/PositionTermScorer.java (working copy) @@ -1,215 +0,0 @@ -package org.apache.lucene.search; - -/** - * 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.DocsAndPositionsEnum; -import org.apache.lucene.search.Similarity.ExactDocScorer; -import org.apache.lucene.search.positions.PositionIntervalIterator; -import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; -import org.apache.lucene.util.BytesRef; - -/** - * Expert: A Scorer for documents matching a Term. - */ -final class PositionTermScorer extends Scorer { - private final DocsAndPositionsEnum docsEnum; - private int doc = -1; - private int freq; - private TermPositions positions; - private ExactDocScorer docScorer; - - /** - * Construct a TermScorer. - * - * @param weight - * The weight of the Term in the query. - * @param td - * An iterator over the documents matching the Term. - * @param similarity - * The Similarity implementation to be used for score - * computations. - * @param norms - * The field norms of the document fields for the Term. - */ - PositionTermScorer(Weight weight, DocsAndPositionsEnum td, Similarity.ExactDocScorer docScorer, boolean doPayloads) { - super(weight); - this.positions = new TermPositions(td, doPayloads); - this.docsEnum = td; - this.docScorer = docScorer; - } - - @Override - public void score(Collector c) throws IOException { - score(c, Integer.MAX_VALUE, nextDoc()); - } - - // firstDocID is ignored since nextDoc() sets 'doc' - @Override - public boolean score(Collector c, int end, int firstDocID) throws IOException { - c.setScorer(this); - while (doc < end) { // for docs in window - c.collect(doc); // collect score - doc = docsEnum.nextDoc(); - if (doc != NO_MORE_DOCS) { - freq = docsEnum.freq(); - } - } - return true; - } - - @Override - public int docID() { - return doc; - } - - @Override - public float freq() { - return freq; - } - - /** - * Advances to the next document matching the query.
- * The iterator over the matching documents is buffered using - * {@link TermDocs#read(int[],int[])}. - * - * @return the document matching the query or NO_MORE_DOCS if there are no - * more documents. - */ - @Override - public int nextDoc() throws IOException { - - doc = docsEnum.nextDoc(); - if (doc != NO_MORE_DOCS) { - positions.positionsPending = freq = docsEnum.freq(); - positions.interval.reset(); - } - return doc; - } - - @Override - public float score() { - assert doc != -1; - return docScorer.score(doc, freq); - } - - /** - * Advances to the first match beyond the current whose document number is - * greater than or equal to a given target.
- * The implementation uses {@link TermDocs#skipTo(int)}. - * - * @param target - * The target document number. - * @return the matching document or NO_MORE_DOCS if none exist. - */ - @Override - public int advance(int target) throws IOException { - doc = docsEnum.advance(target); - if (doc != NO_MORE_DOCS) { - positions.positionsPending = freq = docsEnum.freq(); - positions.interval.reset(); - } - return doc; - } - - /** Returns a string representation of this TermScorer. */ - @Override - public String toString() { - return "scorer(" + weight + ")"; - } - - @Override - public PositionIntervalIterator positions() throws IOException { - return positions; - } - - @SuppressWarnings("serial") - private final class TermPositions extends PositionIntervalIterator { - private final PositionInterval interval; - int positionsPending; - private final DocsAndPositionsEnum docsAndPos; - - public TermPositions(DocsAndPositionsEnum docsAndPos, boolean doPayloads) { - super(PositionTermScorer.this); - this.docsAndPos = docsAndPos; - this.interval = doPayloads ? new PayloadPosInterval(docsAndPos, this) - : new PositionInterval(); - - } - - @Override - public PositionInterval next() throws IOException { - if (--positionsPending >= 0) { - interval.begin = interval.end = docsAndPos.nextPosition(); - return interval; - } - interval.reset(); - positionsPending = 0; - return null; - } - - @Override - public int docID() { - return doc; - } - - @Override - public PositionIntervalIterator[] subs(boolean inOrder) { - return EMPTY; - } - } - - private static final class PayloadPosInterval extends PositionInterval { - private int pos = -1; - private final DocsAndPositionsEnum payloads; - private final TermPositions termPos; - - public PayloadPosInterval(DocsAndPositionsEnum payloads, TermPositions pos) { - this.payloads = payloads; - this.termPos = pos; - } - - @Override - public boolean payloadAvailable() { - return payloads.hasPayload(); - } - - @Override - public boolean nextPayload(BytesRef ref) throws IOException { - if (pos == termPos.positionsPending) { - return false; - } else { - pos = termPos.positionsPending; - final BytesRef payload = payloads.getPayload(); - ref.bytes = payload.bytes; - ref.length = payload.length; - ref.offset = payload.offset; - return true; - } - } - - @Override - public void reset() { - super.reset(); - pos = -1; - } - - } - -} Index: lucene/src/java/org/apache/lucene/search/Scorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/Scorer.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/Scorer.java (working copy) @@ -142,8 +142,8 @@ * @param scorer the current scorer */ public void visitProhibited(P parent, C child, S scorer) {} - } - + } + /** * Expert: call this to gather details for all sub-scorers for this query. * This can be used, in conjunction with a custom {@link Collector} to gather Index: lucene/src/java/org/apache/lucene/search/TermQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/TermQuery.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/TermQuery.java (working copy) @@ -81,15 +81,16 @@ assert termNotInReader(reader, field, term.bytes()) : "no termstate found but term exists in reader"; return null; } + final DocsEnum docs = reader.termDocsEnum(reader.getLiveDocs(), field, + term.bytes(), state); if (scorerContext.needsPositions) { - final DocsAndPositionsEnum docs = reader.termPositionsEnum( + final DocsAndPositionsEnum docsAndPos = reader.termPositionsEnum( reader.getLiveDocs(), field, term.bytes(), state); assert docs != null; - return new PositionTermScorer(this, docs, similarity.exactDocScorer( - stats, field, context), scorerContext.needsPayloads); + assert docsAndPos != null; + return new TermScorer(this, docs, docsAndPos, scorerContext.needsPayloads, similarity.exactDocScorer( + stats, field, context)); } else { - final DocsEnum docs = reader.termDocsEnum(reader.getLiveDocs(), field, - term.bytes(), state); assert docs != null; return new TermScorer(this, docs, similarity.exactDocScorer(stats, field, context)); Index: lucene/src/java/org/apache/lucene/search/TermScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/TermScorer.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/TermScorer.java (working copy) @@ -19,7 +19,11 @@ import java.io.IOException; +import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.search.positions.PositionIntervalIterator; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; +import org.apache.lucene.util.BytesRef; /** Expert: A Scorer for documents matching a Term. */ @@ -35,6 +39,7 @@ private int[] freqs; private final DocsEnum.BulkReadResult bulkResult; private final Similarity.ExactDocScorer docScorer; + private final TermPositions positions; /** * Construct a TermScorer. @@ -48,10 +53,15 @@ * to be used for score computations. */ TermScorer(Weight weight, DocsEnum td, Similarity.ExactDocScorer docScorer) throws IOException { + this(weight, td, null, false, docScorer); + } + + TermScorer(Weight weight, DocsEnum td, DocsAndPositionsEnum docsAndPos, boolean doPayloads, Similarity.ExactDocScorer docScorer) throws IOException { super(weight); this.docScorer = docScorer; this.docsEnum = td; bulkResult = td.getBulkResult(); + positions = docsAndPos != null ? new TermPositions(docsAndPos, doPayloads) : null; } @Override @@ -160,5 +170,96 @@ /** Returns a string representation of this TermScorer. */ @Override public String toString() { return "scorer(" + weight + ")"; } + + @Override + public PositionIntervalIterator positions() throws IOException { + assert positions != null; + return positions; + } + @SuppressWarnings("serial") + private final class TermPositions extends PositionIntervalIterator { + private final PositionInterval interval; + int positionsPending; + private final DocsAndPositionsEnum docsAndPos; + + public TermPositions(DocsAndPositionsEnum docsAndPos, boolean doPayloads) { + super(TermScorer.this); + this.docsAndPos = docsAndPos; + this.interval = doPayloads ? new PayloadPosInterval(docsAndPos, this) + : new PositionInterval(); + } + + @Override + public PositionInterval next() throws IOException { + if (--positionsPending >= 0) { + interval.begin = interval.end = docsAndPos.nextPosition(); + return interval; + } + positionsPending = 0; + return null; + } + + @Override + public int docID() { + return doc; + } + + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return EMPTY; + } + + @Override + public void collect() { + collector.collect(scorer, interval, doc); + } + + @Override + public int advanceTo(int docId) throws IOException { + int advance = docsAndPos.advance(docId); + if (advance != NO_MORE_DOCS) { + positionsPending = freq = docsAndPos.freq(); + } + interval.reset(); + return advance; + } + } + + private static final class PayloadPosInterval extends PositionInterval { + private int pos = -1; + private final DocsAndPositionsEnum payloads; + private final TermPositions termPos; + + public PayloadPosInterval(DocsAndPositionsEnum payloads, TermPositions pos) { + this.payloads = payloads; + this.termPos = pos; + } + + @Override + public boolean payloadAvailable() { + return payloads.hasPayload(); + } + + @Override + public boolean nextPayload(BytesRef ref) throws IOException { + if (pos == termPos.positionsPending) { + return false; + } else { + pos = termPos.positionsPending; + final BytesRef payload = payloads.getPayload(); + ref.bytes = payload.bytes; + ref.length = payload.length; + ref.offset = payload.offset; + return true; + } + } + + @Override + public void reset() { + super.reset(); + pos = -1; + } + + } } Index: lucene/src/java/org/apache/lucene/search/positions/BlockPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/BlockPositionIterator.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/BlockPositionIterator.java (working copy) @@ -17,12 +17,14 @@ * limitations under the License. */ import java.io.IOException; -import java.util.Arrays; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; +@SuppressWarnings("serial") public class BlockPositionIterator extends PositionIntervalIterator { private final PositionIntervalIterator[] iterators; + private static final PositionInterval INFINITE_INTERVAL = new PositionInterval( Integer.MIN_VALUE, Integer.MIN_VALUE); private final PositionInterval[] intervals; @@ -56,11 +58,6 @@ @Override public PositionInterval next() throws IOException { - int currentDocID = scorer.docID(); - if (currentDocID != docID) { - Arrays.fill(intervals, INFINITE_INTERVAL); - docID = currentDocID; - } if (advance()) { interval.begin = intervals[0].begin; interval.end = intervals[lastIter].end; @@ -100,4 +97,41 @@ public PositionIntervalIterator[] subs(boolean inOrder) { return iterators; } + + @Override + public void collect() { + compositeCollector.collect(scorer, interval, docID); + for (PositionIntervalIterator iter : iterators) { + iter.collect(); + } + } + + @Override + public void setPositionCollector(PositionCollector collector) { + super.setPositionCollector(collector); + for (PositionIntervalIterator iter : iterators) { + iter.setPositionCollector(collector); + } + } + + public void setCompositePositionCollector(PositionCollector collector) { + super.setCompositePositionCollector(collector); + for (PositionIntervalIterator iter : iterators) { + iter.setCompositePositionCollector(collector); + } + } + + + @Override + public int advanceTo(int docId) throws IOException { + if (docId == docID) { + return docId; + } + for (int i = 0; i < iterators.length; i++) { + int advancedTo = iterators[i].advanceTo(docId); + intervals[i] = INFINITE_INTERVAL; + this.docID = advancedTo; + } + return docId; + } } Index: lucene/src/java/org/apache/lucene/search/positions/BooleanPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/BooleanPositionIterator.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/BooleanPositionIterator.java (working copy) @@ -3,10 +3,28 @@ import java.io.IOException; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; @SuppressWarnings("serial") abstract class BooleanPositionIterator extends PositionIntervalIterator { + @Override + public void setPositionCollector(PositionCollector collector) { + super.setPositionCollector(collector); + for (PositionIntervalIterator iter : iterators) { + iter.setPositionCollector(collector); + } + } + + public void setCompositePositionCollector(PositionCollector collector) { + super.setCompositePositionCollector(collector); + for (PositionIntervalIterator iter : iterators) { + iter.setCompositePositionCollector(collector); + } + } + + + protected int docId = -1; protected final PositionIntervalIterator[] iterators; protected final IntervalQueue queue; Index: lucene/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/ConjunctionPositionIterator.java (working copy) @@ -22,7 +22,7 @@ /** * ConjuctionPositionIterator based on minimal interval semantics for AND - * operator + * operator. * * 0 && queue.topContainsQueueInterval()) { advance(); } @@ -75,4 +64,29 @@ return iterators; } + @Override + public void collect() { + compositeCollector.collect(scorer, queue.queueInterval, docId); + collector.collect(scorer, queue.queueInterval, docId); + } + + @Override + public int advanceTo(int docId) throws IOException { + //this.docId = docId; + this.docId = Integer.MAX_VALUE; + queue.clear(); + for (int i = 0; i < iterators.length; i++) { + int advanceTo = iterators[i].advanceTo(docId); + if (advanceTo == docId) { + final PositionInterval interval = iterators[i].next(); + assert interval != null; + queue.add(new IntervalRef(interval, i)); + this.docId = docId; + } else if (advanceTo < this.docId) { + this.docId = advanceTo; + } + } + return this.docId; + } + } \ No newline at end of file Index: lucene/src/java/org/apache/lucene/search/positions/IntervalQueue.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/IntervalQueue.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/IntervalQueue.java (working copy) @@ -1,6 +1,7 @@ package org.apache.lucene.search.positions; import org.apache.lucene.search.positions.IntervalQueue.IntervalRef; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; import org.apache.lucene.util.PriorityQueue; @@ -30,4 +31,5 @@ this.index = index; } } + } \ No newline at end of file Index: lucene/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/IntervalQueueAnd.java (working copy) @@ -1,4 +1,20 @@ 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 + * 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 org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; @@ -39,5 +55,4 @@ final PositionInterval b = right.interval; return a.begin < b.begin || (a.begin == b.begin && a.end >= b.end); } - } Index: lucene/src/java/org/apache/lucene/search/positions/IntervalQueueOr.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/IntervalQueueOr.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/IntervalQueueOr.java (working copy) @@ -1,5 +1,6 @@ package org.apache.lucene.search.positions; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; final class IntervalQueueOr extends IntervalQueue { @@ -30,4 +31,5 @@ final PositionInterval b = right.interval; return a.end < b.end|| (a.end == b.end && a.begin >= b.begin); } + } Index: lucene/src/java/org/apache/lucene/search/positions/OrderedConjunctionPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/OrderedConjunctionPositionIterator.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/OrderedConjunctionPositionIterator.java (working copy) @@ -17,8 +17,13 @@ * limitations under the License. */ import java.io.IOException; -import java.util.Arrays; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; + +/** + * @lucene.experimental + */ +@SuppressWarnings("serial") public final class OrderedConjunctionPositionIterator extends PositionIntervalIterator { @@ -43,15 +48,8 @@ @Override public PositionInterval next() throws IOException { - final int currentDocId = scorer.docID(); - if (docId != currentDocId) { - docId = currentDocId; - // TODO maybe use null instead? - Arrays.fill(intervals, 1, intervals.length, INFINITE_INTERVAL); - intervals[0] = iterators[0].next(); - index = 1; - } + if(intervals[0] == null) { return null; } @@ -95,4 +93,39 @@ return iterators; } + @Override + public void collect() { + compositeCollector.collect(scorer, interval, docId); + for (PositionIntervalIterator iter : iterators) { + iter.collect(); + } + } + public void setPositionCollector(PositionCollector collector) { + super.setPositionCollector(collector); + for (PositionIntervalIterator iter : iterators) { + iter.setPositionCollector(collector); + } + } + public void setCompositePositionCollector(PositionCollector collector) { + super.setCompositePositionCollector(collector); + for (PositionIntervalIterator iter : iterators) { + iter.setCompositePositionCollector(collector); + } + } + + @Override + public int advanceTo(int docId) throws IOException { + if (docId == this.docId) { + return docId; + } + for (int i = 0; i < iterators.length; i++) { + int advanceTo = iterators[i].advanceTo(docId); + assert advanceTo == docId; + intervals[i] = INFINITE_INTERVAL; + } + intervals[0] = iterators[0].next(); + index = 1; + return docId; + } + } Index: lucene/src/java/org/apache/lucene/search/positions/PositionFilterQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/PositionFilterQuery.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/PositionFilterQuery.java (working copy) @@ -28,7 +28,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; -import org.apache.lucene.search.Weight.ScorerContext; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; /** @@ -115,12 +115,14 @@ private final Scorer other; private PositionIntervalIterator filter; + private final FilteredPositions filtered; public PositionFilterScorer(Weight weight, Scorer other) throws IOException { super(weight); this.other = other; this.filter = PositionFilterQuery.this.filter != null ? PositionFilterQuery.this.filter.filter(other.positions()) : other.positions(); + filtered = new FilteredPositions(this, filter); } @Override @@ -130,7 +132,7 @@ @Override public PositionIntervalIterator positions() throws IOException { - return filter; + return filtered; } @Override @@ -140,8 +142,10 @@ @Override public int nextDoc() throws IOException { - while (other.nextDoc() != Scorer.NO_MORE_DOCS) { - if (filter.next() != null) { // just check if there is a position that matches! + int docId = -1; + while ((docId = other.nextDoc()) != Scorer.NO_MORE_DOCS) { + filter.advanceTo(docId); + if ((filtered.interval = filter.next()) != null) { // just check if there is a position that matches! return other.docID(); } } @@ -150,14 +154,16 @@ @Override public int advance(int target) throws IOException { - int advance = other.advance(target); - if (advance == Scorer.NO_MORE_DOCS) + int docId = other.advance(target); + if (docId == Scorer.NO_MORE_DOCS) { return NO_MORE_DOCS; + } do { - if (filter.next() != null) { + filter.advanceTo(docId); + if ((filtered.interval = filter.next()) != null) { return other.docID(); } - } while (other.nextDoc() != Scorer.NO_MORE_DOCS); + } while ((docId = other.nextDoc()) != Scorer.NO_MORE_DOCS); return NO_MORE_DOCS; } @@ -167,5 +173,45 @@ public String toString(String field) { return inner.toString(); } + + @SuppressWarnings("serial") + private final class FilteredPositions extends PositionIntervalIterator { + public FilteredPositions(Scorer scorer, PositionIntervalIterator other) { + super(scorer); + this.other = other; + } + private final PositionIntervalIterator other; + PositionInterval interval; + @Override + public int advanceTo(int docId) throws IOException { + return other.docID(); + } + + @Override + public PositionInterval next() throws IOException { + PositionInterval current = this.interval; + this.interval = null; + return current; + } + + @Override + public void collect() { + other.collect(); + + } + @Override + public PositionIntervalIterator[] subs(boolean inOrder) { + return null; + } + + public void setPositionCollector(PositionCollector collector) { + other.setPositionCollector(collector); + } + public void setCompositePositionCollector(PositionCollector collector) { + other.setCompositePositionCollector(collector); + } + + } + } \ No newline at end of file Index: lucene/src/java/org/apache/lucene/search/positions/PositionIntervalIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/PositionIntervalIterator.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/PositionIntervalIterator.java (working copy) @@ -27,20 +27,48 @@ * TODO add documentation */ @SuppressWarnings("serial") -public abstract class PositionIntervalIterator implements Serializable{ +public abstract class PositionIntervalIterator implements Serializable { public static final PositionIntervalIterator[] EMPTY = new PositionIntervalIterator[0]; protected final Scorer scorer; - + protected PositionCollector collector = new PositionCollector() { + + @Override + public void collect(Scorer scorer, PositionInterval interval, int docID) { + } + + }; + + protected PositionCollector compositeCollector = new PositionCollector() { + + @Override + public void collect(Scorer scorer, PositionInterval interval, int docID) { + } + + }; + public PositionIntervalIterator(Scorer scorer) { this.scorer = scorer; } + public abstract int advanceTo(int docId) throws IOException; + public abstract PositionInterval next() throws IOException; + public void setPositionCollector(PositionCollector collector) { + this.collector = collector; + } + + public void setCompositePositionCollector(PositionCollector collector) { + this.compositeCollector = collector; + } + + public abstract void collect(); + public abstract PositionIntervalIterator[] subs(boolean inOrder); public int docID() { + // nocommit this is not necessarily correct we need to make this abstract return scorer.docID(); } @@ -54,7 +82,6 @@ } public static class PositionInterval implements Cloneable { - public int begin; public int end; @@ -77,9 +104,9 @@ } public void reset() { - begin = end = 0; + begin = end = -1; } - + @Override public Object clone() { try { @@ -88,11 +115,16 @@ throw new RuntimeException(); // should not happen } } - + @Override public String toString() { return "PositionInterval [begin=" + begin + ", end=" + end + "]"; } } + + public static interface PositionCollector { + public void collect(Scorer scorer, PositionInterval interval, int docID); + } + } Index: lucene/src/java/org/apache/lucene/search/positions/RangePositionsIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/RangePositionsIterator.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/RangePositionsIterator.java (working copy) @@ -16,15 +16,32 @@ * limitations under the License. */ - import java.io.IOException; +import org.apache.lucene.search.positions.PositionIntervalIterator.PositionCollector; import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; - +/** + * + * @lucene.experimental + * + */ +@SuppressWarnings("serial") public class RangePositionsIterator extends PositionIntervalIterator implements PositionIntervalFilter { + @Override + public void setPositionCollector(PositionCollector collector) { + super.setPositionCollector(collector); + iterator.setPositionCollector(collector); + } + + public void setCompositePositionCollector(PositionCollector collector) { + super.setCompositePositionCollector(collector); + iterator.setCompositePositionCollector(collector); + } + private final PositionIntervalIterator iterator; + private PositionInterval interval; private int start; private int end; @@ -45,7 +62,6 @@ @Override public PositionInterval next() throws IOException { - PositionInterval interval = null; while ((interval = iterator.next()) != null) { if(interval.end > end) { return null; @@ -60,6 +76,17 @@ public PositionIntervalIterator[] subs(boolean inOrder) { return new PositionIntervalIterator[] { iterator }; } + + @Override + public void collect() { + compositeCollector.collect(null, interval, iterator.docID()); + iterator.collect(); + } + + @Override + public int advanceTo(int docId) throws IOException { + return iterator.advanceTo(docId); + } Index: lucene/src/java/org/apache/lucene/search/positions/WithinPositionIterator.java =================================================================== --- lucene/src/java/org/apache/lucene/search/positions/WithinPositionIterator.java (revision 1144713) +++ lucene/src/java/org/apache/lucene/search/positions/WithinPositionIterator.java (working copy) @@ -1,14 +1,34 @@ 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 + * 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.search.Scorer; import org.apache.lucene.search.positions.PositionIntervalIterator.PositionIntervalFilter; +/** + * @lucene.experimental + */ +@SuppressWarnings("serial") public class WithinPositionIterator extends PositionIntervalIterator implements PositionIntervalFilter { private int howMany; private PositionIntervalIterator iterator; + private PositionInterval interval; + public WithinPositionIterator(int howMany, PositionIntervalIterator iterator) { super(iterator != null ? iterator.scorer : null); this.howMany = howMany; @@ -20,7 +40,6 @@ } @Override public PositionInterval next() throws IOException { - PositionInterval interval = null; while ((interval = iterator.next()) != null) { if((interval.end - interval.begin) <= howMany){ return interval; @@ -38,4 +57,26 @@ return new WithinPositionIterator(howMany, iter); } + @Override + public void collect() { + compositeCollector.collect(null, interval, iterator.docID()); + iterator.collect(); + } + + @Override + public void setPositionCollector(PositionCollector collector) { + super.setPositionCollector(collector); + iterator.setPositionCollector(collector); + } + + public void setCompositePositionCollector(PositionCollector collector) { + super.setCompositePositionCollector(collector); + iterator.setCompositePositionCollector(collector); + } + + @Override + public int advanceTo(int docId) throws IOException { + return iterator.advanceTo(docId); + } + } Index: lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java (revision 1144961) +++ lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java (working copy) @@ -19,8 +19,6 @@ import java.io.IOException; -import org.apache.lucene.search.Explanation; -import org.apache.lucene.search.TFIDFSimilarity; import org.apache.lucene.search.Weight; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Similarity; Index: lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java =================================================================== --- lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java (revision 1144961) +++ lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java (working copy) @@ -40,17 +40,18 @@ public boolean next() throws IOException { if (doc == -1) { doc = scorer.nextDoc(); + positions.advanceTo(doc); } if (doc == Scorer.NO_MORE_DOCS) { return false; } - if ((current = positions.next()) == null) { doc = scorer.nextDoc(); if (doc == Scorer.NO_MORE_DOCS) { return false; } + positions.advanceTo(doc); return (current = positions.next()) != null; } return true; @@ -62,6 +63,7 @@ if (doc == Scorer.NO_MORE_DOCS) { return false; } + positions.advanceTo(doc); return (current = positions.next()) != null; } Index: lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java (revision 1144713) +++ lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java (working copy) @@ -33,6 +33,10 @@ import org.apache.lucene.search.positions.PositionFilterQuery; import org.apache.lucene.search.positions.RangePositionsIterator; import org.apache.lucene.search.positions.WithinPositionIterator; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.positions.PositionFilterQuery; +import org.apache.lucene.search.positions.RangePositionsIterator; +import org.apache.lucene.search.positions.WithinPositionIterator; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.NamedThreadFactory; Index: lucene/src/test/org/apache/lucene/search/TestTermQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestTermQuery.java (revision 1144713) +++ lucene/src/test/org/apache/lucene/search/TestTermQuery.java (working copy) @@ -22,16 +22,15 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.MockAnalyzer; -import org.apache.lucene.analysis.MockPayloadAnalyzer; import org.apache.lucene.analysis.MockTokenFilter; import org.apache.lucene.analysis.MockTokenizer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; -import org.apache.lucene.index.IndexReader.AtomicReaderContext; -import org.apache.lucene.index.IndexReader.ReaderContext; import org.apache.lucene.search.Weight.ScorerContext; import org.apache.lucene.search.positions.PositionIntervalIterator; import org.apache.lucene.search.positions.PositionIntervalIterator.PositionInterval; @@ -86,6 +85,8 @@ PositionIntervalIterator positions = scorer.positions(); do { + assertEquals(scorer.docID(), positions.advanceTo(scorer.docID())); + PositionInterval interval = null; String msg = "Advanced to: " + advance + " current doc: " + scorer.docID() + " usePayloads: " + usePayload; @@ -115,7 +116,6 @@ checkPayload(30, interval); assertNull(msg, (interval = positions.next())); - } while (scorer.nextDoc() != Scorer.NO_MORE_DOCS); } } @@ -205,7 +205,7 @@ final int howMany = random.nextInt(20) == 0 ? pos.length - random.nextInt(pos.length) : pos.length; PositionInterval interval = null; - + assertEquals(scorer.docID(), positions.advanceTo(scorer.docID())); for (int j = 0; j < howMany; j++) { assertNotNull((interval = positions.next())); assertEquals("iteration: " + i + " initDoc: " + initDoc + " doc: " @@ -282,6 +282,7 @@ + usePayload; PositionIntervalIterator positions = scorer.positions(); assertEquals(howMany / 2.f, positions.getScorer().freq(), 0.0); + assertEquals(scorer.docID(), positions.advanceTo(scorer.docID())); for (int j = 0; j < howMany; j += 2) { assertNotNull("next returned nullat index: " + j + " with freq: " + positions.getScorer().freq() + " -- " + msg,(interval = positions.next())); Index: lucene/src/test/org/apache/lucene/search/positions/TestBlockPositionsIterator.java =================================================================== --- lucene/src/test/org/apache/lucene/search/positions/TestBlockPositionsIterator.java (revision 1144713) +++ lucene/src/test/org/apache/lucene/search/positions/TestBlockPositionsIterator.java (working copy) @@ -134,6 +134,7 @@ int nextDoc = scorer.nextDoc(); assertEquals(0, nextDoc); PositionIntervalIterator positions = new BlockPositionIterator(scorer.positions()); + assertEquals(0, positions.advanceTo(0)); PositionInterval interval = null; int[] start = new int[] {0, 31}; int[] end = new int[] {2, 33}; @@ -152,6 +153,7 @@ int nextDoc = scorer.nextDoc(); assertEquals(1, nextDoc); PositionIntervalIterator positions = new BlockPositionIterator(scorer.positions()); + assertEquals(1, positions.advanceTo(1)); PositionInterval interval = null; int[] start = new int[] {3, 34}; int[] end = new int[] {5, 36}; Index: lucene/src/test/org/apache/lucene/search/positions/TestConjunctionPositionsIterator.java =================================================================== --- lucene/src/test/org/apache/lucene/search/positions/TestConjunctionPositionsIterator.java (revision 1144713) +++ lucene/src/test/org/apache/lucene/search/positions/TestConjunctionPositionsIterator.java (working copy) @@ -121,6 +121,7 @@ int nextDoc = scorer.nextDoc(); assertEquals(0, nextDoc); PositionIntervalIterator positions = scorer.positions(); + assertEquals(0, positions.advanceTo(nextDoc)); PositionInterval interval = null; int[] start = new int[] {0, 1, 2, 3, 4, 6, 7, 31, 32, 33}; int[] end = new int[] {2, 3, 4, 33, 33, 33, 33, 33, 34, 35}; @@ -129,7 +130,7 @@ // like it cold, Some like it in the pot nine days old! {7}Pease {8}porridge {9}hot!{3,4,5,6,7} Pease{8} porridge{9} cold!", for (int j = 0; j < end.length; j++) { interval = positions.next(); - assertNotNull(interval); + assertNotNull("" + j, interval); assertEquals(start[j], interval.begin); assertEquals(end[j], interval.end); } @@ -139,6 +140,7 @@ int nextDoc = scorer.nextDoc(); assertEquals(1, nextDoc); PositionIntervalIterator positions = scorer.positions(); + assertEquals(1, positions.advanceTo(nextDoc)); PositionInterval interval = null; int[] start = new int[] {0, 1, 3, 4, 5, 6, 7, 31, 32, 34 }; int[] end = new int[] {5, 5, 5, 6, 7, 36, 36, 36, 36, 36 }; Index: lucene/src/test/org/apache/lucene/search/positions/TestOrderedConjunctionPositionsIterator.java =================================================================== --- lucene/src/test/org/apache/lucene/search/positions/TestOrderedConjunctionPositionsIterator.java (revision 1144713) +++ lucene/src/test/org/apache/lucene/search/positions/TestOrderedConjunctionPositionsIterator.java (working copy) @@ -126,6 +126,7 @@ int nextDoc = scorer.nextDoc(); assertEquals(0, nextDoc); PositionIntervalIterator positions = new OrderedConjunctionPositionIterator(scorer.positions()); + assertEquals(0, positions.advanceTo(nextDoc)); PositionInterval interval = null; int[] start = new int[] {0, 31}; int[] end = new int[] {2, 33}; @@ -146,6 +147,7 @@ int nextDoc = scorer.nextDoc(); assertEquals(1, nextDoc); PositionIntervalIterator positions = new OrderedConjunctionPositionIterator(scorer.positions()); + assertEquals(1, positions.advanceTo(nextDoc)); PositionInterval interval = null; int[] start = new int[] {3, 34}; int[] end = new int[] {5, 36}; Index: solr/src/java/org/apache/solr/analysis/XmlCharFilter.java =================================================================== --- solr/src/java/org/apache/solr/analysis/XmlCharFilter.java (revision 0) +++ solr/src/java/org/apache/solr/analysis/XmlCharFilter.java (revision 0) @@ -0,0 +1,377 @@ +package org.apache.solr.analysis; + +import static javax.xml.stream.XMLStreamConstants.CDATA; +import static javax.xml.stream.XMLStreamConstants.CHARACTERS; +import static javax.xml.stream.XMLStreamConstants.COMMENT; +import static javax.xml.stream.XMLStreamConstants.END_ELEMENT; +import static javax.xml.stream.XMLStreamConstants.ENTITY_REFERENCE; +import static javax.xml.stream.XMLStreamConstants.PROCESSING_INSTRUCTION; +import static javax.xml.stream.XMLStreamConstants.SPACE; +import static javax.xml.stream.XMLStreamConstants.START_ELEMENT; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +import javax.xml.stream.XMLInputFactory; +import javax.xml.stream.XMLResolver; +import javax.xml.stream.XMLStreamException; + +import org.apache.lucene.analysis.CharStream; +import org.apache.lucene.analysis.charfilter.BaseCharFilter; +import org.codehaus.stax2.XMLInputFactory2; +import org.codehaus.stax2.XMLStreamReader2; + +/** + * Uses a StAX (XML pull-style) parser to retrieve a subset of + * characters from an XML character Reader. The basic usage retrieves + * all text from the input stream, preserving character offsets into the + * original XML stream. + * + *

Why would you want to use this rather than + * org.apache.lucene.analysis.charfilter.HTMLStripCharFilter? + *

+ *

For one thing, it supports a few standard XML features that the HTML filter doesn't: + * CDATA + * XML entity substitution (internal or external via DTD) + *

+ * + *

But probably the best additional feature here is the ability to exclude and include + * the contents of various elements (in a namespace-aware way). So for example, if you only + * want to search the content of headers, or only the body of a document (ignoring metadata tags), + * but you would like to be able to retrieve the entire original document when highlighting, this + * will be helpful. See {@link XmlCharFilterFactory} for configuration docs. + *

+ * + *

Note: DTDs referenced in XML DOCTYPE declarations are looked for in the solr/conf/dtd folder + * regardless of what their full path in the XML may be. This is a useful, but fairly rigid mechanism that + * could be extended in the future to allow external configuration using a catalog, say. + * Dependent files referenced by the DTD are looked for relative to that folder: their paths are retained. This makes it + * possible to manage DTDs bundled with dependent modules in subdirectories. + *

+ * + ** BUGS: the character set specified in XML declaration is ignored by the parser, but we can't fix this + * here since the input is a character stream, not a byte stream. + * + */ +public class XmlCharFilter extends BaseCharFilter { + + private static final int CDATA_LENGTH = " includedElements; + private ArrayList including; // whether text is indexed or not + private HashMap qnameTable; + + /** + * Creates a new filter using a + * + * @param reader + * the stream with incoming text + * @param inputFactory + * a StaX XML parser factory + */ + public XmlCharFilter(CharStream reader, XMLInputFactory2 inputFactory) { + super(reader); + try { + xmlReader = (XMLStreamReader2) inputFactory.createXMLStreamReader(reader); + xmlReader.setProperty(XMLInputFactory.RESOLVER, new Resolver()); + } catch (XMLStreamException e) { + e.printStackTrace(); + xmlReader = null; + } + } + + /** + * Creates a new filter using a + * + * @param reader the stream with incoming text + * @param inputFactory a StaX XML parser factory + * @param includes a list of XML element names to include + * @param excludes a list of XML element names to exclude + */ + public XmlCharFilter(CharStream reader, XMLInputFactory2 inputFactory, + String[] includes, String[] excludes) { + this(reader, inputFactory); + if (includes != null) + addIncludes(includes, true); + if (excludes != null) + addIncludes(excludes, false); + if (includedElements != null) { + including = new ArrayList(); + Boolean includeRoot = includedElements.get("/"); + if (includeRoot == null) { + // default to include root if no included elements are specified, + // and *not* to include root otherwise + includeRoot = (includes == null); + } + if (!includeRoot) + including.add(false); + qnameTable = new HashMap(); + } + } + + @Override + public void close() throws IOException { + try { + if (xmlReader != null) + xmlReader.close(); + } catch (XMLStreamException e) { + throw new IOException("error closing xml stream"); + } + } + + @Override + public int read(char[] cbuf, int off, int len) throws IOException { + while (xmlReader != null) { + try { + int nread = 0; + int event = xmlReader.getEventType(); + if (event == ENTITY_REFERENCE) { + // ??? + String text = xmlReader.getText(); + int max = cbuf.length - off; + nread = text.length() - xmlReaderOffset; + if (nread > len) + nread = len; + if (nread > max) + nread = max; + if (nread > 0) { + System.arraycopy(text.toCharArray(), 0, cbuf, off, nread); + } + } else if (event == CHARACTERS || event == CDATA || event == SPACE) { + if (isIncluding()) + nread = xmlReader + .getTextCharacters(xmlReaderOffset, cbuf, off, len); + } else if (xmlReaderOffset == -1) { + // insert whitespace for other tags to avoid including + // them as part of a word token + cbuf[off] = ' '; + ++totalOut; + xmlReaderOffset = 0; + return 1; + } + if (nread > 0) { + xmlReaderOffset += nread; + totalOut += nread; + return nread; + } + if (!next()) + return -1; + } catch (XMLStreamException e) { + throw new IOException("error getting text from xml reader"); + } + } + // an error occurred + return -1; + } + + // read events from the StAX input until we get some text + // @return whether any text was read, or false if we are at the end + private boolean next() throws XMLStreamException { + while (xmlReader.hasNext()) { + int event = xmlReader.next(); + int pos = xmlReader.getLocation().getCharacterOffset(); + + if (includedElements != null + && (event == START_ELEMENT || event == END_ELEMENT)) { + // get the element name and namespace + String name = xmlReader.getLocalName(); + String namespace = xmlReader.getNamespaceURI(); + Boolean inc = includedElements.get(QName.get(name, namespace, + qnameTable)); + if (inc != null) { + if (event == START_ELEMENT) + including.add(inc); + else + including.remove(including.size() - 1); + } + } + if (!isIncluding()) + continue; + + if (event == START_ELEMENT || event == END_ELEMENT || event == COMMENT + || event == PROCESSING_INSTRUCTION) { + pushOffset(pos - totalOut); + xmlReaderOffset = -1; // indicates to push a single space on to the + // stream + return true; + } + if (event == CHARACTERS || event == SPACE) { + xmlReaderOffset = 0; + pushOffset(pos - totalOut); + return true; + } + if (event == CDATA) { + xmlReaderOffset = 0; + pushOffset(pos - totalOut + CDATA_LENGTH); + return true; + } + if (event == ENTITY_REFERENCE) { + xmlReaderOffset = 0; + pushOffset(pos - totalOut); + return true; + } + } + return false; + } + + // record the offset of the current text block + private void pushOffset(int delta) { + addOffCorrectMap(totalOut, delta); + } + + /** + * Looks in the solr/conf/dtd folder for a folder with the same basename as + * the dtd. For example, xhtml11.dtd should be placed in + * solr/conf/dtd/xhtml11/xhtml11.dtd. Keeps track of the DTD folder name and + * expects other files to be in the same folder. Note: relative paths are not + * respected. If your DTD has dependent files in subfolders, you would need to + * move them into the same folder with the DTD. + * + * There's no support for providing an external catalog as of yet, which could + * provide a more standard and flexible mechanism. + * + * @author sokolov + * + */ + public class Resolver implements XMLResolver { + + // not thread safe + private String currentDirectory; + + public Object resolveEntity(String publicID, String systemID, + String baseURI, String namespace) throws XMLStreamException { + String solrHome = System.getProperty("solr.solr.home"); + if (solrHome == null) + solrHome = "solr"; + String filename = systemID; + int islash = filename.lastIndexOf('/'); + if (islash >= 0) + filename = filename.substring(islash + 1); + if (filename.endsWith(".dtd")) { + String basename = filename; + int idot = basename.lastIndexOf('.'); + if (idot >= 0) + basename = basename.substring(0, idot); + basename = solrHome + "/conf/dtd/" + basename; + currentDirectory = basename; + filename = basename + "/" + filename; + } else { + filename = currentDirectory + "/" + filename; + } + File dtd = new File(filename); + if (!dtd.exists()) + throw new XMLStreamException(dtd.getAbsolutePath() + " not found"); + if (dtd.isDirectory()) + throw new XMLStreamException(dtd.getAbsolutePath() + + " should be a dtd, but is a directory"); + if (!dtd.canRead()) + throw new XMLStreamException(dtd.getAbsolutePath() + + " could not be read"); + try { + return new FileInputStream(dtd); + } catch (FileNotFoundException e) { + throw new XMLStreamException(dtd.getAbsolutePath() + " not found", e); + } + } + } + + private boolean isIncluding() { + return including == null || including.isEmpty() + || including.get(including.size() - 1); + } + + private void addIncludes(String[] qnames, boolean include) { + if (includedElements == null) { + includedElements = new HashMap(); + } + for (QName q : QName.parseArray(qnames)) { + includedElements.put(q, include); + } + } + + static class QName { + String namespace; + String name; + + static QName get(String name, String namespace, + HashMap interned) { + QName[] names = interned.get(name); + if (names != null) { + if (namespace == null) + namespace = ""; + for (QName qname : names) { + if (qname.namespace.equals(namespace)) + return qname; + } + QName[] newnames = new QName[names.length + 1]; + System.arraycopy(names, 0, newnames, 0, names.length); + names = newnames; + } else { + names = new QName[1]; + } + QName qname = new QName(name, namespace); + names[names.length - 1] = qname; + interned.put(name, names); + return qname; + } + + QName(String name, String namespace) { + this.namespace = namespace == null ? "" : namespace; + this.name = name == null ? "" : name; + } + + QName(String qname) { + if (qname.matches("\\{.+\\}.+")) { + int ibrace = qname.lastIndexOf('}'); + namespace = qname.substring(1, ibrace); + name = qname.substring(ibrace + 1); + } else { + name = qname; + namespace = ""; + } + } + + static QName[] parseArray(String[] arr) { + if (arr == null) { + return null; + } + QName[] qnames = new QName[arr.length]; + int i = 0; + for (String include : arr) { + qnames[i++] = new QName(include); + } + return qnames; + } + + @Override + public boolean equals(Object other) { + if (super.equals(other)) + return true; + if (other == null) + return false; + return (namespace.equals(((QName) other).namespace) && name + .equals(((QName) other).name)); + } + + @Override + public int hashCode() { + return (name + namespace).hashCode(); + } + + @Override + public String toString() { + if (namespace.length() > 0) + return String.format("{%s}%s", namespace, name); + return name; + } + } +} Index: solr/src/java/org/apache/solr/analysis/XmlCharFilterFactory.java =================================================================== --- solr/src/java/org/apache/solr/analysis/XmlCharFilterFactory.java (revision 0) +++ solr/src/java/org/apache/solr/analysis/XmlCharFilterFactory.java (revision 0) @@ -0,0 +1,59 @@ +package org.apache.solr.analysis; + +import javax.xml.stream.XMLInputFactory; + +import org.apache.lucene.analysis.CharStream; +import org.codehaus.stax2.XMLInputFactory2; + +import com.ctc.wstx.api.WstxInputProperties; +import com.ctc.wstx.stax.WstxInputFactory; + +/** + * Support for configuring an XmlCharFilter. Supply attributes include and exclude + * to strip XML elements by name. Text is indexed if its nearest included-or-excluded ancestor element is in + * fact included. For example, you might exclude "head" from an XHTML document, but include "title". This would + * have the effect of indexing the title and the body. The include and exclude attributes + * accept comma-separated lists of XML element names. Each such name may include a namespace wrapped in curly braces, like: + * {http://www.w3.org/1999/xhtml}head. Finally, if no includes are specified, the default is to include all text + * starting from the root of the document. If some includes are specified, the default is swapped: text is excluded until + * an included element is found. + */ +public class XmlCharFilterFactory extends BaseCharFilterFactory { + + private XMLInputFactory2 inputFactory; + + protected XMLInputFactory2 getXMLInputFactory () { + if (inputFactory != null) + return inputFactory; + + inputFactory = (WstxInputFactory) WstxInputFactory.newInstance(); + inputFactory.setProperty (XMLInputFactory.IS_COALESCING, false); + inputFactory.setProperty (XMLInputFactory.IS_REPLACING_ENTITY_REFERENCES, true); + // Workaround for Woodstox config bug + // inputFactory.setProperty (WstxInputProperties.P_NORMALIZE_LFS, false); + ((WstxInputFactory)inputFactory).getConfig().doNormalizeLFs(false); + inputFactory.setProperty (WstxInputProperties.P_TREAT_CHAR_REFS_AS_ENTS, true); + // must set this to 1 in order to get TREAT_CHAR_REFS_AS_ENTS to report entities? + inputFactory.setProperty (WstxInputProperties.P_MIN_TEXT_SEGMENT, Integer.valueOf(1)); + + return inputFactory; + } + + public XmlCharFilter create(CharStream input) { + + String incAttr=null, excAttr=null; + if (getArgs() != null) { + incAttr = getArgs().get("include"); + excAttr = getArgs().get("exclude"); + } + String[] includes=null, excludes=null; + if (incAttr != null) { + includes = incAttr.split("\\s*,\\s*"); + } + if (excAttr != null) { + excludes = excAttr.split("\\s*,\\s*"); + } + XmlCharFilter filter = new XmlCharFilter(input, getXMLInputFactory(), includes, excludes); + return filter; + } +} Index: solr/src/java/org/apache/solr/core/Config.java =================================================================== --- solr/src/java/org/apache/solr/core/Config.java (revision 1144713) +++ solr/src/java/org/apache/solr/core/Config.java (working copy) @@ -26,22 +26,36 @@ import org.slf4j.LoggerFactory; import org.w3c.dom.Document; import org.w3c.dom.Node; +import org.xml.sax.ErrorHandler; import org.xml.sax.InputSource; import org.xml.sax.SAXException; +import org.xml.sax.SAXParseException; +import org.xml.sax.XMLReader; import org.apache.commons.io.IOUtils; + +import javax.xml.XMLConstants; import javax.xml.namespace.QName; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; +import javax.xml.parsers.SAXParser; +import javax.xml.parsers.SAXParserFactory; +import javax.xml.transform.Source; +import javax.xml.transform.stream.StreamSource; +import javax.xml.validation.Schema; +import javax.xml.validation.SchemaFactory; import javax.xml.xpath.XPath; import javax.xml.xpath.XPathConstants; import javax.xml.xpath.XPathExpressionException; import javax.xml.xpath.XPathFactory; + +import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.StringReader; +import java.net.URL; import java.util.Arrays; -import java.util.List; import java.util.Locale; import java.util.concurrent.atomic.AtomicBoolean; @@ -101,33 +115,50 @@ this.prefix = (prefix != null && !prefix.endsWith("/"))? prefix + '/' : prefix; try { javax.xml.parsers.DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance(); - + dbf.setNamespaceAware(true); if (is == null) { is = new InputSource(loader.openConfig(name)); is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name)); } - // only enable xinclude, if a SystemId is available + // only enable xinclude if a SystemId is available if (is.getSystemId() != null) { try { dbf.setXIncludeAware(true); - dbf.setNamespaceAware(true); } catch(UnsupportedOperationException e) { log.warn(name + " XML parser doesn't support XInclude option"); } } + // read config into memory and create a new InputSource; + // the input may be a non-rewindable stream + InputStream instream = is.getByteStream(); + if (instream == null && is.getSystemId() != null) { + instream = new URL(is.getSystemId()).openStream(); + } + if (instream == null) { + throw new IOException ("Error reading config " + name); + } + byte[] config = org.apache.commons.io.IOUtils.toByteArray(instream); + IOUtils.closeQuietly(is.getByteStream()); + InputSource source = new InputSource (new ByteArrayInputStream (config)); + source.setSystemId(is.getSystemId()); + final DocumentBuilder db = dbf.newDocumentBuilder(); db.setEntityResolver(new SystemIdResolver(loader)); db.setErrorHandler(xmllog); - try { - doc = db.parse(is); - } finally { - // some XML parsers are broken and don't close the byte stream (but they should according to spec) - IOUtils.closeQuietly(is.getByteStream()); - } - + doc = db.parse(source); DOMUtil.substituteProperties(doc, loader.getCoreProperties()); + + // Perform substitutions on the document in its original serialized form and validate that. + // This is so we can report line /column numbers accurately, and because validating while + // parsing fails since substitutions are performed *after* parsing. + // Properties appearing in XML comments are allowed to be undefined, so allow + // failed substitutions. + String configString = DOMUtil.substituteProperty + (new String (config, doc.getInputEncoding()), loader.getCoreProperties(), false); + validateConfig(configString, is.getSystemId()); + } catch (ParserConfigurationException e) { SolrException.log(log, "Exception during parsing file: " + name, e); throw e; @@ -137,9 +168,74 @@ } catch( SolrException e ){ SolrException.log(log,"Error in "+name,e); throw e; - } + } } + + protected Schema getSchema () throws SAXException { + SchemaFactory factory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); + //SchemaFactory factory = new CompactSyntaxSchemaFactory(); //CompactSyntaxSchemaFactory.newInstance(CompactSyntaxSchemaFactory.SCHEMA_LANGUAGE); + // load a WXS schema, represented by a Schema instance + //String schemaFileName = "config.rnc"; + String schemaFileName = "config.xsd"; + Source schemaSource, xmlSchemaSource; + try { + schemaSource= new StreamSource(loader.openConfig(schemaFileName)); + xmlSchemaSource = new StreamSource(loader.openConfig("xml.xsd")); + } catch (RuntimeException e) { + return null; + } + schemaSource.setSystemId(SystemIdResolver.createSystemIdFromResourceName(schemaFileName)); + return factory.newSchema(new Source[] {xmlSchemaSource, schemaSource}); + } + + protected void validateConfig (String config, String systemId) throws IOException, SAXException, ParserConfigurationException { + Schema schema = getSchema(); + // Don't validate if we can't find the schema: + if (schema == null) + return; + // Create a parser explicitly so we can enable XInclude processing + SAXParserFactory spf = SAXParserFactory.newInstance(); + spf.setSchema(schema); + if (systemId != null) { + spf.setXIncludeAware(true); + spf.setNamespaceAware(true); + } + SAXParser sp = spf.newSAXParser(); + XMLReader xr = sp.getXMLReader(); + xr.setEntityResolver(new SystemIdResolver(loader)); + xr.setErrorHandler(new ValidationErrorHandler()); + InputSource is = new InputSource (new StringReader (config)); + is.setSystemId(systemId); + xr.parse(is); + } + + class ValidationErrorHandler implements ErrorHandler { + @Override + public void warning(SAXParseException e) throws SAXException { + log (e); + } + + @Override + public void error(SAXParseException e) throws SAXException { + log (e); + throw e; + } + + @Override + public void fatalError(SAXParseException e) throws SAXException { + log (e); + throw (e); + } + + private void log (SAXParseException e) { + String file = name != null ? name : e.getSystemId() != null ? e.getSystemId() : "configuration file"; + SolrException.log(log, String.format("Exception during validation of %s at line %d char %d: %s", + file, e.getLineNumber(), e.getColumnNumber(), e.getMessage()), e); + } + + } + /** * @since solr 1.3 */