You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2013/10/24 01:19:12 UTC

svn commit: r1535208 - in /lucene/dev/trunk/solr: ./ core/ core/src/java/org/apache/solr/search/ core/src/test-files/solr/collection1/conf/ core/src/test/org/apache/solr/search/

Author: jbernste
Date: Wed Oct 23 23:19:12 2013
New Revision: 1535208

URL: http://svn.apache.org/r1535208
Log:
SOLR-5027 CollapsingQParserPlugin

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/ivy.xml
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1535208&r1=1535207&r2=1535208&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Wed Oct 23 23:19:12 2013
@@ -100,6 +100,10 @@ New Features
 * SOLR-5353: Enhance CoreAdmin api to split a route key's documents from an index
   and leave behind all other documents. (shalin)
 
+* SOLR-5027: CollapsingQParserPlugin for high performance field collapsing on high cardinality fields.
+  (Joel Bernstein)
+
+
 Bug Fixes
 ----------------------
   

Modified: lucene/dev/trunk/solr/core/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/ivy.xml?rev=1535208&r1=1535207&r2=1535208&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/ivy.xml (original)
+++ lucene/dev/trunk/solr/core/ivy.xml Wed Oct 23 23:19:12 2013
@@ -39,6 +39,7 @@
     <dependency org="org.restlet.jee" name="org.restlet.ext.servlet" rev="${/org.restlet.jee/org.restlet.ext.servlet}" conf="compile->*"/>
     <dependency org="joda-time" name="joda-time" rev="${/joda-time/joda-time}" conf="compile->*"/>
     <dependency org="dom4j" name="dom4j" rev="${/dom4j/dom4j}" transitive="false"/>
+    <dependency org="com.carrotsearch" name="hppc" rev="${/com.carrotsearch/hppc}"/>
 
     <dependency org="javax.servlet" name="javax.servlet-api" rev="${/javax.servlet/javax.servlet-api}" conf="test->*"/>
     <dependency org="org.easymock" name="easymock" rev="${/org.easymock/easymock}" conf="test->*"/>

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java?rev=1535208&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java Wed Oct 23 23:19:12 2013
@@ -0,0 +1,917 @@
+/*
+ * 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.schema.TrieFloatField;
+import org.apache.solr.schema.TrieIntField;
+import org.apache.solr.schema.TrieLongField;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.handler.component.QueryElevationComponent;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.search.*;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.Bits;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+
+import com.carrotsearch.hppc.FloatArrayList;
+import com.carrotsearch.hppc.IntOpenHashSet;
+import com.carrotsearch.hppc.cursors.IntCursor;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Iterator;
+
+/**
+
+ The <b>CollapsingQParserPlugin</b> is a PostFilter that performs field collapsing.
+ This is a high performance alternative to standard Solr
+ field collapsing (with ngroups) when the number of distinct groups
+ in the result set is high.
+ <p/>
+ Sample syntax:
+ <p/>
+ Collapse based on the highest scoring document:
+ <p/>
+
+ fq=(!collapse field=field_name}
+
+ <p/>
+ Collapse based on the min value of a numeric field:
+ <p/>
+ fq={!collapse field=field_name min=field_name}
+ <p/>
+ Collapse based on the max value of a numeric field:
+ <p/>
+ fq={!collapse field=field_name max=field_name}
+ <p/>
+ Collapse with a null policy:
+ <p/>
+ fq={!collapse field=field_name nullPolicy=nullPolicy}
+ <p/>
+ There are three null policies: <br/>
+ ignore : removes docs with a null value in the collapse field (default).<br/>
+ expand : treats each doc with a null value in the collapse field as a separate group.<br/>
+ collapse : collapses all docs with a null value into a single group using either highest score, or min/max.
+ <p/>
+ The CollapsingQParserPlugin fully supports the QueryElevationComponent
+
+
+ **/
+
+public class CollapsingQParserPlugin extends QParserPlugin {
+
+  public static final String NAME = "collapse";
+  public static final String NULL_COLLAPSE = "collapse";
+  public static final String NULL_IGNORE = "ignore";
+  public static final String NULL_EXPAND = "expand";
+
+
+  public void init(NamedList namedList) {
+
+  }
+
+  public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
+    return new CollapsingQParser(qstr, localParams, params, request);
+  }
+
+  private class CollapsingQParser extends QParser {
+
+    public CollapsingQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
+      super(qstr, localParams, params, request);
+    }
+
+    public Query parse() throws SyntaxError {
+      try {
+        return new CollapsingPostFilter(localParams, params, req);
+      } catch (Exception e) {
+        throw new SyntaxError(e.getMessage(), e);
+      }
+    }
+  }
+
+  private class CollapsingPostFilter extends ExtendedQueryBase implements PostFilter {
+
+    private Object cacheId;
+    private String field;
+    private int leafCount;
+    private SortedDocValues docValues;
+    private int maxDoc;
+    private String max;
+    private String min;
+    private FieldType fieldType;
+    private int nullPolicy;
+    private SolrIndexSearcher searcher;
+    private SolrParams solrParams;
+    private Map context;
+    private IndexSchema schema;
+    public static final int NULL_POLICY_IGNORE = 0;
+    public static final int NULL_POLICY_COLLAPSE = 1;
+    public static final int NULL_POLICY_EXPAND = 2;
+
+    public void setCache(boolean cache) {
+
+    }
+
+    public void setCacheSep(boolean cacheSep) {
+
+    }
+
+    public boolean getCacheSep() {
+      return false;
+    }
+
+    public boolean getCache() {
+      return false;
+    }
+
+    public int hashCode() {
+      return this.cacheId.hashCode()*((1+Float.floatToIntBits(this.getBoost()))*31);
+    }
+
+    public boolean equals(Object o) {
+      //Uses the unique id for equals to ensure that the query result cache always fails.
+      if(o instanceof CollapsingPostFilter) {
+        CollapsingPostFilter c = (CollapsingPostFilter)o;
+        //Do object comparison to be sure only the same object will return true.
+        if(this.cacheId == c.cacheId && this.getBoost()==c.getBoost()) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public int getCost() {
+      return Math.max(super.getCost(), 100);
+    }
+
+    public String toString(String s) {
+      return s;
+    }
+
+    public CollapsingPostFilter(SolrParams localParams, SolrParams params, SolrQueryRequest request) throws IOException {
+      this.cacheId = new Object();
+      this.field = localParams.get("field");
+      this.solrParams = params;
+      String nPolicy = localParams.get("nullPolicy", NULL_IGNORE);
+      if(nPolicy.equals(NULL_IGNORE)) {
+        this.nullPolicy = NULL_POLICY_IGNORE;
+      } else if (nPolicy.equals(NULL_COLLAPSE)) {
+        this.nullPolicy = NULL_POLICY_COLLAPSE;
+      } else if(nPolicy.equals((NULL_EXPAND))) {
+        this.nullPolicy = NULL_POLICY_EXPAND;
+      }
+      this.searcher = request.getSearcher();
+      this.leafCount = searcher.getTopReaderContext().leaves().size();
+      this.maxDoc = searcher.maxDoc();
+      this.schema = searcher.getSchema();
+      SchemaField schemaField = schema.getField(this.field);
+      if(schemaField.hasDocValues()) {
+        this.docValues = searcher.getAtomicReader().getSortedDocValues(this.field);
+      } else {
+        this.docValues = FieldCache.DEFAULT.getTermsIndex(searcher.getAtomicReader(), this.field);
+      }
+
+      this.max = localParams.get("max");
+      if(this.max != null) {
+        this.fieldType = searcher.getSchema().getField(this.max).getType();
+      }
+
+      this.min = localParams.get("min");
+      if(this.min != null) {
+        this.fieldType = searcher.getSchema().getField(this.min).getType();
+      }
+
+      this.context = request.getContext();
+    }
+
+    private IntOpenHashSet getBoostDocs(IndexSearcher indexSearcher, Set<String> boosted) throws IOException {
+      IntOpenHashSet boostDocs = null;
+      if(boosted != null) {
+        SchemaField idField = this.schema.getUniqueKeyField();
+        String fieldName = idField.getName();
+        HashSet<BytesRef> localBoosts = new HashSet(boosted.size()*2);
+        Iterator<String> boostedIt = boosted.iterator();
+        while(boostedIt.hasNext()) {
+          localBoosts.add(new BytesRef(boostedIt.next()));
+        }
+
+        boostDocs = new IntOpenHashSet(boosted.size()*2);
+
+        List<AtomicReaderContext>leaves = indexSearcher.getTopReaderContext().leaves();
+        TermsEnum termsEnum = null;
+        DocsEnum docsEnum = null;
+        for(AtomicReaderContext leaf : leaves) {
+          AtomicReader reader = leaf.reader();
+          int docBase = leaf.docBase;
+          Bits liveDocs = reader.getLiveDocs();
+          Terms terms = reader.terms(fieldName);
+          termsEnum = terms.iterator(termsEnum);
+          Iterator<BytesRef> it = localBoosts.iterator();
+          while(it.hasNext()) {
+            BytesRef ref = it.next();
+            if(termsEnum.seekExact(ref)) {
+              docsEnum = termsEnum.docs(liveDocs, docsEnum);
+              int doc = docsEnum.nextDoc();
+              if(doc != -1) {
+                //Found the document.
+                boostDocs.add(doc+docBase);
+                it.remove();
+              }
+            }
+          }
+        }
+      }
+
+      return boostDocs;
+    }
+
+    public DelegatingCollector getFilterCollector(IndexSearcher indexSearcher) {
+      try {
+        IntOpenHashSet boostDocs = getBoostDocs(indexSearcher, (Set<String>) (this.context.get(QueryElevationComponent.BOOSTED)));
+
+        if(this.min != null || this.max != null) {
+
+          return new CollapsingFieldValueCollector(this.maxDoc,
+              this.leafCount,
+              this.docValues,
+              this.searcher,
+              this.nullPolicy,
+              max != null ? this.max : this.min,
+              max != null,
+              needsScores(this.solrParams),
+              this.fieldType,
+              boostDocs);
+        } else {
+          return new CollapsingScoreCollector(this.maxDoc, this.leafCount, this.docValues, this.nullPolicy, boostDocs);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    private boolean needsScores(SolrParams params) {
+
+      String sortSpec = params.get("sort");
+      if(sortSpec != null) {
+        String[] sorts = sortSpec.split(",");
+        for(String s: sorts) {
+          String parts[] = s.split(" ");
+          if(parts[0].equals("score")) {
+            return true;
+          }
+        }
+      } else {
+        //No sort specified so it defaults to score.
+        return true;
+      }
+
+      String fl = params.get("fl");
+      if(fl != null) {
+        String[] fls = fl.split(",");
+        for(String f : fls) {
+          if(f.trim().equals("score")) {
+            return true;
+          }
+        }
+      }
+
+      if(this.context.containsKey(QueryElevationComponent.BOOSTED)) {
+        return true;
+      }
+
+      return false;
+    }
+  }
+
+  private class DummyScorer extends Scorer {
+
+    public float score;
+
+    public DummyScorer() {
+      super(null);
+    }
+
+    public float score() {
+      return score;
+    }
+
+    public int freq() {
+      return 0;
+    }
+
+    public int advance(int i) {
+      return -1;
+    }
+
+    public int nextDoc() {
+      return 0;
+    }
+
+    public int docID() {
+      return 0;
+    }
+
+    public long cost() {
+      return 0;
+    }
+  }
+
+
+  private class CollapsingScoreCollector extends DelegatingCollector {
+
+    private AtomicReaderContext[] contexts;
+    private OpenBitSet collapsedSet;
+    private SortedDocValues values;
+    private int[] ords;
+    private float[] scores;
+    private int docBase;
+    private int maxDoc;
+    private int nullPolicy;
+    private float nullScore = -Float.MAX_VALUE;
+    private int nullDoc;
+    private FloatArrayList nullScores;
+    private IntOpenHashSet boostDocs;
+
+    public CollapsingScoreCollector(int maxDoc,
+                                    int segments,
+                                    SortedDocValues values,
+                                    int nullPolicy,
+                                    IntOpenHashSet boostDocs) {
+      this.maxDoc = maxDoc;
+      this.contexts = new AtomicReaderContext[segments];
+      this.collapsedSet = new OpenBitSet(maxDoc);
+      this.boostDocs = boostDocs;
+      if(this.boostDocs != null) {
+        //Set the elevated docs now.
+        Iterator<IntCursor> it = this.boostDocs.iterator();
+        while(it.hasNext()) {
+          IntCursor cursor = it.next();
+          this.collapsedSet.fastSet(cursor.value);
+        }
+      }
+      this.values = values;
+      int valueCount = values.getValueCount();
+      this.ords = new int[valueCount];
+      Arrays.fill(this.ords, -1);
+      this.scores = new float[valueCount];
+      Arrays.fill(this.scores, -Float.MAX_VALUE);
+      this.nullPolicy = nullPolicy;
+      if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        nullScores = new FloatArrayList();
+      }
+    }
+
+    public boolean acceptsDocsOutOfOrder() {
+      //Documents must be sent in order to this collector.
+      return false;
+    }
+
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      this.contexts[context.ord] = context;
+      this.docBase = context.docBase;
+    }
+
+    public void collect(int docId) throws IOException {
+      int globalDoc = docId+this.docBase;
+      int ord = values.getOrd(globalDoc);
+      if(ord > -1) {
+        float score = scorer.score();
+        if(score > scores[ord]) {
+          ords[ord] = globalDoc;
+          scores[ord] = score;
+        }
+      } else if (this.collapsedSet.fastGet(globalDoc)) {
+        //The doc is elevated so score does not matter
+        //We just want to be sure it doesn't fall into the null policy
+      } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        float score = scorer.score();
+        if(score > nullScore) {
+          nullScore = score;
+          nullDoc = globalDoc;
+        }
+      } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        collapsedSet.fastSet(globalDoc);
+        nullScores.add(scorer.score());
+      }
+    }
+
+    public void finish() throws IOException {
+      if(contexts.length == 0) {
+        return;
+      }
+
+      if(nullScore > 0) {
+        this.collapsedSet.fastSet(nullDoc);
+      }
+
+      for(int i=0; i<ords.length; i++) {
+        int doc = ords[i];
+        if(doc > -1) {
+          collapsedSet.fastSet(doc);
+        }
+      }
+
+      int currentContext = 0;
+      int currentDocBase = 0;
+      int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+      delegate.setNextReader(contexts[currentContext]);
+      DummyScorer dummy = new DummyScorer();
+      delegate.setScorer(dummy);
+      DocIdSetIterator it = collapsedSet.iterator();
+      int docId = -1;
+      int nullScoreIndex = 0;
+      while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+
+        int ord = values.getOrd(docId);
+        if(ord > -1) {
+          dummy.score = scores[ord];
+        } else if(this.boostDocs != null && boostDocs.contains(docId)) {
+          //Elevated docs don't need a score.
+          dummy.score = 0F;
+        } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+          dummy.score = nullScore;
+        } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          dummy.score = nullScores.get(nullScoreIndex++);
+        }
+
+        while(docId >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts[currentContext].docBase;
+          nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+          delegate.setNextReader(contexts[currentContext]);
+        }
+
+        int contextDoc = docId-currentDocBase;
+        delegate.collect(contextDoc);
+      }
+
+      if(delegate instanceof DelegatingCollector) {
+        ((DelegatingCollector) delegate).finish();
+      }
+    }
+  }
+
+  private class CollapsingFieldValueCollector extends DelegatingCollector {
+    private AtomicReaderContext[] contexts;
+    private SortedDocValues values;
+
+    private int docBase;
+    private int maxDoc;
+    private int nullPolicy;
+
+    private FieldValueCollapse fieldValueCollapse;
+    private boolean needsScores;
+    private IntOpenHashSet boostDocs;
+
+    public CollapsingFieldValueCollector(int maxDoc,
+                                         int segments,
+                                         SortedDocValues values,
+                                         SolrIndexSearcher searcher,
+                                         int nullPolicy,
+                                         String field,
+                                         boolean max,
+                                         boolean needsScores,
+                                         FieldType fieldType,
+                                         IntOpenHashSet boostDocs) throws IOException{
+
+      this.maxDoc = maxDoc;
+      this.contexts = new AtomicReaderContext[segments];
+      this.values = values;
+      int valueCount = values.getValueCount();
+      this.nullPolicy = nullPolicy;
+      this.needsScores = needsScores;
+      this.boostDocs = boostDocs;
+      if(fieldType instanceof TrieIntField) {
+        this.fieldValueCollapse = new IntValueCollapse(searcher, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs);
+      } else if(fieldType instanceof TrieLongField) {
+        this.fieldValueCollapse =  new LongValueCollapse(searcher, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs);
+      } else if(fieldType instanceof TrieFloatField) {
+        this.fieldValueCollapse =  new FloatValueCollapse(searcher, field, nullPolicy, new int[valueCount], max, this.needsScores, boostDocs);
+      }
+    }
+
+    public boolean acceptsDocsOutOfOrder() {
+      //Documents must be sent in order to this collector.
+      return false;
+    }
+
+    public void setScorer(Scorer scorer) {
+      this.fieldValueCollapse.setScorer(scorer);
+    }
+
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      this.contexts[context.ord] = context;
+      this.docBase = context.docBase;
+      this.fieldValueCollapse.setNextReader(context);
+    }
+
+    public void collect(int docId) throws IOException {
+      int globalDoc = docId+this.docBase;
+      int ord = values.getOrd(globalDoc);
+      fieldValueCollapse.collapse(ord, docId, globalDoc);
+    }
+
+    public void finish() throws IOException {
+      if(contexts.length == 0) {
+        return;
+      }
+
+      int currentContext = 0;
+      int currentDocBase = 0;
+      int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+      delegate.setNextReader(contexts[currentContext]);
+      DummyScorer dummy = new DummyScorer();
+      delegate.setScorer(dummy);
+      DocIdSetIterator it = fieldValueCollapse.getCollapsedSet().iterator();
+      int docId = -1;
+      int nullScoreIndex = 0;
+      float[] scores = fieldValueCollapse.getScores();
+      FloatArrayList nullScores = fieldValueCollapse.getNullScores();
+      float nullScore = fieldValueCollapse.getNullScore();
+      while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+
+        if(this.needsScores){
+          int ord = values.getOrd(docId);
+          if(ord > -1) {
+            dummy.score = scores[ord];
+          } else if (boostDocs != null && boostDocs.contains(docId)) {
+            //Its an elevated doc so no score is needed
+            dummy.score = 0F;
+          } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+            dummy.score = nullScore;
+          } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+            dummy.score = nullScores.get(nullScoreIndex++);
+          }
+        }
+
+        while(docId >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts[currentContext].docBase;
+          nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
+          delegate.setNextReader(contexts[currentContext]);
+        }
+
+        int contextDoc = docId-currentDocBase;
+        delegate.collect(contextDoc);
+      }
+
+      if(delegate instanceof DelegatingCollector) {
+        ((DelegatingCollector) delegate).finish();
+      }
+    }
+  }
+
+  private abstract class FieldValueCollapse {
+    protected int nullPolicy;
+    protected int[] ords;
+    protected Scorer scorer;
+    protected FloatArrayList nullScores;
+    protected float nullScore;
+    protected float[] scores;
+    protected OpenBitSet collapsedSet;
+    protected IntOpenHashSet boostDocs;
+    protected int nullDoc = -1;
+    protected boolean needsScores;
+    protected boolean max;
+    protected String field;
+
+    public abstract void collapse(int ord, int contextDoc, int globalDoc) throws IOException;
+    public abstract void setNextReader(AtomicReaderContext context) throws IOException;
+
+    public FieldValueCollapse(SolrIndexSearcher searcher,
+                              String field,
+                              int nullPolicy,
+                              boolean max,
+                              boolean needsScores,
+                              IntOpenHashSet boostDocs) {
+      this.field = field;
+      this.nullPolicy = nullPolicy;
+      this.max = max;
+      this.needsScores = needsScores;
+      this.collapsedSet = new OpenBitSet(searcher.maxDoc());
+      this.boostDocs = boostDocs;
+      if(this.boostDocs != null) {
+        Iterator<IntCursor> it = boostDocs.iterator();
+        while(it.hasNext()) {
+          IntCursor cursor = it.next();
+          this.collapsedSet.fastSet(cursor.value);
+        }
+      }
+    }
+
+    public OpenBitSet getCollapsedSet() {
+      if(nullDoc > -1) {
+        this.collapsedSet.fastSet(nullDoc);
+      }
+
+      for(int i=0; i<ords.length; i++) {
+        int doc = ords[i];
+        if(doc > -1) {
+          collapsedSet.fastSet(doc);
+        }
+      }
+
+      return collapsedSet;
+    }
+
+    public void setScorer(Scorer scorer) {
+      this.scorer = scorer;
+    }
+
+    public FloatArrayList getNullScores() {
+      return nullScores;
+    }
+
+    public float getNullScore() {
+      return this.nullScore;
+    }
+
+    public float[] getScores() {
+      return scores;
+    }
+  }
+
+  private class IntValueCollapse extends FieldValueCollapse {
+
+    private FieldCache.Ints vals;
+    private IntCompare comp;
+    private int nullVal;
+    private int[] ordVals;
+
+    public IntValueCollapse(SolrIndexSearcher searcher,
+                            String field,
+                            int nullPolicy,
+                            int[] ords,
+                            boolean max,
+                            boolean needsScores,
+                            IntOpenHashSet boostDocs) throws IOException {
+      super(searcher, field, nullPolicy, max, needsScores, boostDocs);
+      this.ords = ords;
+      this.ordVals = new int[ords.length];
+      Arrays.fill(ords, -1);
+
+      if(max) {
+        comp = new MaxIntComp();
+        Arrays.fill(ordVals, Integer.MIN_VALUE);
+      } else {
+        comp = new MinIntComp();
+        Arrays.fill(ordVals, Integer.MAX_VALUE);
+        this.nullVal = Integer.MAX_VALUE;
+      }
+
+      if(needsScores) {
+        this.scores = new float[ords.length];
+        if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          nullScores = new FloatArrayList();
+        }
+      }
+    }
+
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      this.vals = FieldCache.DEFAULT.getInts(context.reader(), this.field, false);
+    }
+
+    public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
+      int val = vals.get(contextDoc);
+      if(ord > -1) {
+        if(comp.test(val, ordVals[ord])) {
+          ords[ord] = globalDoc;
+          ordVals[ord] = val;
+          if(needsScores) {
+            scores[ord] = scorer.score();
+          }
+        }
+      } else if(this.collapsedSet.fastGet(globalDoc)) {
+        // Elevated doc so do nothing.
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        if(comp.test(val, nullVal)) {
+          nullVal = val;
+          nullDoc = globalDoc;
+          if(needsScores) {
+            nullScore = scorer.score();
+          }
+        }
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        this.collapsedSet.fastSet(globalDoc);
+        if(needsScores) {
+          nullScores.add(scorer.score());
+        }
+      }
+    }
+  }
+
+  private class LongValueCollapse extends FieldValueCollapse {
+
+    private FieldCache.Longs vals;
+    private LongCompare comp;
+    private long nullVal;
+    private long[] ordVals;
+
+    public LongValueCollapse(SolrIndexSearcher searcher,
+                             String field,
+                             int nullPolicy,
+                             int[] ords,
+                             boolean max,
+                             boolean needsScores,
+                             IntOpenHashSet boostDocs) throws IOException {
+      super(searcher, field, nullPolicy, max, needsScores, boostDocs);
+      this.ords = ords;
+      this.ordVals = new long[ords.length];
+      Arrays.fill(ords, -1);
+
+      if(max) {
+        comp = new MaxLongComp();
+        Arrays.fill(ordVals, Long.MIN_VALUE);
+      } else {
+        this.nullVal = Long.MAX_VALUE;
+        comp = new MinLongComp();
+        Arrays.fill(ordVals, Long.MAX_VALUE);
+      }
+
+      if(needsScores) {
+        this.scores = new float[ords.length];
+        if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          nullScores = new FloatArrayList();
+        }
+      }
+    }
+
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      this.vals = FieldCache.DEFAULT.getLongs(context.reader(), this.field, false);
+    }
+
+    public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
+      long val = vals.get(contextDoc);
+      if(ord > -1) {
+        if(comp.test(val, ordVals[ord])) {
+          ords[ord] = globalDoc;
+          ordVals[ord] = val;
+          if(needsScores) {
+            scores[ord] = scorer.score();
+          }
+        }
+      } else if (this.collapsedSet.fastGet(globalDoc)) {
+        //Elevated doc so do nothing
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        if(comp.test(val, nullVal)) {
+          nullVal = val;
+          nullDoc = globalDoc;
+          if(needsScores) {
+            nullScore = scorer.score();
+          }
+        }
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        this.collapsedSet.fastSet(globalDoc);
+        if(needsScores) {
+          nullScores.add(scorer.score());
+        }
+      }
+    }
+  }
+
+  private class FloatValueCollapse extends FieldValueCollapse {
+
+    private FieldCache.Floats vals;
+    private FloatCompare comp;
+    private float nullVal;
+    private float[] ordVals;
+
+    public FloatValueCollapse(SolrIndexSearcher searcher,
+                              String field,
+                              int nullPolicy,
+                              int[] ords,
+                              boolean max,
+                              boolean needsScores,
+                              IntOpenHashSet boostDocs) throws IOException {
+      super(searcher, field, nullPolicy, max, needsScores, boostDocs);
+      this.ords = ords;
+      this.ordVals = new float[ords.length];
+      Arrays.fill(ords, -1);
+
+      if(max) {
+        comp = new MaxFloatComp();
+        Arrays.fill(ordVals, -Float.MAX_VALUE );
+      } else {
+        this.nullVal = Float.MAX_VALUE;
+        comp = new MinFloatComp();
+        Arrays.fill(ordVals, Float.MAX_VALUE);
+      }
+
+      if(needsScores) {
+        this.scores = new float[ords.length];
+        if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+          nullScores = new FloatArrayList();
+        }
+      }
+    }
+
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      this.vals = FieldCache.DEFAULT.getFloats(context.reader(), this.field, false);
+    }
+
+    public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
+      float val = vals.get(contextDoc);
+      if(ord > -1) {
+        if(comp.test(val, ordVals[ord])) {
+          ords[ord] = globalDoc;
+          ordVals[ord] = val;
+          if(needsScores) {
+            scores[ord] = scorer.score();
+          }
+        }
+      } else if (this.collapsedSet.fastGet(globalDoc)) {
+        //Elevated doc so do nothing
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
+        if(comp.test(val, nullVal)) {
+          nullVal = val;
+          nullDoc = globalDoc;
+          if(needsScores) {
+            nullScore = scorer.score();
+          }
+        }
+      } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
+        this.collapsedSet.fastSet(globalDoc);
+        if(needsScores) {
+          nullScores.add(scorer.score());
+        }
+      }
+    }
+  }
+
+  private interface IntCompare {
+    public boolean test(int i1, int i2);
+  }
+
+  private interface FloatCompare {
+    public boolean test(float i1, float i2);
+  }
+
+  private interface LongCompare {
+    public boolean test(long i1, long i2);
+  }
+
+  private class MaxIntComp implements IntCompare {
+    public boolean test(int i1, int i2) {
+      return i1 > i2;
+    }
+  }
+
+  private class MinIntComp implements IntCompare {
+    public boolean test(int i1, int i2) {
+      return i1 < i2;
+    }
+  }
+
+  private class MaxFloatComp implements FloatCompare {
+    public boolean test(float i1, float i2) {
+      return i1 > i2;
+    }
+  }
+
+  private class MinFloatComp implements FloatCompare {
+    public boolean test(float i1, float i2) {
+      return i1 < i2;
+    }
+  }
+
+  private class MaxLongComp implements LongCompare {
+    public boolean test(long i1, long i2) {
+      return i1 > i2;
+    }
+  }
+
+  private class MinLongComp implements LongCompare {
+    public boolean test(long i1, long i2) {
+      return i1 < i2;
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java?rev=1535208&r1=1535207&r2=1535208&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/QParserPlugin.java Wed Oct 23 23:19:12 2013
@@ -51,7 +51,8 @@ public abstract class QParserPlugin impl
     SwitchQParserPlugin.NAME, SwitchQParserPlugin.class,
     MaxScoreQParserPlugin.NAME, MaxScoreQParserPlugin.class,
     BlockJoinParentQParserPlugin.NAME, BlockJoinParentQParserPlugin.class,
-    BlockJoinChildQParserPlugin.NAME, BlockJoinChildQParserPlugin.class
+    BlockJoinChildQParserPlugin.NAME, BlockJoinChildQParserPlugin.class,
+    CollapsingQParserPlugin.NAME, CollapsingQParserPlugin.class
   };
 
   /** return a {@link QParser} */

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml?rev=1535208&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-collapseqparser.xml Wed Oct 23 23:19:12 2013
@@ -0,0 +1,578 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<!-- This is a "kitchen sink" config file that tests can use.
+     When writting a new test, feel free to add *new* items (plugins,
+     config options, etc...) as long as they don't break any existing
+     tests.  if you need to test something esoteric please add a new
+     "solrconfig-your-esoteric-purpose.xml" config file.
+
+     Note in particular that this test is used by MinimalSchemaTest so
+     Anything added to this file needs to work correctly even if there
+     is now uniqueKey or defaultSearch Field.
+  -->
+
+<config>
+
+  <jmx />
+
+  <!-- Used to specify an alternate directory to hold all index data.
+       It defaults to "index" if not present, and should probably
+       not be changed if replication is in use. -->
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <!--  The DirectoryFactory to use for indexes.
+        solr.StandardDirectoryFactory, the default, is filesystem based.
+        solr.RAMDirectoryFactory is memory based and not persistent. -->
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}">
+    <double name="maxWriteMBPerSecDefault">1000000</double>
+    <double name="maxWriteMBPerSecFlush">2000000</double>
+    <double name="maxWriteMBPerSecMerge">3000000</double>
+    <double name="maxWriteMBPerSecRead">4000000</double>
+  </directoryFactory>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+
+    <!-- autocommit pending docs if certain criteria are met
+    <autoCommit>
+      <maxDocs>10000</maxDocs>
+      <maxTime>3600000</maxTime>
+    </autoCommit>
+    -->
+    <!-- represents a lower bound on the frequency that commits may
+    occur (in seconds). NOTE: not yet implemented
+
+    <commitIntervalLowerBound>0</commitIntervalLowerBound>
+    -->
+
+    <!-- The RunExecutableListener executes an external command.
+         exe - the name of the executable to run
+         dir - dir to use as the current working directory. default="."
+         wait - the calling thread waits until the executable returns. default="true"
+         args - the arguments to pass to the program.  default=nothing
+         env - environment variables to set.  default=nothing
+      -->
+    <!-- A postCommit event is fired after every commit
+    <listener event="postCommit" class="solr.RunExecutableListener">
+      <str name="exe">/var/opt/resin3/__PORT__/scripts/solr/snapshooter</str>
+      <str name="dir">/var/opt/resin3/__PORT__</str>
+      <bool name="wait">true</bool>
+      <arr name="args"> <str>arg1</str> <str>arg2</str> </arr>
+      <arr name="env"> <str>MYVAR=val1</str> </arr>
+    </listener>
+    -->
+
+    <updateLog enable="${enable.update.log:true}">
+      <str name="dir">${solr.ulog.dir:}</str>
+    </updateLog>
+
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+
+  </updateHandler>
+
+  <query>
+    <!-- Maximum number of clauses in a boolean query... can affect
+        range or wildcard queries that expand to big boolean
+        queries.  An exception is thrown if exceeded.
+    -->
+    <maxBooleanClauses>1024</maxBooleanClauses>
+
+    <!-- Cache specification for Filters or DocSets - unordered set of *all* documents
+         that match a particular query.
+      -->
+    <filterCache
+        class="solr.search.FastLRUCache"
+        size="512"
+        initialSize="512"
+        autowarmCount="2"/>
+
+    <queryResultCache
+        class="solr.search.LRUCache"
+        size="512"
+        initialSize="512"
+        autowarmCount="2"/>
+
+    <documentCache
+        class="solr.search.LRUCache"
+        size="512"
+        initialSize="512"
+        autowarmCount="0"/>
+
+    <cache name="perSegFilter"
+           class="solr.search.LRUCache"
+           size="10"
+           initialSize="0"
+           autowarmCount="10" />
+
+    <!-- If true, stored fields that are not requested will be loaded lazily.
+    -->
+    <enableLazyFieldLoading>true</enableLazyFieldLoading>
+
+    <!--
+
+    <cache name="myUserCache"
+      class="solr.search.LRUCache"
+      size="4096"
+      initialSize="1024"
+      autowarmCount="1024"
+      regenerator="MyRegenerator"
+      />
+    -->
+
+    <!--
+    <useFilterForSortedQuery>true</useFilterForSortedQuery>
+    -->
+
+    <queryResultWindowSize>10</queryResultWindowSize>
+
+    <!-- set maxSize artificially low to exercise both types of sets -->
+    <HashDocSet maxSize="3" loadFactor="0.75"/>
+
+    <!-- boolToFilterOptimizer converts boolean clauses with zero boost
+         into cached filters if the number of docs selected by the clause exceeds
+         the threshold (represented as a fraction of the total index)
+    -->
+    <boolTofilterOptimizer enabled="false" cacheSize="32" threshold=".05"/>
+
+    <!-- a newSearcher event is fired whenever a new searcher is being prepared
+         and there is a current searcher handling requests (aka registered). -->
+    <!-- QuerySenderListener takes an array of NamedList and executes a
+         local query request for each NamedList in sequence. -->
+    <!--
+    <listener event="newSearcher" class="solr.QuerySenderListener">
+      <arr name="queries">
+        <lst> <str name="q">solr</str> <str name="start">0</str> <str name="rows">10</str> </lst>
+        <lst> <str name="q">rocks</str> <str name="start">0</str> <str name="rows">10</str> </lst>
+      </arr>
+    </listener>
+    -->
+
+    <!-- a firstSearcher event is fired whenever a new searcher is being
+         prepared but there is no current registered searcher to handle
+         requests or to gain prewarming data from. -->
+    <!--
+    <listener event="firstSearcher" class="solr.QuerySenderListener">
+      <arr name="queries">
+        <lst> <str name="q">fast_warm</str> <str name="start">0</str> <str name="rows">10</str> </lst>
+      </arr>
+    </listener>
+    -->
+
+  </query>
+
+  <queryResponseWriter name="xml" default="true"
+                       class="solr.XMLResponseWriter" />
+
+  <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
+
+  <!-- An alternate set representation that uses an integer hash to store filters (sets of docids).
+If the set cardinality <= maxSize elements, then HashDocSet will be used instead of the bitset
+based HashBitset. -->
+
+  <!-- requestHandler plugins... incoming queries will be dispatched to the
+     correct handler based on the 'qt' param matching the
+     name of registered handlers.
+      The "standard" request handler is the default and will be used if qt
+     is not specified in the request.
+  -->
+  <requestHandler name="standard" class="solr.StandardRequestHandler">
+    <bool name="httpCaching">true</bool>
+  </requestHandler>
+
+  <requestHandler name="/get" class="solr.RealTimeGetHandler">
+    <lst name="defaults">
+      <str name="omitHeader">true</str>
+    </lst>
+  </requestHandler>
+
+  <requestHandler name="dismax" class="solr.SearchHandler" >
+    <lst name="defaults">
+      <str name="defType">dismax</str>
+      <str name="q.alt">*:*</str>
+      <float name="tie">0.01</float>
+      <str name="qf">
+        text^0.5 features_t^1.0 subject^1.4 title_stemmed^2.0
+      </str>
+      <str name="pf">
+        text^0.2 features_t^1.1 subject^1.4 title_stemmed^2.0 title^1.5
+      </str>
+      <str name="bf">
+        ord(weight)^0.5 recip(rord(iind),1,1000,1000)^0.3
+      </str>
+      <str name="mm">
+        3&lt;-1 5&lt;-2 6&lt;90%
+      </str>
+      <int name="ps">100</int>
+    </lst>
+  </requestHandler>
+
+  <requestHandler name="mock" class="org.apache.solr.core.MockQuerySenderListenerReqHandler"/>
+
+  <requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
+
+  <!-- test query parameter defaults -->
+  <requestHandler name="defaults" class="solr.StandardRequestHandler">
+    <lst name="defaults">
+      <int name="rows">4</int>
+      <bool name="hl">true</bool>
+      <str name="hl.fl">text,name,subject,title,whitetok</str>
+    </lst>
+  </requestHandler>
+
+  <!-- test query parameter defaults -->
+  <requestHandler name="lazy" class="solr.StandardRequestHandler" startup="lazy">
+    <lst name="defaults">
+      <int name="rows">4</int>
+      <bool name="hl">true</bool>
+      <str name="hl.fl">text,name,subject,title,whitetok</str>
+    </lst>
+  </requestHandler>
+
+  <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
+
+  <searchComponent name="spellcheck" class="org.apache.solr.handler.component.SpellCheckComponent">
+    <!-- This is slightly different from the field value so we can test dealing with token offset changes -->
+    <str name="queryAnalyzerFieldType">lowerpunctfilt</str>
+
+    <lst name="spellchecker">
+      <str name="name">default</str>
+      <str name="field">lowerfilt</str>
+      <str name="spellcheckIndexDir">spellchecker1</str>
+      <str name="buildOnCommit">false</str>
+    </lst>
+    <lst name="spellchecker">
+      <str name="name">direct</str>
+      <str name="classname">DirectSolrSpellChecker</str>
+      <str name="field">lowerfilt</str>
+      <int name="minQueryLength">3</int>
+    </lst>
+    <lst name="spellchecker">
+      <str name="name">wordbreak</str>
+      <str name="classname">solr.WordBreakSolrSpellChecker</str>
+      <str name="field">lowerfilt</str>
+      <str name="combineWords">true</str>
+      <str name="breakWords">true</str>
+      <int name="maxChanges">10</int>
+    </lst>
+    <lst name="spellchecker">
+      <str name="name">multipleFields</str>
+      <str name="field">lowerfilt1and2</str>
+      <str name="spellcheckIndexDir">spellcheckerMultipleFields</str>
+      <str name="buildOnCommit">false</str>
+    </lst>
+    <!-- Example of using different distance measure -->
+    <lst name="spellchecker">
+      <str name="name">jarowinkler</str>
+      <str name="field">lowerfilt</str>
+      <!-- Use a different Distance Measure -->
+      <str name="distanceMeasure">org.apache.lucene.search.spell.JaroWinklerDistance</str>
+      <str name="spellcheckIndexDir">spellchecker2</str>
+
+    </lst>
+    <lst name="spellchecker">
+      <str name="classname">solr.FileBasedSpellChecker</str>
+      <str name="name">external</str>
+      <str name="sourceLocation">spellings.txt</str>
+      <str name="characterEncoding">UTF-8</str>
+      <str name="spellcheckIndexDir">spellchecker3</str>
+    </lst>
+    <!-- Comparator -->
+    <lst name="spellchecker">
+      <str name="name">freq</str>
+      <str name="field">lowerfilt</str>
+      <str name="spellcheckIndexDir">spellcheckerFreq</str>
+      <!-- comparatorClass be one of:
+        1. score (default)
+        2. freq (Frequency first, then score)
+        3. A fully qualified class name
+       -->
+      <str name="comparatorClass">freq</str>
+      <str name="buildOnCommit">false</str>
+    </lst>
+    <lst name="spellchecker">
+      <str name="name">fqcn</str>
+      <str name="field">lowerfilt</str>
+      <str name="spellcheckIndexDir">spellcheckerFQCN</str>
+      <str name="comparatorClass">org.apache.solr.spelling.SampleComparator</str>
+      <str name="buildOnCommit">false</str>
+    </lst>
+    <lst name="spellchecker">
+      <str name="name">perDict</str>
+      <str name="classname">org.apache.solr.handler.component.DummyCustomParamSpellChecker</str>
+      <str name="field">lowerfilt</str>
+    </lst>
+  </searchComponent>
+
+  <searchComponent name="termsComp" class="org.apache.solr.handler.component.TermsComponent"/>
+
+  <requestHandler name="/terms" class="org.apache.solr.handler.component.SearchHandler">
+    <arr name="components">
+      <str>termsComp</str>
+    </arr>
+  </requestHandler>
+  <!--
+  The SpellingQueryConverter to convert raw (CommonParams.Q) queries into tokens.  Uses a simple regular expression
+   to strip off field markup, boosts, ranges, etc. but it is not guaranteed to match an exact parse from the query parser.
+   -->
+  <queryConverter name="queryConverter" class="org.apache.solr.spelling.SpellingQueryConverter"/>
+
+  <requestHandler name="spellCheckCompRH" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <!-- omp = Only More Popular -->
+      <str name="spellcheck.onlyMorePopular">false</str>
+      <!-- exr = Extended Results -->
+      <str name="spellcheck.extendedResults">false</str>
+      <!--  The number of suggestions to return -->
+      <str name="spellcheck.count">1</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
+  <requestHandler name="spellCheckCompRH_Direct" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="spellcheck.dictionary">direct</str>
+      <str name="spellcheck.onlyMorePopular">false</str>
+      <str name="spellcheck.extendedResults">false</str>
+      <str name="spellcheck.count">1</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
+  <requestHandler name="spellCheckWithWordbreak" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="spellcheck.dictionary">default</str>
+      <str name="spellcheck.dictionary">wordbreak</str>
+      <str name="spellcheck.count">20</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
+  <requestHandler name="spellCheckWithWordbreak_Direct" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="spellcheck.dictionary">direct</str>
+      <str name="spellcheck.dictionary">wordbreak</str>
+      <str name="spellcheck.count">20</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
+  <requestHandler name="spellCheckCompRH1" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="defType">dismax</str>
+      <str name="qf">lowerfilt1^1</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
+
+  <requestHandler name="mltrh" class="org.apache.solr.handler.component.SearchHandler">
+
+  </requestHandler>
+
+  <searchComponent name="tvComponent" class="org.apache.solr.handler.component.TermVectorComponent"/>
+
+  <requestHandler name="tvrh" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+
+    </lst>
+    <arr name="last-components">
+      <str>tvComponent</str>
+    </arr>
+  </requestHandler>
+
+  <!-- test elevation -->
+  <searchComponent name="elevate" class="org.apache.solr.handler.component.QueryElevationComponent" >
+    <str name="queryFieldType">string</str>
+    <str name="config-file">elevate.xml</str>
+  </searchComponent>
+
+
+  <requestHandler name="/elevate" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+    </lst>
+    <arr name="last-components">
+      <str>elevate</str>
+    </arr>
+  </requestHandler>
+
+  <requestHandler name="/mlt" class="solr.MoreLikeThisHandler">
+  </requestHandler>
+
+  <searchComponent class="solr.HighlightComponent" name="highlight">
+    <highlighting>
+      <!-- Configure the standard fragmenter -->
+      <fragmenter name="gap" class="org.apache.solr.highlight.GapFragmenter" default="true">
+        <lst name="defaults">
+          <int name="hl.fragsize">100</int>
+        </lst>
+      </fragmenter>
+
+      <fragmenter name="regex" class="org.apache.solr.highlight.RegexFragmenter">
+        <lst name="defaults">
+          <int name="hl.fragsize">70</int>
+        </lst>
+      </fragmenter>
+
+      <!-- Configure the standard formatter -->
+      <formatter name="html" class="org.apache.solr.highlight.HtmlFormatter" default="true">
+        <lst name="defaults">
+          <str name="hl.simple.pre"><![CDATA[<em>]]></str>
+          <str name="hl.simple.post"><![CDATA[</em>]]></str>
+        </lst>
+      </formatter>
+
+      <!-- Configure the standard fragListBuilder -->
+      <fragListBuilder name="simple" class="org.apache.solr.highlight.SimpleFragListBuilder" default="true"/>
+
+      <!-- Configure the standard fragmentsBuilder -->
+      <fragmentsBuilder name="simple" class="org.apache.solr.highlight.SimpleFragmentsBuilder" default="true"/>
+      <fragmentsBuilder name="scoreOrder" class="org.apache.solr.highlight.ScoreOrderFragmentsBuilder"/>
+
+      <boundaryScanner name="simple" class="solr.highlight.SimpleBoundaryScanner" default="true">
+        <lst name="defaults">
+          <str name="hl.bs.maxScan">10</str>
+          <str name="hl.bs.chars">.,!? &#9;&#10;&#13;</str>
+        </lst>
+      </boundaryScanner>
+
+      <boundaryScanner name="breakIterator" class="solr.highlight.BreakIteratorBoundaryScanner">
+        <lst name="defaults">
+          <str name="hl.bs.type">WORD</str>
+          <str name="hl.bs.language">en</str>
+          <str name="hl.bs.country">US</str>
+        </lst>
+      </boundaryScanner>
+    </highlighting>
+  </searchComponent>
+
+  <!-- enable streaming for testing... -->
+  <requestDispatcher handleSelect="true" >
+    <requestParsers enableRemoteStreaming="true" multipartUploadLimitInKB="2048" />
+    <httpCaching lastModifiedFrom="openTime" etagSeed="Solr" never304="false">
+      <cacheControl>max-age=30, public</cacheControl>
+    </httpCaching>
+  </requestDispatcher>
+
+  <!-- Echo the request contents back to the client -->
+  <requestHandler name="/debug/dump" class="solr.DumpRequestHandler" >
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="echoHandler">true</str>
+    </lst>
+  </requestHandler>
+
+  <admin>
+    <defaultQuery>solr</defaultQuery>
+    <gettableFiles>solrconfig.xml schema.xml admin-extra.html</gettableFiles>
+  </admin>
+
+  <!-- test getting system property -->
+  <propTest attr1="${solr.test.sys.prop1}-$${literal}"
+            attr2="${non.existent.sys.prop:default-from-config}">prefix-${solr.test.sys.prop2}-suffix</propTest>
+
+  <queryParser name="foo" class="FooQParserPlugin"/>
+
+  <updateRequestProcessorChain name="dedupe">
+    <processor class="org.apache.solr.update.processor.SignatureUpdateProcessorFactory">
+      <bool name="enabled">false</bool>
+      <bool name="overwriteDupes">true</bool>
+      <str name="fields">v_t,t_field</str>
+      <str name="signatureClass">org.apache.solr.update.processor.TextProfileSignature</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+  <updateRequestProcessorChain name="dedupe-allfields">
+    <processor class="org.apache.solr.update.processor.SignatureUpdateProcessorFactory">
+      <bool name="enabled">false</bool>
+      <bool name="overwriteDupes">false</bool>
+      <str name="signatureField">id</str>
+      <str name="fields"></str>
+      <str name="signatureClass">org.apache.solr.update.processor.Lookup3Signature</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+  <updateRequestProcessorChain name="stored_sig">
+    <!-- this chain is valid even though the signature field is not
+         indexed, because we are not asking for dups to be overwritten
+      -->
+    <processor class="org.apache.solr.update.processor.SignatureUpdateProcessorFactory">
+      <bool name="enabled">true</bool>
+      <str name="signatureField">non_indexed_signature_sS</str>
+      <bool name="overwriteDupes">false</bool>
+      <str name="fields">v_t,t_field</str>
+      <str name="signatureClass">org.apache.solr.update.processor.TextProfileSignature</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+  <updateRequestProcessorChain name="uniq-fields">
+    <processor class="org.apache.solr.update.processor.UniqFieldsUpdateProcessorFactory">
+      <arr name="fieldName">
+        <str>uniq</str>
+        <str>uniq2</str>
+        <str>uniq3</str>
+      </arr>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="distrib-dup-test-chain-explicit">
+    <!-- explicit test using processors before and after distrib -->
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_A_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_B_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="distrib-dup-test-chain-implicit">
+    <!-- implicit test w/o distrib declared-->
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_A_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_B_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+</config>
+

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java?rev=1535208&r1=1535207&r2=1535208&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java Wed Oct 23 23:19:12 2013
@@ -195,6 +195,16 @@ public class QueryEqualityTest extends S
     }
   }
 
+  public void testQueryCollapse() throws Exception {
+    SolrQueryRequest req = req("myField","foo_s");
+    try {
+      assertQueryEquals("collapse", req,
+          "{!collapse field=$myField}");
+    } finally {
+      req.close();
+    }
+  }
+
   public void testQueryNested() throws Exception {
     SolrQueryRequest req = req("df", "foo_s");
     try {

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java?rev=1535208&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java Wed Oct 23 23:19:12 2013
@@ -0,0 +1,128 @@
+/*
+ * 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.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+
+public class TestCollapseQParserPlugin extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-collapseqparser.xml", "schema11.xml");
+  }
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    // if you override setUp or tearDown, you better call
+    // the super classes version
+    super.setUp();
+    clearIndex();
+    assertU(commit());
+  }
+
+  @Test
+  public void testCollapseQueries() throws Exception {
+    String[] doc = {"id","1", "term_s", "YYYY", "group_s", "group1", "test_ti", "5", "test_tl", "10", "test_tf", "2000"};
+    assertU(adoc(doc));
+    String[] doc1 = {"id","2", "term_s","YYYY", "group_s", "group1", "test_ti", "50", "test_tl", "100", "test_tf", "200"};
+    assertU(adoc(doc1));
+
+    String[] doc2 = {"id","3", "term_s", "YYYY", "test_ti", "5000", "test_tl", "100", "test_tf", "200"};
+    assertU(adoc(doc2));
+
+    String[] doc3 = {"id","4", "term_s", "YYYY", "test_ti", "500", "test_tl", "1000", "test_tf", "2000"};
+    assertU(adoc(doc3));
+
+    assertU(commit());
+
+    //Test collapse by score
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s}");
+    params.add("defType", "edismax");
+    params.add("bf", "field(test_ti)");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='50']");
+
+    //Test collapse by score with elevation
+
+    params = new ModifiableSolrParams();
+    params.add("q", "YYYY");
+    params.add("fq", "{!collapse field=group_s nullPolicy=collapse}");
+    params.add("defType", "edismax");
+    params.add("bf", "field(test_ti)");
+    params.add("qf", "term_s");
+    params.add("qt", "/elevate");
+    assertQ(req(params), "*[count(//doc)=3]", "//doc[./int[1][@name='test_ti']='5']");
+
+    //Test collapse by min int field
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s min=test_ti}");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='5']");
+
+    //Test collapse by max int field
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s max=test_ti}");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='50']");
+
+    //Test collapse by min long field
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s min=test_tl}");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='5']");
+
+    //Test collapse by max long field
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s max=test_tl}");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='50']");
+
+    //Test collapse by min float field
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s min=test_tf}");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='50']");
+
+    //Test collapse by min float field
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s max=test_tf}");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='5']");
+
+    //Test nullPolicy expand
+    params = new ModifiableSolrParams();
+    params.add("q", "*:*");
+    params.add("fq", "{!collapse field=group_s max=test_tf nullPolicy=expand}");
+    assertQ(req(params), "*[count(//doc)=3]");
+
+    //Test nullPolicy collapse
+    params = new ModifiableSolrParams();
+    params.add("q", "test_ti:(500 5000)");
+    params.add("fq", "{!collapse field=group_s max=test_tf nullPolicy=collapse}");
+    assertQ(req(params), "*[count(//doc)=1]", "//doc[./int[@name='test_ti']='500']");
+  }
+}