You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2019/08/05 09:51:15 UTC

[lucene-solr] branch branch_8x updated: SOLR-11866: QueryElevationComponent match="subset" feature Closes #780

This is an automated email from the ASF dual-hosted git repository.

dsmiley pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 445f4bf  SOLR-11866: QueryElevationComponent match="subset" feature Closes #780
445f4bf is described below

commit 445f4bf5c28b91ddf02f25d61f0b282995c99891
Author: Bruno Roustant <br...@salesforce.com>
AuthorDate: Mon Aug 5 11:23:47 2019 +0200

    SOLR-11866: QueryElevationComponent match="subset" feature
    Closes #780
    
    (cherry picked from commit d97912529d5ec4e86a8b6def4103bc6f4fbfd24b)
---
 solr/CHANGES.txt                                   |   3 +
 .../handler/component/QueryElevationComponent.java | 514 ++++++++++++++++++---
 .../test-files/solr/collection1/conf/elevate.xml   |  22 +
 .../solr/collection1/conf/solrconfig-elevate.xml   |   2 +-
 .../component/QueryElevationComponentTest.java     | 166 ++++++-
 .../src/the-query-elevation-component.adoc         |   8 +
 6 files changed, 640 insertions(+), 75 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9179888..73e4750 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -54,6 +54,9 @@ New Features
 
 * SOLR-13622: Add files() stream source to create tuples from lines in local files (Jason Gerlowski and Joel Bernstein)
 
+* SOLR-11866: QueryElevationComponent can have query rules configured with match="subset" wherein the words need only
+  match a subset of the query's words and in any order.  (Bruno Roustant via David Smiley)
+
 Improvements
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
index d396c84..3ce531b 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
@@ -25,26 +25,35 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Queue;
 import java.util.Set;
+import java.util.SortedSet;
 import java.util.WeakHashMap;
+import java.util.function.Consumer;
 
 import com.carrotsearch.hppc.IntIntHashMap;
 import com.carrotsearch.hppc.cursors.IntIntCursor;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
 import com.google.common.collect.Collections2;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
+import com.google.common.collect.ImmutableSortedSet;
+import com.google.common.collect.ObjectArrays;
+import com.google.common.collect.Sets;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -208,7 +217,6 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
    * @param core The core holding this component.
    * @return The number of elevation rules parsed.
    */
-  @SuppressWarnings("WeakerAccess")
   protected int loadElevationConfiguration(SolrCore core) throws Exception {
     synchronized (elevationProviderCache) {
       elevationProviderCache.clear();
@@ -376,12 +384,10 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
   /**
    * Loads the {@link ElevationProvider}.
    *
-   * @throws java.io.IOException      If an I/O error occurs while analyzing the triggering queries.
    * @throws RuntimeException If the config does not provide an XML content of the expected format
    *                          (either {@link RuntimeException} or {@link org.apache.solr.common.SolrException}).
    */
-  @SuppressWarnings("WeakerAccess")
-  protected ElevationProvider loadElevationProvider(XmlConfigFile config) throws IOException {
+  protected ElevationProvider loadElevationProvider(XmlConfigFile config) {
     Map<ElevatingQuery, ElevationBuilder> elevationBuilderMap = new LinkedHashMap<>();
     XPath xpath = XPathFactory.newInstance().newXPath();
     NodeList nodes = (NodeList) config.evaluate("elevate/query", XPathConstants.NODESET);
@@ -389,7 +395,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
       Node node = nodes.item(i);
       String queryString = DOMUtil.getAttr(node, "text", "missing query 'text'");
       String matchString = DOMUtil.getAttr(node, "match");
-      ElevatingQuery elevatingQuery = new ElevatingQuery(queryString, parseMatchPolicy(matchString));
+      ElevatingQuery elevatingQuery = new ElevatingQuery(queryString, isSubsetMatchPolicy(matchString));
 
       NodeList children;
       try {
@@ -425,10 +431,10 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
         previousElevationBuilder.merge(elevationBuilder);
       }
     }
-    return createElevationProvider(queryAnalyzer, elevationBuilderMap);
+    return createElevationProvider(elevationBuilderMap);
   }
 
-  private boolean parseMatchPolicy(String matchString) {
+  protected boolean isSubsetMatchPolicy(String matchString) {
     if (matchString == null) {
       return DEFAULT_SUBSET_MATCH;
     } else if (matchString.equalsIgnoreCase("exact")) {
@@ -591,7 +597,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
       fields.add(currentFields.get(i));
     }
     return modify ?
-            new SortSpec(new Sort(sorts.toArray(new SortField[sorts.size()])),
+            new SortSpec(new Sort(sorts.toArray(new SortField[0])),
                     fields,
                     current.getCount(),
                     current.getOffset())
@@ -682,12 +688,11 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
    * Creates the {@link ElevationProvider} to set during configuration loading. The same instance will be used later
    * when elevating results for queries.
    *
-   * @param queryAnalyzer to analyze and tokenize the query.
    * @param elevationBuilderMap map of all {@link ElevatingQuery} and their corresponding {@link ElevationBuilder}.
    * @return The created {@link ElevationProvider}.
    */
-  protected ElevationProvider createElevationProvider(Analyzer queryAnalyzer, Map<ElevatingQuery, ElevationBuilder> elevationBuilderMap) {
-    return new MapElevationProvider(elevationBuilderMap);
+  protected ElevationProvider createElevationProvider(Map<ElevatingQuery, ElevationBuilder> elevationBuilderMap) {
+    return new DefaultElevationProvider(new TrieSubsetMatcher.Builder<>(), elevationBuilderMap);
   }
 
   //---------------------------------------------------------------------------------
@@ -695,22 +700,28 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
   //---------------------------------------------------------------------------------
 
   /**
-   * Analyzes the provided query string and returns a space concatenation of the analyzed tokens.
+   * Analyzes the provided query string and returns a concatenation of the analyzed tokens.
    */
   public String analyzeQuery(String query) {
-    //split query terms with analyzer then join
-    StringBuilder norm = new StringBuilder();
+    StringBuilder concatTerms = new StringBuilder();
+    analyzeQuery(query, concatTerms::append);
+    return concatTerms.toString();
+  }
+
+  /**
+   * Analyzes the provided query string, tokenizes the terms, and adds them to the provided {@link Consumer}.
+   */
+  protected void analyzeQuery(String query, Consumer<CharSequence> termsConsumer) {
     try (TokenStream tokens = queryAnalyzer.tokenStream("", query)) {
       tokens.reset();
       CharTermAttribute termAtt = tokens.addAttribute(CharTermAttribute.class);
       while (tokens.incrementToken()) {
-        norm.append(termAtt);
+        termsConsumer.accept(termAtt);
       }
       tokens.end();
     } catch (IOException e) {
-      Throwables.propagate(e);
+      throw new RuntimeException(e);
     }
-    return norm.toString();
   }
 
   //---------------------------------------------------------------------------------
@@ -726,11 +737,10 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
    * @param subsetMatch <code>true</code> for query subset match; <code>false</code> for query exact match.
    * @param elevatedIds The readable ids of the documents to set as top results for the provided query.
    * @param excludedIds The readable ids of the document to exclude from results for the provided query.
-   * @throws java.io.IOException If there is a low-level I/O error.
    */
   @VisibleForTesting
   void setTopQueryResults(IndexReader reader, String queryString, boolean subsetMatch,
-                          String[] elevatedIds, String[] excludedIds) throws IOException {
+                          String[] elevatedIds, String[] excludedIds) {
     clearElevationProviderCache();
     ElevatingQuery elevatingQuery = new ElevatingQuery(queryString, subsetMatch);
     ElevationBuilder elevationBuilder = new ElevationBuilder();
@@ -738,7 +748,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
     elevationBuilder.addExcludedIds(excludedIds == null ? Collections.emptyList() : Arrays.asList(excludedIds));
     Map<ElevatingQuery, ElevationBuilder> elevationBuilderMap = ImmutableMap.of(elevatingQuery, elevationBuilder);
     synchronized (elevationProviderCache) {
-      elevationProviderCache.computeIfAbsent(reader, k -> createElevationProvider(queryAnalyzer, elevationBuilderMap));
+      elevationProviderCache.computeIfAbsent(reader, k -> createElevationProvider(elevationBuilderMap));
     }
   }
 
@@ -834,48 +844,80 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
   };
 
   /**
-   * Simple query exact match {@link ElevationProvider}.
-   * <p>
-   * It does not support subset matching (see {@link #parseMatchPolicy(String)}).
-   * <p>
-   * Immutable.
+   * Provides elevations with either:
+   * <ul>
+   * <li><b>subset match</b> - all the elevating terms are matched in the search query, in any order.</li>
+   * <li><b>exact match</b> - the elevating query matches fully (all terms in same order) the search query.</li>
+   * </ul>
+   * The terms are tokenized with the query analyzer.
    */
-  @SuppressWarnings("WeakerAccess")
-  protected class MapElevationProvider implements ElevationProvider {
+  protected class DefaultElevationProvider implements ElevationProvider {
 
-    private final Map<String, Elevation> elevationMap;
-
-    public MapElevationProvider(Map<ElevatingQuery, ElevationBuilder> elevationBuilderMap) {
-      elevationMap = buildElevationMap(elevationBuilderMap);
-    }
+    private final TrieSubsetMatcher<String, Elevation> subsetMatcher;
+    private final Map<String, Elevation> exactMatchElevationMap;
 
-    private Map<String, Elevation> buildElevationMap(Map<ElevatingQuery, ElevationBuilder> elevationBuilderMap) {
-      Map<String, Elevation> elevationMap = Maps.newHashMapWithExpectedSize(elevationBuilderMap.size());
+    /**
+     * @param subsetMatcherBuilder The {@link TrieSubsetMatcher.Builder} to build the {@link TrieSubsetMatcher}.
+     * @param elevationBuilderMap The map of elevation rules.
+     */
+    protected DefaultElevationProvider(TrieSubsetMatcher.Builder<String, Elevation> subsetMatcherBuilder,
+                                       Map<ElevatingQuery, ElevationBuilder> elevationBuilderMap) {
+      exactMatchElevationMap = new LinkedHashMap<>();
+      Collection<String> queryTerms = new ArrayList<>();
+      Consumer<CharSequence> termsConsumer = term -> queryTerms.add(term.toString());
+      StringBuilder concatTerms = new StringBuilder();
+      Consumer<CharSequence> concatConsumer = concatTerms::append;
       for (Map.Entry<ElevatingQuery, ElevationBuilder> entry : elevationBuilderMap.entrySet()) {
         ElevatingQuery elevatingQuery = entry.getKey();
-        if (elevatingQuery.subsetMatch) {
-          throw new UnsupportedOperationException("Subset matching is not supported by " + getClass().getName());
-        }
-        String analyzedQuery = analyzeQuery(elevatingQuery.queryString);
         Elevation elevation = entry.getValue().build();
-        Elevation duplicateElevation = elevationMap.put(analyzedQuery, elevation);
-        if (duplicateElevation != null) {
-          throw new IllegalArgumentException("Duplicate elevation for query.  Analyzed: \"" + analyzedQuery + "\"" +
-              " Original: \"" + elevatingQuery.queryString + "\"");
+        if (elevatingQuery.subsetMatch) {
+          queryTerms.clear();
+          analyzeQuery(elevatingQuery.queryString, termsConsumer);
+          subsetMatcherBuilder.addSubset(queryTerms, elevation);
+        } else {
+          concatTerms.setLength(0);
+          analyzeQuery(elevatingQuery.queryString, concatConsumer);
+          exactMatchElevationMap.put(concatTerms.toString(), elevation);
         }
       }
-      return Collections.unmodifiableMap(elevationMap);
+      this.subsetMatcher = subsetMatcherBuilder.build();
     }
 
     @Override
     public Elevation getElevationForQuery(String queryString) {
-      String analyzedQuery = analyzeQuery(queryString);
-      return elevationMap.get(analyzedQuery);
+      boolean hasExactMatchElevationRules = exactMatchElevationMap.size() != 0;
+      if (subsetMatcher.getSubsetCount() == 0) {
+        if (!hasExactMatchElevationRules) {
+          return null;
+        }
+        return exactMatchElevationMap.get(analyzeQuery(queryString));
+      }
+      Collection<String> queryTerms = new ArrayList<>();
+      Consumer<CharSequence> termsConsumer = term -> queryTerms.add(term.toString());
+      StringBuilder concatTerms = null;
+      if (hasExactMatchElevationRules) {
+        concatTerms = new StringBuilder();
+        termsConsumer = termsConsumer.andThen(concatTerms::append);
+      }
+      analyzeQuery(queryString, termsConsumer);
+      Elevation mergedElevation = null;
+
+      if (hasExactMatchElevationRules) {
+        mergedElevation = exactMatchElevationMap.get(concatTerms.toString());
+      }
+
+      Iterator<Elevation> elevationIterator = subsetMatcher.findSubsetsMatching(queryTerms);
+      while (elevationIterator.hasNext()) {
+        Elevation elevation = elevationIterator.next();
+        mergedElevation = mergedElevation == null ? elevation : mergedElevation.mergeWith(elevation);
+      }
+
+      return mergedElevation;
     }
 
     @Override
     public int size() {
-      return elevationMap.size();
+      return exactMatchElevationMap.size() + subsetMatcher.getSubsetCount();
     }
   }
 
@@ -892,7 +934,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
      * @param queryString The query to elevate documents for (not the analyzed form).
      * @param subsetMatch Whether to match a subset of query terms.
      */
-    protected ElevatingQuery(String queryString, boolean subsetMatch) throws IOException {
+    protected ElevatingQuery(String queryString, boolean subsetMatch) {
       this.queryString = queryString;
       this.subsetMatch = subsetMatch;
     }
@@ -944,7 +986,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
 
     public ElevationBuilder addExcludedIds(Collection<String> ids) {
       if (excludedIds == null) {
-        excludedIds = new LinkedHashSet<>(Math.max(10, ids.size()));
+        excludedIds = new HashSet<>(Math.max(10, ids.size()));
       }
       for (String id : ids) {
         excludedIds.add(toBytesRef(id));
@@ -979,6 +1021,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
 
   /**
    * Elevation of some documents in search results, with potential exclusion of others.
+   * Immutable.
    */
   protected static class Elevation {
 
@@ -992,18 +1035,18 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
 
     /**
      * Constructs an elevation.
-     * @param elevatedIds           The ids of the elevated documents that should appear on top of search results; can be <code>null</code>.
-     *                              In configured order.
-     * @param excludedIds           The ids of the excluded documents that should not appear in search results; can be <code>null</code>.
-     * @param queryFieldName        The field name to use to create query terms.
+     *
+     * @param elevatedIds    The ids of the elevated documents that should appear on top of search results, in configured order;
+     *                       can be <code>null</code>.
+     * @param excludedIds    The ids of the excluded documents that should not appear in search results; can be <code>null</code>.
+     * @param queryFieldName The field name to use to create query terms.
      */
-    public Elevation(Set<BytesRef> elevatedIds, Set<BytesRef> excludedIds,
-                      String queryFieldName) {
+    public Elevation(Set<BytesRef> elevatedIds, Set<BytesRef> excludedIds, String queryFieldName) {
       if (elevatedIds == null || elevatedIds.isEmpty()) {
         includeQuery = EMPTY_QUERY;
         this.elevatedIds = Collections.emptySet();
       } else {
-        this.elevatedIds = new LinkedHashSet<>(elevatedIds);
+        this.elevatedIds = ImmutableSet.copyOf(elevatedIds);
         BooleanQuery.Builder includeQueryBuilder = new BooleanQuery.Builder();
         for (BytesRef elevatedId : elevatedIds) {
           includeQueryBuilder.add(new TermQuery(new Term(queryFieldName, elevatedId)), BooleanClause.Occur.SHOULD);
@@ -1020,10 +1063,59 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
         for (BytesRef excludedId : excludedIds) {
           excludeQueriesBuilder.add(new TermQuery(new Term(queryFieldName, excludedId)));
         }
-        excludeQueries = excludeQueriesBuilder.toArray(new TermQuery[excludeQueriesBuilder.size()]);
+        excludeQueries = excludeQueriesBuilder.toArray(new TermQuery[0]);
       }
     }
 
+    protected Elevation(Set<BytesRef> elevatedIds, BooleanQuery includeQuery, Set<BytesRef> excludedIds, TermQuery[] excludeQueries) {
+      this.elevatedIds = elevatedIds;
+      this.includeQuery = includeQuery;
+      this.excludedIds = excludedIds;
+      this.excludeQueries = excludeQueries;
+    }
+
+    /**
+     * Merges this {@link Elevation} with another and creates a new {@link Elevation}.
+
+     * @return A new instance containing the merging of the two elevations; or directly this elevation if the other
+     *         is <code>null</code>.
+     */
+    protected Elevation mergeWith(Elevation elevation) {
+      if (elevation == null) {
+        return this;
+      }
+      Set<BytesRef> elevatedIds = ImmutableSet.<BytesRef>builder().addAll(this.elevatedIds).addAll(elevation.elevatedIds).build();
+      boolean overlappingElevatedIds = elevatedIds.size() != (this.elevatedIds.size() + elevation.elevatedIds.size());
+      BooleanQuery.Builder includeQueryBuilder = new BooleanQuery.Builder();
+      Set<BooleanClause> clauseSet = (overlappingElevatedIds ? Sets.newHashSetWithExpectedSize(elevatedIds.size()) : null);
+      for (BooleanClause clause : this.includeQuery.clauses()) {
+        if (!overlappingElevatedIds || clauseSet.add(clause)) {
+          includeQueryBuilder.add(clause);
+        }
+      }
+      for (BooleanClause clause : elevation.includeQuery.clauses()) {
+        if (!overlappingElevatedIds || clauseSet.add(clause)) {
+          includeQueryBuilder.add(clause);
+        }
+      }
+      Set<BytesRef> excludedIds = ImmutableSet.<BytesRef>builder().addAll(this.excludedIds).addAll(elevation.excludedIds).build();
+      TermQuery[] excludeQueries;
+      if (this.excludeQueries == null) {
+        excludeQueries = elevation.excludeQueries;
+      } else if (elevation.excludeQueries == null) {
+        excludeQueries = this.excludeQueries;
+      } else {
+        boolean overlappingExcludedIds = excludedIds.size() != (this.excludedIds.size() + elevation.excludedIds.size());
+        if (overlappingExcludedIds) {
+          excludeQueries = ImmutableSet.<TermQuery>builder().add(this.excludeQueries).add(elevation.excludeQueries)
+              .build().toArray(new TermQuery[0]);
+        } else {
+          excludeQueries = ObjectArrays.concat(this.excludeQueries, elevation.excludeQueries, TermQuery.class);
+        }
+      }
+      return new Elevation(elevatedIds, includeQueryBuilder.build(), excludedIds, excludeQueries);
+    }
+
     @Override
     public String toString() {
       return "{elevatedIds=" + Collections2.transform(elevatedIds, BytesRef::utf8ToString) +
@@ -1064,7 +1156,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
         boolean hasElevatedDocsThisSegment;
 
         @Override
-        protected void doSetNextReader(LeafReaderContext context) throws IOException {
+        protected void doSetNextReader(LeafReaderContext context) {
           docBase = context.docBase;
           // ascertain if hasElevatedDocsThisSegment
           final int idx = Arrays.binarySearch(sortedElevatedDocIds, docBase);
@@ -1131,4 +1223,308 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
       };
     }
   }
+
+  /**
+   * Matches a potentially large collection of subsets with a trie implementation.
+   * <p>
+   * Given a collection of subsets <code>N</code>, finds all the subsets that are contained (ignoring duplicate elements)
+   * by a provided set <code>s</code>.
+   * That is, finds all subsets <code>n</code> in <code>N</code> for which <code>s.containsAll(n)</code>
+   * (<code>s</code> contains all the elements of <code>n</code>, in any order).
+   * <p>
+   * Associates a match value of type &lt;M&gt; to each subset and provides it each time the subset matches (i.e. is
+   * contained by the provided set).
+   * <p>
+   * This matcher imposes the elements are {@link Comparable}.
+   * It does not keep the subset insertion order.
+   * Duplicate subsets stack their match values.
+   * <p>
+   * The time complexity of adding a subset is <code>O(n.log(n))</code>, where <code>n</code> is the size of the subset.
+   * <p>
+   * The worst case time complexity of the subset matching is <code>O(2^s)</code>, however a more typical case time
+   * complexity is <code>O(s^3)</code> where s is the size of the set to partially match.
+   * Note it does not depend on <code>N</code>, the size of the collection of subsets, nor on <code>n</code>, the size of
+   * a subset.
+   *
+   * @param <E> Subset element type.
+   * @param <M> Subset match value type.
+   */
+  protected static class TrieSubsetMatcher<E extends Comparable<? super E>, M> {
+
+      /*
+      Trie structure:
+      ---------------
+      - A subset element on each edge.
+      - Each node may contain zero or more match values.
+
+      Sample construction:
+      --------------------
+      - given the subsets "B A C", "A B", "A B A", "B", "D B".
+      - remove duplicates and sort each subset => "A B C", "A B", "A B", "B", "B D".
+      - N() means a node with no match value.
+      - N(x, y) means a node with 2 match values x and y.
+
+        root
+          --A--> N()
+                   --B--> N("A B", "A B A")
+                            --C--> N("B A C")
+          --B--> N("B")
+                   --D--> N("D B")
+
+      Subset matching algorithm:
+      --------------------------
+      - given a set s
+
+      In the above sample, with s="A B C B", then the matching subsets are "B A C", "A B", "A B A", "B"
+
+      remove duplicates in s
+      sort s
+      keep a queue Q of current nodes
+      Add root node to Q
+      Another queue Q' will hold the child nodes (initially empty)
+      for each element e in s {
+        for each current node in Q {
+          if current node has a child for edge e {
+            add the child to Q'
+            record the child match values
+          }
+          if e is greater than or equal to current node greatest edge {
+            remove current node from Q (as we are sure this current node children cannot match anymore)
+          }
+        }
+        Move all child nodes from Q' to Q
+      }
+
+      Time complexity:
+      ----------------
+      s = size of the set to partially match
+      N = size of the collection of subsets
+      n = size of a subset
+
+      The time complexity depends on the number of current nodes in Q.
+
+      The worst case time complexity:
+      For a given set s:
+      - initially Q contains only 1 current node, the root
+        => 1 node
+      - for first element e1 in s, at most 1 node is added to Q
+        => 2 nodes
+      - for element e2 in s, at most 2 new nodes are added to Q
+        => 4 nodes
+      - for element e3 in s, at most 4 new nodes are added to Q
+        => 8 nodes
+      - for element ek in s, at most 2^(k-1) new nodes are added to Q
+        => 2^k nodes
+      - however there are, in worst case, a maximum of N.n nodes
+      Sum[k=0 to s](2^k) = 2^(s+1)-1
+      So the worst case time complexity is: min(O(2^s), O(s.N.n))
+
+      A more typical case time complexity:
+      For a given set s:
+      - initially Q contains only 1 current node, the root
+        => 1 node
+      - for first element e1 in s, 1 node is added to Q
+        => 2 nodes
+      - for element e2 in s, 2 new nodes are added to Q
+        => 4 nodes
+      - for element e3 in s, 3 new nodes are added to Q
+        => 7 nodes
+      - for element ek in s, k new nodes are added to Q
+        => previous nodes + k : q(k) = q(k-1) + k
+
+      Solution is q(k) = 1/2 (k^2+k+2)
+      Sum[k=0 to s](k^2+k+2)/2 = 1/6 (s+1) (s^2+2s+6)
+      So a more typical case time complexity is: min(O(s^3), O(s.N.n))
+      */
+
+    public static class Builder<E extends Comparable<? super E>, M> {
+
+      private final TrieSubsetMatcher.Node<E, M> root = new TrieSubsetMatcher.Node<>();
+      private int subsetCount;
+
+      /**
+       * Adds a subset. If the subset is already registered, the new match value is added to the previous one(s).
+       *
+       * @param subset     The subset of {@link Comparable} elements; it is copied. It is ignored if its size is <code>0</code>.
+       *                   Any subset added is guaranteed to be returned by {@link TrieSubsetMatcher#findSubsetsMatching}
+       *                   if it matches (i.e. is contained), even if two or more subsets are equal, or equal when ignoring
+       *                   duplicate elements.
+       * @param matchValue The match value provided each time the subset matches.
+       * @return This builder.
+       */
+      public Builder<E, M> addSubset(Collection<E> subset, M matchValue) {
+        if (!subset.isEmpty()) {
+          TrieSubsetMatcher.Node<E, M> node = root;
+          for (E e : ImmutableSortedSet.copyOf(subset)) {
+            node = node.getOrCreateChild(e);
+          }
+          node.addMatchValue(matchValue);
+          subsetCount++;
+        }
+        return this;
+      }
+
+      public TrieSubsetMatcher<E, M> build() {
+        root.trimAndMakeImmutable();
+        return new TrieSubsetMatcher<>(root, subsetCount);
+      }
+    }
+
+    private final Node<E, M> root;
+    private final int subsetCount;
+
+    private TrieSubsetMatcher(Node<E, M> root, int subsetCount) {
+      this.root = root;
+      this.subsetCount = subsetCount;
+    }
+
+    /**
+     * Gets the number of subsets in this matcher.
+     */
+    public int getSubsetCount() {
+      return subsetCount;
+    }
+
+    /**
+     * Returns an iterator over all the subsets that are contained by the provided set.
+     * The returned iterator does not support removal.
+     *
+     * @param set This set is copied to a new {@link ImmutableSortedSet} with natural ordering.
+     */
+    public Iterator<M> findSubsetsMatching(Collection<E> set) {
+      return new MatchIterator(ImmutableSortedSet.copyOf(set));
+    }
+
+    /**
+     * Trie node.
+     */
+    private static class Node<E extends Comparable<? super E>, M> {
+
+      private Map<E, Node<E, M>> children;
+      private E greatestEdge;
+      private List<M> matchValues;
+
+      /**
+       * Gets the child node for the provided element; or <code>null</code> if none.
+       */
+      Node<E, M> getChild(E e) {
+        return (children == null ? null : children.get(e));
+      }
+
+      /**
+       * Gets the child node for the provided element, or creates it if it does not exist.
+       */
+      Node<E, M> getOrCreateChild(E e) {
+        if (children == null) {
+          children = new HashMap<>(4);
+        }
+        Node<E, M> child = children.get(e);
+        if (child == null) {
+          child = new Node<>();
+          children.put(e, child);
+          if (greatestEdge == null || e.compareTo(greatestEdge) > 0) {
+            greatestEdge = e;
+          }
+        }
+        return child;
+      }
+
+      /**
+       * Indicates whether this node has more children for edges greater than the given element.
+       *
+       * @return <code>true</code> if this node has more children for edges greater than the given element;
+       * <code>false</code> otherwise.
+       */
+      boolean hasMorePotentialChildren(E e) {
+        return greatestEdge != null && e.compareTo(greatestEdge) < 0;
+      }
+
+      /**
+       * Decorates this node with an additional match value.
+       */
+      void addMatchValue(M matchValue) {
+        if (matchValues == null) {
+          matchValues = new ArrayList<>(1);
+        }
+        matchValues.add(matchValue);
+      }
+
+      /**
+       * Gets the match values decorating this node.
+       */
+      List<M> getMatchValues() {
+        return (matchValues == null ? Collections.emptyList() : matchValues);
+      }
+
+      /**
+       * Trims and makes this node, as well as all descendant nodes, immutable.
+       * This may reduce its memory usage and make it more efficient.
+       */
+      void trimAndMakeImmutable() {
+        if (children != null && !(children instanceof ImmutableMap)) {
+          for (Node<E, M> child : children.values())
+            child.trimAndMakeImmutable();
+          children = ImmutableMap.copyOf(children);
+        }
+        if (matchValues != null && !(matchValues instanceof ImmutableList)) {
+          matchValues = ImmutableList.copyOf(matchValues);
+        }
+      }
+    }
+
+    private class MatchIterator implements Iterator<M> {
+
+      private final Iterator<E> sortedSetIterator;
+      private final Queue<TrieSubsetMatcher.Node<E, M>> currentNodes;
+      private final Queue<M> nextMatchValues;
+
+      MatchIterator(SortedSet<E> set) {
+        sortedSetIterator = set.iterator();
+        currentNodes = new ArrayDeque<>();
+        currentNodes.offer(root);
+        nextMatchValues = new ArrayDeque<>();
+      }
+
+      @Override
+      public boolean hasNext() {
+        return !nextMatchValues.isEmpty() || nextSubsetMatch();
+      }
+
+      @Override
+      public M next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        assert !nextMatchValues.isEmpty();
+        return nextMatchValues.poll();
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+
+      private boolean nextSubsetMatch() {
+        while (sortedSetIterator.hasNext()) {
+          E e = sortedSetIterator.next();
+          int currentNodeCount = currentNodes.size();
+          for (int i = 0; i < currentNodeCount; i++) {
+            TrieSubsetMatcher.Node<E, M> currentNode = currentNodes.remove();
+            TrieSubsetMatcher.Node<E, M> child = currentNode.getChild(e);
+            if (child != null) {
+              currentNodes.offer(child);
+              nextMatchValues.addAll(child.getMatchValues());
+            }
+            if (currentNode.hasMorePotentialChildren(e)) {
+              currentNodes.offer(currentNode);
+            }
+          }
+          if (!nextMatchValues.isEmpty()) {
+            return true;
+          }
+        }
+        return false;
+      }
+    }
+  }
 }
\ No newline at end of file
diff --git a/solr/core/src/test-files/solr/collection1/conf/elevate.xml b/solr/core/src/test-files/solr/collection1/conf/elevate.xml
index 1befc54..ddf1e9f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/elevate.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/elevate.xml
@@ -51,4 +51,26 @@
   <doc id="7" />
  </query>
 
+ <query text="AA BB CC" match="subset">
+  <doc id="10"/>
+ </query>
+
+ <query text="BB CC" match="subset">
+  <doc id="10"/>
+  <doc id="12"/>
+ </query>
+
+ <query text="BB CC" match="subset">
+  <doc id="11"/>
+ </query>
+
+ <query text="BB CC" match="exact">
+  <doc id="13"/>
+ </query>
+
+ <query text="DD AA" match="subset">
+  <doc id="12"/>
+  <doc id="14"/>
+ </query>
+
 </elevate>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml
index 3e92ab2..ae8b4ea 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml
@@ -92,7 +92,7 @@
 
   <!-- test elevation -->
   <searchComponent name="elevate" class="org.apache.solr.handler.component.QueryElevationComponent" >
-    <str name="queryFieldType">string</str>
+    <str name="queryFieldType">text</str>
     <str name="config-file">${elevate.file:elevate.xml}</str>
   </searchComponent>
 
diff --git a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
index 7c7495d..ad0816b 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/QueryElevationComponentTest.java
@@ -22,6 +22,9 @@ import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 import org.apache.lucene.index.IndexReader;
@@ -54,7 +57,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   @BeforeClass
-  public static void beforeClass() throws Exception {
+  public static void beforeClass() {
     switch (random().nextInt(3)) {
       case 0:
         System.setProperty("solr.tests.id.stored", "true");
@@ -69,7 +72,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
         System.setProperty("solr.tests.id.docValues", "true");
         break;
       default:
-        fail("Bad random number generatged not between 0-2 iunclusive");
+        fail("Bad random number generated not between 0-2 inclusive");
         break;
     }
   }
@@ -98,7 +101,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
   }
 
   //TODO should be @After ?
-  private void delete() throws Exception {
+  private void delete() {
     deleteCore();
   }
 
@@ -391,13 +394,13 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
       req.close();
 
       // Make sure the boosts loaded properly
-      assertEquals(7, elevationProvider.size());
+      assertEquals(11, elevationProvider.size());
       assertEquals(1, elevationProvider.getElevationForQuery("XXXX").elevatedIds.size());
       assertEquals(2, elevationProvider.getElevationForQuery("YYYY").elevatedIds.size());
       assertEquals(3, elevationProvider.getElevationForQuery("ZZZZ").elevatedIds.size());
-      assertEquals(null, elevationProvider.getElevationForQuery("xxxx"));
-      assertEquals(null, elevationProvider.getElevationForQuery("yyyy"));
-      assertEquals(null, elevationProvider.getElevationForQuery("zzzz"));
+      assertNull(elevationProvider.getElevationForQuery("xxxx"));
+      assertNull(elevationProvider.getElevationForQuery("yyyy"));
+      assertNull(elevationProvider.getElevationForQuery("zzzz"));
 
       // Now test the same thing with a lowercase filter: 'lowerfilt'
       args = new NamedList<>();
@@ -408,7 +411,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
       comp.init(args);
       comp.inform(core);
       elevationProvider = comp.getElevationProvider(reader, core);
-      assertEquals(7, elevationProvider.size());
+      assertEquals(11, elevationProvider.size());
       assertEquals(1, elevationProvider.getElevationForQuery("XXXX").elevatedIds.size());
       assertEquals(2, elevationProvider.getElevationForQuery("YYYY").elevatedIds.size());
       assertEquals(3, elevationProvider.getElevationForQuery("ZZZZ").elevatedIds.size());
@@ -605,7 +608,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
 
       // Try normal sort by 'id'
       // default 'forceBoost' should be false
-      assertEquals(false, booster.forceElevation);
+      assertFalse(booster.forceElevation);
       assertQ(req(baseParams, "sort", "id asc")
           , "//*[@numFound='4']"
           , "//result/doc[1]/str[@name='id'][.='a']"
@@ -685,8 +688,8 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
     }
   }
 
-  // write a test file to boost some docs
-  private void writeFile(File file, String query, String... ids) throws Exception {
+  // write an elevation config file to boost some docs
+  private void writeElevationConfigFile(File file, String query, String... ids) throws Exception {
     PrintWriter out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(file), StandardCharsets.UTF_8));
     out.println("<?xml version=\"1.0\" encoding=\"UTF-8\" ?>");
     out.println("<elevate>");
@@ -708,7 +711,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
       init("schema12.xml");
       String testfile = "data-elevation.xml";
       File configFile = new File(h.getCore().getDataDir(), testfile);
-      writeFile(configFile, "aaa", "A");
+      writeElevationConfigFile(configFile, "aaa", "A");
 
       QueryElevationComponent comp = (QueryElevationComponent) h.getCore().getSearchComponent("elevate");
       NamedList<String> args = new NamedList<>();
@@ -716,7 +719,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
       comp.init(args);
       comp.inform(h.getCore());
 
-      QueryElevationComponent.ElevationProvider elevationProvider = null;
+      QueryElevationComponent.ElevationProvider elevationProvider;
 
       try (SolrQueryRequest req = req()) {
         elevationProvider = comp.getElevationProvider(req.getSearcher().getIndexReader(), req.getCore());
@@ -725,7 +728,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
       }
 
       // now change the file
-      writeFile(configFile, "bbb", "B");
+      writeElevationConfigFile(configFile, "bbb", "B");
 
       // With no index change, we get the same index reader, so the elevationProviderCache returns the previous ElevationProvider without the change.
       try (SolrQueryRequest req = req()) {
@@ -746,7 +749,7 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
       }
 
       // Now change the config file again.
-      writeFile(configFile, "ccc", "C");
+      writeElevationConfigFile(configFile, "ccc", "C");
 
       // Without index change, but calling a different method that clears the elevationProviderCache, so we should load a new ElevationProvider.
       int elevationRuleNumber = comp.loadElevationConfiguration(h.getCore());
@@ -794,4 +797,137 @@ public class QueryElevationComponentTest extends SolrTestCaseJ4 {
     }
   }
 
+  @Test
+  public void testQuerySubsetMatching() throws Exception {
+    try {
+      init("schema12.xml");
+      assertU(adoc("id", "1", "title", "XXXX", "str_s1", "a"));
+      assertU(adoc("id", "2", "title", "YYYY", "str_s1", "b"));
+      assertU(adoc("id", "3", "title", "ZZZZ", "str_s1", "c"));
+
+      assertU(adoc("id", "4", "title", "XXXX XXXX", "str_s1", "x"));
+      assertU(adoc("id", "5", "title", "YYYY YYYY", "str_s1", "y"));
+      assertU(adoc("id", "6", "title", "XXXX XXXX", "str_s1", "z"));
+      assertU(adoc("id", "7", "title", "AAAA", "str_s1", "a"));
+
+      assertU(adoc("id", "10", "title", "RR", "str_s1", "r"));
+      assertU(adoc("id", "11", "title", "SS", "str_s1", "r"));
+      assertU(adoc("id", "12", "title", "TT", "str_s1", "r"));
+      assertU(adoc("id", "13", "title", "UU", "str_s1", "r"));
+      assertU(adoc("id", "14", "title", "VV", "str_s1", "r"));
+      assertU(commit());
+
+      // Exact matching.
+      assertQ("", req(CommonParams.Q, "XXXX", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]"),
+          "//*[@numFound='3']",
+          "//result/doc[1]/str[@name='id'][.='1']",
+          "//result/doc[2]/str[@name='id'][.='4']",
+          "//result/doc[3]/str[@name='id'][.='6']",
+          "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[2]/bool[@name='[elevated]'][.='false']",
+          "//result/doc[3]/bool[@name='[elevated]'][.='false']"
+      );
+
+      // Exact matching.
+      assertQ("", req(CommonParams.Q, "QQQQ EE", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]"),
+          "//*[@numFound='0']"
+      );
+
+      // Subset matching.
+      assertQ("", req(CommonParams.Q, "BB DD CC VV", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]"),
+          "//*[@numFound='4']",
+          "//result/doc[1]/str[@name='id'][.='10']",
+          "//result/doc[2]/str[@name='id'][.='12']",
+          "//result/doc[3]/str[@name='id'][.='11']",
+          "//result/doc[4]/str[@name='id'][.='14']",
+          "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[2]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[3]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[4]/bool[@name='[elevated]'][.='false']"
+      );
+
+      // Subset + exact matching.
+      assertQ("", req(CommonParams.Q, "BB CC", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]"),
+          "//*[@numFound='4']",
+          "//result/doc[1]/str[@name='id'][.='13']",
+          "//result/doc[2]/str[@name='id'][.='10']",
+          "//result/doc[3]/str[@name='id'][.='12']",
+          "//result/doc[4]/str[@name='id'][.='11']",
+          "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[2]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[3]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[4]/bool[@name='[elevated]'][.='true']"
+      );
+
+      // Subset matching.
+      assertQ("", req(CommonParams.Q, "AA BB DD CC AA", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]"),
+          "//*[@numFound='4']",
+          "//result/doc[1]/str[@name='id'][.='10']",
+          "//result/doc[2]/str[@name='id'][.='12']",
+          "//result/doc[3]/str[@name='id'][.='11']",
+          "//result/doc[4]/str[@name='id'][.='14']",
+          "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[2]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[3]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[4]/bool[@name='[elevated]'][.='true']"
+      );
+
+      // Subset matching.
+      assertQ("", req(CommonParams.Q, "AA RR BB DD AA", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]"),
+          "//*[@numFound='3']",
+          "//result/doc[1]/str[@name='id'][.='12']",
+          "//result/doc[2]/str[@name='id'][.='14']",
+          "//result/doc[3]/str[@name='id'][.='10']",
+          "//result/doc[1]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[2]/bool[@name='[elevated]'][.='true']",
+          "//result/doc[3]/bool[@name='[elevated]'][.='false']"
+      );
+
+      // Subset matching.
+      assertQ("", req(CommonParams.Q, "AA BB EE", CommonParams.QT, "/elevate",
+          CommonParams.FL, "id, score, [elevated]")
+          , "//*[@numFound='0']"
+          );
+    } finally {
+      delete();
+    }
+  }
+
+  @Test
+  public void testElevatedIds() throws Exception {
+    try {
+      init("schema12.xml");
+      SolrCore core = h.getCore();
+
+      NamedList<String> args = new NamedList<>();
+      args.add(QueryElevationComponent.FIELD_TYPE, "text");
+      args.add(QueryElevationComponent.CONFIG_FILE, "elevate.xml");
+
+      QueryElevationComponent comp = new QueryElevationComponent();
+      comp.init(args);
+      comp.inform(core);
+
+      SolrQueryRequest req = req();
+      IndexReader reader = req.getSearcher().getIndexReader();
+      QueryElevationComponent.ElevationProvider elevationProvider = comp.getElevationProvider(reader, core);
+      req.close();
+
+      assertEquals(toIdSet("1"), elevationProvider.getElevationForQuery("xxxx").elevatedIds);
+      assertEquals(toIdSet("10", "11", "12"), elevationProvider.getElevationForQuery("bb DD CC vv").elevatedIds);
+      assertEquals(toIdSet("10", "11", "12", "13"), elevationProvider.getElevationForQuery("BB Cc").elevatedIds);
+      assertEquals(toIdSet("10", "11", "12", "14"), elevationProvider.getElevationForQuery("aa bb dd cc aa").elevatedIds);
+    } finally {
+      delete();
+    }
+  }
+
+  private static Set<BytesRef> toIdSet(String... ids) {
+    return Arrays.stream(ids).map(BytesRef::new).collect(Collectors.toSet());
+  }
 }
diff --git a/solr/solr-ref-guide/src/the-query-elevation-component.adoc b/solr/solr-ref-guide/src/the-query-elevation-component.adoc
index 3f9bf63..a5b98bd 100644
--- a/solr/solr-ref-guide/src/the-query-elevation-component.adoc
+++ b/solr/solr-ref-guide/src/the-query-elevation-component.adoc
@@ -93,11 +93,19 @@ Elevated query results can be configured in an external XML file specified in th
     <doc id="MA147LL/A" />  <!-- put the actual ipod at the top -->
     <doc id="IW-02" exclude="true" /> <!-- exclude this cable -->
   </query>
+
+  <query text="foo bill" match="subset">
+    <doc id="11" />
+  </query>
 </elevate>
 ----
 
 In this example, the query "foo bar" would first return documents 1, 2 and 3, then whatever normally appears for the same query. For the query "ipod", it would first return "MA147LL/A", and would make sure that "IW-02" is not in the result set.
 
+Notice the `match` parameter with the value `"subset"` for the third rule. A query "bill bar foo" would trigger this rule because the rule defines a subset of terms to appear in the query, in any order. This query would elevate document 11 on top.
+The `match` parameter accepts either `"exact"` (by default) or `"subset"` values.
+Subset matching is scalable, one can add many rules with the `match="subset"` parameter.
+
 If documents to be elevated are not defined in the `elevate.xml` file, they should be passed in at query time with the <<The elevateIds and excludeIds Parameters,`elevateIds` parameter>>.
 
 == Using the Query Elevation Component