You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/01/11 08:00:43 UTC

[07/50] [abbrv] lucene-solr:jira/solr-11702: SOLR-11801: Support customisation of the highlighting query response element. (Ramsey Haddad, Pranav Murugappan, David Smiley, Christine Poerschke)

SOLR-11801: Support customisation of the highlighting query response element.
(Ramsey Haddad, Pranav Murugappan, David Smiley, Christine Poerschke)


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

Branch: refs/heads/jira/solr-11702
Commit: 65c842f9faed1c9de9dce38a247a43c36b82873f
Parents: 5d4f029
Author: Christine Poerschke <cp...@apache.org>
Authored: Thu Jan 4 13:17:25 2018 +0000
Committer: Christine Poerschke <cp...@apache.org>
Committed: Thu Jan 4 14:54:48 2018 +0000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../handler/component/HighlightComponent.java   |  41 ++-
 .../component/CustomHighlightComponentTest.java | 310 +++++++++++++++++++
 3 files changed, 348 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/65c842f9/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7137b80..61551e0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -113,6 +113,9 @@ Other Changes
 * SOLR-11798: Formally deprecate top-level <highlighting> syntax in solrconfig.xml
   in favour of <searchComponent> equivalent syntax. (Christine Poerschke)
 
+* SOLR-11801: Support customisation of the "highlighting" query response element.
+  (Ramsey Haddad, Pranav Murugappan, David Smiley, Christine Poerschke)
+
 ==================  7.2.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/65c842f9/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java b/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
index 698cc05..1602502 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
@@ -187,7 +187,7 @@ public class HighlightComponent extends SearchComponent implements PluginInfoIni
         
         if(sumData != null) {
           // TODO ???? add this directly to the response?
-          rb.rsp.add("highlighting", sumData);
+          rb.rsp.add(highlightingResponseField(), convertHighlights(sumData));
         }
       }
     }
@@ -245,7 +245,8 @@ public class HighlightComponent extends SearchComponent implements PluginInfoIni
   public void finishStage(ResponseBuilder rb) {
     if (rb.doHighlights && rb.stage == ResponseBuilder.STAGE_GET_FIELDS) {
 
-      NamedList.NamedListEntry[] arr = new NamedList.NamedListEntry[rb.resultIds.size()];
+      final Object[] objArr = newHighlightsArray(rb.resultIds.size());
+      final String highlightingResponseField = highlightingResponseField();
 
       // TODO: make a generic routine to do automatic merging of id keyed data
       for (ShardRequest sreq : rb.finished) {
@@ -256,13 +257,12 @@ public class HighlightComponent extends SearchComponent implements PluginInfoIni
             // this should only happen when using shards.tolerant=true
             continue;
           }
-          NamedList hl = (NamedList)srsp.getSolrResponse().getResponse().get("highlighting");
-          SolrPluginUtils.copyNamedListIntoArrayByDocPosInResponse(hl, rb.resultIds, arr);
+          Object hl = srsp.getSolrResponse().getResponse().get(highlightingResponseField);
+          addHighlights(objArr, hl, rb.resultIds);
         }
       }
 
-      // remove nulls in case not all docs were able to be retrieved
-      rb.rsp.add("highlighting", SolrPluginUtils.removeNulls(arr, new SimpleOrderedMap<>()));
+      rb.rsp.add(highlightingResponseField, getAllHighlights(objArr));
     }
   }
 
@@ -279,4 +279,33 @@ public class HighlightComponent extends SearchComponent implements PluginInfoIni
   public Category getCategory() {
     return Category.HIGHLIGHTER;
   }
+
+  ////////////////////////////////////////////
+  ///  highlighting response collation
+  ////////////////////////////////////////////
+
+  protected String highlightingResponseField() {
+    return "highlighting";
+  }
+
+  protected Object convertHighlights(NamedList hl) {
+    return hl;
+  }
+
+  protected Object[] newHighlightsArray(int size) {
+    return new NamedList.NamedListEntry[size];
+  }
+
+  protected void addHighlights(Object[] objArr, Object obj, Map<Object, ShardDoc> resultIds) {
+    Map.Entry<String, Object>[] arr = (Map.Entry<String, Object>[])objArr;
+    NamedList hl = (NamedList)obj;
+    SolrPluginUtils.copyNamedListIntoArrayByDocPosInResponse(hl, resultIds, arr);
+  }
+
+  protected Object getAllHighlights(Object[] objArr) {
+      final Map.Entry<String, Object>[] arr = (Map.Entry<String, Object>[])objArr;
+      // remove nulls in case not all docs were able to be retrieved
+      return SolrPluginUtils.removeNulls(arr, new SimpleOrderedMap<>());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/65c842f9/solr/core/src/test/org/apache/solr/handler/component/CustomHighlightComponentTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/component/CustomHighlightComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/CustomHighlightComponentTest.java
new file mode 100644
index 0000000..5f74064
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/component/CustomHighlightComponentTest.java
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.AbstractDistribZkTestBase;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.cloud.TestCloudSearcherWarming.ConfigRequest;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.highlight.SolrFragmentsBuilder;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class CustomHighlightComponentTest extends SolrCloudTestCase {
+
+  public static class CustomHighlightComponent extends HighlightComponent {
+
+    protected String id_key = "id";
+    protected String snippets_key = "snippets";
+
+    @Override
+    protected String highlightingResponseField() {
+      return "custom_highlighting";
+    }
+
+    @Override
+    protected Object convertHighlights(NamedList hl) {
+      final ArrayList<SimpleOrderedMap> hlMaps = new ArrayList<>();
+      for (int i=0; i<hl.size(); ++i) {
+          SimpleOrderedMap hlMap = new SimpleOrderedMap<Object>();
+          hlMap.add(id_key, hl.getName(i));
+          hlMap.add(snippets_key, hl.getVal(i));
+          hlMaps.add(hlMap);
+      }
+      return hlMaps;
+    }
+
+    @Override
+    protected Object[] newHighlightsArray(int size) {
+      return new SimpleOrderedMap[size];
+    }
+
+    @Override
+    protected void addHighlights(Object[] objArr, Object obj, Map<Object, ShardDoc> resultIds) {
+      SimpleOrderedMap[] mapArr = (SimpleOrderedMap[])objArr;
+      final ArrayList<SimpleOrderedMap> hlMaps = (ArrayList<SimpleOrderedMap>)obj;
+      for (SimpleOrderedMap hlMap : hlMaps) {
+        String id = (String)hlMap.get(id_key);
+        ShardDoc sdoc = resultIds.get(id);
+        int idx = sdoc.positionInResponse;
+        mapArr[idx] = hlMap;
+      }
+    }
+
+    @Override
+    protected Object getAllHighlights(Object[] objArr) {
+      final SimpleOrderedMap[] mapArr = (SimpleOrderedMap[])objArr;
+      // remove nulls in case not all docs were able to be retrieved
+      ArrayList<SimpleOrderedMap> mapList = new ArrayList<>();
+      for (SimpleOrderedMap map : mapArr) {
+        if (map != null) {
+          mapList.add((SimpleOrderedMap)map);
+        }
+      }
+      return mapList;
+    }
+
+  }
+
+  protected String customHighlightComponentClassName() {
+    return CustomHighlightComponent.class.getName();
+  }
+
+  protected String id_key = "id";
+  protected String snippets_key = "snippets";
+
+  private static String COLLECTION;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+
+    // decide collection name ...
+    COLLECTION = "collection"+(1+random().nextInt(100)) ;
+    // ... and shard/replica/node numbers
+    final int numShards = 3;
+    final int numReplicas = 2;
+    final int maxShardsPerNode = 2;
+    final int nodeCount = (numShards*numReplicas + (maxShardsPerNode-1))/maxShardsPerNode;
+
+    // create and configure cluster
+    configureCluster(nodeCount)
+        .addConfig("conf", configset("cloud-dynamic"))
+        .configure();
+
+    // create an empty collection
+    CollectionAdminRequest
+    .createCollection(COLLECTION, "conf", numShards, numReplicas)
+    .setMaxShardsPerNode(maxShardsPerNode)
+    .processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), false, true, DEFAULT_TIMEOUT);
+  }
+
+  @Test
+  public void test() throws Exception {
+
+    // determine custom search handler name (the exact name should not matter)
+    final String customSearchHandlerName = "/custom_select"+random().nextInt();
+
+    final String defaultHighlightComponentName = HighlightComponent.COMPONENT_NAME;
+    final String highlightComponentName;
+
+    // add custom component (if needed) and handler
+    {
+      if (random().nextBoolean()) {
+        // default component
+        highlightComponentName = defaultHighlightComponentName;
+      } else {
+        // custom component
+        highlightComponentName = "customhighlight"+random().nextInt();
+        cluster.getSolrClient().request(
+            new ConfigRequest(
+                SolrRequest.METHOD.POST,
+                "/config",
+                "{\n" +
+                "  'add-searchcomponent': {\n" +
+                "    'name': '"+highlightComponentName+"',\n" +
+                "    'class': '"+customHighlightComponentClassName()+"'\n" +
+                "  }\n" +
+                "}"),
+            COLLECTION);
+      }
+      // handler
+      cluster.getSolrClient().request(
+          new ConfigRequest(
+              SolrRequest.METHOD.POST,
+              "/config",
+              "{\n" +
+              "  'add-requesthandler': {\n" +
+              "    'name' : '"+customSearchHandlerName+"',\n" +
+              "    'class' : 'org.apache.solr.handler.component.SearchHandler',\n" +
+              "    'components' : [ '"+QueryComponent.COMPONENT_NAME+"', '"+highlightComponentName+"' ]\n" +
+              "  }\n" +
+              "}"),
+          COLLECTION);
+    }
+
+    // add some documents
+    final String id = "id";
+    final String t1 = "a_t";
+    final String t2 = "b_t";
+    {
+      new UpdateRequest()
+          .add(sdoc(id, 1, t1, "bumble bee", t2, "bumble bee"))
+          .add(sdoc(id, 2, t1, "honey bee", t2, "honey bee"))
+          .add(sdoc(id, 3, t1, "solitary bee", t2, "solitary bee"))
+          .commit(cluster.getSolrClient(), COLLECTION);
+    }
+
+    // search for the documents
+    {
+      // compose the query
+      final SolrQuery solrQuery =  new SolrQuery(t1+":bee");
+      solrQuery.setRequestHandler(customSearchHandlerName);
+      solrQuery.setHighlight(true);
+      final boolean t1Highlights = random().nextBoolean();
+      if (t1Highlights) {
+        solrQuery.addHighlightField(t1);
+      }
+      final boolean t2Highlights = random().nextBoolean();
+      if (t2Highlights) {
+        solrQuery.addHighlightField(t2);
+      }
+
+      // make the query
+      final QueryResponse queryResponse = new QueryRequest(solrQuery)
+          .process(cluster.getSolrClient(), COLLECTION);
+
+      // analyse the response
+      final Map<String, Map<String, List<String>>> highlighting = queryResponse.getHighlighting();
+      final ArrayList<SimpleOrderedMap<Object>> custom_highlighting =
+          (ArrayList<SimpleOrderedMap<Object>>)queryResponse.getResponse().get("custom_highlighting");
+
+      if (defaultHighlightComponentName.equals(highlightComponentName)) {
+        // regular 'highlighting' ...
+        if (t1Highlights) {
+          checkHighlightingResponseMap(highlighting, t1);
+        }
+        if (t2Highlights) {
+          checkHighlightingResponseMap(highlighting, t2);
+        }
+        if (!t1Highlights && !t2Highlights) {
+          checkHighlightingResponseMap(highlighting, null);
+        }
+        // ... and no 'custom_highlighting'
+        assertNull(custom_highlighting);
+      } else {
+        // no regular 'highlighting' ...
+        assertNull(highlighting);
+        // ... but 'custom_highlighting'
+        assertNotNull(custom_highlighting);
+        if (t1Highlights) {
+          checkHighlightingResponseList(custom_highlighting, t1);
+        }
+        if (t2Highlights) {
+          checkHighlightingResponseList(custom_highlighting, t2);
+        }
+        if (!t1Highlights && !t2Highlights) {
+          checkHighlightingResponseList(custom_highlighting, null);
+        }
+      }
+    }
+  }
+
+  protected void checkHighlightingResponseMap(Map<String, Map<String, List<String>>> highlightingMap,
+      String highlightedField) throws Exception {
+    assertEquals("too few or too many keys: "+highlightingMap.keySet(),
+        3, highlightingMap.size());
+    checkHighlightingResponseMapElement(highlightingMap.get("1"), highlightedField, "bumble ", "bee");
+    checkHighlightingResponseMapElement(highlightingMap.get("2"), highlightedField, "honey ", "bee");
+    checkHighlightingResponseMapElement(highlightingMap.get("3"), highlightedField, "solitary ", "bee");
+  }
+
+  protected void checkHighlightingResponseMapElement(Map<String, List<String>> docHighlights,
+      String highlightedField, String preHighlightText, String highlightedText) throws Exception {
+    if (highlightedField == null) {
+      assertEquals(0, docHighlights.size());
+    } else {
+      List<String> docHighlightsList = docHighlights.get(highlightedField);
+      assertEquals(1, docHighlightsList.size());
+      assertEquals(preHighlightText
+          + SolrFragmentsBuilder.DEFAULT_PRE_TAGS
+          + highlightedText
+          + SolrFragmentsBuilder.DEFAULT_POST_TAGS, docHighlightsList.get(0));
+    }
+  }
+
+  protected void checkHighlightingResponseList(ArrayList<SimpleOrderedMap<Object>> highlightingList,
+      String highlightedField) throws Exception {
+    assertEquals("too few or too many elements: "+highlightingList.size(),
+        3, highlightingList.size());
+    final Set<String> seenDocIds = new HashSet<>();
+    for (SimpleOrderedMap<Object> highlightingListElementMap : highlightingList) {
+      final String expectedHighlightText;
+      final String actualHighlightText;
+      // two elements in total: id and snippets
+      assertEquals(highlightingList.toString(), 2, highlightingListElementMap.size());
+      // id element
+      {
+        final String docId = (String)highlightingListElementMap.get(id_key);
+        seenDocIds.add(docId);
+        final String preHighlightText;
+        final String highlightedText = "bee";
+        if ("1".equals(docId)) {
+          preHighlightText = "bumble ";
+        } else if ("2".equals(docId)) {
+          preHighlightText = "honey ";
+        } else if ("3".equals(docId)) {
+          preHighlightText = "solitary ";
+        } else  {
+          preHighlightText = null;
+          fail("unknown docId "+docId);
+        }
+        expectedHighlightText = preHighlightText
+            + SolrFragmentsBuilder.DEFAULT_PRE_TAGS
+            + highlightedText
+            + SolrFragmentsBuilder.DEFAULT_POST_TAGS;
+      }
+      // snippets element
+      {
+        SimpleOrderedMap<Object> snippets = (SimpleOrderedMap<Object>)highlightingListElementMap.get(snippets_key);
+        if (highlightedField == null) {
+          assertEquals(0, snippets.size());
+        } else {
+          ArrayList<String> docHighlights = (ArrayList<String>)(snippets).get(highlightedField);
+          assertEquals(1, docHighlights.size());
+          actualHighlightText = (String)docHighlights.get(0);
+          assertEquals(expectedHighlightText, actualHighlightText);
+        }
+      }
+    }
+    assertEquals(3, seenDocIds.size());
+  }
+
+}