You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by Koji Sekiguchi <ko...@r.email.ne.jp> on 2010/08/21 05:51:41 UTC

Re: svn commit: r987690 - in /lucene/dev/trunk/solr: ./ src/common/org/apache/solr/common/params/ src/java/org/apache/solr/handler/component/ src/java/org/apache/solr/response/ src/java/org/apache/solr/schema/ src/java/org/apache/solr/search/ src/java/org/...

  Can't this be back ported to branch_3x? Are there something depend on flex?

Thanks,

Koji

-- 
http://www.rondhuit.com/en/


(10/08/21 11:36), yonik@apache.org wrote:
> Author: yonik
> Date: Sat Aug 21 02:36:09 2010
> New Revision: 987690
>
> URL: http://svn.apache.org/viewvc?rev=987690&view=rev
> Log:
> SOLR-1682: field collapsing / grouping
>
> Added:
>      lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java   (with props)
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java   (with props)
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java   (with props)
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java   (with props)
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java   (with props)
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java   (with props)
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java   (with props)
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueLong.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java   (with props)
>      lucene/dev/trunk/solr/src/test/org/apache/solr/TestGroupingSearch.java   (with props)
> Modified:
>      lucene/dev/trunk/solr/CHANGES.txt
>      lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/QueryComponent.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/response/JSONResponseWriter.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/schema/TrieField.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocValues.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DoubleFieldSource.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/FloatFieldSource.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/IntFieldSource.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/LongFieldSource.java
>      lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/StringIndexDocValues.java
>
> Modified: lucene/dev/trunk/solr/CHANGES.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/CHANGES.txt (original)
> +++ lucene/dev/trunk/solr/CHANGES.txt Sat Aug 21 02:36:09 2010
> @@ -232,6 +232,13 @@ New Features
>
>   * SOLR-2053: Add support for custom comparators in Solr spellchecker, per LUCENE-2479 (gsingers)
>
> +* SOLR-1682: (SOLR-236, SOLR-237, SOLR-1773, SOLR-1311) Search grouping / Field collapsing.
> +  (Martijn van Groningen, Emmanuel Keller, Shalin Shekhar Mangar,
> +   Koji Sekiguchi, Iván de Prado, Ryan McKinley, Marc Sturlese, Peter Karich,
> +   Bojan Smid, Charles Hornberger, Dieter Grad, Dmitry Lihachev, Doug Steigerwald,
> +   Karsten Sperling, Michael Gundlach, Oleg Gnatovskiy, Thomas Traeger, yonik)
> +
> +
>   Optimizations
>   ----------------------
>
>
> Added: lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java (added)
> +++ lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,34 @@
> +/**
> + * 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.
> + */
> +
> +package org.apache.solr.common.params;
> +
> +/**
> + * Facet parameters
> + */
> +public interface GroupParams {
> +  public static final String GROUP = "group";
> +
> +  public static final String GROUP_QUERY = GROUP + ".query";
> +  public static final String GROUP_FIELD = GROUP + ".field";
> +  public static final String GROUP_FUNC = GROUP + ".func";
> +  public static final String GROUP_SORT = GROUP + ".sort";
> +
> +  /** the limit for the number of documents in each group */
> +  public static final String GROUP_LIMIT = GROUP + ".limit";
> +}
> +
>
> Propchange: lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/common/org/apache/solr/common/params/GroupParams.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/QueryComponent.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/handler/component/QueryComponent.java Sat Aug 21 02:36:09 2010
> @@ -28,10 +28,7 @@ import org.apache.lucene.util.BytesRef;
>   import org.apache.solr.common.SolrDocument;
>   import org.apache.solr.common.SolrDocumentList;
>   import org.apache.solr.common.SolrException;
> -import org.apache.solr.common.params.CommonParams;
> -import org.apache.solr.common.params.ModifiableSolrParams;
> -import org.apache.solr.common.params.ShardParams;
> -import org.apache.solr.common.params.SolrParams;
> +import org.apache.solr.common.params.*;
>   import org.apache.solr.common.util.NamedList;
>   import org.apache.solr.common.util.StrUtils;
>   import org.apache.solr.request.SolrQueryRequest;
> @@ -39,6 +36,10 @@ import org.apache.solr.response.SolrQuer
>   import org.apache.solr.schema.FieldType;
>   import org.apache.solr.schema.SchemaField;
>   import org.apache.solr.search.*;
> +import org.apache.solr.search.function.BoostedQuery;
> +import org.apache.solr.search.function.FunctionQuery;
> +import org.apache.solr.search.function.QueryValueSource;
> +import org.apache.solr.search.function.ValueSource;
>   import org.apache.solr.util.SolrPluginUtils;
>
>   import java.io.IOException;
> @@ -175,6 +176,77 @@ public class QueryComponent extends Sear
>       SolrIndexSearcher.QueryCommand cmd = rb.getQueryCommand();
>       cmd.setTimeAllowed(timeAllowed);
>       SolrIndexSearcher.QueryResult result = new SolrIndexSearcher.QueryResult();
> +
> +    //
> +    // grouping / field collapsing
> +    //
> +    boolean doGroup = params.getBool(GroupParams.GROUP, false);
> +    if (doGroup) {
> +      try {
> +        cmd.groupCommands = new ArrayList<SolrIndexSearcher.GroupCommand>();
> +
> +        String[] fields = params.getParams(GroupParams.GROUP_FIELD);
> +        String[] funcs = params.getParams(GroupParams.GROUP_FUNC);
> +        String[] queries = params.getParams(GroupParams.GROUP_QUERY);
> +        String groupSortStr = params.get(GroupParams.GROUP_SORT);
> +        Sort groupSort = groupSortStr != null ? QueryParsing.parseSort(groupSortStr, req.getSchema()) : null;
> +
> +        int limitDefault = cmd.getLen(); // this is normally from "rows"
> +        int docsPerGroupDefault = params.getInt(GroupParams.GROUP_LIMIT, 1);
> +
> +        // temporary: implement all group-by-field as group-by-func
> +        if (funcs == null) {
> +          funcs = fields;
> +        } else if (fields != null) {
> +          // catenate functions and fields
> +          String[] both = new String[fields.length + funcs.length];
> +          System.arraycopy(fields, 0, both, 0, fields.length);
> +          System.arraycopy(funcs, 0, both, fields.length, funcs.length);
> +          funcs = both;
> +        }
> +
> +
> +        if (funcs != null) {
> +          for (String groupByStr : funcs) {
> +            QParser parser = QParser.getParser(groupByStr, "func", rb.req);
> +            Query q = parser.getQuery();
> +            SolrIndexSearcher.GroupCommandFunc gc;
> +            if (groupSort != null) {
> +              SolrIndexSearcher.GroupSortCommand gcSort = new SolrIndexSearcher.GroupSortCommand();
> +              gcSort.sort = groupSort;
> +              gc = gcSort;
> +            } else {
> +              gc =  new SolrIndexSearcher.GroupCommandFunc();
> +            }
> +
> +            if (q instanceof FunctionQuery) {
> +              gc.groupBy = ((FunctionQuery)q).getValueSource();
> +            } else {
> +              gc.groupBy = new QueryValueSource(q, 0.0f);
> +            }
> +            gc.key = groupByStr;
> +            gc.groupLimit = limitDefault;
> +            gc.docsPerGroup = docsPerGroupDefault;
> +
> +            cmd.groupCommands.add(gc);
> +          }
> +        }
> +
> +
> +        if (cmd.groupCommands.size() == 0)
> +          cmd.groupCommands = null;
> +
> +        if (cmd.groupCommands != null) {
> +          searcher.search(result,cmd);
> +          rsp.add("grouped", result.groupedResults);
> +          return;
> +        }
> +      } catch (ParseException e) {
> +        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
> +      }
> +    }
> +
> +    // normal search result
>       searcher.search(result,cmd);
>       rb.setResult( result );
>
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/response/JSONResponseWriter.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/response/JSONResponseWriter.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/response/JSONResponseWriter.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/response/JSONResponseWriter.java Sat Aug 21 02:36:09 2010
> @@ -483,6 +483,8 @@ class JSONWriter extends TextResponseWri
>       boolean first=true;
>
>       SolrIndexSearcher searcher = req.getSearcher();
> +    // be defensive... write out the doc even if we don't have the scores like we should
> +    includeScore = includeScore&&  ids.hasScores();
>       DocIterator iterator = ids.iterator();
>       for (int i=0; i<sz; i++) {
>         int id = iterator.nextDoc();
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/schema/TrieField.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/schema/TrieField.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/schema/TrieField.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/schema/TrieField.java Sat Aug 21 02:36:09 2010
> @@ -28,6 +28,8 @@ import org.apache.solr.analysis.*;
>   import org.apache.solr.common.SolrException;
>   import org.apache.solr.response.TextResponseWriter;
>   import org.apache.solr.response.XMLWriter;
> +import org.apache.solr.search.MutableValueDate;
> +import org.apache.solr.search.MutableValueLong;
>   import org.apache.solr.search.QParser;
>   import org.apache.solr.search.function.*;
>
> @@ -558,6 +560,11 @@ class TrieDateFieldSource extends LongFi
>     }
>
>     @Override
> +  protected MutableValueLong newMutableValueLong() {
> +    return new MutableValueDate();
> +  }
> +
> +  @Override
>     public long externalToLong(String extVal) {
>       return TrieField.dateField.parseMath(null, extVal).getTime();
>     }
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,543 @@
> +/**
> + * 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.
> + */
> +
> +package org.apache.solr.search;
> +
> +import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.*;
> +import org.apache.solr.search.function.DocValues;
> +import org.apache.solr.search.function.ValueSource;
> +
> +import java.io.IOException;
> +import java.util.*;
> +
> +public class MultiCollector extends Collector {
> +  final Collector[] collectors;
> +  final boolean acceptsDocsOutOfOrder;
> +
> +  public static Collector wrap(List<? extends Collector>  collectors) {
> +    return collectors.size() == 1 ? collectors.get(0) : new MultiCollector(collectors);
> +  }
> +
> +  public static Collector[] subCollectors(Collector collector) {
> +    if (collector instanceof MultiCollector)
> +      return ((MultiCollector)collector).collectors;
> +    return new Collector[]{collector};
> +  }
> +
> +  public MultiCollector(List<? extends Collector>  collectors) {
> +    this(collectors.toArray(new Collector[collectors.size()]));
> +  }
> +
> +  public MultiCollector(Collector[] collectors) {
> +    this.collectors = collectors;
> +
> +    boolean acceptsDocsOutOfOrder = true;
> +    for (Collector collector : collectors) {
> +      if (collector.acceptsDocsOutOfOrder() == false) {
> +        acceptsDocsOutOfOrder = false;
> +        break;
> +      }
> +    }
> +    this.acceptsDocsOutOfOrder = acceptsDocsOutOfOrder;
> +  }
> +
> +  @Override
> +  public void setScorer(Scorer scorer) throws IOException {
> +    for (Collector collector : collectors)
> +      collector.setScorer(scorer);
> +  }
> +
> +  @Override
> +  public void collect(int doc) throws IOException {
> +    for (Collector collector : collectors)
> +      collector.collect(doc);
> +  }
> +
> +  @Override
> +  public void setNextReader(IndexReader reader, int docBase) throws IOException {
> +    for (Collector collector : collectors)
> +      collector.setNextReader(reader, docBase);
> +  }
> +
> +  @Override
> +  public boolean acceptsDocsOutOfOrder() {
> +    return acceptsDocsOutOfOrder;
> +  }
> +}
> +
> +
> +
> +
> +
> +
> +class SearchGroup {
> +  public MutableValue groupValue;
> +  int matches;
> +  int topDoc;
> +  // float topDocScore;  // currently unused
> +  int comparatorSlot;
> +
> +  // currently only used when sort != sort.group
> +  FieldComparator[] sortGroupComparators;
> +  int[] sortGroupReversed;
> +
> +  /***
> +  @Override
> +  public int hashCode() {
> +    return super.hashCode();
> +  }
> +
> +  @Override
> +  public boolean equals(Object obj) {
> +    return groupValue.equalsSameType(((SearchGroup)obj).groupValue);
> +  }
> +  ***/
> +}
> +
> +
> +
> +/** Finds the top set of groups, grouped by groupByVS when sort == group.sort */
> +class TopGroupCollector extends Collector {
> +  final int nGroups;
> +  final HashMap<MutableValue, SearchGroup>  groupMap;
> +  TreeSet<SearchGroup>  orderedGroups;
> +  final ValueSource vs;
> +  final Map context;
> +  final FieldComparator[] comparators;
> +  final int[] reversed;
> +
> +  DocValues docValues;
> +  DocValues.ValueFiller filler;
> +  MutableValue mval;
> +  Scorer scorer;
> +  int docBase;
> +  int spareSlot;
> +
> +  int matches;
> +
> +  public TopGroupCollector(ValueSource groupByVS, Map vsContext, Sort sort, int nGroups) throws IOException {
> +    this.vs = groupByVS;
> +    this.context = vsContext;
> +    this.nGroups = nGroups;
> +
> +    SortField[] sortFields = sort.getSort();
> +    this.comparators = new FieldComparator[sortFields.length];
> +    this.reversed = new int[sortFields.length];
> +    for (int i = 0; i<  sortFields.length; i++) {
> +      SortField sortField = sortFields[i];
> +      reversed[i] = sortField.getReverse() ? -1 : 1;
> +      // use nGroups + 1 so we have a spare slot to use for comparing (tracked by this.spareSlot)
> +      comparators[i] = sortField.getComparator(nGroups + 1, i);
> +    }
> +    this.spareSlot = nGroups;
> +
> +    this.groupMap = new HashMap<MutableValue, SearchGroup>(nGroups);
> +  }
> +
> +  @Override
> +  public void setScorer(Scorer scorer) throws IOException {
> +    this.scorer = scorer;
> +    for (FieldComparator fc : comparators)
> +      fc.setScorer(scorer);
> +  }
> +
> +  @Override
> +  public void collect(int doc) throws IOException {
> +    matches++;
> +    filler.fillValue(doc);
> +    SearchGroup group = groupMap.get(mval);
> +    if (group == null) {
> +      int num = groupMap.size();
> +      if (groupMap.size()<  nGroups) {
> +        SearchGroup sg = new SearchGroup();
> +        sg.groupValue = mval.duplicate();
> +        sg.comparatorSlot = num++;
> +        sg.matches = 1;
> +        sg.topDoc = docBase + doc;
> +        // sg.topDocScore = scorer.score();
> +        for (FieldComparator fc : comparators)
> +          fc.copy(sg.comparatorSlot, doc);
> +        groupMap.put(sg.groupValue, sg);
> +        return;
> +      }
> +
> +      if (orderedGroups == null) {
> +        buildSet();
> +      }
> +
> +
> +      for (int i = 0;; i++) {
> +        final int c = reversed[i] * comparators[i].compareBottom(doc);
> +        if (c<  0) {
> +          // Definitely not competitive.
> +          return;
> +        } else if (c>  0) {
> +          // Definitely competitive.
> +          break;
> +        } else if (i == comparators.length - 1) {
> +          // Here c=0. If we're at the last comparator, this doc is not
> +          // competitive, since docs are visited in doc Id order, which means
> +          // this doc cannot compete with any other document in the queue.
> +          return;
> +        }
> +      }
> +
> +      // remove current smallest group
> +      SearchGroup smallest = orderedGroups.pollLast();
> +      groupMap.remove(smallest.groupValue);
> +
> +      // reuse the removed SearchGroup
> +      smallest.groupValue.copy(mval);
> +      smallest.matches = 1;
> +      smallest.topDoc = docBase + doc;
> +      // smallest.topDocScore = scorer.score();
> +      for (FieldComparator fc : comparators)
> +        fc.copy(smallest.comparatorSlot, doc);
> +
> +      groupMap.put(smallest.groupValue, smallest);
> +      orderedGroups.add(smallest);
> +
> +      for (FieldComparator fc : comparators)
> +        fc.setBottom(orderedGroups.last().comparatorSlot);
> +
> +      return;
> +    }
> +
> +    //
> +    // update existing group
> +    //
> +
> +    group.matches++; // TODO: these aren't valid if the group is every discarded then re-added.  keep track if there have been discards?
> +
> +    for (int i = 0;; i++) {
> +      FieldComparator fc = comparators[i];
> +      fc.copy(spareSlot, doc);
> +
> +      final int c = reversed[i] * fc.compare(group.comparatorSlot, spareSlot);
> +      if (c<  0) {
> +        // Definitely not competitive.
> +        return;
> +      } else if (c>  0) {
> +        // Definitely competitive.
> +        // Set remaining comparators
> +        for (int j=i+1; j<comparators.length; j++)
> +          comparators[j].copy(spareSlot, doc);
> +        break;
> +      } else if (i == comparators.length - 1) {
> +        // Here c=0. If we're at the last comparator, this doc is not
> +        // competitive, since docs are visited in doc Id order, which means
> +        // this doc cannot compete with any other document in the queue.
> +        return;
> +      }
> +    }
> +
> +    // remove before updating the group since lookup is done via comparators
> +    // TODO: optimize this
> +    if (orderedGroups != null)
> +      orderedGroups.remove(group);
> +
> +    group.topDoc = docBase + doc;
> +    // group.topDocScore = scorer.score();
> +    int tmp = spareSlot; spareSlot = group.comparatorSlot; group.comparatorSlot=tmp;  // swap slots
> +
> +    // re-add the changed group
> +    if (orderedGroups != null)
> +      orderedGroups.add(group);
> +  }
> +
> +  void buildSet() {
> +    Comparator<SearchGroup>  comparator = new Comparator<SearchGroup>() {
> +      public int compare(SearchGroup o1, SearchGroup o2) {
> +        for (int i = 0;; i++) {
> +          FieldComparator fc = comparators[i];
> +          int c = reversed[i] * fc.compare(o1.comparatorSlot, o2.comparatorSlot);
> +          if (c != 0) {
> +            return c;
> +          } else if (i == comparators.length - 1) {
> +            return o1.topDoc - o2.topDoc;
> +          }
> +        }
> +      }
> +    };
> +
> +    orderedGroups = new TreeSet<SearchGroup>(comparator);
> +    orderedGroups.addAll(groupMap.values());
> +    if (orderedGroups.size() == 0) return;
> +    for (FieldComparator fc : comparators)
> +      fc.setBottom(orderedGroups.last().comparatorSlot);
> +  }
> +
> +  @Override
> +  public void setNextReader(IndexReader reader, int docBase) throws IOException {
> +    this.docBase = docBase;
> +    docValues = vs.getValues(context, reader);
> +    filler = docValues.getValueFiller();
> +    mval = filler.getValue();
> +    for (FieldComparator fc : comparators)
> +      fc.setNextReader(reader, docBase);
> +  }
> +
> +  @Override
> +  public boolean acceptsDocsOutOfOrder() {
> +    return false;
> +  }
> +
> +  public int getMatches() {
> +    return matches;
> +  }
> +}
> +
> +
> +/**
> + * This class allows a different sort within a group than what is used between groups.
> + * Sorting between groups is done by the sort value of the first (highest ranking)
> + * document in that group.
> + */
> +class TopGroupSortCollector extends TopGroupCollector {
> +
> +  IndexReader reader;
> +  Sort groupSort;
> +
> +  public TopGroupSortCollector(ValueSource groupByVS, Map vsContext, Sort sort, Sort groupSort, int nGroups) throws IOException {
> +    super(groupByVS, vsContext, sort, nGroups);
> +    this.groupSort = groupSort;
> +  }
> +
> +  void constructComparators(FieldComparator[] comparators, int[] reversed, SortField[] sortFields, int size) throws IOException {
> +    for (int i = 0; i<  sortFields.length; i++) {
> +      SortField sortField = sortFields[i];
> +      reversed[i] = sortField.getReverse() ? -1 : 1;
> +      comparators[i] = sortField.getComparator(size, i);
> +      if (scorer != null) comparators[i].setScorer(scorer);
> +      if (reader != null) comparators[i].setNextReader(reader, docBase);
> +    }
> +  }
> +
> +  @Override
> +  public void setScorer(Scorer scorer) throws IOException {
> +    super.setScorer(scorer);
> +    for (SearchGroup searchGroup : groupMap.values()) {
> +      for (FieldComparator fc : searchGroup.sortGroupComparators) {
> +        fc.setScorer(scorer);
> +      }
> +    }
> +  }
> +
> +  @Override
> +  public void collect(int doc) throws IOException {
> +    matches++;
> +    filler.fillValue(doc);
> +    SearchGroup group = groupMap.get(mval);
> +    if (group == null) {
> +      int num = groupMap.size();
> +      if (groupMap.size()<  nGroups) {
> +        SearchGroup sg = new SearchGroup();
> +        SortField[] sortGroupFields = groupSort.getSort();
> +        sg.sortGroupComparators = new FieldComparator[sortGroupFields.length];
> +        sg.sortGroupReversed = new int[sortGroupFields.length];
> +        constructComparators(sg.sortGroupComparators, sg.sortGroupReversed, sortGroupFields, 1);
> +
> +        sg.groupValue = mval.duplicate();
> +        sg.comparatorSlot = num++;
> +        sg.matches = 1;
> +        sg.topDoc = docBase + doc;
> +        // sg.topDocScore = scorer.score();
> +        for (FieldComparator fc : comparators)
> +          fc.copy(sg.comparatorSlot, doc);
> +        for (FieldComparator fc : sg.sortGroupComparators) {
> +          fc.copy(0, doc);
> +          fc.setBottom(0);
> +        }
> +        groupMap.put(sg.groupValue, sg);
> +        return;
> +      }
> +
> +      if (orderedGroups == null) {
> +        buildSet();
> +      }
> +
> +      SearchGroup leastSignificantGroup = orderedGroups.last();
> +      for (int i = 0;; i++) {
> +        final int c = leastSignificantGroup.sortGroupReversed[i] * leastSignificantGroup.sortGroupComparators[i].compareBottom(doc);
> +        if (c<  0) {
> +          // Definitely not competitive.
> +          return;
> +        } else if (c>  0) {
> +          // Definitely competitive.
> +          break;
> +        } else if (i == leastSignificantGroup.sortGroupComparators.length - 1) {
> +          // Here c=0. If we're at the last comparator, this doc is not
> +          // competitive, since docs are visited in doc Id order, which means
> +          // this doc cannot compete with any other document in the queue.
> +          return;
> +        }
> +      }
> +
> +      // remove current smallest group
> +      SearchGroup smallest = orderedGroups.pollLast();
> +      groupMap.remove(smallest.groupValue);
> +
> +      // reuse the removed SearchGroup
> +      smallest.groupValue.copy(mval);
> +      smallest.matches = 1;
> +      smallest.topDoc = docBase + doc;
> +      // smallest.topDocScore = scorer.score();
> +      for (FieldComparator fc : comparators)
> +        fc.copy(smallest.comparatorSlot, doc);
> +      for (FieldComparator fc : smallest.sortGroupComparators) {
> +        fc.copy(0, doc);
> +        fc.setBottom(0);
> +      }
> +
> +      groupMap.put(smallest.groupValue, smallest);
> +      orderedGroups.add(smallest);
> +
> +      for (FieldComparator fc : comparators)
> +        fc.setBottom(orderedGroups.last().comparatorSlot);
> +      for (FieldComparator fc : smallest.sortGroupComparators)
> +        fc.setBottom(0);
> +
> +      return;
> +    }
> +
> +    //
> +    // update existing group
> +    //
> +
> +    group.matches++; // TODO: these aren't valid if the group is every discarded then re-added.  keep track if there have been discards?
> +
> +    for (int i = 0;; i++) {
> +      FieldComparator fc = group.sortGroupComparators[i];
> +
> +      final int c = group.sortGroupReversed[i] * fc.compareBottom(doc);
> +      if (c<  0) {
> +        // Definitely not competitive.
> +        return;
> +      } else if (c>  0) {
> +        // Definitely competitive.
> +        // Set remaining comparators
> +        for (int j = 0; j<  group.sortGroupComparators.length; j++) {
> +          group.sortGroupComparators[j].copy(0, doc);
> +          group.sortGroupComparators[j].setBottom(0);
> +        }
> +        for (FieldComparator comparator : comparators) comparator.copy(spareSlot, doc);
> +        break;
> +      } else if (i == group.sortGroupComparators.length - 1) {
> +        // Here c=0. If we're at the last comparator, this doc is not
> +        // competitive, since docs are visited in doc Id order, which means
> +        // this doc cannot compete with any other document in the queue.
> +        return;
> +      }
> +    }
> +
> +    // remove before updating the group since lookup is done via comparators
> +    // TODO: optimize this
> +    if (orderedGroups != null)
> +      orderedGroups.remove(group);
> +
> +    group.topDoc = docBase + doc;
> +    // group.topDocScore = scorer.score();
> +    int tmp = spareSlot; spareSlot = group.comparatorSlot; group.comparatorSlot=tmp;  // swap slots
> +
> +    // re-add the changed group
> +    if (orderedGroups != null)
> +      orderedGroups.add(group);
> +  }
> +
> +  @Override
> +  public void setNextReader(IndexReader reader, int docBase) throws IOException {
> +    super.setNextReader(reader, docBase);
> +    this.reader = reader;
> +    for (SearchGroup searchGroup : groupMap.values()) {
> +      for (FieldComparator fc : searchGroup.sortGroupComparators) {
> +        fc.setNextReader(reader, docBase);
> +      }
> +    }
> +  }
> +
> +}
> +
> +
> +class Phase2GroupCollector extends Collector {
> +  final HashMap<MutableValue, SearchGroupDocs>  groupMap;
> +  final ValueSource vs;
> +  final Map context;
> +
> +  DocValues docValues;
> +  DocValues.ValueFiller filler;
> +  MutableValue mval;
> +  Scorer scorer;
> +  int docBase;
> +
> +  // TODO: may want to decouple from the phase1 collector
> +  public Phase2GroupCollector(TopGroupCollector topGroups, ValueSource groupByVS, Map vsContext, Sort sort, int docsPerGroup, boolean getScores) throws IOException {
> +    boolean getSortFields = false;
> +
> +    groupMap = new HashMap<MutableValue, SearchGroupDocs>(topGroups.groupMap.size());
> +    for (SearchGroup group : topGroups.groupMap.values()) {
> +      SearchGroupDocs groupDocs = new SearchGroupDocs();
> +      groupDocs.groupValue = group.groupValue;
> +      groupDocs.collector = TopFieldCollector.create(sort, docsPerGroup, getSortFields, getScores, getScores, true);
> +      groupMap.put(groupDocs.groupValue, groupDocs);
> +    }
> +
> +    this.vs = groupByVS;
> +    this.context = vsContext;
> +  }
> +
> +  @Override
> +  public void setScorer(Scorer scorer) throws IOException {
> +    this.scorer = scorer;
> +    for (SearchGroupDocs group : groupMap.values())
> +      group.collector.setScorer(scorer);
> +  }
> +
> +  @Override
> +  public void collect(int doc) throws IOException {
> +    filler.fillValue(doc);
> +    SearchGroupDocs group = groupMap.get(mval);
> +    if (group == null) return;
> +    group.matches++;
> +    group.collector.collect(doc);
> +  }
> +
> +  @Override
> +  public void setNextReader(IndexReader reader, int docBase) throws IOException {
> +    this.docBase = docBase;
> +    docValues = vs.getValues(context, reader);
> +    filler = docValues.getValueFiller();
> +    mval = filler.getValue();
> +    for (SearchGroupDocs group : groupMap.values())
> +      group.collector.setNextReader(reader, docBase);
> +  }
> +
> +  @Override
> +  public boolean acceptsDocsOutOfOrder() {
> +    return false;
> +  }
> +}
> +
> +// TODO: merge with SearchGroup or not?
> +// ad: don't need to build a new hashmap
> +// disad: blows up the size of SearchGroup if we need many of them, and couples implementations
> +class SearchGroupDocs {
> +  public MutableValue groupValue;
> +  int matches;
> +  TopFieldCollector collector;
> +}
> +
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MultiCollector.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,61 @@
> +/**
> + * 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.
> + */
> +package org.apache.solr.search;
> +
> +/** @lucene.internal */
> +public abstract class MutableValue implements Comparable {
> +  protected boolean exists = true;
> +
> +  public abstract void copy(MutableValue source);
> +  public abstract MutableValue duplicate();
> +  public abstract boolean equalsSameType(Object other);
> +  public abstract int compareSameType(Object other);
> +  public abstract Object toObject();
> +
> +  public boolean exists() {
> +    return exists;
> +  }
> +
> +  public int compareTo(Object other) {
> +    Class c1 = this.getClass();
> +    Class c2 = other.getClass();
> +    if (c1 != c2) {
> +      int c = c1.hashCode() - c2.hashCode();
> +      if (c == 0) {
> +        c = c1.getCanonicalName().compareTo(c2.getCanonicalName());
> +      }
> +      return c;
> +    }
> +    return compareSameType(other);
> +  }
> +
> +  @Override
> +  public boolean equals(Object other) {
> +    Class c1 = this.getClass();
> +    Class c2 = other.getClass();
> +    return (c1 == c2) ? this.equalsSameType(other) : false;
> +  }
> +
> +  public abstract int hashCode();
> +
> +  @Override
> +  public String toString() {
> +    return exists() ? toObject().toString() : "(null)";
> +  }
> +}
> +
> +
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValue.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,33 @@
> +/**
> + * 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.
> + */
> +package org.apache.solr.search;
> +
> +import java.util.Date;
> +
> +public class MutableValueDate extends MutableValueLong {
> +  @Override
> +  public Object toObject() {
> +    return new Date(value);
> +  }
> +
> +  @Override
> +  public MutableValue duplicate() {
> +    MutableValueDate v = new MutableValueDate();
> +    v.value = this.value;
> +    return v;
> +  }
> +}
> \ No newline at end of file
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDate.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,54 @@
> +/**
> + * 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.
> + */
> +package org.apache.solr.search;
> +
> +public class MutableValueDouble extends MutableValue {
> +  public double value;
> +
> +  @Override
> +  public Object toObject() {
> +    return value;
> +  }
> +
> +  @Override
> +  public void copy(MutableValue source) {
> +    value = ((MutableValueDouble)source).value;
> +  }
> +
> +  @Override
> +  public MutableValue duplicate() {
> +    MutableValueDouble v = new MutableValueDouble();
> +    v.value = this.value;
> +    return v;
> +  }
> +
> +  @Override
> +  public boolean equalsSameType(Object other) {
> +    return value == ((MutableValueDouble)other).value;
> +  }
> +
> +  @Override
> +  public int compareSameType(Object other) {
> +    return Double.compare(value, ((MutableValueDouble)other).value);  // handles NaN
> +  }
> +
> +  @Override
> +  public int hashCode() {
> +    long x = Double.doubleToLongBits(value);
> +    return (int)x + (int)(x>>>32);
> +  }
> +}
> \ No newline at end of file
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueDouble.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,53 @@
> +/**
> + * 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.
> + */
> +package org.apache.solr.search;
> +
> +public class MutableValueFloat extends MutableValue {
> +  public float value;
> +
> +  @Override
> +  public Object toObject() {
> +    return value;
> +  }
> +
> +  @Override
> +  public void copy(MutableValue source) {
> +    value = ((MutableValueFloat)source).value;
> +  }
> +
> +  @Override
> +  public MutableValue duplicate() {
> +    MutableValueFloat v = new MutableValueFloat();
> +    v.value = this.value;
> +    return v;
> +  }
> +
> +  @Override
> +  public boolean equalsSameType(Object other) {
> +    return value == ((MutableValueFloat)other).value;
> +  }
> +
> +  @Override
> +  public int compareSameType(Object other) {
> +    return Float.compare(value, ((MutableValueFloat)other).value);  // handles NaN
> +  }
> +
> +  @Override
> +  public int hashCode() {
> +    return Float.floatToIntBits(value);
> +  }
> +}
> \ No newline at end of file
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueFloat.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,64 @@
> +/**
> + * 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.
> + */
> +package org.apache.solr.search;
> +
> +public class MutableValueInt extends MutableValue {
> +  public int value;
> +
> +  @Override
> +  public Object toObject() {
> +    return value;
> +  }
> +
> +  @Override
> +  public void copy(MutableValue source) {
> +    value = ((MutableValueInt)source).value;
> +  }
> +
> +  @Override
> +  public MutableValue duplicate() {
> +    MutableValueInt v = new MutableValueInt();
> +    v.value = this.value;
> +    return v;
> +  }
> +
> +  @Override
> +  public boolean equalsSameType(Object other) {
> +    return value == ((MutableValueInt)other).value;
> +  }
> +
> +  @Override
> +  public int compareSameType(Object other) {
> +    int a = value;
> +    int b = ((MutableValueInt)other).value;
> +    return (int)((((long)a) - ((long)b))>>  32);  // any shift>= 32 should do.
> +
> +    /* is there any pattern that the compiler would recognize as a single native CMP instruction? */
> +    /***
> +    if (a<b) return -1;
> +    else if (a>b) return 1;
> +    else return 0;
> +    ***/
> +  }
> +
> +
> +  @Override
> +  public int hashCode() {
> +    // TODO: if used in HashMap, it already mixes the value... maybe use a straight value?
> +    return (value>>8) + (value>>16);
> +  }
> +}
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueInt.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueLong.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueLong.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueLong.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueLong.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,57 @@
> +/**
> + * 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.
> + */
> +package org.apache.solr.search;
> +
> +public class MutableValueLong extends MutableValue {
> +  public long value;
> +
> +  @Override
> +  public Object toObject() {
> +    return value;
> +  }
> +
> +  @Override
> +  public void copy(MutableValue source) {
> +    value = ((MutableValueLong)source).value;
> +  }
> +
> +  @Override
> +  public MutableValue duplicate() {
> +    MutableValueLong v = new MutableValueLong();
> +    v.value = this.value;
> +    return v;
> +  }
> +
> +  @Override
> +  public boolean equalsSameType(Object other) {
> +    return value == ((MutableValueLong)other).value;
> +  }
> +
> +  @Override
> +  public int compareSameType(Object other) {
> +    long b = ((MutableValueLong)other).value;
> +    if (value<b) return -1;
> +    else if (value>b) return 1;
> +    else return 0;
> +  }
> +
> +
> +  @Override
> +  public int hashCode() {
> +    return (int)value + (int)(value>>32);
> +  }
> +}
> \ No newline at end of file
>
> Added: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java (added)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,57 @@
> +/**
> + * 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.
> + */
> +package org.apache.solr.search;
> +
> +import org.apache.lucene.util.BytesRef;
> +import org.apache.solr.util.ByteUtils;
> +
> +public class MutableValueStr extends MutableValue {
> +  public BytesRef value = new BytesRef();
> +
> +  @Override
> +  public Object toObject() {
> +    return ByteUtils.UTF8toUTF16(value);
> +  }
> +
> +  @Override
> +  public void copy(MutableValue source) {
> +    value.copy(((MutableValueStr)source).value);
> +  }
> +
> +  @Override
> +  public MutableValue duplicate() {
> +    MutableValueStr v = new MutableValueStr();
> +    v.value = new BytesRef(value);
> +    return v;
> +  }
> +
> +  @Override
> +  public boolean equalsSameType(Object other) {
> +    return value.equals(((MutableValueStr)other).value);
> +  }
> +
> +  @Override
> +  public int compareSameType(Object other) {
> +    return value.compareTo(((MutableValueStr)other).value);
> +  }
> +
> +
> +  @Override
> +  public int hashCode() {
> +    return value.hashCode();
> +  }
> +}
> \ No newline at end of file
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java
> ------------------------------------------------------------------------------
>      svn:executable = *
>
> Propchange: lucene/dev/trunk/solr/src/java/org/apache/solr/search/MutableValueStr.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java Sat Aug 21 02:36:09 2010
> @@ -37,6 +37,9 @@ import org.apache.lucene.util.OpenBitSet
>   import java.io.IOException;
>   import java.net.URL;
>   import java.util.*;
> +
> +import org.apache.solr.search.function.DocValues;
> +import org.apache.solr.search.function.ValueSource;
>   import org.slf4j.Logger;
>   import org.slf4j.LoggerFactory;
>
> @@ -898,12 +901,138 @@ public class SolrIndexSearcher extends I
>
>     public static final int GET_SCORES             =       0x01;
>
> +
> +  private void groupBy(QueryResult qr, QueryCommand cmd) throws IOException {
> +    DocListAndSet out = new DocListAndSet();
> +    qr.setDocListAndSet(out);
> +
> +    DocSet filter = cmd.getFilter()!=null ? cmd.getFilter() : getDocSet(cmd.getFilterList());
> +
> +    int last = cmd.getOffset() + cmd.getLen();
> +    if (last<  0 || last>  maxDoc()) last=maxDoc();
> +
> +    boolean needScores = (cmd.getFlags()&  GET_SCORES) != 0;
> +
> +    Query query = QueryUtils.makeQueryable(cmd.getQuery());
> +
> +    final Filter luceneFilter = filter==null ? null : filter.getTopFilter();
> +
> +    Sort sort = cmd.getSort();
> +    if (sort == null) sort = new Sort();
> +
> +    // TODO: make this a generic collector list
> +    List<TopGroupCollector>  collectors = new ArrayList<TopGroupCollector>(cmd.groupCommands.size());
> +    for (GroupCommand groupCommand : cmd.groupCommands) {
> +      // TODO: perhaps use some methods rather than instanceof
> +      if (groupCommand instanceof GroupCommandFunc) {
> +        GroupCommandFunc gc = (GroupCommandFunc)groupCommand;
> +        Map context = ValueSource.newContext();
> +        gc.groupBy.createWeight(context, this);
> +        TopGroupCollector collector;
> +        if (gc instanceof GroupSortCommand) {
> +          GroupSortCommand sortGc = (GroupSortCommand) gc;
> +          collector = new TopGroupSortCollector(gc.groupBy, context, sort, sortGc.sort, last);
> +        } else {
> +          collector = new TopGroupCollector(gc.groupBy, context, sort, last);
> +        }
> +        collectors.add(collector);
> +
> +        // for next phase
> +        gc.context = context;
> +        gc.collector = collector;
> +      }
> +    }
> +
> +    search(query, luceneFilter, MultiCollector.wrap(collectors));
> +
> +    // TODO: make this a generic collector list
> +    List<Phase2GroupCollector>  phase2Collectors = new ArrayList<Phase2GroupCollector>(cmd.groupCommands.size());
> +    for (GroupCommand groupCommand : cmd.groupCommands) {
> +      if (groupCommand instanceof GroupCommandFunc) {
> +        GroupCommandFunc gc = (GroupCommandFunc)groupCommand;
> +        Sort collectorSort;
> +        if (gc instanceof GroupSortCommand) {
> +          collectorSort = ((GroupSortCommand) gc).sort;
> +        } else {
> +          collectorSort = sort;
> +        }
> +
> +        Phase2GroupCollector collector = new Phase2GroupCollector((TopGroupCollector)gc.collector, gc.groupBy, gc.context, collectorSort, gc.docsPerGroup, needScores);
> +        phase2Collectors.add(collector);
> +      }
> +    }
> +
> +    // TODO: optionally cache docs and feed them back through rather than re-searching
> +    search(query, luceneFilter, MultiCollector.wrap(phase2Collectors));
> +
> +
> +    NamedList grouped = new SimpleOrderedMap();
> +    for (int cmdnum=0; cmdnum<cmd.groupCommands.size(); cmdnum++) {
> +      GroupCommand groupCommand = cmd.groupCommands.get(cmdnum);
> +      GroupCommandFunc groupCommandFunc = (GroupCommandFunc)groupCommand;
> +      TopGroupCollector collector = collectors.get(cmdnum);
> +      Phase2GroupCollector collector2 = phase2Collectors.get(cmdnum);
> +
> +      if (collector.orderedGroups == null) collector.buildSet();
> +
> +      NamedList groupResult = new SimpleOrderedMap();
> +      grouped.add(groupCommand.key, groupResult);  // grouped={ key={
> +
> +      groupResult.add("matches", collector.getMatches());
> +
> +      List groupList = new ArrayList();
> +      groupResult.add("groups", groupList);        // grouped={ key={ groups=[
> +
> +      for (SearchGroup group : collector.orderedGroups) {
> +        NamedList nl = new SimpleOrderedMap();
> +        groupList.add(nl);                         // grouped={ key={ groups=[ {
> +
> +        nl.add("groupValue", group.groupValue.toObject());
> +
> +        SearchGroupDocs groupDocs = collector2.groupMap.get(group.groupValue);
> +        // nl.add("matches", groupDocs.matches);  // redundant with doclist.numFound from the doc list
> +
> +        TopDocs topDocs = groupDocs.collector.topDocs(0, groupCommandFunc.docsPerGroup);
> +        //topDocs.totalHits
> +        int ids[] = new int[topDocs.scoreDocs.length];
> +        float[] scores = needScores ? new float[topDocs.scoreDocs.length] : null;
> +        for (int i=0; i<ids.length; i++) {
> +          ids[i] = topDocs.scoreDocs[i].doc;
> +          if (scores != null)
> +            scores[i] = topDocs.scoreDocs[i].score;
> +        }
> +
> +        DocSlice docs = new DocSlice(0, ids.length, ids, scores, topDocs.totalHits, topDocs.getMaxScore());
> +        nl.add("doclist", docs);
> +
> +
> +        /*** values from stage 1
> +         DocSlice docs = new DocSlice(0, 1, new int[] {group.topDoc}, null, 1, 0);
> +         nl.add("docs", docs);
> +
> +         Object[] vals = new Object[collector.comparators.length];
> +         for (int i=0; i<vals.length; i++) {
> +         vals[i] = collector.comparators[i].value(group.comparatorSlot);
> +         }
> +         nl.add("groupSortValue", vals);
> +         groupResult.add(nl);
> +         ***/
> +      }
> +      qr.groupedResults = grouped;
> +
> +    }
> +  }
> +
>     /**
>      * getDocList version that uses+populates query and filter caches.
>      * In the event of a timeout, the cache is not populated.
>      */
>     private void getDocListC(QueryResult qr, QueryCommand cmd) throws IOException {
> -    // old parameters: DocListAndSet out, Query query, List<Query>  filterList, DocSet filter, Sort lsort, int offset, int len, int flags, long timeAllowed, NamedList<Object>  responseHeader
> +    if (cmd.groupCommands != null) {
> +      groupBy(qr, cmd);
> +      return;
> +    }
> +
>       DocListAndSet out = new DocListAndSet();
>       qr.setDocListAndSet(out);
>       QueryResultKey key=null;
> @@ -1714,6 +1843,8 @@ public class SolrIndexSearcher extends I
>       private int flags;
>       private long timeAllowed = -1;
>
> +    public List<GroupCommand>  groupCommands;
> +
>       public Query getQuery() { return query; }
>       public QueryCommand setQuery(Query query) {
>         this.query = query;
> @@ -1814,12 +1945,37 @@ public class SolrIndexSearcher extends I
>       }
>     }
>
> +  public static class GroupCommand {
> +    public String key;  // the name to use for this group in the response
> +    public Sort groupSort;  // the sort of the documents *within* a single group.
> +    public int groupLimit;   // how many groups - defaults to the "rows" parameter
> +    public int docsPerGroup; // how many docs in each group - from "group.limit" param, default=1
> +
> +
> +
> +  }
> +
> +  public static class GroupCommandFunc extends GroupCommand {
> +    public ValueSource groupBy;
> +
> +
> +    // todo - find a better place to store these
> +    transient Map context;
> +    transient Collector collector;
> +  }
> +
> +  public static class GroupSortCommand extends GroupCommandFunc {
> +    public Sort sort;
> +  }
> +
>     /**
>      * The result of a search.
>      */
>     public static class QueryResult {
>       private boolean partialResults;
>       private DocListAndSet docListAndSet;
> +
> +    public Object groupedResults;   // TODO: currently for testing
>
>       public DocList getDocList() { return docListAndSet.docList; }
>       public void setDocList(DocList list) {
> @@ -1847,5 +2003,3 @@ public class SolrIndexSearcher extends I
>   }
>
>
> -
> -
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java Sat Aug 21 02:36:09 2010
> @@ -20,8 +20,9 @@ package org.apache.solr.search.function;
>   import org.apache.lucene.index.IndexReader;
>   import org.apache.lucene.index.Term;
>   import org.apache.lucene.search.Searcher;
> -import org.apache.lucene.search.Similarity;
>   import org.apache.lucene.util.BytesRef;
> +import org.apache.solr.search.MutableValueInt;
> +import org.apache.solr.search.MutableValue;
>   import org.apache.solr.util.ByteUtils;
>
>   import java.io.IOException;
> @@ -192,6 +193,23 @@ abstract class IntDocValues extends DocV
>     public String toString(int doc) {
>       return vs.description() + '=' + strVal(doc);
>     }
> +
> +  @Override
> +  public ValueFiller getValueFiller() {
> +    return new ValueFiller() {
> +      private final MutableValueInt mval = new MutableValueInt();
> +
> +      @Override
> +      public MutableValue getValue() {
> +        return mval;
> +      }
> +
> +      @Override
> +      public void fillValue(int doc) {
> +        mval.value = intVal(doc);
> +      }
> +    };
> +  }
>   }
>
>
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocValues.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocValues.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocValues.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DocValues.java Sat Aug 21 02:36:09 2010
> @@ -19,6 +19,8 @@ package org.apache.solr.search.function;
>
>   import org.apache.lucene.search.*;
>   import org.apache.lucene.index.IndexReader;
> +import org.apache.solr.search.MutableValue;
> +import org.apache.solr.search.MutableValueFloat;
>
>   /**
>    * Represents field values as different types.
> @@ -46,6 +48,32 @@ public abstract class DocValues {
>     public String strVal(int doc) { throw new UnsupportedOperationException(); }
>     public abstract String toString(int doc);
>
> +  /** @lucene.experimental  */
> +  public static abstract class ValueFiller {
> +    /** MutableValue will be reused across calls */
> +    public abstract MutableValue getValue();
> +
> +    /** MutableValue will be reused across calls.  Returns true if the value exists. */
> +    public abstract void fillValue(int doc);
> +  }
> +
> +  /** @lucene.experimental  */
> +  public ValueFiller getValueFiller() {
> +    return new ValueFiller() {
> +      private final MutableValueFloat mval = new MutableValueFloat();
> +
> +      @Override
> +      public MutableValue getValue() {
> +        return mval;
> +      }
> +
> +      @Override
> +      public void fillValue(int doc) {
> +        mval.value = floatVal(doc);
> +      }
> +    };
> +  }
> +
>     //For Functions that can work with multiple values from the same document.  This does not apply to all functions
>     public void byteVal(int doc, byte [] vals) { throw new UnsupportedOperationException(); }
>     public void shortVal(int doc, short [] vals) { throw new UnsupportedOperationException(); }
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DoubleFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DoubleFieldSource.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DoubleFieldSource.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/DoubleFieldSource.java Sat Aug 21 02:36:09 2010
> @@ -19,6 +19,8 @@ package org.apache.solr.search.function;
>
>   import org.apache.lucene.index.IndexReader;
>   import org.apache.lucene.search.FieldCache;
> +import org.apache.solr.search.MutableValue;
> +import org.apache.solr.search.MutableValueDouble;
>
>   import java.io.IOException;
>   import java.util.Map;
> @@ -134,6 +136,24 @@ public class DoubleFieldSource extends F
>           }
>         }
>
> +            @Override
> +      public ValueFiller getValueFiller() {
> +        return new ValueFiller() {
> +          private final double[] doubleArr = arr;
> +          private final MutableValueDouble mval = new MutableValueDouble();
> +
> +          @Override
> +          public MutableValue getValue() {
> +            return mval;
> +          }
> +
> +          @Override
> +          public void fillValue(int doc) {
> +            mval.value = doubleArr[doc];
> +          }
> +        };
> +      }
> +
>
>         };
>
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/FloatFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/FloatFieldSource.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/FloatFieldSource.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/FloatFieldSource.java Sat Aug 21 02:36:09 2010
> @@ -18,6 +18,8 @@
>   package org.apache.solr.search.function;
>
>   import org.apache.lucene.index.IndexReader;
> +import org.apache.solr.search.MutableValue;
> +import org.apache.solr.search.MutableValueFloat;
>   import org.apache.solr.search.function.DocValues;
>   import org.apache.lucene.search.FieldCache;
>
> @@ -76,6 +78,25 @@ public class FloatFieldSource extends Fi
>         public String toString(int doc) {
>           return description() + '=' + floatVal(doc);
>         }
> +
> +      @Override
> +      public ValueFiller getValueFiller() {
> +        return new ValueFiller() {
> +          private final float[] floatArr = arr;
> +          private final MutableValueFloat mval = new MutableValueFloat();
> +
> +          @Override
> +          public MutableValue getValue() {
> +            return mval;
> +          }
> +
> +          @Override
> +          public void fillValue(int doc) {
> +            mval.value = floatArr[doc];
> +          }
> +        };
> +      }
> +
>       };
>     }
>
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/IntFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/IntFieldSource.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/IntFieldSource.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/IntFieldSource.java Sat Aug 21 02:36:09 2010
> @@ -18,7 +18,8 @@
>   package org.apache.solr.search.function;
>
>   import org.apache.lucene.index.IndexReader;
> -import org.apache.solr.search.function.DocValues;
> +import org.apache.solr.search.MutableValueInt;
> +import org.apache.solr.search.MutableValue;
>   import org.apache.lucene.search.FieldCache;
>
>   import java.io.IOException;
> @@ -32,7 +33,7 @@ import java.util.Map;
>    */
>
>   public class IntFieldSource extends FieldCacheSource {
> -  FieldCache.IntParser parser;
> +  final FieldCache.IntParser parser;
>
>     public IntFieldSource(String field) {
>       this(field, null);
> @@ -53,6 +54,8 @@ public class IntFieldSource extends Fiel
>               cache.getInts(reader, field) :
>               cache.getInts(reader, field, parser);
>       return new DocValues() {
> +      final MutableValueInt val = new MutableValueInt();
> +
>         public float floatVal(int doc) {
>           return (float)arr[doc];
>         }
> @@ -110,6 +113,26 @@ public class IntFieldSource extends Fiel
>             }
>           };
>         }
> +
> +      @Override
> +      public ValueFiller getValueFiller() {
> +        return new ValueFiller() {
> +          private final int[] intArr = arr;
> +          private final MutableValueInt mval = new MutableValueInt();
> +
> +          @Override
> +          public MutableValue getValue() {
> +            return mval;
> +          }
> +
> +          @Override
> +          public void fillValue(int doc) {
> +            mval.value = intArr[doc];
> +          }
> +        };
> +      }
> +
> +
>       };
>     }
>
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/LongFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/LongFieldSource.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/LongFieldSource.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/LongFieldSource.java Sat Aug 21 02:36:09 2010
> @@ -19,6 +19,8 @@ package org.apache.solr.search.function;
>
>   import org.apache.lucene.index.IndexReader;
>   import org.apache.lucene.search.FieldCache;
> +import org.apache.solr.search.MutableValue;
> +import org.apache.solr.search.MutableValueLong;
>
>
>   import java.io.IOException;
> @@ -116,10 +118,33 @@ public class LongFieldSource extends Fie
>           };
>         }
>
> +      @Override
> +      public ValueFiller getValueFiller() {
> +        return new ValueFiller() {
> +          private final long[] longArr = arr;
> +          private final MutableValueLong mval = newMutableValueLong();
> +
> +          @Override
> +          public MutableValue getValue() {
> +            return mval;
> +          }
> +
> +          @Override
> +          public void fillValue(int doc) {
> +            mval.value = longArr[doc];
> +          }
> +        };
> +      }
> +
> +
>
>       };
>     }
>
> +  protected MutableValueLong newMutableValueLong() {
> +    return new MutableValueLong();
> +  }
> +
>     public boolean equals(Object o) {
>       if (o.getClass() != this.getClass()) return false;
>       LongFieldSource other = (LongFieldSource) o;
>
> Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/StringIndexDocValues.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/StringIndexDocValues.java?rev=987690&r1=987689&r2=987690&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/StringIndexDocValues.java (original)
> +++ lucene/dev/trunk/solr/src/java/org/apache/solr/search/function/StringIndexDocValues.java Sat Aug 21 02:36:09 2010
> @@ -20,6 +20,8 @@ package org.apache.solr.search.function;
>   import org.apache.lucene.search.FieldCache;
>   import org.apache.lucene.index.IndexReader;
>   import org.apache.lucene.util.BytesRef;
> +import org.apache.solr.search.MutableValue;
> +import org.apache.solr.search.MutableValueStr;
>
>   import java.io.IOException;
>
> @@ -29,6 +31,7 @@ import java.io.IOException;
>   public abstract class StringIndexDocValues extends DocValues {
>       protected final FieldCache.DocTermsIndex termsIndex;
>       protected final ValueSource vs;
> +    protected final MutableValueStr val = new MutableValueStr();
>
>       public StringIndexDocValues(ValueSource vs, IndexReader reader, String field) throws IOException {
>         try {
> @@ -81,9 +84,26 @@ public abstract class StringIndexDocValu
>         };
>       }
>
> -    public String toString(int doc) {
> -      return vs.description() + '=' + strVal(doc);
> -    }
> +  public String toString(int doc) {
> +    return vs.description() + '=' + strVal(doc);
> +  }
> +
> +  @Override
> +  public ValueFiller getValueFiller() {
> +    return new ValueFiller() {
> +      private final MutableValueStr mval = new MutableValueStr();
> +
> +      @Override
> +      public MutableValue getValue() {
> +        return mval;
> +      }
> +
> +      @Override
> +      public void fillValue(int doc) {
> +        mval.value = termsIndex.getTerm(doc, val.value);
> +      }
> +    };
> +  }
>
>     public static final class StringIndexException extends RuntimeException {
>       public StringIndexException(final String fieldName,
> @@ -92,5 +112,6 @@ public abstract class StringIndexDocValu
>               "DocValues for field: " + fieldName, cause);
>       }
>     }
> -
> +
> +
>   }
>
> Added: lucene/dev/trunk/solr/src/test/org/apache/solr/TestGroupingSearch.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/test/org/apache/solr/TestGroupingSearch.java?rev=987690&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/src/test/org/apache/solr/TestGroupingSearch.java (added)
> +++ lucene/dev/trunk/solr/src/test/org/apache/solr/TestGroupingSearch.java Sat Aug 21 02:36:09 2010
> @@ -0,0 +1,217 @@
> +/**
> + * 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.
> + */
> +
> +package org.apache.solr;
> +
> +import org.junit.Before;
> +import org.junit.BeforeClass;
> +import org.junit.Test;
> +
> +public class TestGroupingSearch extends SolrTestCaseJ4 {
> +
> +  @BeforeClass
> +  public static void beforeTests() throws Exception {
> +    initCore("solrconfig.xml","schema12.xml");
> +  }
> +
> +  @Before
> +  public void cleanIndex() {
> +    assertU(delQ("*:*"));
> +    assertU(commit());
> +  }
> +
> +  @Test
> +  public void testGroupingGroupSortingScore_basic() {
> +    assertU(add(doc("id", "1","name", "author1", "title", "a book title")));
> +    assertU(add(doc("id", "2","name", "author1", "title", "the title")));
> +    assertU(add(doc("id", "3","name", "author2", "title", "a book title")));
> +    assertU(add(doc("id", "4","name", "author2", "title", "title")));
> +    assertU(add(doc("id", "5","name", "author3", "title", "the title of a title")));
> +    assertU(commit());
> +
> +    assertQ(req("q","title:title", "group", "true", "group.field","name")
> +            ,"//lst[@name='grouped']/lst[@name='name']"
> +            ,"*[count(//arr[@name='groups']/lst) = 3]"
> +
> +            ,"//arr[@name='groups']/lst[1]/str[@name='groupValue'][.='author2']"
> +    //        ,"//arr[@name='groups']/lst[1]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[1]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[1]/result/doc/*[@name='id'][.='4']"
> +
> +            ,"//arr[@name='groups']/lst[2]/str[@name='groupValue'][.='author1']"
> +    //       ,"//arr[@name='groups']/lst[2]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[2]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[2]/result/doc/*[@name='id'][.='2']"
> +
> +            ,"//arr[@name='groups']/lst[3]/str[@name='groupValue'][.='author3']"
> +    //        ,"//arr[@name='groups']/lst[3]/int[@name='matches'][.='1']"
> +            ,"//arr[@name='groups']/lst[3]/result[@numFound='1']"
> +            ,"//arr[@name='groups']/lst[3]/result/doc/*[@name='id'][.='5']"
> +            );
> +  }
> +
> +  @Test
> +  public void testGroupingGroupSortingScore_basicWithGroupSortEqualToSort() {
> +    assertU(add(doc("id", "1","name", "author1", "title", "a book title")));
> +    assertU(add(doc("id", "2","name", "author1", "title", "the title")));
> +    assertU(add(doc("id", "3","name", "author2", "title", "a book title")));
> +    assertU(add(doc("id", "4","name", "author2", "title", "title")));
> +    assertU(add(doc("id", "5","name", "author3", "title", "the title of a title")));
> +    assertU(commit());
> +
> +    assertQ(req("q","title:title", "group", "true", "group.field","name", "sort", "score desc", "group.sort", "score desc")
> +            ,"//arr[@name='groups']/lst[1]/str[@name='groupValue'][.='author2']"
> +    //        ,"//arr[@name='groups']/lst[1]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[1]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[1]/result/doc/*[@name='id'][.='4']"
> +
> +            ,"//arr[@name='groups']/lst[2]/str[@name='groupValue'][.='author1']"
> +    //        ,"//arr[@name='groups']/lst[2]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[2]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[2]/result/doc/*[@name='id'][.='2']"
> +
> +            ,"//arr[@name='groups']/lst[3]/str[@name='groupValue'][.='author3']"
> +    //        ,"//arr[@name='groups']/lst[3]/int[@name='matches'][.='1']"
> +            ,"//arr[@name='groups']/lst[3]/result[@numFound='1']"
> +            ,"//arr[@name='groups']/lst[3]/result/doc/*[@name='id'][.='5']"
> +            );
> +  }
> +
> +  @Test
> +  public void testGroupingGroupSortingName() {
> +    assertU(add(doc("id", "1","name", "author1", "title", "a book title")));
> +    assertU(add(doc("id", "2","name", "author1", "title", "the title")));
> +    assertU(add(doc("id", "3","name", "author2", "title", "book title")));
> +    assertU(add(doc("id", "4","name", "author2", "title", "the title")));
> +    assertU(commit());
> +
> +    assertQ(req("q","title:title", "group", "true", "group.field","name", "group.sort", "title asc")
> +            ,"*[count(//arr[@name='groups']/lst) = 2]"
> +            ,"//arr[@name='groups']/lst[1]/str[@name='groupValue'][.='author2']"
> +    //       ,"//arr[@name='groups']/lst[1]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[1]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[1]/result/doc/*[@name='id'][.='3']"
> +
> +            ,"//arr[@name='groups']/lst[2]/str[@name='groupValue'][.='author1']"
> +    //        ,"//arr[@name='groups']/lst[2]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[2]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[2]/result/doc/*[@name='id'][.='1']"
> +            );
> +  }
> +
> +  @Test
> +  public void testGroupingGroupSortingWeight() {
> +    assertU(add(doc("id", "1","name", "author1", "weight", "12.1")));
> +    assertU(add(doc("id", "2","name", "author1", "weight", "2.1")));
> +    assertU(add(doc("id", "3","name", "author2", "weight", "0.1")));
> +    assertU(add(doc("id", "4","name", "author2", "weight", "0.11")));
> +    assertU(commit());
> +
> +    assertQ(req("q","*:*", "group", "true", "group.field","name", "sort", "id asc", "group.sort", "weight desc")
> +            ,"*[count(//arr[@name='groups']/lst) = 2]"
> +            ,"//arr[@name='groups']/lst[1]/str[@name='groupValue'][.='author1']"
> +    //        ,"//arr[@name='groups']/lst[1]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[1]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[1]/result/doc/*[@name='id'][.='1']"
> +
> +            ,"//arr[@name='groups']/lst[2]/str[@name='groupValue'][.='author2']"
> +    //        ,"//arr[@name='groups']/lst[2]/int[@name='matches'][.='2']"
> +            ,"//arr[@name='groups']/lst[2]/result[@numFound='2']"
> +            ,"//arr[@name='groups']/lst[2]/result/doc/*[@name='id'][.='4']"
> +            );
> +  }
> +
> +
> +
> +  static String f = "foo_i";
> +  static String f2 = "foo2_i";
> +
> +  public static void createIndex() {
> +    assertU(adoc("id","1", f,"5",  f2,"4"));
> +    assertU(adoc("id","2", f,"4",  f2,"2"));
> +    assertU(adoc("id","3", f,"3",  f2,"7"));
> +    assertU(adoc("id","4", f,"2",  f2,"6"));
> +    assertU(adoc("id","5", f,"1",  f2,"2"));
> +    assertU(adoc("id","6", f,"3",  f2,"2"));
> +    assertU(adoc("id","7", f,"2",  f2,"3"));
> +    assertU(adoc("id","8", f,"1",  f2,"10"));
> +    assertU(adoc("id","9", f,"2",  f2,"1"));
> +    assertU(adoc("id","10", f,"1", f2,"3"));
> +    assertU(commit());
> +  }
> +
> +  @Test
> +  public void testGroupAPI() throws Exception {
> +    createIndex();
> +    String filt = f + ":[* TO *]";
> +
> +    assertQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f)
> +        ,"/response/lst[@name='grouped']/lst[@name='"+f+"']/arr[@name='groups']"
> +    );
> +
> +    assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id")
> +      ,"/responseHeader/status:0"                         // exact match
> +      ,"/responseHeader:{'_SKIP_':'QTime', 'status':0}"   // partial match by skipping some elements
> +      ,"/responseHeader:{'_MATCH_':'status', 'status':0}" // partial match by only including some elements
> +      ,"/grouped:{'foo_i':{'matches':10,'groups':[\n" +
> +              "{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'}]}}," +
> +              "{'groupValue':3,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}," +
> +              "{'groupValue':2,'doclist':{'numFound':3,'start':0,'docs':[{'id':'4'}]}}," +
> +              "{'groupValue':5,'doclist':{'numFound':1,'start':0,'docs':[{'id':'1'}]}}," +
> +              "{'groupValue':4,'doclist':{'numFound':1,'start':0,'docs':[{'id':'2'}]}}" +
> +            "]}}"
> +    );
> +
> +    // test limiting the number of groups returned
> +    assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2")
> +      ,"/grouped:{'foo_i':{'matches':10,'groups':[" +
> +              "{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'}]}}," +
> +              "{'groupValue':3,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}" +
> +            "]}}"
> +    );
> +
> +    // test increasing the docs per group returned
> +    assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id", "rows","2", "group.limit","3")
> +      ,"/grouped:{'foo_i':{'matches':10,'groups':[" +
> +            "{'groupValue':1,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'},{'id':'10'},{'id':'5'}]}}," +
> +            "{'groupValue':3,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'},{'id':'6'}]}}" +
> +          "]}}"
> +    );
> +
> +    // test adding in scores
> +    assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.field",f, "fl","id,score", "rows","2", "group.limit","2", "indent","off")
> +      ,"/grouped/foo_i/groups:" +
> +            "[" +
> +              "{'groupValue':1,'doclist':{'numFound':3,'start':0,'maxScore':10.0,'docs':[{'id':'8','score':10.0},{'id':'10','score':3.0}]}}," +
> +              "{'groupValue':3,'doclist':{'numFound':2,'start':0,'maxScore':7.0,'docs':[{'id':'3','score':7.0},{'id':'6','score':2.0}]}}" +
> +            "]"
> +
> +    );
> +
> +    // test function (functions are currently all float - this may change)
> +    String func = "add("+f+","+f+")";
> +    assertJQ(req("fq",filt,  "q","{!func}"+f2, "group","true", "group.func", func  , "fl","id", "rows","2")
> +      ,"/grouped:{'"+func+"':{'matches':10,'groups':[" +
> +              "{'groupValue':2.0,'doclist':{'numFound':3,'start':0,'docs':[{'id':'8'}]}}," +
> +              "{'groupValue':6.0,'doclist':{'numFound':2,'start':0,'docs':[{'id':'3'}]}}" +
> +            "]}}"
> +    );
> +
> +  };
> +
> +
> +}
>
> Propchange: lucene/dev/trunk/solr/src/test/org/apache/solr/TestGroupingSearch.java
> ------------------------------------------------------------------------------
>      svn:eol-style = native
>
> Propchange: lucene/dev/trunk/solr/src/test/org/apache/solr/TestGroupingSearch.java
> ------------------------------------------------------------------------------
>      svn:keywords = Date Author Id Revision HeadURL
>
>
>



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org