You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2015/03/26 13:38:23 UTC

svn commit: r1669336 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/test/org/apache/lucene/util/ solr/ solr/core/ solr/core/src/java/org/apache/solr/request/ solr/core/src/test/...

Author: romseygeek
Date: Thu Mar 26 12:38:23 2015
New Revision: 1669336

URL: http://svn.apache.org/r1669336
Log:
SOLR-1387: Add facet.contains and facet.contains.ignoreCase

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
    lucene/dev/branches/branch_5x/solr/solrj/   (props changed)
    lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Thu Mar 26 12:38:23 2015
@@ -22,6 +22,7 @@ import java.math.BigInteger;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Arrays;
+import java.util.Locale;
 import java.util.Properties;
 
 /**
@@ -33,11 +34,11 @@ public abstract class StringHelper {
 
   /**
    * Compares two {@link BytesRef}, element by element, and returns the
-   * number of elements common to both arrays.
+   * number of elements common to both arrays (from the start of each).
    *
    * @param left The first {@link BytesRef} to compare
    * @param right The second {@link BytesRef} to compare
-   * @return The number of common elements.
+   * @return The number of common elements (from the start of each).
    */
   public static int bytesDifference(BytesRef left, BytesRef right) {
     int len = left.length < right.length ? left.length : right.length;
@@ -134,6 +135,34 @@ public abstract class StringHelper {
     return sliceEquals(ref, suffix, ref.length - suffix.length);
   }
   
+  /**
+   * Returns <code>true</code> iff the ref contains the given slice. Otherwise
+   * <code>false</code>.
+   * 
+   * @param ref
+   *          the {@link BytesRef} to test
+   * @param slice
+   *          the slice to look for
+   * @param ignoreCase
+   *          whether the comparison should be case-insensitive
+   * @return Returns <code>true</code> iff the ref contains the given slice.
+   *         Otherwise <code>false</code>.
+   */
+  public static boolean contains(BytesRef ref, BytesRef slice, boolean ignoreCase) {
+    if (ignoreCase) {
+      String s1 = ref.utf8ToString();
+      String s2 = slice.utf8ToString();
+      return s1.toLowerCase(Locale.ENGLISH).contains(s2.toLowerCase(Locale.ENGLISH));
+    } else {
+      for (int pos = 0; pos <= ref.length - slice.length; ++pos) {
+        if (sliceEquals(ref, slice, pos)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  
   private static boolean sliceEquals(BytesRef sliceToTest, BytesRef other, int pos) {
     if (pos < 0 || sliceToTest.length - pos < other.length) {
       return false;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/TestStringHelper.java Thu Mar 26 12:38:23 2015
@@ -18,6 +18,73 @@ package org.apache.lucene.util;
  */
 
 public class TestStringHelper extends LuceneTestCase {
+  
+  public void testBytesDifference() {
+    BytesRef left = new BytesRef("foobar");
+    BytesRef right = new BytesRef("foozo");
+    assertEquals(3, StringHelper.bytesDifference(left, right));
+  }
+
+  public void testEquals() {
+    assertTrue(StringHelper.equals("foo", "foo"));
+    assertFalse(StringHelper.equals("foo", null));
+    assertFalse(StringHelper.equals(null, "foo"));
+    assertTrue(StringHelper.equals(null, null));
+  }
+  
+  public void testStartsWith() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("foo");
+    assertTrue(StringHelper.startsWith(ref, slice));
+  }
+  
+  public void testEndsWith() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("bar");
+    assertTrue(StringHelper.endsWith(ref, slice));
+  }
+
+  public void testStartsWithWhole() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("foobar");
+    assertTrue(StringHelper.startsWith(ref, slice));
+  }
+  
+  public void testEndsWithWhole() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("foobar");
+    assertTrue(StringHelper.endsWith(ref, slice));
+  }
+
+  public void testContainsAtStart() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("foo");
+    assertTrue(StringHelper.contains(ref, slice, false));
+  }
+  
+  public void testContains() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("ooba");
+    assertTrue(StringHelper.contains(ref, slice, false));
+  }
+  
+  public void testContainsAtEnd() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("bar");
+    assertTrue(StringHelper.contains(ref, slice, false));
+  }
+  
+  public void testContainsWhole() {
+    BytesRef ref = new BytesRef("foobar");
+    BytesRef slice = new BytesRef("foobar");
+    assertTrue(StringHelper.contains(ref, slice, false));
+  }
+  
+  public void testContainsIgnoreCase() {
+    BytesRef ref = new BytesRef("FooBar");
+    BytesRef slice = new BytesRef("bar");
+    assertTrue(StringHelper.contains(ref, slice, true));
+  }
 
   public void testMurmurHash3() throws Exception {
     // Hashes computed using murmur3_32 from https://code.google.com/p/pyfasthash

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Thu Mar 26 12:38:23 2015
@@ -159,6 +159,9 @@ New Features
 * SOLR-7307: EmbeddedSolrServer can now be started up by passing a path to a
   solr home directory, or a NodeConfig object (Alan Woodward, Mike Drob)
 
+* SOLR-1387: Add facet.contains and facet.contains.ignoreCase options (Tom Winch
+  via Alan Woodward)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java Thu Mar 26 12:38:23 2015
@@ -34,6 +34,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.util.NamedList;
@@ -58,7 +59,7 @@ import org.apache.solr.util.LongPriority
 public class DocValuesFacets {
   private DocValuesFacets() {}
   
-  public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix) throws IOException {
+  public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase) throws IOException {
     SchemaField schemaField = searcher.getSchema().getField(fieldName);
     FieldType ft = schemaField.getType();
     NamedList<Integer> res = new NamedList<>();
@@ -97,6 +98,8 @@ public class DocValuesFacets {
       prefixRef = new BytesRefBuilder();
       prefixRef.copyChars(prefix);
     }
+    
+    final BytesRef containsBR = contains != null ? new BytesRef(contains) : null;
 
     int startTermIndex, endTermIndex;
     if (prefix!=null) {
@@ -170,6 +173,12 @@ public class DocValuesFacets {
         int min=mincount-1;  // the smallest value in the top 'N' values
         for (int i=(startTermIndex==-1)?1:0; i<nTerms; i++) {
           int c = counts[i];
+          if (containsBR != null) {
+            final BytesRef term = si.lookupOrd(startTermIndex+i);
+            if (!StringHelper.contains(term, containsBR, ignoreCase)) {
+              continue;
+            }
+          }
           if (c>min) {
             // NOTE: we use c>min rather than c>=min as an optimization because we are going in
             // index order, so we already know that the keys are ordered.  This can be very
@@ -203,18 +212,28 @@ public class DocValuesFacets {
       } else {
         // add results in index order
         int i=(startTermIndex==-1)?1:0;
-        if (mincount<=0) {
-          // if mincount<=0, then we won't discard any terms and we know exactly
-          // where to start.
+        if (mincount<=0 && containsBR == null) {
+          // if mincount<=0 and we're not examining the values for contains, then
+          // we won't discard any terms and we know exactly where to start.
           i+=off;
           off=0;
         }
 
         for (; i<nTerms; i++) {          
           int c = counts[i];
-          if (c<mincount || --off>=0) continue;
+          if (c<mincount) continue;
+          BytesRef term = null;
+          if (containsBR != null) {
+            term = si.lookupOrd(startTermIndex+i);
+            if (!StringHelper.contains(term, containsBR, ignoreCase)) {
+              continue;
+            }
+          }
+          if (--off>=0) continue;
           if (--lim<0) break;
-          final BytesRef term = si.lookupOrd(startTermIndex+i);
+          if (term == null) {
+            term = si.lookupOrd(startTermIndex+i);
+          }
           ft.indexedToReadable(term, charsRef);
           res.add(charsRef.toString(), c);
         }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java Thu Mar 26 12:38:23 2015
@@ -32,6 +32,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.FacetParams;
@@ -54,12 +55,14 @@ class PerSegmentSingleValuedFaceting {
   boolean missing;
   String sort;
   String prefix;
+  BytesRef containsBR;
+  boolean ignoreCase;
 
   Filter baseSet;
 
   int nThreads;
 
-  public PerSegmentSingleValuedFaceting(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix) {
+  public PerSegmentSingleValuedFaceting(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase) {
     this.searcher = searcher;
     this.docs = docs;
     this.fieldName = fieldName;
@@ -69,6 +72,8 @@ class PerSegmentSingleValuedFaceting {
     this.missing = missing;
     this.sort = sort;
     this.prefix = prefix;
+    this.containsBR = contains != null ? new BytesRef(contains) : null;
+    this.ignoreCase = ignoreCase;
   }
 
   public void setNumThreads(int threads) {
@@ -173,16 +178,23 @@ class PerSegmentSingleValuedFaceting {
 
     while (queue.size() > 0) {
       SegFacet seg = queue.top();
-
+      
+      // if facet.contains specified, only actually collect the count if substring contained
+      boolean collect = containsBR == null || StringHelper.contains(seg.tempBR, containsBR, ignoreCase);
+      
       // we will normally end up advancing the term enum for this segment
       // while still using "val", so we need to make a copy since the BytesRef
       // may be shared across calls.
-      val.copyBytes(seg.tempBR);
-
+      if (collect) {
+        val.copyBytes(seg.tempBR);
+      }
+      
       int count = 0;
 
       do {
-        count += seg.counts[seg.pos - seg.startTermIndex];
+        if (collect) {
+          count += seg.counts[seg.pos - seg.startTermIndex];
+        }
 
         // TODO: OPTIMIZATION...
         // if mincount>0 then seg.pos++ can skip ahead to the next non-zero entry.
@@ -190,14 +202,16 @@ class PerSegmentSingleValuedFaceting {
         if (seg.pos >= seg.endTermIndex) {
           queue.pop();
           seg = queue.top();
-        }  else {
+        } else {
           seg.tempBR = seg.tenum.next();
           seg = queue.updateTop();
         }
       } while (seg != null && val.get().compareTo(seg.tempBR) == 0);
 
-      boolean stop = collector.collect(val.get(), count);
-      if (stop) break;
+      if (collect) {
+        boolean stop = collector.collect(val.get(), count);
+        if (stop) break;
+      }
     }
 
     NamedList<Integer> res = collector.getFacetCounts();
@@ -253,31 +267,28 @@ class PerSegmentSingleValuedFaceting {
         startTermIndex=-1;
         endTermIndex=si.getValueCount();
       }
-
       final int nTerms=endTermIndex-startTermIndex;
-      if (nTerms>0) {
-        // count collection array only needs to be as big as the number of terms we are
-        // going to collect counts for.
-        final int[] counts = this.counts = new int[nTerms];
-        DocIdSet idSet = baseSet.getDocIdSet(context, null);  // this set only includes live docs
-        DocIdSetIterator iter = idSet.iterator();
+      if (nTerms == 0) return;
 
+      // count collection array only needs to be as big as the number of terms we are
+      // going to collect counts for.
+      final int[] counts = this.counts = new int[nTerms];
+      DocIdSet idSet = baseSet.getDocIdSet(context, null);  // this set only includes live docs
+      DocIdSetIterator iter = idSet.iterator();
 
-        ////
+      if (prefix==null) {
+        // specialized version when collecting counts for all terms
         int doc;
-
-        if (prefix==null) {
-          // specialized version when collecting counts for all terms
-          while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
-            counts[1+si.getOrd(doc)]++;
-          }
-        } else {
-          // version that adjusts term numbers because we aren't collecting the full range
-          while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
-            int term = si.getOrd(doc);
-            int arrIdx = term-startTermIndex;
-            if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
-          }
+        while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
+          counts[1+si.getOrd(doc)]++;
+        }
+      } else {
+        // version that adjusts term numbers because we aren't collecting the full range
+        int doc;
+        while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
+          int term = si.getOrd(doc);
+          int arrIdx = term-startTermIndex;
+          if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
         }
       }
     }

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Thu Mar 26 12:38:23 2015
@@ -17,26 +17,6 @@
 
 package org.apache.solr.request;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.IdentityHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Future;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -97,6 +77,26 @@ import org.apache.solr.util.BoundedTreeS
 import org.apache.solr.util.DateMathParser;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
 /**
  * A class that generates simple Facet information for a request.
  *
@@ -289,7 +289,6 @@ public class SimpleFacets {
 
     String[] facetQs = params.getParams(FacetParams.FACET_QUERY);
 
-    
     if (null != facetQs && 0 != facetQs.length) {
       for (String q : facetQs) {
         parseParams(FacetParams.FACET_QUERY, q);
@@ -382,8 +381,9 @@ public class SimpleFacets {
     boolean missing = params.getFieldBool(field, FacetParams.FACET_MISSING, false);
     // default to sorting if there is a limit.
     String sort = params.getFieldParam(field, FacetParams.FACET_SORT, limit>0 ? FacetParams.FACET_SORT_COUNT : FacetParams.FACET_SORT_INDEX);
-    String prefix = params.getFieldParam(field,FacetParams.FACET_PREFIX);
-
+    String prefix = params.getFieldParam(field, FacetParams.FACET_PREFIX);
+    String contains = params.getFieldParam(field, FacetParams.FACET_CONTAINS);
+    boolean ignoreCase = params.getFieldBool(field, FacetParams.FACET_CONTAINS_IGNORE_CASE, false);
 
     NamedList<Integer> counts;
     SchemaField sf = searcher.getSchema().getField(field);
@@ -435,13 +435,13 @@ public class SimpleFacets {
     }
 
     if (params.getFieldBool(field, GroupParams.GROUP_FACET, false)) {
-      counts = getGroupedCounts(searcher, base, field, multiToken, offset,limit, mincount, missing, sort, prefix);
+      counts = getGroupedCounts(searcher, base, field, multiToken, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
     } else {
       assert method != null;
       switch (method) {
         case ENUM:
           assert TrieField.getMainValuePrefix(ft) == null;
-          counts = getFacetTermEnumCounts(searcher, base, field, offset, limit, mincount,missing,sort,prefix);
+          counts = getFacetTermEnumCounts(searcher, base, field, offset, limit, mincount, missing, sort, prefix, contains, ignoreCase);
           break;
         case FCS:
           assert !multiToken;
@@ -450,16 +450,19 @@ public class SimpleFacets {
             if (prefix != null && !prefix.isEmpty()) {
               throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_PREFIX + " is not supported on numeric types");
             }
+            if (contains != null && !contains.isEmpty()) {
+              throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_CONTAINS + " is not supported on numeric types");
+            }
             counts = NumericFacets.getCounts(searcher, base, field, offset, limit, mincount, missing, sort);
           } else {
-            PerSegmentSingleValuedFaceting ps = new PerSegmentSingleValuedFaceting(searcher, base, field, offset,limit, mincount, missing, sort, prefix);
+            PerSegmentSingleValuedFaceting ps = new PerSegmentSingleValuedFaceting(searcher, base, field, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
             Executor executor = threads == 0 ? directExecutor : facetExecutor;
             ps.setNumThreads(threads);
             counts = ps.getFacetCounts(executor);
           }
           break;
         case FC:
-          counts = DocValuesFacets.getCounts(searcher, base, field, offset,limit, mincount, missing, sort, prefix);
+          counts = DocValuesFacets.getCounts(searcher, base, field, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
           break;
         default:
           throw new AssertionError();
@@ -478,7 +481,9 @@ public class SimpleFacets {
                                              int mincount,
                                              boolean missing,
                                              String sort,
-                                             String prefix) throws IOException {
+                                             String prefix,
+                                             String contains,
+                                             boolean ignoreCase) throws IOException {
     GroupingSpecification groupingSpecification = rb.getGroupingSpec();
     final String groupField  = groupingSpecification != null ? groupingSpecification.getFields()[0] : null;
     if (groupField == null) {
@@ -488,8 +493,9 @@ public class SimpleFacets {
       );
     }
 
-    BytesRef prefixBR = prefix != null ? new BytesRef(prefix) : null;
-    final TermGroupFacetCollector collector = TermGroupFacetCollector.createTermGroupFacetCollector(groupField, field, multiToken, prefixBR, 128);
+    BytesRef prefixBytesRef = prefix != null ? new BytesRef(prefix) : null;
+    BytesRef containsRef = contains != null ? new BytesRef(contains) : null;
+    final TermGroupFacetCollector collector = TermGroupFacetCollector.createTermGroupFacetCollector(groupField, field, multiToken, prefixBytesRef, 128);
     
     SchemaField sf = searcher.getSchema().getFieldOrNull(groupField);
     
@@ -519,6 +525,10 @@ public class SimpleFacets {
     List<TermGroupFacetCollector.FacetEntry> scopedEntries 
       = result.getFacetEntries(offset, limit < 0 ? Integer.MAX_VALUE : limit);
     for (TermGroupFacetCollector.FacetEntry facetEntry : scopedEntries) {
+      //:TODO:can we do contains earlier than this to make it more efficient?
+      if (containsRef != null && !StringHelper.contains(facetEntry.getValue(), containsRef, ignoreCase)) {
+        continue;
+      }
       facetFieldType.indexedToReadable(facetEntry.getValue(), charsRef);
       facetCounts.add(charsRef.toString(), facetEntry.getCount());
     }
@@ -683,7 +693,7 @@ public class SimpleFacets {
    * @see FacetParams#FACET_ZEROS
    * @see FacetParams#FACET_MISSING
    */
-  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, String sort, String prefix)
+  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase)
     throws IOException {
 
     /* :TODO: potential optimization...
@@ -715,10 +725,16 @@ public class SimpleFacets {
     int off=offset;
     int lim=limit>=0 ? limit : Integer.MAX_VALUE;
 
-    BytesRef startTermBytes = null;
+    BytesRef prefixTermBytes = null;
     if (prefix != null) {
       String indexedPrefix = ft.toInternal(prefix);
-      startTermBytes = new BytesRef(indexedPrefix);
+      prefixTermBytes = new BytesRef(indexedPrefix);
+    }
+    
+    BytesRef containsTermBytes = null;
+    if (contains != null) {
+      String indexedContains = ft.toInternal(contains);
+      containsTermBytes = new BytesRef(indexedContains);
     }
 
     Fields fields = r.fields();
@@ -732,8 +748,8 @@ public class SimpleFacets {
       // TODO: OPT: if seek(ord) is supported for this termsEnum, then we could use it for
       // facet.offset when sorting by index order.
 
-      if (startTermBytes != null) {
-        if (termsEnum.seekCeil(startTermBytes) == TermsEnum.SeekStatus.END) {
+      if (prefixTermBytes != null) {
+        if (termsEnum.seekCeil(prefixTermBytes) == TermsEnum.SeekStatus.END) {
           termsEnum = null;
         } else {
           term = termsEnum.term();
@@ -750,77 +766,77 @@ public class SimpleFacets {
     if (docs.size() >= mincount) {
       while (term != null) {
 
-        if (startTermBytes != null && !StringHelper.startsWith(term, startTermBytes))
+        if (prefixTermBytes != null && !StringHelper.startsWith(term, prefixTermBytes))
           break;
 
-        int df = termsEnum.docFreq();
+        if (containsTermBytes == null || StringHelper.contains(term, containsTermBytes, ignoreCase)) {
+          int df = termsEnum.docFreq();
 
-        // If we are sorting, we can use df>min (rather than >=) since we
-        // are going in index order.  For certain term distributions this can
-        // make a large difference (for example, many terms with df=1).
-        if (df>0 && df>min) {
-          int c;
-
-          if (df >= minDfFilterCache) {
-            // use the filter cache
-
-            if (deState==null) {
-              deState = new SolrIndexSearcher.DocsEnumState();
-              deState.fieldName = field;
-              deState.liveDocs = r.getLiveDocs();
-              deState.termsEnum = termsEnum;
-              deState.postingsEnum = postingsEnum;
-            }
+          // If we are sorting, we can use df>min (rather than >=) since we
+          // are going in index order.  For certain term distributions this can
+          // make a large difference (for example, many terms with df=1).
+          if (df > 0 && df > min) {
+            int c;
+
+            if (df >= minDfFilterCache) {
+              // use the filter cache
+
+              if (deState == null) {
+                deState = new SolrIndexSearcher.DocsEnumState();
+                deState.fieldName = field;
+                deState.liveDocs = r.getLiveDocs();
+                deState.termsEnum = termsEnum;
+                deState.postingsEnum = postingsEnum;
+              }
 
-            c = searcher.numDocs(docs, deState);
+              c = searcher.numDocs(docs, deState);
 
-            postingsEnum = deState.postingsEnum;
-          } else {
-            // iterate over TermDocs to calculate the intersection
+              postingsEnum = deState.postingsEnum;
+            } else {
+              // iterate over TermDocs to calculate the intersection
 
-            // TODO: specialize when base docset is a bitset or hash set (skipDocs)?  or does it matter for this?
-            // TODO: do this per-segment for better efficiency (MultiDocsEnum just uses base class impl)
-            // TODO: would passing deleted docs lead to better efficiency over checking the fastForRandomSet?
-            postingsEnum = termsEnum.postings(null, postingsEnum, PostingsEnum.NONE);
-            c=0;
-
-            if (postingsEnum instanceof MultiPostingsEnum) {
-              MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
-              int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
-              for (int subindex = 0; subindex<numSubs; subindex++) {
-                MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
-                if (sub.postingsEnum == null) continue;
-                int base = sub.slice.start;
+              // TODO: specialize when base docset is a bitset or hash set (skipDocs)?  or does it matter for this?
+              // TODO: do this per-segment for better efficiency (MultiDocsEnum just uses base class impl)
+              // TODO: would passing deleted docs lead to better efficiency over checking the fastForRandomSet?
+              postingsEnum = termsEnum.postings(null, postingsEnum, PostingsEnum.NONE);
+              c = 0;
+
+              if (postingsEnum instanceof MultiPostingsEnum) {
+                MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
+                int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
+                for (int subindex = 0; subindex < numSubs; subindex++) {
+                  MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
+                  if (sub.postingsEnum == null) continue;
+                  int base = sub.slice.start;
+                  int docid;
+                  while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+                    if (fastForRandomSet.exists(docid + base)) c++;
+                  }
+                }
+              } else {
                 int docid;
-                while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-                  if (fastForRandomSet.exists(docid+base)) c++;
+                while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+                  if (fastForRandomSet.exists(docid)) c++;
                 }
               }
-            } else {
-              int docid;
-              while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-                if (fastForRandomSet.exists(docid)) c++;
-              }
-            }
-            
 
-          }
-
-          if (sortByCount) {
-            if (c>min) {
-              BytesRef termCopy = BytesRef.deepCopyOf(term);
-              queue.add(new CountPair<>(termCopy, c));
-              if (queue.size()>=maxsize) min=queue.last().val;
             }
-          } else {
-            if (c >= mincount && --off<0) {
-              if (--lim<0) break;
-              ft.indexedToReadable(term, charsRef);
-              res.add(charsRef.toString(), c);
+
+            if (sortByCount) {
+              if (c > min) {
+                BytesRef termCopy = BytesRef.deepCopyOf(term);
+                queue.add(new CountPair<>(termCopy, c));
+                if (queue.size() >= maxsize) min = queue.last().val;
+              }
+            } else {
+              if (c >= mincount && --off < 0) {
+                if (--lim < 0) break;
+                ft.indexedToReadable(term, charsRef);
+                res.add(charsRef.toString(), c);
+              }
             }
           }
         }
-
         term = termsEnum.next();
       }
     }

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/request/SimpleFacetsTest.java Thu Mar 26 12:38:23 2015
@@ -17,12 +17,6 @@
 
 package org.apache.solr.request;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -32,6 +26,12 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.noggit.ObjectBuilder;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
 
 public class SimpleFacetsTest extends SolrTestCaseJ4 {
 
@@ -85,6 +85,7 @@ public class SimpleFacetsTest extends So
     indexFacetSingleValued();
     indexFacetPrefixMultiValued();
     indexFacetPrefixSingleValued();
+    indexFacetContains();
     indexSimpleGroupedFacetCounts();
 
     Collections.shuffle(pendingDocs, random());
@@ -1866,54 +1867,74 @@ public class SimpleFacetsTest extends So
 
 
   static void indexFacetPrefixMultiValued() {
-    indexFacetPrefix("50","t_s");
+    indexFacetPrefix("50","t_s","","ignore_s");
   }
 
   @Test
   public void testFacetPrefixMultiValued() {
-    doFacetPrefix("t_s", null, "facet.method","enum");
-    doFacetPrefix("t_s", null, "facet.method", "enum", "facet.enum.cache.minDf", "3");
-    doFacetPrefix("t_s", null, "facet.method", "enum", "facet.enum.cache.minDf", "100");
-    doFacetPrefix("t_s", null, "facet.method", "fc");
+    doFacetPrefix("t_s", null, "", "facet.method","enum");
+    doFacetPrefix("t_s", null, "", "facet.method", "enum", "facet.enum.cache.minDf", "3");
+    doFacetPrefix("t_s", null, "", "facet.method", "enum", "facet.enum.cache.minDf", "100");
+    doFacetPrefix("t_s", null, "", "facet.method", "fc");
   }
 
   static void indexFacetPrefixSingleValued() {
-    indexFacetPrefix("60","tt_s1");
+    indexFacetPrefix("60","tt_s1","","ignore_s");
   }
 
   @Test
   public void testFacetPrefixSingleValued() {
-    doFacetPrefix("tt_s1", null);
+    doFacetPrefix("tt_s1", null, "");
   }
+  
   @Test
   public void testFacetPrefixSingleValuedFcs() {
-    doFacetPrefix("tt_s1", null, "facet.method","fcs");
-    doFacetPrefix("tt_s1", "{!threads=0}", "facet.method","fcs");   // direct execution
-    doFacetPrefix("tt_s1", "{!threads=-1}", "facet.method","fcs");  // default / unlimited threads
-    doFacetPrefix("tt_s1", "{!threads=2}", "facet.method","fcs");   // specific number of threads
+    doFacetPrefix("tt_s1", null, "", "facet.method","fcs");
+    doFacetPrefix("tt_s1", "{!threads=0}", "", "facet.method","fcs");   // direct execution
+    doFacetPrefix("tt_s1", "{!threads=-1}", "", "facet.method","fcs");  // default / unlimited threads
+    doFacetPrefix("tt_s1", "{!threads=2}", "", "facet.method","fcs");   // specific number of threads
   }
 
-
-  static void indexFacetPrefix(String idPrefix, String f) {
-    add_doc("id", idPrefix+"1",  f, "AAA");
-    add_doc("id", idPrefix+"2",  f, "B");
-    add_doc("id", idPrefix+"3",  f, "BB");
-    add_doc("id", idPrefix+"4",  f, "BB");
-    add_doc("id", idPrefix+"5",  f, "BBB");
-    add_doc("id", idPrefix+"6",  f, "BBB");
-    add_doc("id", idPrefix+"7",  f, "BBB");
-    add_doc("id", idPrefix+"8",  f, "CC");
-    add_doc("id", idPrefix+"9",  f, "CC");
-    add_doc("id", idPrefix+"10", f, "CCC");
-    add_doc("id", idPrefix+"11", f, "CCC");
-    add_doc("id", idPrefix+"12", f, "CCC");
+  static void indexFacetContains() {
+    indexFacetPrefix("70","contains_s1","","contains_group_s1");
+    indexFacetPrefix("80","contains_s1","Astra","contains_group_s1");
+  }
+  
+  @Test
+  public void testFacetContains() {
+    doFacetContains("contains_s1", "contains_group_s1", "Astra", "BAst", "Ast", "facet.method", "enum");
+    doFacetContains("contains_s1", "contains_group_s1", "Astra", "BAst", "Ast", "facet.method", "fcs");
+    doFacetContains("contains_s1", "contains_group_s1", "Astra", "BAst", "Ast", "facet.method", "fc");
+    doFacetContains("contains_s1", "contains_group_s1", "Astra", "bAst", "ast", "facet.method", "enum", "facet.contains.ignoreCase", "true");
+    doFacetContains("contains_s1", "contains_group_s1", "Astra", "baSt", "ast", "facet.method", "fcs", "facet.contains.ignoreCase", "true");
+    doFacetContains("contains_s1", "contains_group_s1", "Astra", "basT", "ast", "facet.method", "fc", "facet.contains.ignoreCase", "true");
+    doFacetPrefix("contains_s1", null, "Astra", "facet.method", "enum", "facet.contains", "Ast");
+    doFacetPrefix("contains_s1", null, "Astra", "facet.method", "fcs", "facet.contains", "Ast");
+    doFacetPrefix("contains_s1", null, "Astra", "facet.method", "fc", "facet.contains", "Ast");
+    doFacetPrefix("contains_s1", null, "Astra", "facet.method", "enum", "facet.contains", "aSt", "facet.contains.ignoreCase", "true");
+    doFacetPrefix("contains_s1", null, "Astra", "facet.method", "fcs", "facet.contains", "asT", "facet.contains.ignoreCase", "true");
+    doFacetPrefix("contains_s1", null, "Astra", "facet.method", "fc", "facet.contains", "aST", "facet.contains.ignoreCase", "true");
+  }
+
+  static void indexFacetPrefix(String idPrefix, String f, String termSuffix, String g) {
+    add_doc("id", idPrefix+"1",  f, "AAA"+termSuffix, g, "A");
+    add_doc("id", idPrefix+"2",  f, "B"+termSuffix,   g, "A");
+    add_doc("id", idPrefix+"3",  f, "BB"+termSuffix,  g, "B");
+    add_doc("id", idPrefix+"4",  f, "BB"+termSuffix,  g, "B");
+    add_doc("id", idPrefix+"5",  f, "BBB"+termSuffix, g, "B");
+    add_doc("id", idPrefix+"6",  f, "BBB"+termSuffix, g, "B");
+    add_doc("id", idPrefix+"7",  f, "BBB"+termSuffix, g, "C");
+    add_doc("id", idPrefix+"8",  f, "CC"+termSuffix,  g, "C");
+    add_doc("id", idPrefix+"9",  f, "CC"+termSuffix,  g, "C");
+    add_doc("id", idPrefix+"10", f, "CCC"+termSuffix, g, "C");
+    add_doc("id", idPrefix+"11", f, "CCC"+termSuffix, g, "D");
+    add_doc("id", idPrefix+"12", f, "CCC"+termSuffix, g, "E");
     assertU(commit());
   }
 
-  public void doFacetPrefix(String f, String local, String... params) {
+  public void doFacetPrefix(String f, String local, String termSuffix, String... params) {
     String indent="on";
     String pre = "//lst[@name='"+f+"']";
-    String notc = "id:[* TO *] -"+f+":C";
     String lf = local==null ? f : local+f;
 
 
@@ -1929,9 +1950,9 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","B"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=3]"
-            ,pre+"/int[1][@name='BBB'][.='3']"
-            ,pre+"/int[2][@name='BB'][.='2']"
-            ,pre+"/int[3][@name='B'][.='1']"
+            ,pre+"/int[1][@name='BBB"+termSuffix+"'][.='3']"
+            ,pre+"/int[2][@name='BB"+termSuffix+"'][.='2']"
+            ,pre+"/int[3][@name='B"+termSuffix+"'][.='1']"
     );
 
     assertQ("test facet.prefix middle, exact match first term, unsorted",
@@ -1946,30 +1967,11 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","B"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=3]"
-            ,pre+"/int[1][@name='B'][.='1']"
-            ,pre+"/int[2][@name='BB'][.='2']"
-            ,pre+"/int[3][@name='BBB'][.='3']"
+            ,pre+"/int[1][@name='B"+termSuffix+"'][.='1']"
+            ,pre+"/int[2][@name='BB"+termSuffix+"'][.='2']"
+            ,pre+"/int[3][@name='BBB"+termSuffix+"'][.='3']"
     );
 
-
-     assertQ("test facet.prefix middle, exact match first term, unsorted",
-            req(params, "q", "id:[* TO *]"
-                    ,"indent",indent
-                    ,"facet","true"
-                    ,"facet.field", lf
-                    ,"facet.mincount","0"
-                    ,"facet.offset","0"
-                    ,"facet.limit","100"
-                    ,"facet.sort","index"
-                    ,"facet.prefix","B"
-            )
-            ,"*[count(//lst[@name='facet_fields']/lst/int)=3]"
-            ,pre+"/int[1][@name='B'][.='1']"
-            ,pre+"/int[2][@name='BB'][.='2']"
-            ,pre+"/int[3][@name='BBB'][.='3']"
-    );
-
-
     assertQ("test facet.prefix middle, paging",
             req(params, "q", "id:[* TO *]"
                     ,"indent",indent
@@ -1982,8 +1984,8 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","B"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=2]"
-            ,pre+"/int[1][@name='BB'][.='2']"
-            ,pre+"/int[2][@name='B'][.='1']"
+            ,pre+"/int[1][@name='BB"+termSuffix+"'][.='2']"
+            ,pre+"/int[2][@name='B"+termSuffix+"'][.='1']"
     );
 
     assertQ("test facet.prefix middle, paging",
@@ -1998,7 +2000,7 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","B"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
-            ,pre+"/int[1][@name='BB'][.='2']"
+            ,pre+"/int[1][@name='BB"+termSuffix+"'][.='2']"
     );
 
     assertQ("test facet.prefix middle, paging",
@@ -2013,7 +2015,7 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","B"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
-            ,pre+"/int[1][@name='BB'][.='2']"
+            ,pre+"/int[1][@name='BB"+termSuffix+"'][.='2']"
     );
 
     assertQ("test facet.prefix end, not exact match",
@@ -2028,8 +2030,8 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","C"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=2]"
-            ,pre+"/int[1][@name='CCC'][.='3']"
-            ,pre+"/int[2][@name='CC'][.='2']"
+            ,pre+"/int[1][@name='CCC"+termSuffix+"'][.='3']"
+            ,pre+"/int[2][@name='CC"+termSuffix+"'][.='2']"
     );
 
     assertQ("test facet.prefix end, exact match",
@@ -2044,8 +2046,8 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","CC"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=2]"
-            ,pre+"/int[1][@name='CCC'][.='3']"
-            ,pre+"/int[2][@name='CC'][.='2']"
+            ,pre+"/int[1][@name='CCC"+termSuffix+"'][.='3']"
+            ,pre+"/int[2][@name='CC"+termSuffix+"'][.='2']"
     );
 
     assertQ("test facet.prefix past end",
@@ -2088,7 +2090,7 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","AAA"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
-            ,pre+"/int[1][@name='AAA'][.='1']"
+            ,pre+"/int[1][@name='AAA"+termSuffix+"'][.='1']"
     );
     assertQ("test facet.prefix at Start, not exact match",
             req(params, "q", "id:[* TO *]"
@@ -2102,7 +2104,7 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","AA"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
-            ,pre+"/int[1][@name='AAA'][.='1']"
+            ,pre+"/int[1][@name='AAA"+termSuffix+"'][.='1']"
     );
     assertQ("test facet.prefix at Start, not exact match",
             req(params, "q", "id:[* TO *]"
@@ -2116,7 +2118,7 @@ public class SimpleFacetsTest extends So
                     ,"facet.prefix","AA"
             )
             ,"*[count(//lst[@name='facet_fields']/lst/int)=1]"
-            ,pre+"/int[1][@name='AAA'][.='1']"
+            ,pre+"/int[1][@name='AAA"+termSuffix+"'][.='1']"
     );    
     assertQ("test facet.prefix before start",
             req(params, "q", "id:[* TO *]"
@@ -2162,9 +2164,54 @@ public class SimpleFacetsTest extends So
     );
   }
 
+  public void doFacetContains(String f, String g, String termSuffix, String contains, String groupContains, String... params) {
+    String indent="on";
+    String pre = "//lst[@name='"+f+"']";
+
+    assertQ("test facet.contains",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.field", f
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.contains",contains
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=3]"
+            ,pre+"/int[1][@name='BBB"+termSuffix+"'][.='3']"
+            ,pre+"/int[2][@name='BB"+termSuffix+"'][.='2']"
+            ,pre+"/int[3][@name='B"+termSuffix+"'][.='1']"
+    );
+
+    assertQ("test facet.contains for grouped facets",
+            req(params, "q", "id:[* TO *]"
+                    ,"indent",indent
+                    ,"facet","true"
+                    ,"facet.field", f
+                    ,"facet.mincount","0"
+                    ,"facet.offset","0"
+                    ,"facet.limit","100"
+                    ,"facet.sort","count"
+                    ,"facet.contains",groupContains
+                    ,"group","true"
+                    ,"group.field",g
+                    ,"group.facet","true"
+            )
+            ,"*[count(//lst[@name='facet_fields']/lst/int)=6]"
+            ,pre+"/int[1][@name='CCC"+termSuffix+"'][.='3']"
+            ,pre+"/int[2][@name='BBB"+termSuffix+"'][.='2']"
+            ,pre+"/int[3][@name='AAA"+termSuffix+"'][.='1']"
+            ,pre+"/int[4][@name='B"+termSuffix+"'][.='1']"
+            ,pre+"/int[5][@name='BB"+termSuffix+"'][.='1']"
+            ,pre+"/int[6][@name='CC"+termSuffix+"'][.='1']"
+    );
+  }
+
   /** 
-   * kind of an absurd tests because if there is an inifnite loop, it 
-   * would ver finish -- but at least it ensures that <i>if</i> one of 
+   * kind of an absurd test because if there is an infinite loop, it 
+   * would never finish -- but at least it ensures that <i>if</i> one of 
    * these requests return, they return an error 
    */
   public void testRangeFacetInfiniteLoopDetection() {

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java?rev=1669336&r1=1669335&r2=1669336&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java Thu Mar 26 12:38:23 2015
@@ -149,6 +149,16 @@ public interface FacetParams {
    */
   public static final String FACET_PREFIX = FACET + ".prefix";
 
+  /**
+   * Only return constraints of a facet field containing the given string.
+   */
+  public static final String FACET_CONTAINS = FACET + ".contains";
+
+  /**
+   * If using facet contains, ignore case when comparing values.
+   */
+  public static final String FACET_CONTAINS_IGNORE_CASE = FACET_CONTAINS + ".ignoreCase";
+
  /**
    * When faceting by enumerating the terms in a field,
    * only use the filterCache for terms with a df &gt;= to this parameter.
@@ -165,7 +175,7 @@ public interface FacetParams {
    */
   public static final String FACET_DATE_START = FACET_DATE + ".start";
   /**
-   * Date string indicating the endinging point for a date facet range.
+   * Date string indicating the ending point for a date facet range.
    * Can be overriden on a per field basis.
    */
   public static final String FACET_DATE_END = FACET_DATE + ".end";