You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2018/10/18 12:53:44 UTC

lucene-solr:master: LUCENE-8535: Drop out of the box Block-Join highlight support

Repository: lucene-solr
Updated Branches:
  refs/heads/master 167c65afa -> 5a4fd86cc


LUCENE-8535: Drop out of the box Block-Join highlight support

Highlighter doesn't support ToParent and ToChildBlockJoinQuery out of the
box anymore. In oder to highlight on Block-Join Queries a custom WeightedSpanTermExtractor
should be used.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/5a4fd86c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/5a4fd86c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/5a4fd86c

Branch: refs/heads/master
Commit: 5a4fd86cce57586dd14fb8f11f9b170b121eebc4
Parents: 167c65a
Author: Simon Willnauer <si...@apache.org>
Authored: Thu Oct 18 10:15:57 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Thu Oct 18 14:53:01 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++
 lucene/MIGRATE.txt                              |  5 ++
 lucene/highlighter/build.xml                    |  2 -
 .../lucene/search/highlight/QueryScorer.java    |  3 +-
 .../highlight/WeightedSpanTermExtractor.java    | 36 +++++++++---
 .../vectorhighlight/FastVectorHighlighter.java  |  6 +-
 .../search/vectorhighlight/FieldQuery.java      | 12 +---
 .../search/highlight/HighlighterTest.java       | 60 --------------------
 .../search/vectorhighlight/FieldQueryTest.java  | 15 -----
 .../solr/highlight/DefaultSolrHighlighter.java  | 49 +++++++++++++++-
 10 files changed, 90 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 50b1cf3..a3af603 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -123,6 +123,10 @@ Changes in Runtime Behavior
 * LUCENE-8505: IndexWriter#addIndices will now fail if the target index is sorted but
   the candidate is not. (Jim Ferenczi)
 
+* LUCENE-8535: Highlighter and FVH doesn't support ToParent and ToChildBlockJoinQuery out of the
+  box anymore. In order to highlight on Block-Join Queries a custom WeightedSpanTermExtractor / FieldQuery
+  should be used. (Simon Willnauer, Jim Ferenczi, Julie Tibshiran)
+
 New Features
 
 * LUCENE-8340: LongPoint#newDistanceQuery may be used to boost scores based on

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/MIGRATE.txt
----------------------------------------------------------------------
diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt
index 1b56b64..0515ff8 100644
--- a/lucene/MIGRATE.txt
+++ b/lucene/MIGRATE.txt
@@ -140,3 +140,8 @@ a LowerCaseFilter
 
 CharTokenizer now only performs tokenization. To perform any type of filtering
 use a TokenFilter chain as you would with any other Tokenizer.
+
+## Highlighter and FastVectorHighlighter no longer support ToParent/ToChildBlockJoinQuery
+
+Both Highlighter and FastVectorHighlighter need a custom WeightedSpanTermExtractor or FieldQuery respectively
+in order to support ToParent/ToChildBlockJoinQuery.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/highlighter/build.xml
----------------------------------------------------------------------
diff --git a/lucene/highlighter/build.xml b/lucene/highlighter/build.xml
index ed6cc74..6ecd793 100644
--- a/lucene/highlighter/build.xml
+++ b/lucene/highlighter/build.xml
@@ -31,14 +31,12 @@
   <path id="classpath">
     <pathelement path="${memory.jar}"/>
     <pathelement path="${queries.jar}"/>
-    <pathelement path="${join.jar}"/>
     <path refid="base.classpath"/>
   </path>
 
   <path id="test.classpath">
     <pathelement path="${memory.jar}"/>
     <pathelement path="${queries.jar}"/>
-    <pathelement path="${join.jar}"/>
     <pathelement path="${analyzers-common.jar}"/>
     <path refid="test.base.classpath"/>
   </path>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
index 25d9b3c..5452fc7 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
@@ -229,8 +229,7 @@ public class QueryScorer implements Scorer {
   }
   
   protected WeightedSpanTermExtractor newTermExtractor(String defaultField) {
-    return defaultField == null ? new WeightedSpanTermExtractor()
-    : new WeightedSpanTermExtractor(defaultField);
+    return new WeightedSpanTermExtractor(defaultField);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
index e0655c5..bc02621 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
@@ -54,8 +54,6 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.SynonymQuery;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.join.ToChildBlockJoinQuery;
-import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.lucene.search.spans.FieldMaskingSpanQuery;
 import org.apache.lucene.search.spans.SpanFirstQuery;
 import org.apache.lucene.search.spans.SpanNearQuery;
@@ -71,6 +69,31 @@ import org.apache.lucene.util.IOUtils;
 /**
  * Class used to extract {@link WeightedSpanTerm}s from a {@link Query} based on whether 
  * {@link Term}s from the {@link Query} are contained in a supplied {@link TokenStream}.
+ *
+ * In order to support additional, by default unsupported queries, subclasses can override
+ * {@link #extract(Query, float, Map)} for extracting wrapped or delegate queries and
+ * {@link #extractUnknownQuery(Query, Map)} to process custom leaf queries:
+ * <pre>
+ * <code>
+ *    WeightedSpanTermExtractor extractor = new WeightedSpanTermExtractor() {
+ *        protected void extract(Query query, float boost, Map&lt;String, WeightedSpanTerm&gt;terms) throws IOException {
+ *          if (query instanceof QueryWrapper) {
+ *            extract(((QueryWrapper)query).getQuery(), boost, terms);
+ *          } else {
+ *            super.extract(query, boost, terms);
+ *          }
+ *        }
+ *
+ *        protected void extractUnknownQuery(Query query, Map&lt;String, WeightedSpanTerm&gt; terms) throws IOException {
+ *          if (query instanceOf CustomTermQuery) {
+ *            Term term = ((CustomTermQuery) query).getTerm();
+ *            terms.put(term.field(), new WeightedSpanTerm(1, term.text()));
+ *          }
+ *        }
+ *    };
+ * }
+ * </code>
+ * </pre>
  */
 public class WeightedSpanTermExtractor {
 
@@ -85,12 +108,11 @@ public class WeightedSpanTermExtractor {
   private LeafReader internalReader = null;
 
   public WeightedSpanTermExtractor() {
+    this(null);
   }
 
   public WeightedSpanTermExtractor(String defaultField) {
-    if (defaultField != null) {
-      this.defaultField = defaultField;
-    }
+    this.defaultField = defaultField;
   }
 
   /**
@@ -154,10 +176,6 @@ public class WeightedSpanTermExtractor {
       for (Query clause : ((DisjunctionMaxQuery) query)) {
         extract(clause, boost, terms);
       }
-    } else if (query instanceof ToParentBlockJoinQuery) {
-      extract(((ToParentBlockJoinQuery) query).getChildQuery(), boost, terms);
-    } else if (query instanceof ToChildBlockJoinQuery) {
-      extract(((ToChildBlockJoinQuery) query).getParentQuery(), boost, terms);
     } else if (query instanceof MultiPhraseQuery) {
       final MultiPhraseQuery mpq = (MultiPhraseQuery) query;
       final Term[][] termArrays = mpq.getTermArrays();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java
index 4845de6..ac7e9d0 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java
@@ -31,8 +31,8 @@ import org.apache.lucene.search.highlight.Encoder;
 public class FastVectorHighlighter {
   public static final boolean DEFAULT_PHRASE_HIGHLIGHT = true;
   public static final boolean DEFAULT_FIELD_MATCH = true;
-  private final boolean phraseHighlight;
-  private final boolean fieldMatch;
+  protected final boolean phraseHighlight;
+  protected final boolean fieldMatch;
   private final FragListBuilder fragListBuilder;
   private final FragmentsBuilder fragmentsBuilder;
   private int phraseLimit = Integer.MAX_VALUE;
@@ -80,7 +80,7 @@ public class FastVectorHighlighter {
     // TODO: should we deprecate this? 
     // because if there is no reader, then we cannot rewrite MTQ.
     try {
-      return new FieldQuery( query, null, phraseHighlight, fieldMatch );
+      return getFieldQuery(query, null);
     } catch (IOException e) {
       // should never be thrown when reader is null
       throw new RuntimeException (e);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
index cb502343..8584b40 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
@@ -38,7 +38,6 @@ import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SynonymQuery;
 import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.lucene.search.vectorhighlight.FieldTermStack.TermInfo;
 
 /**
@@ -62,7 +61,7 @@ public class FieldQuery {
   // The maximum number of different matching terms accumulated from any one MultiTermQuery
   private static final int MAX_MTQ_TERMS = 1024;
 
-  FieldQuery( Query query, IndexReader reader, boolean phraseHighlight, boolean fieldMatch ) throws IOException {
+  public FieldQuery(Query query, IndexReader reader, boolean phraseHighlight, boolean fieldMatch) throws IOException {
     this.fieldMatch = fieldMatch;
     Set<Query> flatQueries = new LinkedHashSet<>();
     flatten( query, reader, flatQueries, 1f );
@@ -95,7 +94,7 @@ public class FieldQuery {
     this (query, null, phraseHighlight, fieldMatch);
   }
 
-  void flatten( Query sourceQuery, IndexReader reader, Collection<Query> flatQueries, float boost ) throws IOException{
+  protected void flatten( Query sourceQuery, IndexReader reader, Collection<Query> flatQueries, float boost ) throws IOException {
     while (sourceQuery instanceof BoostQuery) {
       BoostQuery bq = (BoostQuery) sourceQuery;
       sourceQuery = bq.getQuery();
@@ -141,15 +140,10 @@ public class FieldQuery {
         flatten( q, reader, flatQueries, boost);
       }
     } else if (sourceQuery instanceof FunctionScoreQuery) {
-      final Query q = ((FunctionScoreQuery)sourceQuery).getWrappedQuery();
+      final Query q = ((FunctionScoreQuery) sourceQuery).getWrappedQuery();
       if (q != null) {
         flatten(q, reader, flatQueries, boost);
       }
-    } else if (sourceQuery instanceof ToParentBlockJoinQuery) {
-      Query childQuery = ((ToParentBlockJoinQuery) sourceQuery).getChildQuery();
-      if (childQuery != null) {
-        flatten(childQuery, reader, flatQueries, boost);
-      }
     } else if (reader != null) {
       Query query = sourceQuery;
       Query rewritten;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
index 89c81db..898846f 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
@@ -81,11 +81,6 @@ import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.search.highlight.SynonymTokenizer.TestHighlightRunner;
-import org.apache.lucene.search.join.BitSetProducer;
-import org.apache.lucene.search.join.QueryBitSetProducer;
-import org.apache.lucene.search.join.ScoreMode;
-import org.apache.lucene.search.join.ToChildBlockJoinQuery;
-import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanNotQuery;
@@ -628,61 +623,6 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
 
   }
   
-  public void testToParentBlockJoinQuery() throws Exception {
-    BitSetProducer parentFilter = new QueryBitSetProducer(
-        new TermQuery(new Term(FIELD_NAME, "parent")));
-    
-    query = new ToParentBlockJoinQuery(new TermQuery(new Term(FIELD_NAME, "child")),
-        parentFilter, ScoreMode.None);
-    searcher = newSearcher(reader);
-    hits = searcher.search(query, 100);
-    int maxNumFragmentsRequired = 2;
-    
-    QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
-    Highlighter highlighter = new Highlighter(this, scorer);
-    
-    for (int i = 0; i < hits.totalHits.value; i++) {
-      String text = "child document";
-      TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, text);
-      
-      highlighter.setTextFragmenter(new SimpleFragmenter(40));
-      highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired, "...");
-    }
-    
-    assertTrue("Failed to find correct number of highlights " + numHighlights + " found",
-        numHighlights == 1);
-  }
-  
-  public void testToChildBlockJoinQuery() throws Exception {
-    BitSetProducer parentFilter = new QueryBitSetProducer(
-        new TermQuery(new Term(FIELD_NAME, "parent")));
-    
-    BooleanQuery.Builder booleanQuery = new BooleanQuery.Builder();
-    booleanQuery.add(new ToChildBlockJoinQuery(new TermQuery(
-        new Term(FIELD_NAME, "parent")), parentFilter), Occur.MUST);
-    booleanQuery.add(new TermQuery(new Term(FIELD_NAME, "child")), Occur.MUST);
-    query = booleanQuery.build();
-    
-    searcher = newSearcher(reader);
-    hits = searcher.search(query, 100);
-    int maxNumFragmentsRequired = 2;
-    
-    QueryScorer scorer = new QueryScorer(query, FIELD_NAME);
-    Highlighter highlighter = new Highlighter(this, scorer);
-    
-    for (int i = 0; i < hits.totalHits.value; i++) {
-      String text = "parent document";
-      final int docId = hits.scoreDocs[i].doc;
-      TokenStream tokenStream = getAnyTokenStream(FIELD_NAME, docId);
-      
-      highlighter.setTextFragmenter(new SimpleFragmenter(40));
-      highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired, "...");
-    }
-    
-    assertTrue("Failed to find correct number of highlights " + numHighlights + " found",
-        numHighlights == 1);
-  }
-
   public void testSimpleQueryScorerPhraseHighlighting2() throws Exception {
     PhraseQuery phraseQuery = new PhraseQuery(5, FIELD_NAME, "text", "piece", "long");
     doSearching(phraseQuery);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java
index 6b1e49d..eb888f6 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java
@@ -27,16 +27,12 @@ import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.RegexpQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.search.WildcardQuery;
-import org.apache.lucene.search.join.QueryBitSetProducer;
-import org.apache.lucene.search.join.ScoreMode;
-import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.lucene.search.vectorhighlight.FieldQuery.QueryPhraseMap;
 import org.apache.lucene.search.vectorhighlight.FieldTermStack.TermInfo;
 import org.apache.lucene.util.BytesRef;
@@ -954,15 +950,4 @@ public class FieldQueryTest extends AbstractTestCase {
     fq.flatten( query, reader, flatQueries, 1f );
     assertCollectionQueries( flatQueries, tq( boost, "A" ) );
   }
-
-  public void testFlattenToParentBlockJoinQuery() throws Exception {
-    initBoost();
-    Query childQuery = tq(boost, "a");
-    Query query = new ToParentBlockJoinQuery(childQuery, new QueryBitSetProducer(new MatchAllDocsQuery()), ScoreMode.None);
-    FieldQuery fq = new FieldQuery(query, true, true );
-    Set<Query> flatQueries = new HashSet<>();
-    fq.flatten(query, reader, flatQueries, 1f);
-    assertCollectionQueries(flatQueries, tq(boost, "a"));
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5a4fd86c/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
index b8fbb25..088170d 100644
--- a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
@@ -19,6 +19,7 @@ package org.apache.solr.highlight;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -52,6 +53,10 @@ import org.apache.lucene.search.highlight.QueryTermScorer;
 import org.apache.lucene.search.highlight.Scorer;
 import org.apache.lucene.search.highlight.TextFragment;
 import org.apache.lucene.search.highlight.TokenSources;
+import org.apache.lucene.search.highlight.WeightedSpanTerm;
+import org.apache.lucene.search.highlight.WeightedSpanTermExtractor;
+import org.apache.lucene.search.join.ToChildBlockJoinQuery;
+import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.lucene.search.vectorhighlight.BoundaryScanner;
 import org.apache.lucene.search.vectorhighlight.FastVectorHighlighter;
 import org.apache.lucene.search.vectorhighlight.FieldQuery;
@@ -238,7 +243,12 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf
    */
   protected QueryScorer getSpanQueryScorer(Query query, String fieldName, TokenStream tokenStream, SolrQueryRequest request) {
     QueryScorer scorer = new QueryScorer(query,
-        request.getParams().getFieldBool(fieldName, HighlightParams.FIELD_MATCH, false) ? fieldName : null);
+        request.getParams().getFieldBool(fieldName, HighlightParams.FIELD_MATCH, false) ? fieldName : null) {
+      @Override
+      protected WeightedSpanTermExtractor newTermExtractor(String defaultField) {
+        return new CustomSpanTermExtractor(defaultField);
+      }
+    };
     scorer.setExpandMultiTermQuery(request.getParams().getBool(HighlightParams.HIGHLIGHT_MULTI_TERM, true));
 
     boolean defaultPayloads = true;//overwritten below
@@ -256,6 +266,24 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf
     return scorer;
   }
 
+  private static class CustomSpanTermExtractor extends WeightedSpanTermExtractor {
+    public CustomSpanTermExtractor(String defaultField) {
+      super(defaultField);
+    }
+
+    @Override
+    protected void extract(Query query, float boost, Map<String, WeightedSpanTerm> terms) throws IOException {
+      // these queries are not supported in lucene highlighting out of the box since 8.0
+      if (query instanceof ToParentBlockJoinQuery) {
+        extract(((ToParentBlockJoinQuery) query).getChildQuery(), boost, terms);
+      } else if (query instanceof ToChildBlockJoinQuery) {
+        extract(((ToChildBlockJoinQuery) query).getParentQuery(), boost, terms);
+      } else {
+        super.extract(query, boost, terms);
+      }
+    }
+  }
+
   /**
    * Return a {@link org.apache.lucene.search.highlight.Scorer} suitable for this Query and field.
    * @param query The current query
@@ -469,7 +497,24 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf
             // FVH cannot process hl.usePhraseHighlighter parameter per-field basis
             params.getBool(HighlightParams.USE_PHRASE_HIGHLIGHTER, true),
             // FVH cannot process hl.requireFieldMatch parameter per-field basis
-            params.getBool(HighlightParams.FIELD_MATCH, false));
+            params.getBool(HighlightParams.FIELD_MATCH, false)) {
+          @Override
+          public FieldQuery getFieldQuery(Query query, IndexReader reader) throws IOException {
+            return new FieldQuery(query, reader, phraseHighlight, fieldMatch) {
+              @Override
+              protected void flatten(Query sourceQuery, IndexReader reader, Collection<Query> flatQueries, float boost) throws IOException {
+                if (sourceQuery instanceof ToParentBlockJoinQuery) {
+                  Query childQuery = ((ToParentBlockJoinQuery) sourceQuery).getChildQuery();
+                  if (childQuery != null) {
+                    flatten(childQuery, reader, flatQueries, boost);
+                  }
+                } else {
+                  super.flatten(sourceQuery, reader, flatQueries, boost);
+                }
+              }
+            };
+          }
+        };
         fvh.setPhraseLimit(params.getInt(HighlightParams.PHRASE_LIMIT, SolrHighlighter.DEFAULT_PHRASE_LIMIT));
         fvhContainer.fvh = fvh;
         fvhContainer.fieldQuery = fvh.getFieldQuery(query, reader);