Return-Path: Delivered-To: apmail-lucene-commits-archive@www.apache.org Received: (qmail 35198 invoked from network); 22 Jul 2010 19:36:50 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 22 Jul 2010 19:36:50 -0000 Received: (qmail 69471 invoked by uid 500); 22 Jul 2010 19:36:50 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 69464 invoked by uid 99); 22 Jul 2010 19:36:50 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 22 Jul 2010 19:36:50 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 22 Jul 2010 19:36:41 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id D65B82388C48; Thu, 22 Jul 2010 19:35:00 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r966819 [18/20] - in /lucene/dev/branches/realtime_search: ./ lucene/ lucene/backwards/ lucene/contrib/ lucene/contrib/benchmark/conf/ lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/contrib/benchmark/src/... Date: Thu, 22 Jul 2010 19:34:52 -0000 To: commits@lucene.apache.org From: buschmi@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100722193500.D65B82388C48@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/request/SimpleFacets.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/request/SimpleFacets.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/request/SimpleFacets.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/request/SimpleFacets.java Thu Jul 22 19:34:35 2010 @@ -21,6 +21,10 @@ import org.apache.lucene.index.*; import org.apache.lucene.queryParser.ParseException; import org.apache.lucene.search.*; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.packed.Direct16; +import org.apache.lucene.util.packed.Direct32; +import org.apache.lucene.util.packed.Direct8; +import org.apache.lucene.util.packed.PackedInts; import org.apache.noggit.CharArr; import org.apache.solr.common.SolrException; import org.apache.solr.common.params.FacetParams; @@ -400,7 +404,7 @@ public class SimpleFacets { assert endTermIndex < 0; endTermIndex = -endTermIndex-1; } else { - startTermIndex=1; + startTermIndex=0; endTermIndex=si.numOrd(); } @@ -415,10 +419,53 @@ public class SimpleFacets { final int[] counts = new int[nTerms]; DocIterator iter = docs.iterator(); - while (iter.hasNext()) { - int term = si.getOrd(iter.nextDoc()); - int arrIdx = term-startTermIndex; - if (arrIdx>=0 && arrIdx=0 && arrIdx=0 && arrIdx=0 && arrIdx=0 && arrIdx> queue = new BoundedTreeSet>(maxsize); int min=mincount-1; // the smallest value in the top 'N' values - for (int i=0; imin) { // NOTE: we use c>min rather than c>=min as an optimization because we are going in @@ -452,11 +499,11 @@ public class SimpleFacets { } } else { // add results in index order - int i=0; + int i=(startTermIndex==0)?1:0; if (mincount<=0) { // if mincount<=0, then we won't discard any terms and we know exactly // where to start. - i=off; + i+=off; off=0; } @@ -530,7 +577,7 @@ public class SimpleFacets { Fields fields = MultiFields.getFields(r); Terms terms = fields==null ? null : fields.terms(field); TermsEnum termsEnum = null; - + SolrIndexSearcher.DocsEnumState deState = null; BytesRef term = null; if (terms != null) { termsEnum = terms.iterator(); @@ -574,7 +621,17 @@ public class SimpleFacets { spare.reset(); ByteUtils.UTF8toUTF16(term, spare); Term t = template.createTerm(spare.toString()); - c = searcher.numDocs(new TermQuery(t), docs); + + if (deState==null) { + deState = new SolrIndexSearcher.DocsEnumState(); + deState.deletedDocs = MultiFields.getDeletedDocs(r); + deState.termsEnum = termsEnum; + deState.reuse = docsEnum; + } + + c = searcher.numDocs(new TermQuery(t), docs, deState); + + docsEnum = deState.reuse; } else { // iterate over TermDocs to calculate the intersection @@ -582,20 +639,43 @@ public class SimpleFacets { // TODO: do this per-segment for better efficiency (MultiDocsEnum just uses base class impl) // TODO: would passing deleted docs lead to better efficiency over checking the fastForRandomSet? docsEnum = termsEnum.docs(null, docsEnum); + c=0; + + if (docsEnum instanceof MultiDocsEnum) { + MultiDocsEnum.EnumWithSlice[] subs = ((MultiDocsEnum)docsEnum).getSubs(); + int numSubs = ((MultiDocsEnum)docsEnum).getNumSubs(); + for (int subindex = 0; subindex= threshold) { TopTerm topTerm = new TopTerm(); - topTerm.term = t; + topTerm.term = new BytesRef(t); topTerm.termNum = termNum; bigTerms.put(topTerm.termNum, topTerm); - DocSet set = searcher.getDocSet(new TermQuery(topTerm.term)); + if (deState == null) { + deState = new SolrIndexSearcher.DocsEnumState(); + deState.termsEnum = te.tenum; + deState.reuse = te.docsEnum; + } + DocSet set = searcher.getDocSet(new TermQuery(new Term(ti.field, topTerm.term)), deState); + te.docsEnum = deState.reuse; + maxTermCounts[termNum] = set.size(); te.next(); @@ -252,17 +267,19 @@ public class UnInvertedField { termsInverted++; - TermDocs td = te.getTermDocs(); - td.seek(te); + DocsEnum docsEnum = te.getDocsEnum(); + + DocsEnum.BulkReadResult bulkResult = docsEnum.getBulkResult(); + for(;;) { - int n = td.read(docs,freqs); + int n = docsEnum.read(); if (n <= 0) break; maxTermCounts[termNum] += n; for (int i=0; i 0) { - te.skipTo(prefix); + te.skipTo(new BytesRef(prefix)); startTerm = te.getTermNumber(); - te.skipTo(prefix + "\uffff\uffff\uffff\uffff"); + te.skipTo(new BytesRef(prefix + "\uffff\uffff\uffff\uffff")); endTerm = te.getTermNumber(); } @@ -497,7 +514,7 @@ public class UnInvertedField { for (TopTerm tt : bigTerms.values()) { // TODO: counts could be deferred if sorted==false if (tt.termNum >= startTerm && tt.termNum < endTerm) { - counts[tt.termNum] = searcher.numDocs(new TermQuery(tt.term), docs); + counts[tt.termNum] = searcher.numDocs(new TermQuery(new Term(ti.field, tt.term)), docs); } } @@ -549,6 +566,8 @@ public class UnInvertedField { } } + CharArr spare = new CharArr(); + int off=offset; int lim=limit>=0 ? limit : Integer.MAX_VALUE; @@ -584,7 +603,7 @@ public class UnInvertedField { int c = -(int)(p.longValue() >>> 32); //int tnum = 0x7fffffff - (int)p.longValue(); // use if priority queue int tnum = (int)p.longValue(); - String label = ft.indexedToReadable(getTermText(te, tnum)); + String label = getReadableValue(getTermValue(te, tnum), ft, spare); res.add(label, c); } } else { @@ -602,7 +621,7 @@ public class UnInvertedField { if (c=0) continue; if (--lim<0) break; - String label = ft.indexedToReadable(getTermText(te, i)); + String label = getReadableValue(getTermValue(te, i), ft, spare); res.add(label, c); } } @@ -669,7 +688,7 @@ public class UnInvertedField { final int[] index = this.index; final int[] counts = new int[numTermsInField];//keep track of the number of times we see each word in the field for all the documents in the docset - NumberedTermEnum te = ti.getEnumerator(searcher.getReader()); + NumberedTermsEnum te = ti.getEnumerator(searcher.getReader()); boolean doNegative = false; @@ -693,12 +712,13 @@ public class UnInvertedField { for (TopTerm tt : bigTerms.values()) { // TODO: counts could be deferred if sorted==false if (tt.termNum >= 0 && tt.termNum < numTermsInField) { + final Term t = new Term(ti.field, tt.term); if (finfo.length == 0) { - counts[tt.termNum] = searcher.numDocs(new TermQuery(tt.term), docs); + counts[tt.termNum] = searcher.numDocs(new TermQuery(t), docs); } else { //COULD BE VERY SLOW //if we're collecting stats for facet fields, we need to iterate on all matching documents - DocSet bigTermDocSet = searcher.getDocSet(new TermQuery(tt.term)).intersection(docs); + DocSet bigTermDocSet = searcher.getDocSet(new TermQuery(t)).intersection(docs); DocIterator iter = bigTermDocSet.iterator(); while (iter.hasNext()) { int doc = iter.nextDoc(); @@ -758,11 +778,15 @@ public class UnInvertedField { } // add results in index order + CharArr spare = new CharArr(); for (i = 0; i < numTermsInField; i++) { int c = doNegative ? maxTermCounts[i] - counts[i] : counts[i]; if (c == 0) continue; - Double value = Double.parseDouble(ft.indexedToReadable(getTermText(te, i))); + String label = getReadableValue(getTermValue(te, i), ft, spare); + // TODO: we should avoid this re-parse + Double value = Double.parseDouble(label); + allstats.accumulate(value, c); //as we've parsed the termnum into a value, lets also accumulate fieldfacet statistics for (FieldFacetStats f : finfo) { @@ -792,20 +816,27 @@ public class UnInvertedField { } + String getReadableValue(BytesRef termval, FieldType ft, CharArr spare) { + if (spare == null) { + spare = new CharArr(); + } else { + spare.reset(); + } + ft.indexedToReadable(termval, spare); + return spare.toString(); + } - - - String getTermText(NumberedTermEnum te, int termNum) throws IOException { + /** may return a reused BytesRef */ + BytesRef getTermValue(NumberedTermsEnum te, int termNum) throws IOException { if (bigTerms.size() > 0) { // see if the term is one of our big terms. TopTerm tt = bigTerms.get(termNum); if (tt != null) { - return tt.term.text(); + return tt.term; } } - te.skipTo(termNum); - return te.term().text(); + return te.skipTo(termNum); } public String toString() { @@ -860,95 +891,99 @@ class TermEnumListener { ***/ -class NumberedTermEnum extends TermEnum { +class NumberedTermsEnum extends TermsEnum { protected final IndexReader reader; protected final TermIndex tindex; - protected TermEnum tenum; + protected TermsEnum tenum; protected int pos=-1; - protected Term t; - protected TermDocs termDocs; + protected BytesRef termText; + protected DocsEnum docsEnum; + protected Bits deletedDocs; - NumberedTermEnum(IndexReader reader, TermIndex tindex) throws IOException { + NumberedTermsEnum(IndexReader reader, TermIndex tindex) throws IOException { this.reader = reader; this.tindex = tindex; } - NumberedTermEnum(IndexReader reader, TermIndex tindex, String termValue, int pos) throws IOException { + NumberedTermsEnum(IndexReader reader, TermIndex tindex, BytesRef termValue, int pos) throws IOException { this.reader = reader; this.tindex = tindex; this.pos = pos; - tenum = reader.terms(tindex.createTerm(termValue)); - setTerm(); + Terms terms = MultiFields.getTerms(reader, tindex.field); + deletedDocs = MultiFields.getDeletedDocs(reader); + if (terms != null) { + tenum = terms.iterator(); + tenum.seek(termValue); + setTerm(); + } } - public TermDocs getTermDocs() throws IOException { - if (termDocs==null) termDocs = reader.termDocs(t); - else termDocs.seek(t); - return termDocs; + @Override + public Comparator getComparator() throws IOException { + return tenum.getComparator(); } - protected boolean setTerm() { - t = tenum.term(); - if (t==null - || t.field() != tindex.fterm.field() // intern'd compare - || (tindex.prefix != null && !t.text().startsWith(tindex.prefix,0)) ) - { - t = null; - return false; - } - return true; + public DocsEnum getDocsEnum() throws IOException { + docsEnum = tenum.docs(deletedDocs, docsEnum); + return docsEnum; } + protected BytesRef setTerm() throws IOException { + termText = tenum.term(); + if (tindex.prefix != null && !termText.startsWith(tindex.prefix)) { + termText = null; + } + return termText; + } - public boolean next() throws IOException { + @Override + public BytesRef next() throws IOException { pos++; - boolean b = tenum.next(); - if (!b) { - t = null; - return false; + if (tenum.next() == null) { + termText = null; + return null; } return setTerm(); // this is extra work if we know we are in bounds... } - public Term term() { - return t; + @Override + public BytesRef term() { + return termText; } + @Override public int docFreq() { return tenum.docFreq(); } - public void close() throws IOException { - if (tenum!=null) tenum.close(); - } + public BytesRef skipTo(BytesRef target) throws IOException { - public boolean skipTo(String target) throws IOException { - return skipTo(tindex.fterm.createTerm(target)); - } - - public boolean skipTo(Term target) throws IOException { // already here - if (t != null && t.equals(target)) return true; + if (termText != null && termText.equals(target)) return termText; - int startIdx = Arrays.binarySearch(tindex.index,target.text()); + if (tenum == null) { + return null; + } + + int startIdx = Arrays.binarySearch(tindex.index,target); if (startIdx >= 0) { // we hit the term exactly... lucky us! - if (tenum != null) tenum.close(); - tenum = reader.terms(target); + TermsEnum.SeekStatus seekStatus = tenum.seek(target); + assert seekStatus == TermsEnum.SeekStatus.FOUND; pos = startIdx << tindex.intervalBits; return setTerm(); } // we didn't hit the term exactly startIdx=-startIdx-1; - + if (startIdx == 0) { // our target occurs *before* the first term - if (tenum != null) tenum.close(); - tenum = reader.terms(target); + TermsEnum.SeekStatus seekStatus = tenum.seek(target); + assert seekStatus == TermsEnum.SeekStatus.NOT_FOUND; pos = 0; return setTerm(); } @@ -956,53 +991,81 @@ class NumberedTermEnum extends TermEnum // back up to the start of the block startIdx--; - if ((pos >> tindex.intervalBits) == startIdx && t != null && t.text().compareTo(target.text())<=0) { + if ((pos >> tindex.intervalBits) == startIdx && termText != null && termText.compareTo(target)<=0) { // we are already in the right block and the current term is before the term we want, // so we don't need to seek. } else { // seek to the right block - if (tenum != null) tenum.close(); - tenum = reader.terms(target.createTerm(tindex.index[startIdx])); + TermsEnum.SeekStatus seekStatus = tenum.seek(tindex.index[startIdx]); + assert seekStatus == TermsEnum.SeekStatus.FOUND; pos = startIdx << tindex.intervalBits; - setTerm(); // should be true since it's in the index + setTerm(); // should be non-null since it's in the index } - - while (t != null && t.text().compareTo(target.text()) < 0) { + while (termText != null && termText.compareTo(target) < 0) { next(); } - return t != null; + return termText; } - - public boolean skipTo(int termNumber) throws IOException { + public BytesRef skipTo(int termNumber) throws IOException { int delta = termNumber - pos; if (delta < 0 || delta > tindex.interval || tenum==null) { int idx = termNumber >>> tindex.intervalBits; - String base = tindex.index[idx]; + BytesRef base = tindex.index[idx]; pos = idx << tindex.intervalBits; delta = termNumber - pos; - if (tenum != null) tenum.close(); - tenum = reader.terms(tindex.createTerm(base)); + TermsEnum.SeekStatus seekStatus = tenum.seek(base); + assert seekStatus == TermsEnum.SeekStatus.FOUND; } while (--delta >= 0) { - boolean b = tenum.next(); - if (b==false) { - t = null; - return false; + BytesRef br = tenum.next(); + if (br == null) { + termText = null; + return null; } ++pos; } return setTerm(); } + protected void close() throws IOException { + // no-op, needed so the anon subclass that does indexing + // can build its index + } + /** The current term number, starting at 0. * Only valid if the previous call to next() or skipTo() returned true. */ public int getTermNumber() { return pos; } + + @Override + public long ord() { + throw new UnsupportedOperationException(); + } + + @Override + public SeekStatus seek(long ord) { + throw new UnsupportedOperationException(); + } + + @Override + public DocsEnum docs(Bits skipDocs, DocsEnum reuse) { + throw new UnsupportedOperationException(); + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) { + throw new UnsupportedOperationException(); + } + + @Override + public SeekStatus seek(BytesRef target, boolean useCache) { + throw new UnsupportedOperationException(); + } } @@ -1018,9 +1081,9 @@ class TermIndex { final static int intervalMask = 0xffffffff >>> (32-intervalBits); final static int interval = 1 << intervalBits; - final Term fterm; // prototype to be used in term construction w/o String.intern overhead - final String prefix; - String[] index; + final String field; + final BytesRef prefix; + BytesRef[] index; int nTerms; long sizeOfStrings; @@ -1029,16 +1092,12 @@ class TermIndex { } TermIndex(String field, String prefix) { - this.fterm = new Term(field, ""); - this.prefix = prefix; - } - - Term createTerm(String termVal) { - return fterm.createTerm(termVal); + this.field = field; + this.prefix = prefix == null ? null : new BytesRef(prefix); } - NumberedTermEnum getEnumerator(IndexReader reader, int termNumber) throws IOException { - NumberedTermEnum te = new NumberedTermEnum(reader, this); + NumberedTermsEnum getEnumerator(IndexReader reader, int termNumber) throws IOException { + NumberedTermsEnum te = new NumberedTermsEnum(reader, this); te.skipTo(termNumber); return te; } @@ -1047,38 +1106,41 @@ class TermIndex { with next() to fully traverse all of the terms so the index will be built. */ - NumberedTermEnum getEnumerator(IndexReader reader) throws IOException { - if (index==null) return new NumberedTermEnum(reader,this, prefix==null?"":prefix, 0) { - ArrayList lst; - - protected boolean setTerm() { - boolean b = super.setTerm(); - if (b && (pos & intervalMask)==0) { - String text = term().text(); - sizeOfStrings += text.length() << 1; + NumberedTermsEnum getEnumerator(IndexReader reader) throws IOException { + if (index==null) return new NumberedTermsEnum(reader,this, prefix==null?new BytesRef():prefix, 0) { + ArrayList lst; + PagedBytes bytes; + + protected BytesRef setTerm() throws IOException { + BytesRef br = super.setTerm(); + if (br != null && (pos & intervalMask)==0) { + sizeOfStrings += br.length; if (lst==null) { - lst = new ArrayList(); + lst = new ArrayList(); + bytes = new PagedBytes(15); } - lst.add(text); + BytesRef out = new BytesRef(); + bytes.copy(br, out); + lst.add(out); } - return b; + return br; } - public boolean skipTo(Term target) throws IOException { + public BytesRef skipTo(Term target) throws IOException { throw new UnsupportedOperationException(); } - public boolean skipTo(int termNumber) throws IOException { + public BytesRef skipTo(int termNumber) throws IOException { throw new UnsupportedOperationException(); } public void close() throws IOException { nTerms=pos; super.close(); - index = lst!=null ? lst.toArray(new String[lst.size()]) : new String[0]; + index = lst!=null ? lst.toArray(new BytesRef[lst.size()]) : new BytesRef[0]; } }; - else return new NumberedTermEnum(reader,this,"",0); + else return new NumberedTermsEnum(reader,this,new BytesRef(),0); } Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/GeoHashField.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/GeoHashField.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/GeoHashField.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/GeoHashField.java Thu Jul 22 19:34:35 2010 @@ -18,13 +18,22 @@ package org.apache.solr.schema; import org.apache.lucene.document.Fieldable; +import org.apache.lucene.search.Query; import org.apache.lucene.search.SortField; import org.apache.lucene.spatial.geohash.GeoHashUtils; +import org.apache.lucene.spatial.DistanceUtils; +import org.apache.lucene.spatial.tier.InvalidGeoException; +import org.apache.solr.common.SolrException; import org.apache.solr.response.TextResponseWriter; import org.apache.solr.response.XMLWriter; import org.apache.solr.search.QParser; +import org.apache.solr.search.SolrConstantScoreQuery; +import org.apache.solr.search.SpatialOptions; +import org.apache.solr.search.function.LiteralValueSource; import org.apache.solr.search.function.ValueSource; -import org.apache.solr.search.function.distance.DistanceUtils; +import org.apache.solr.search.function.ValueSourceRangeFilter; +import org.apache.solr.search.function.distance.GeohashHaversineFunction; + import java.io.IOException; @@ -33,9 +42,9 @@ import java.io.IOException; * href="http://en.wikipedia.org/wiki/Geohash">Geohash field. The field is * provided as a lat/lon pair and is internally represented as a string. * - * @see org.apache.solr.search.function.distance.DistanceUtils#parseLatitudeLongitude(double[], String) + * @see org.apache.lucene.spatial.DistanceUtils#parseLatitudeLongitude(double[], String) */ -public class GeoHashField extends FieldType { +public class GeoHashField extends FieldType implements SpatialQueryable { @Override @@ -43,6 +52,22 @@ public class GeoHashField extends FieldT return getStringSort(field, top); } + //QUESTION: Should we do a fast and crude one? Or actually check distances + //Fast and crude could use EdgeNGrams, but that would require a different + //encoding. Plus there are issues around the Equator/Prime Meridian + public Query createSpatialQuery(QParser parser, SpatialOptions options) { + double [] point = new double[0]; + try { + point = DistanceUtils.parsePointDouble(null, options.pointStr, 2); + } catch (InvalidGeoException e) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); + } + String geohash = GeoHashUtils.encode(point[0], point[1]); + //TODO: optimize this + return new SolrConstantScoreQuery(new ValueSourceRangeFilter(new GeohashHaversineFunction(getValueSource(options.field, parser), + new LiteralValueSource(geohash), options.radius), "0", String.valueOf(options.distance), true, true)); + } + @Override public void write(XMLWriter xmlWriter, String name, Fieldable f) throws IOException { @@ -67,7 +92,12 @@ public class GeoHashField extends FieldT public String toInternal(String val) { // validate that the string is of the form // latitude, longitude - double[] latLon = DistanceUtils.parseLatitudeLongitude(null, val); + double[] latLon = new double[0]; + try { + latLon = DistanceUtils.parseLatitudeLongitude(null, val); + } catch (InvalidGeoException e) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); + } return GeoHashUtils.encode(latLon[0], latLon[1]); } Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/PointType.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/PointType.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/PointType.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/PointType.java Thu Jul 22 19:34:35 2010 @@ -23,15 +23,17 @@ import org.apache.lucene.search.BooleanC import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.SortField; +import org.apache.lucene.spatial.DistanceUtils; +import org.apache.lucene.spatial.tier.InvalidGeoException; import org.apache.solr.common.SolrException; import org.apache.solr.common.params.MapSolrParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.response.TextResponseWriter; import org.apache.solr.response.XMLWriter; import org.apache.solr.search.QParser; +import org.apache.solr.search.SpatialOptions; import org.apache.solr.search.function.VectorValueSource; import org.apache.solr.search.function.ValueSource; -import org.apache.solr.search.function.distance.DistanceUtils; import java.io.IOException; import java.util.Map; @@ -45,7 +47,7 @@ import java.util.ArrayList; *

* NOTE: There can only be one sub type */ -public class PointType extends CoordinateFieldType { +public class PointType extends CoordinateFieldType implements SpatialQueryable { @Override protected void init(IndexSchema schema, Map args) { @@ -71,7 +73,12 @@ public class PointType extends Coordinat @Override public Fieldable[] createFields(SchemaField field, String externalVal, float boost) { - String[] point = DistanceUtils.parsePoint(null, externalVal, dimension); + String[] point = new String[0]; + try { + point = DistanceUtils.parsePoint(null, externalVal, dimension); + } catch (InvalidGeoException e) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); + } // TODO: this doesn't currently support polyFields as sub-field types Fieldable[] f = new Fieldable[ (field.indexed() ? dimension : 0) + (field.stored() ? 1 : 0) ]; @@ -103,7 +110,11 @@ public class PointType extends Coordinat } - //It never makes sense to create a single field, so make it impossible to happen + /** + * It never makes sense to create a single field, so make it impossible to happen by + * throwing UnsupportedOperationException + * + */ @Override public Field createField(SchemaField field, String externalVal, float boost) { throw new UnsupportedOperationException("PointType uses multiple fields. field=" + field.getName()); @@ -131,8 +142,14 @@ public class PointType extends Coordinat public Query getRangeQuery(QParser parser, SchemaField field, String part1, String part2, boolean minInclusive, boolean maxInclusive) { //Query could look like: [x1,y1 TO x2,y2] for 2 dimension, but could look like: [x1,y1,z1 TO x2,y2,z2], and can be extrapolated to n-dimensions //thus, this query essentially creates a box, cube, etc. - String[] p1 = DistanceUtils.parsePoint(null, part1, dimension); - String[] p2 = DistanceUtils.parsePoint(null, part2, dimension); + String[] p1; + String[] p2; + try { + p1 = DistanceUtils.parsePoint(null, part1, dimension); + p2 = DistanceUtils.parsePoint(null, part2, dimension); + } catch (InvalidGeoException e) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); + } BooleanQuery result = new BooleanQuery(true); for (int i = 0; i < dimension; i++) { SchemaField subSF = subField(field, i); @@ -144,7 +161,12 @@ public class PointType extends Coordinat @Override public Query getFieldQuery(QParser parser, SchemaField field, String externalVal) { - String[] p1 = DistanceUtils.parsePoint(null, externalVal, dimension); + String[] p1 = new String[0]; + try { + p1 = DistanceUtils.parsePoint(null, externalVal, dimension); + } catch (InvalidGeoException e) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); + } //TODO: should we assert that p1.length == dimension? BooleanQuery bq = new BooleanQuery(true); for (int i = 0; i < dimension; i++) { @@ -154,6 +176,43 @@ public class PointType extends Coordinat } return bq; } + + /** + * Calculates the range and creates a RangeQuery (bounding box) wrapped in a BooleanQuery (unless the dimension is 1, one range for every dimension, AND'd together by a Boolean + * @param parser The parser + * @param options The {@link org.apache.solr.search.SpatialOptions} for this filter. + * @return The Query representing the bounding box around the point. + */ + public Query createSpatialQuery(QParser parser, SpatialOptions options) { + Query result = null; + double [] point = new double[0]; + try { + point = DistanceUtils.parsePointDouble(null, options.pointStr, dimension); + } catch (InvalidGeoException e) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e); + } + if (dimension == 1){ + //TODO: Handle distance measures + String lower = String.valueOf(point[0] - options.distance); + String upper = String.valueOf(point[0] + options.distance); + SchemaField subSF = subField(options.field, 0); + // points must currently be ordered... should we support specifying any two opposite corner points? + result = subSF.getType().getRangeQuery(parser, subSF, lower, upper, true, true); + } else { + BooleanQuery tmp = new BooleanQuery(); + //TODO: Handle distance measures, as this assumes Euclidean + double [] ur = DistanceUtils.vectorBoxCorner(point, null, options.distance, true); + double [] ll = DistanceUtils.vectorBoxCorner(point, null, options.distance, false); + for (int i = 0; i < ur.length; i++) { + SchemaField subSF = subField(options.field, i); + Query range = subSF.getType().getRangeQuery(parser, subSF, String.valueOf(ll[i]), String.valueOf(ur[i]), true, true); + tmp.add(range, BooleanClause.Occur.MUST); + + } + result = tmp; + } + return result; + } } Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/SpatialTileField.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/SpatialTileField.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/SpatialTileField.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/schema/SpatialTileField.java Thu Jul 22 19:34:35 2010 @@ -1,190 +0,0 @@ -package org.apache.solr.schema; -/** - * 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.document.Field; -import org.apache.lucene.document.Fieldable; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.SortField; -import org.apache.lucene.spatial.tier.projections.CartesianTierPlotter; -import org.apache.lucene.spatial.tier.projections.IProjector; -import org.apache.lucene.spatial.tier.projections.SinusoidalProjector; -import org.apache.solr.common.ResourceLoader; -import org.apache.solr.common.SolrException; -import org.apache.solr.common.params.MapSolrParams; -import org.apache.solr.common.params.SolrParams; -import org.apache.solr.response.TextResponseWriter; -import org.apache.solr.response.XMLWriter; -import org.apache.solr.search.QParser; -import org.apache.solr.search.function.ValueSource; -import org.apache.solr.search.function.distance.DistanceUtils; -import org.apache.solr.util.plugin.ResourceLoaderAware; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - - -/** - * Represents a Tiling system for spatial data representation (lat/lon). A Tile is like a zoom level on an - * interactive map. - *

- * Specify a lower and upper tile, and this will create tiles for all the levels in between, inclusive of the upper tile. - *

- * Querying directly against this field is probably not all that useful unless you specifically know the box id - *

- *

- * See http://wiki.apache.org/solr/SpatialSearch - */ -public class SpatialTileField extends AbstractSubTypeFieldType implements ResourceLoaderAware { - - public static final String START_LEVEL = "start"; - public static final String END_LEVEL = "end"; - public static final String PROJECTOR_CLASS = "projector"; - - private static final int DEFAULT_END_LEVEL = 15; - - private static final int DEFAULT_START_LEVEL = 4; - - private int start = DEFAULT_START_LEVEL, end = DEFAULT_END_LEVEL; - private int tileDiff;//we're going to need this over and over, so cache it. - private String projectorName; - protected List plotters; - - - @Override - protected void init(IndexSchema schema, Map args) { - SolrParams p = new MapSolrParams(args); - start = p.getInt(START_LEVEL, DEFAULT_START_LEVEL); - end = p.getInt(END_LEVEL, DEFAULT_END_LEVEL); - if (end < start) { - //flip them around - int tmp = start; - start = end; - end = tmp; - } - args.remove(START_LEVEL); - args.remove(END_LEVEL); - projectorName = p.get(PROJECTOR_CLASS, SinusoidalProjector.class.getName()); - args.remove(PROJECTOR_CLASS); - super.init(schema, args); - tileDiff = (end - start) + 1;//add one since we are inclusive of the upper tier - createSuffixCache(tileDiff); - - - } - - public void inform(ResourceLoader loader) { - IProjector projector = (IProjector) loader.newInstance(projectorName); - if (projector != null) { - plotters = new ArrayList(tileDiff); - for (int i = start; i <= end; i++) { - plotters.add(new CartesianTierPlotter(i, projector, "")); - } - } else { - throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not instantiate a Projector Instance for: " - + projectorName + ". Make sure the " + PROJECTOR_CLASS + " attribute is set properly in the schema"); - } - - } - - @Override - public Fieldable[] createFields(SchemaField field, String externalVal, float boost) { - Fieldable[] f = new Fieldable[(field.indexed() ? tileDiff : 0) + (field.stored() ? 1 : 0)]; - if (field.indexed()) { - int i = 0; - double[] latLon = DistanceUtils.parseLatitudeLongitude(null, externalVal); - for (CartesianTierPlotter plotter : plotters) { - double boxId = plotter.getTierBoxId(latLon[0], latLon[1]); - f[i] = subField(field, i).createField(String.valueOf(boxId), boost); - i++; - } - } - - if (field.stored()) { - String storedVal = externalVal; // normalize or not? - f[f.length - 1] = createField(field.getName(), storedVal, - getFieldStore(field, storedVal), Field.Index.NO, Field.TermVector.NO, - false, false, boost); - } - return f; - } - - //The externalVal here is a box id, as it doesn't make sense to pick a specific tile since that requires a distance - //so, just OR together a search against all the tile - - @Override - public Query getRangeQuery(QParser parser, SchemaField field, String part1, String part2, boolean minInclusive, - boolean maxInclusive) { - BooleanQuery bq = new BooleanQuery(true); - for (int i = 0; i < tileDiff; i++) { - SchemaField sf = subField(field, i); - Query tq = sf.getType().getRangeQuery(parser, sf, part1, part2, minInclusive, maxInclusive); - bq.add(tq, BooleanClause.Occur.SHOULD); - } - return bq; - } - - @Override - public Query getFieldQuery(QParser parser, SchemaField field, String externalVal) { - //The externalVal here is a box id, as it doesn't make sense to pick a specific tile since that requires a distance - //so, just OR together a search against all the tiles - BooleanQuery bq = new BooleanQuery(true); - for (int i = 0; i < tileDiff; i++) { - SchemaField sf = subField(field, i); - Query tq = sf.getType().getFieldQuery(parser, sf, externalVal); - bq.add(tq, BooleanClause.Occur.SHOULD); - } - return bq; - } - - @Override - public boolean isPolyField() { - return true; - } - - @Override - public void write(XMLWriter xmlWriter, String name, Fieldable f) throws IOException { - xmlWriter.writeStr(name, f.stringValue()); - } - - @Override - public void write(TextResponseWriter writer, String name, Fieldable f) throws IOException { - writer.writeStr(name, f.stringValue(), false); - } - - @Override - public SortField getSortField(SchemaField field, boolean top) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Sorting not supported on SpatialTileField " + field.getName()); - } - - @Override - public ValueSource getValueSource(SchemaField field, QParser parser) { - //TODO: Should this really throw UOE? What does it mean for a function to use the values of a tier? Let's leave it unsupported for now - throw new UnsupportedOperationException("SpatialTileField uses multiple fields and does not support ValueSource"); - } - - //It never makes sense to create a single field, so make it impossible to happen - - @Override - public Field createField(SchemaField field, String externalVal, float boost) { - throw new UnsupportedOperationException("SpatialTileField uses multiple fields. field=" + field.getName()); - } -} Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/BitDocSet.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/BitDocSet.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/BitDocSet.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/BitDocSet.java Thu Jul 22 19:34:35 2010 @@ -140,8 +140,11 @@ public class BitDocSet extends DocSetBas size=-1; } + /** Returns true of the doc exists in the set. + * Should only be called when doc < OpenBitSet.size() + */ public boolean exists(int doc) { - return bits.get(doc); + return bits.fastGet(doc); } @Override Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java Thu Jul 22 19:34:35 2010 @@ -869,7 +869,7 @@ class ExtendedDismaxQParser extends QPar int slop; @Override - protected Query getFieldQuery(String field, String val) throws ParseException { + protected Query getFieldQuery(String field, String val, boolean quoted) throws ParseException { //System.out.println("getFieldQuery: val="+val); this.type = QType.FIELD; @@ -1004,7 +1004,7 @@ class ExtendedDismaxQParser extends QPar switch (type) { case FIELD: // fallthrough case PHRASE: - Query query = super.getFieldQuery(field, val); + Query query = super.getFieldQuery(field, val, type == QType.PHRASE); if (query instanceof PhraseQuery) { PhraseQuery pq = (PhraseQuery)query; if (minClauseSize > 1 && pq.getTerms().length < minClauseSize) return null; Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/QParserPlugin.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/QParserPlugin.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/QParserPlugin.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/QParserPlugin.java Thu Jul 22 19:34:35 2010 @@ -37,6 +37,7 @@ public abstract class QParserPlugin impl RawQParserPlugin.NAME, RawQParserPlugin.class, NestedQParserPlugin.NAME, NestedQParserPlugin.class, FunctionRangeQParserPlugin.NAME, FunctionRangeQParserPlugin.class, + SpatialFilterQParserPlugin.NAME, SpatialFilterQParserPlugin.class, }; /** return a {@link QParser} */ Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexReader.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexReader.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexReader.java Thu Jul 22 19:34:35 2010 @@ -485,11 +485,6 @@ public class SolrIndexReader extends Fil } @Override - public TermPositions termPositions(Term term) throws IOException { - return in.termPositions(term); - } - - @Override public void undeleteAll() throws StaleReaderException, CorruptIndexException, LockObtainFailedException, IOException { in.undeleteAll(); } Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java Thu Jul 22 19:34:35 2010 @@ -480,7 +480,7 @@ public class SolrIndexSearcher extends I if (fields == null) return -1; Terms terms = fields.terms(t.field()); if (terms == null) return -1; - BytesRef termBytes = new BytesRef(t.text()); + BytesRef termBytes = t.bytes(); DocsEnum docs = terms.docs(MultiFields.getDeletedDocs(reader), termBytes, null); if (docs == null) return -1; int id = docs.nextDoc(); @@ -554,6 +554,32 @@ public class SolrIndexSearcher extends I return answer; } + /** lucene.internal */ + public DocSet getDocSet(Query query, DocsEnumState deState) throws IOException { + // Get the absolute value (positive version) of this query. If we + // get back the same reference, we know it's positive. + Query absQ = QueryUtils.getAbs(query); + boolean positive = query==absQ; + + if (filterCache != null) { + DocSet absAnswer = (DocSet)filterCache.get(absQ); + if (absAnswer!=null) { + if (positive) return absAnswer; + else return getPositiveDocSet(matchAllDocsQuery).andNot(absAnswer); + } + } + + DocSet absAnswer = getDocSetNC(absQ, null, deState); + DocSet answer = positive ? absAnswer : getPositiveDocSet(matchAllDocsQuery, deState).andNot(absAnswer); + + if (filterCache != null) { + // cache negative queries as positive + filterCache.put(absQ, absAnswer); + } + + return answer; + } + // only handle positive (non negative) queries DocSet getPositiveDocSet(Query q) throws IOException { DocSet answer; @@ -566,6 +592,17 @@ public class SolrIndexSearcher extends I return answer; } + // only handle positive (non negative) queries + DocSet getPositiveDocSet(Query q, DocsEnumState deState) throws IOException { + DocSet answer; + if (filterCache != null) { + answer = (DocSet)filterCache.get(q); + if (answer!=null) return answer; + } + answer = getDocSetNC(q,null,deState); + if (filterCache != null) filterCache.put(q,answer); + return answer; + } private static Query matchAllDocsQuery = new MatchAllDocsQuery(); @@ -624,6 +661,83 @@ public class SolrIndexSearcher extends I } // query must be positive + protected DocSet getDocSetNC(Query query, DocSet filter, DocsEnumState deState) throws IOException { + if (filter != null) return getDocSetNC(query, filter, null); + + int smallSetSize = maxDoc()>>6; + int largestPossible = deState.termsEnum.docFreq(); + + int[] docs = new int[Math.min(smallSetSize, largestPossible)]; + int upto = 0; + int bitsSet = 0; + OpenBitSet obs = null; + + DocsEnum docsEnum = deState.termsEnum.docs(deState.deletedDocs, deState.reuse); + if (deState.reuse == null) { + deState.reuse = docsEnum; + } + + if (docsEnum instanceof MultiDocsEnum) { + MultiDocsEnum.EnumWithSlice[] subs = ((MultiDocsEnum)docsEnum).getSubs(); + int numSubs = ((MultiDocsEnum)docsEnum).getNumSubs(); + for (int subindex = 0; subindex docs.length) { + if (obs == null) obs = new OpenBitSet(maxDoc()); + for (int i=bulk.docs.offset; i docs.length) { + if (obs == null) obs = new OpenBitSet(maxDoc()); + for (int i=bulk.docs.offset; i>6, maxDoc()); @@ -640,7 +754,7 @@ public class SolrIndexSearcher extends I Fields fields = sir.fields(); Terms terms = fields.terms(t.field()); - BytesRef termBytes = new BytesRef(t.text()); + BytesRef termBytes = t.bytes(); Bits skipDocs = sir.getDeletedDocs(); DocsEnum docsEnum = terms==null ? null : terms.docs(skipDocs, termBytes, null); @@ -1436,6 +1550,20 @@ public class SolrIndexSearcher extends I return a==absQ ? b.intersectionSize(positiveA) : b.andNotSize(positiveA); } + /** @lucene.internal */ + public int numDocs(Query a, DocSet b, DocsEnumState deState) throws IOException { + // Negative query if absolute value different from original + Query absQ = QueryUtils.getAbs(a); + DocSet positiveA = getPositiveDocSet(absQ, deState); + return a==absQ ? b.intersectionSize(positiveA) : b.andNotSize(positiveA); + } + + public static class DocsEnumState { + public TermsEnum termsEnum; + public Bits deletedDocs; + public DocsEnum reuse; + } + /** * Returns the number of documents that match both a and b. *

Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrQueryParser.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrQueryParser.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrQueryParser.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/SolrQueryParser.java Thu Jul 22 19:34:35 2010 @@ -54,8 +54,6 @@ import org.apache.solr.schema.TextField; *

* * @see QueryParsing#parseFunction - * @see ConstantScoreRangeQuery - * @see ConstantScorePrefixQuery */ public class SolrQueryParser extends QueryParser { protected final IndexSchema schema; @@ -89,7 +87,7 @@ public class SolrQueryParser extends Que } public SolrQueryParser(QParser parser, String defaultField, Analyzer analyzer) { - super(Version.LUCENE_24, defaultField, analyzer); + super(parser.getReq().getSchema().getSolrConfig().getLuceneVersion("luceneMatchVersion", Version.LUCENE_24), defaultField, analyzer); this.schema = parser.getReq().getSchema(); this.parser = parser; this.defaultField = defaultField; @@ -128,7 +126,7 @@ public class SolrQueryParser extends Que } } - protected Query getFieldQuery(String field, String queryText) throws ParseException { + protected Query getFieldQuery(String field, String queryText, boolean quoted) throws ParseException { checkNullField(field); // intercept magic field name of "_" to use as a hook for our // own functions. @@ -152,7 +150,7 @@ public class SolrQueryParser extends Que } // default to a normal field query - return super.getFieldQuery(field, queryText); + return super.getFieldQuery(field, queryText, quoted); } protected Query getRangeQuery(String field, String part1, String part2, boolean inclusive) throws ParseException { Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ValueSourceParser.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ValueSourceParser.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ValueSourceParser.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/ValueSourceParser.java Thu Jul 22 19:34:35 2010 @@ -17,19 +17,21 @@ package org.apache.solr.search; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.Term; import org.apache.lucene.queryParser.ParseException; import org.apache.lucene.search.Query; import org.apache.lucene.search.Searcher; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.spell.JaroWinklerDistance; import org.apache.lucene.search.spell.LevensteinDistance; import org.apache.lucene.search.spell.NGramDistance; import org.apache.lucene.search.spell.StringDistance; +import org.apache.lucene.spatial.DistanceUtils; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.UnicodeUtil; import org.apache.solr.common.SolrException; import org.apache.solr.common.util.NamedList; -import org.apache.solr.schema.DateField; -import org.apache.solr.schema.LegacyDateField; -import org.apache.solr.schema.SchemaField; -import org.apache.solr.schema.TrieDateField; +import org.apache.solr.schema.*; import org.apache.solr.search.function.*; import org.apache.solr.search.function.distance.*; @@ -45,7 +47,7 @@ import java.util.Collections; /** * A factory that parses user queries to generate ValueSource instances. - * Intented usage is to create pluggable, named functions for use in function queries. + * Intended usage is to create pluggable, named functions for use in function queries. */ public abstract class ValueSourceParser implements NamedListInitializedPlugin { /** @@ -451,6 +453,81 @@ public abstract class ValueSourceParser return new DoubleConstValueSource(Math.E); } }); + + + addParser("docfreq", new ValueSourceParser() { + public ValueSource parse(FunctionQParser fp) throws ParseException { + TInfo tinfo = parseTerm(fp); + return new DocFreqValueSource(tinfo.field, tinfo.val, tinfo.indexedField, tinfo.indexedBytes); + } + }); + + addParser("idf", new ValueSourceParser() { + public ValueSource parse(FunctionQParser fp) throws ParseException { + TInfo tinfo = parseTerm(fp); + return new IDFValueSource(tinfo.field, tinfo.val, tinfo.indexedField, tinfo.indexedBytes); + } + }); + + addParser("termfreq", new ValueSourceParser() { + public ValueSource parse(FunctionQParser fp) throws ParseException { + TInfo tinfo = parseTerm(fp); + return new TermFreqValueSource(tinfo.field, tinfo.val, tinfo.indexedField, tinfo.indexedBytes); + } + }); + + addParser("tf", new ValueSourceParser() { + public ValueSource parse(FunctionQParser fp) throws ParseException { + TInfo tinfo = parseTerm(fp); + return new TFValueSource(tinfo.field, tinfo.val, tinfo.indexedField, tinfo.indexedBytes); + } + }); + + addParser("norm", new ValueSourceParser() { + public ValueSource parse(FunctionQParser fp) throws ParseException { + String field = fp.parseArg(); + return new NormValueSource(field); + } + }); + + addParser("maxdoc", new ValueSourceParser() { + public ValueSource parse(FunctionQParser fp) throws ParseException { + return new MaxDocValueSource(); + } + }); + + addParser("numdocs", new ValueSourceParser() { + public ValueSource parse(FunctionQParser fp) throws ParseException { + return new NumDocsValueSource(); + } + }); + } + + private static TInfo parseTerm(FunctionQParser fp) throws ParseException { + TInfo tinfo = new TInfo(); + + tinfo.indexedField = tinfo.field = fp.parseArg(); + tinfo.val = fp.parseArg(); + tinfo.indexedBytes = new BytesRef(); + + FieldType ft = fp.getReq().getSchema().getFieldTypeNoEx(tinfo.field); + if (ft == null) ft = new StrField(); + + if (ft instanceof TextField) { + // need to do analyisis on the term + String indexedVal = tinfo.val; + Query q = ft.getFieldQuery(fp, fp.getReq().getSchema().getFieldOrNull(tinfo.field), tinfo.val); + if (q instanceof TermQuery) { + Term term = ((TermQuery)q).getTerm(); + tinfo.indexedField = term.field(); + indexedVal = term.text(); + } + UnicodeUtil.UTF16toUTF8(indexedVal, 0, indexedVal.length(), tinfo.indexedBytes); + } else { + ft.readableToIndexed(tinfo.val, tinfo.indexedBytes); + } + + return tinfo; } private static void splitSources(int dim, List sources, List dest1, List dest2) { @@ -502,6 +579,14 @@ public abstract class ValueSourceParser MultiValueSource mv1; MultiValueSource mv2; } + + private static class TInfo { + String field; + String val; + String indexedField; + BytesRef indexedBytes; + } + } Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/FileFloatSource.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/FileFloatSource.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/FileFloatSource.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/FileFloatSource.java Thu Jul 22 19:34:35 2010 @@ -17,10 +17,12 @@ package org.apache.solr.search.function; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.TermDocs; -import org.apache.lucene.index.TermEnum; -import org.apache.lucene.index.Term; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; import org.apache.solr.core.SolrCore; import org.apache.solr.schema.SchemaField; import org.apache.solr.schema.FieldType; @@ -220,29 +222,22 @@ public class FileFloatSource extends Val int notFoundCount=0; int otherErrors=0; - TermDocs termDocs = null; - Term protoTerm = new Term(idName, ""); - TermEnum termEnum = null; // Number of times to try termEnum.next() before resorting to skip int numTimesNext = 10; char delimiter='='; - String termVal; - boolean hasNext=true; - String prevKey=""; - String lastVal="\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF"; + BytesRef lastVal=new BytesRef("\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF\uFFFF"); + BytesRef internalKey = new BytesRef(); + BytesRef prevKey=new BytesRef(); + BytesRef tmp; try { - termDocs = reader.termDocs(); - termEnum = reader.terms(protoTerm); - Term t = termEnum.term(); - if (t != null && t.field() == idName) { // intern'd comparison - termVal = t.text(); - } else { - termVal = lastVal; - } - + TermsEnum termsEnum = MultiFields.getTerms(reader, idName).iterator(); + DocsEnum docsEnum = null; + BytesRef t = termsEnum.next(); + if (t==null) t=lastVal; + final Bits delDocs = MultiFields.getDeletedDocs(reader); for (String line; (line=r.readLine())!=null;) { int delimIndex = line.indexOf(delimiter); @@ -258,7 +253,9 @@ public class FileFloatSource extends Val String key = line.substring(0, delimIndex); String val = line.substring(delimIndex+1, endIndex); - String internalKey = idType.toInternal(key); + tmp = prevKey; prevKey=internalKey; internalKey=tmp; + idType.readableToIndexed(key, internalKey); + float fval; try { fval=Float.parseFloat(val); @@ -274,16 +271,16 @@ public class FileFloatSource extends Val if (sorted) { // make sure this key is greater than the previous key sorted = internalKey.compareTo(prevKey) >= 0; - prevKey = internalKey; if (sorted) { int countNext = 0; for(;;) { - int cmp = internalKey.compareTo(termVal); + int cmp = internalKey.compareTo(t); if (cmp == 0) { - termDocs.seek(termEnum); - while (termDocs.next()) { - vals[termDocs.doc()] = fval; + docsEnum = termsEnum.docs(delDocs, docsEnum); + int doc; + while ((doc = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) { + vals[doc] = fval; } break; } else if (cmp < 0) { @@ -301,32 +298,26 @@ public class FileFloatSource extends Val // so the best thing is to simply ask the reader for a new termEnum(target) // if we really need to skip. if (++countNext > numTimesNext) { - termEnum = reader.terms(protoTerm.createTerm(internalKey)); - t = termEnum.term(); + termsEnum.seek(internalKey); + t = termsEnum.term(); } else { - hasNext = termEnum.next(); - t = hasNext ? termEnum.term() : null; + t = termsEnum.next(); } - if (t != null && t.field() == idName) { // intern'd comparison - termVal = t.text(); - } else { - termVal = lastVal; - } + if (t==null) t = lastVal; } } // end for(;;) } } if (!sorted) { - termEnum = reader.terms(protoTerm.createTerm(internalKey)); - t = termEnum.term(); - if (t != null && t.field() == idName // intern'd comparison - && internalKey.equals(t.text())) - { - termDocs.seek (termEnum); - while (termDocs.next()) { - vals[termDocs.doc()] = fval; + TermsEnum.SeekStatus result = termsEnum.seek(internalKey); + t = termsEnum.term(); + if (result == TermsEnum.SeekStatus.FOUND) { + docsEnum = termsEnum.docs(delDocs, docsEnum); + int doc; + while ((doc = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) { + vals[doc] = fval; } } else { if (notFoundCount<10) { // collect first 10 not found for logging @@ -342,8 +333,6 @@ public class FileFloatSource extends Val } finally { // swallow exceptions on close so we don't override any // exceptions that happened in the loop - if (termDocs!=null) try{termDocs.close();}catch(Exception e){} - if (termEnum!=null) try{termEnum.close();}catch(Exception e){} try{r.close();}catch(Exception e){} } Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/Constants.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/Constants.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/Constants.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/Constants.java Thu Jul 22 19:34:35 2010 @@ -1,27 +0,0 @@ -package org.apache.solr.search.function.distance; -/** - * 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. - */ - - -/** - * - * - **/ -public interface Constants { - public static final double EARTH_RADIUS_KM = 6378.160187; - public static final double EARTH_RADIUS_MI = 3963.205; -} Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/DistanceUtils.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/DistanceUtils.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/DistanceUtils.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/DistanceUtils.java Thu Jul 22 19:34:35 2010 @@ -1,239 +0,0 @@ -package org.apache.solr.search.function.distance; - -import org.apache.solr.common.SolrException; -/** - * 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. - */ - - -/** - * Useful distance utiltities. - * solr-internal: subject to change w/o notification. - */ -public class DistanceUtils { - public static final double DEGREES_TO_RADIANS = Math.PI / 180.0; - public static final double RADIANS_TO_DEGREES = 180.0 / Math.PI; - - public static final double KM_TO_MILES = 0.621371192; - public static final double MILES_TO_KM = 1.609344; - - /** - * Calculate the p-norm (i.e. length) between two vectors - * - * @param vec1 The first vector - * @param vec2 The second vector - * @param power The power (2 for Euclidean distance, 1 for manhattan, etc.) - * @return The length. - *

- * See http://en.wikipedia.org/wiki/Lp_space - * @see #vectorDistance(double[], double[], double, double) - */ - public static double vectorDistance(double[] vec1, double[] vec2, double power) { - return vectorDistance(vec1, vec2, power, 1.0 / power); - } - - /** - * Calculate the p-norm (i.e. length) between two vectors - * - * @param vec1 The first vector - * @param vec2 The second vector - * @param power The power (2 for Euclidean distance, 1 for manhattan, etc.) - * @param oneOverPower If you've precalculated oneOverPower and cached it, use this method to save one division operation over {@link #vectorDistance(double[], double[], double)}. - * @return The length. - */ - public static double vectorDistance(double[] vec1, double[] vec2, double power, double oneOverPower) { - double result = 0; - - if (power == 0) { - for (int i = 0; i < vec1.length; i++) { - result += vec1[i] - vec2[i] == 0 ? 0 : 1; - } - - } else if (power == 1.0) { - for (int i = 0; i < vec1.length; i++) { - result += vec1[i] - vec2[i]; - } - } else if (power == 2.0) { - result = Math.sqrt(squaredEuclideanDistance(vec1, vec2)); - } else if (power == Integer.MAX_VALUE || Double.isInfinite(power)) {//infininte norm? - for (int i = 0; i < vec1.length; i++) { - result = Math.max(result, Math.max(vec1[i], vec2[i])); - } - } else { - for (int i = 0; i < vec1.length; i++) { - result += Math.pow(vec1[i] - vec2[i], power); - } - result = Math.pow(result, oneOverPower); - } - return result; - } - - public static double squaredEuclideanDistance(double[] vec1, double[] vec2) { - double result = 0; - for (int i = 0; i < vec1.length; i++) { - double v = vec1[i] - vec2[i]; - result += v * v; - } - return result; - } - - /** - * @param x1 The x coordinate of the first point - * @param y1 The y coordinate of the first point - * @param x2 The x coordinate of the second point - * @param y2 The y coordinate of the second point - * @param radius The radius of the sphere - * @return The distance between the two points, as determined by the Haversine formula. - * @see org.apache.solr.search.function.distance.HaversineFunction - */ - public static double haversine(double x1, double y1, double x2, double y2, double radius) { - double result = 0; - //make sure they aren't all the same, as then we can just return 0 - if ((x1 != x2) || (y1 != y2)) { - double diffX = x1 - x2; - double diffY = y1 - y2; - double hsinX = Math.sin(diffX * 0.5); - double hsinY = Math.sin(diffY * 0.5); - double h = hsinX * hsinX + - (Math.cos(x1) * Math.cos(x2) * hsinY * hsinY); - result = (radius * 2 * Math.atan2(Math.sqrt(h), Math.sqrt(1 - h))); - } - return result; - } - - /** - * Given a string containing dimension values encoded in it, separated by commas, return a String array of length dimension - * containing the values. - * - * @param out A preallocated array. Must be size dimension. If it is not it will be resized. - * @param externalVal The value to parse - * @param dimension The expected number of values for the point - * @return An array of the values that make up the point (aka vector) - * @throws {@link SolrException} if the dimension specified does not match the number of values in the externalValue. - */ - public static String[] parsePoint(String[] out, String externalVal, int dimension) { - //TODO: Should we support sparse vectors? - if (out == null || out.length != dimension) out = new String[dimension]; - int idx = externalVal.indexOf(','); - int end = idx; - int start = 0; - int i = 0; - if (idx == -1 && dimension == 1 && externalVal.length() > 0) {//we have a single point, dimension better be 1 - out[0] = externalVal.trim(); - i = 1; - } else if (idx > 0) {//if it is zero, that is an error - //Parse out a comma separated list of point values, as in: 73.5,89.2,7773.4 - for (; i < dimension; i++) { - while (start < end && externalVal.charAt(start) == ' ') start++; - while (end > start && externalVal.charAt(end - 1) == ' ') end--; - if (start == end){ - break; - } - out[i] = externalVal.substring(start, end); - start = idx + 1; - end = externalVal.indexOf(',', start); - idx = end; - if (end == -1) { - end = externalVal.length(); - } - } - } - if (i != dimension) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "incompatible dimension (" + dimension + - ") and values (" + externalVal + "). Only " + i + " values specified"); - } - return out; - } - - /** - * Given a string containing dimension values encoded in it, separated by commas, return a double array of length dimension - * containing the values. - * - * @param out A preallocated array. Must be size dimension. If it is not it will be resized. - * @param externalVal The value to parse - * @param dimension The expected number of values for the point - * @return An array of the values that make up the point (aka vector) - * @throws {@link SolrException} if the dimension specified does not match the number of values in the externalValue. - */ - public static double[] parsePointDouble(double[] out, String externalVal, int dimension) { - if (out == null || out.length != dimension) out = new double[dimension]; - int idx = externalVal.indexOf(','); - int end = idx; - int start = 0; - int i = 0; - if (idx == -1 && dimension == 1 && externalVal.length() > 0) {//we have a single point, dimension better be 1 - out[0] = Double.parseDouble(externalVal.trim()); - i = 1; - } else if (idx > 0) {//if it is zero, that is an error - //Parse out a comma separated list of point values, as in: 73.5,89.2,7773.4 - for (; i < dimension; i++) { - //TODO: abstract common code with other parsePoint - while (start < end && externalVal.charAt(start) == ' ') start++; - while (end > start && externalVal.charAt(end - 1) == ' ') end--; - if (start == end){ - break; - } - out[i] = Double.parseDouble(externalVal.substring(start, end)); - start = idx + 1; - end = externalVal.indexOf(',', start); - idx = end; - if (end == -1) { - end = externalVal.length(); - } - } - } - if (i != dimension) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "incompatible dimension (" + dimension + - ") and values (" + externalVal + "). Only " + i + " values specified"); - } - return out; - } - - /** - * extract (by calling {@link #parsePoint(String[], String, int)} and validate the latitude and longitude contained - * in the String by making sure the latitude is between 90 & -90 and longitude is between -180 and 180. - *

- * The latitude is assumed to be the first part of the string and the longitude the second part. - * - * @param latLon A preallocated array to hold the result - * @param latLonStr The string to parse - * @return The lat long - */ - public static final double[] parseLatitudeLongitude(double[] latLon, String latLonStr) { - if (latLon == null) { - latLon = new double[2]; - } - double[] toks = DistanceUtils.parsePointDouble(null, latLonStr, 2); - latLon[0] = Double.valueOf(toks[0]); - - if (latLon[0] < -90.0 || latLon[0] > 90.0) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, - "Invalid latitude: latitudes are range -90 to 90: provided lat: [" - + latLon[0] + "]"); - } - - latLon[1] = Double.valueOf(toks[1]); - - if (latLon[1] < -180.0 || latLon[1] > 180.0) { - - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, - "Invalid longitude: longitudes are range -180 to 180: provided lon: [" - + latLon[1] + "]"); - } - - return latLon; - } -} Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java Thu Jul 22 19:34:35 2010 @@ -17,6 +17,7 @@ package org.apache.solr.search.function. */ +import org.apache.lucene.spatial.DistanceUtils; import org.apache.solr.search.function.ValueSource; import org.apache.solr.search.function.DocValues; import org.apache.lucene.index.IndexReader; @@ -94,6 +95,8 @@ public class GeohashHaversineFunction ex String h1 = gh1DV.strVal(doc); String h2 = gh2DV.strVal(doc); if (h1 != null && h2 != null && h1.equals(h2) == false){ + //TODO: If one of the hashes is a literal value source, seems like we could cache it + //and avoid decoding every time double[] h1Pair = GeoHashUtils.decode(h1); double[] h2Pair = GeoHashUtils.decode(h2); result = DistanceUtils.haversine(Math.toRadians(h1Pair[0]), Math.toRadians(h1Pair[1]), Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/HaversineFunction.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/HaversineFunction.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/HaversineFunction.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/HaversineFunction.java Thu Jul 22 19:34:35 2010 @@ -18,6 +18,7 @@ package org.apache.solr.search.function. import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.Searcher; +import org.apache.lucene.spatial.DistanceUtils; import org.apache.solr.common.SolrException; import org.apache.solr.search.function.MultiValueSource; import org.apache.solr.search.function.DocValues; Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/SquaredEuclideanFunction.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/SquaredEuclideanFunction.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/SquaredEuclideanFunction.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/SquaredEuclideanFunction.java Thu Jul 22 19:34:35 2010 @@ -16,6 +16,7 @@ package org.apache.solr.search.function. * limitations under the License. */ +import org.apache.lucene.spatial.DistanceUtils; import org.apache.solr.search.function.DocValues; import org.apache.solr.search.function.MultiValueSource; Modified: lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/VectorDistanceFunction.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/VectorDistanceFunction.java?rev=966819&r1=966818&r2=966819&view=diff ============================================================================== --- lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/VectorDistanceFunction.java (original) +++ lucene/dev/branches/realtime_search/solr/src/java/org/apache/solr/search/function/distance/VectorDistanceFunction.java Thu Jul 22 19:34:35 2010 @@ -18,6 +18,7 @@ package org.apache.solr.search.function. import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.Searcher; +import org.apache.lucene.spatial.DistanceUtils; import org.apache.solr.common.SolrException; import org.apache.solr.search.function.DocValues; import org.apache.solr.search.function.MultiValueSource;